[jira] [Commented] (SPARK-40939) Release a shaded version of Apache Spark / shade jars on main jar

2023-12-07 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-40939:
-

No movement/discussion that I've seen in the community since my last comment in 
2022, but we're still interested in this over at LinkedIn and might make a push 
for it ... 

> Release a shaded version of Apache Spark / shade jars on main jar
> -
>
> Key: SPARK-40939
> URL: https://issues.apache.org/jira/browse/SPARK-40939
> Project: Spark
>  Issue Type: Improvement
>  Components: Deploy
>Affects Versions: 3.4.0
>Reporter: Almog Tavor
>Priority: Major
>
> I suggest shading in Apache Spark, to resolve the dependency hell that may 
> occur when building / deploying Apache Spark. This mainly occurs on Java 
> projects and on Hadoop environments, but shading will help for using Spark 
> with Scala & even Python either.
> Flink has a similar solution, delivering 
> [flink-shaded|https://github.com/apache/flink-shaded/blob/master/README.md].
> The dependencies I think that are relevant for shading are Jackson, Guava, 
> Netty & any of the Hadoop ecosystems if possible.
> As for releasing sources for the shaded version, I think the [issue that has 
> been raised in Flink|https://github.com/apache/flink-shaded/issues/25] is 
> relevant and unanswered here too, hence I don't think that's an option 
> currently (personally I don't see any value for it either).



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

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



[jira] [Resolved] (SPARK-42539) User-provided JARs can override Spark's Hive metadata client JARs when using "builtin"

2023-04-18 Thread Erik Krogen (Jira)


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

Erik Krogen resolved SPARK-42539.
-
Resolution: Fixed

[~csun] it looks like this didn't get marked as closed / fix-version updated 
when the PR was merged. I believe this went only into 3.5.0; the original PR 
went into branch-3.4 but was reverted and the second PR didn't make it to 
branch-3.4. I've marked the fix version as 3.5.0 but please correct me if I'm 
wrong here:
{code:java}
> glog apache/branch-3.4 | grep SPARK-42539
* 26009d47c1f 2023-02-28 Revert "[SPARK-42539][SQL][HIVE] Eliminate separate 
classloader when using 'builtin' Hive version for metadata client" [Hyukjin 
Kwon ]
* 40a4019dfc5 2023-02-27 [SPARK-42539][SQL][HIVE] Eliminate separate 
classloader when using 'builtin' Hive version for metadata client [Erik Krogen 
]


> glog apache/master | grep SPARK-42539
* 2e34427d4f3 2023-03-01 [SPARK-42539][SQL][HIVE] Eliminate separate 
classloader when using 'builtin' Hive version for metadata client [Erik Krogen 
]
* 5627ceeddb4 2023-02-28 Revert "[SPARK-42539][SQL][HIVE] Eliminate separate 
classloader when using 'builtin' Hive version for metadata client" [Hyukjin 
Kwon ]
* 27ad5830f9a 2023-02-27 [SPARK-42539][SQL][HIVE] Eliminate separate 
classloader when using 'builtin' Hive version for metadata client [Erik Krogen 
] {code}

> User-provided JARs can override Spark's Hive metadata client JARs when using 
> "builtin"
> --
>
> Key: SPARK-42539
> URL: https://issues.apache.org/jira/browse/SPARK-42539
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.1.3, 3.2.3, 3.3.2
>Reporter: Erik Krogen
>Priority: Major
> Fix For: 3.5.0
>
>
> Recently we observed that on version 3.2.0 and Java 8, it is possible for 
> user-provided Hive JARs to break the ability for Spark, via the Hive metadata 
> client / {{IsolatedClientLoader}}, to communicate with Hive Metastore, when 
> using the default behavior of the "builtin" Hive version. After SPARK-35321, 
> when Spark is compiled against Hive >= 2.3.9 and the "builtin" Hive client 
> version is used, we will call the method {{Hive.getWithoutRegisterFns()}} 
> (from HIVE-21563) instead of {{Hive.get()}}. If the user has included, for 
> example, {{hive-exec-2.3.8.jar}} on their classpath, the client will break 
> with a {{NoSuchMethodError}}. This particular failure mode was resolved in 
> 3.2.1 by SPARK-37446, but while investigating, we found a general issue that 
> it's possible for user JARs to override Spark's own JARs -- but only inside 
> of the IsolatedClientLoader when using "builtin". This happens because even 
> when Spark is configured to use the "builtin" Hive classes, it still creates 
> a separate URLClassLoader for the HiveClientImpl used for HMS communication. 
> To get the set of JAR URLs to use for this classloader, Spark [collects all 
> of the JARs used by the user classloader (and its parent, and that 
> classloader's parent, and so 
> on)|https://github.com/apache/spark/blob/87e3d5625e76bb734b8dd753bfb25002822c8585/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala#L412-L438].
>  Thus the newly created classloader will have all of the same JARs as the 
> user classloader, but the ordering has been reversed! User JARs get 
> prioritized ahead of system JARs, because the classloader hierarchy is 
> traversed from bottom-to-top. For example let's say we have user JARs 
> "foo.jar" and "hive-exec-2.3.8.jar". The user classloader will look like this:
> {code}
> MutableURLClassLoader
> -- foo.jar
> -- hive-exec-2.3.8.jar
> -- parent: URLClassLoader
> - spark-core_2.12-3.2.0.jar
> - ...
> - hive-exec-2.3.9.jar
> - ...
> {code}
> This setup provides the expected behavior within the user classloader; it 
> will first check the parent, so hive-exec-2.3.9.jar takes precedence, and the 
> MutableURLClassLoader is only checked if the class doesn't exist in the 
> parent. But when a JAR list is constructed for the IsolatedClientLoader, it 
> traverses the URLs from MutableURLClassLoader first, then it's parent, so the 
> final list looks like (in order):
> {code}
> URLClassLoader [IsolatedClientLoader]
> -- foo.jar
> -- hive-exec-2.3.8.jar
> -- spark-core_2.12-3.2.0.jar
> -- ...
> -- hive-exec-2.3.9.jar
> -- ...
> -- parent: boot classloader (JVM classes)
> {code}
> Now when a lookup happens, all of the JARs are within the same 
> URLClassLoader, and the user JARs are in front of the Spark ones, so the user 
> JARs get prioritized. This is the opposite of the expected behavior when 
> using the default user/application classloader in Spark, which has 
> parent-first behavior, prioritizing the Spark/system classes over the user 
> classes. (Note that 

[jira] [Updated] (SPARK-42539) User-provided JARs can override Spark's Hive metadata client JARs when using "builtin"

2023-04-18 Thread Erik Krogen (Jira)


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

Erik Krogen updated SPARK-42539:

Fix Version/s: 3.5.0

> User-provided JARs can override Spark's Hive metadata client JARs when using 
> "builtin"
> --
>
> Key: SPARK-42539
> URL: https://issues.apache.org/jira/browse/SPARK-42539
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.1.3, 3.2.3, 3.3.2
>Reporter: Erik Krogen
>Priority: Major
> Fix For: 3.5.0
>
>
> Recently we observed that on version 3.2.0 and Java 8, it is possible for 
> user-provided Hive JARs to break the ability for Spark, via the Hive metadata 
> client / {{IsolatedClientLoader}}, to communicate with Hive Metastore, when 
> using the default behavior of the "builtin" Hive version. After SPARK-35321, 
> when Spark is compiled against Hive >= 2.3.9 and the "builtin" Hive client 
> version is used, we will call the method {{Hive.getWithoutRegisterFns()}} 
> (from HIVE-21563) instead of {{Hive.get()}}. If the user has included, for 
> example, {{hive-exec-2.3.8.jar}} on their classpath, the client will break 
> with a {{NoSuchMethodError}}. This particular failure mode was resolved in 
> 3.2.1 by SPARK-37446, but while investigating, we found a general issue that 
> it's possible for user JARs to override Spark's own JARs -- but only inside 
> of the IsolatedClientLoader when using "builtin". This happens because even 
> when Spark is configured to use the "builtin" Hive classes, it still creates 
> a separate URLClassLoader for the HiveClientImpl used for HMS communication. 
> To get the set of JAR URLs to use for this classloader, Spark [collects all 
> of the JARs used by the user classloader (and its parent, and that 
> classloader's parent, and so 
> on)|https://github.com/apache/spark/blob/87e3d5625e76bb734b8dd753bfb25002822c8585/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala#L412-L438].
>  Thus the newly created classloader will have all of the same JARs as the 
> user classloader, but the ordering has been reversed! User JARs get 
> prioritized ahead of system JARs, because the classloader hierarchy is 
> traversed from bottom-to-top. For example let's say we have user JARs 
> "foo.jar" and "hive-exec-2.3.8.jar". The user classloader will look like this:
> {code}
> MutableURLClassLoader
> -- foo.jar
> -- hive-exec-2.3.8.jar
> -- parent: URLClassLoader
> - spark-core_2.12-3.2.0.jar
> - ...
> - hive-exec-2.3.9.jar
> - ...
> {code}
> This setup provides the expected behavior within the user classloader; it 
> will first check the parent, so hive-exec-2.3.9.jar takes precedence, and the 
> MutableURLClassLoader is only checked if the class doesn't exist in the 
> parent. But when a JAR list is constructed for the IsolatedClientLoader, it 
> traverses the URLs from MutableURLClassLoader first, then it's parent, so the 
> final list looks like (in order):
> {code}
> URLClassLoader [IsolatedClientLoader]
> -- foo.jar
> -- hive-exec-2.3.8.jar
> -- spark-core_2.12-3.2.0.jar
> -- ...
> -- hive-exec-2.3.9.jar
> -- ...
> -- parent: boot classloader (JVM classes)
> {code}
> Now when a lookup happens, all of the JARs are within the same 
> URLClassLoader, and the user JARs are in front of the Spark ones, so the user 
> JARs get prioritized. This is the opposite of the expected behavior when 
> using the default user/application classloader in Spark, which has 
> parent-first behavior, prioritizing the Spark/system classes over the user 
> classes. (Note that this behavior is correct when using the 
> {{ChildFirstURLClassLoader}}.)
> After SPARK-37446, the NoSuchMethodError is no longer an issue, but this 
> still breaks assumptions about how user JARs should be treated vs. system 
> JARs, and presents the ability for the client to break in other ways. For 
> example in SPARK-37446 it describes a scenario whereby Hive 2.3.8 JARs have 
> been included; the changes in Hive 2.3.9 were needed to improve compatibility 
> with older HMS, so if a user were to accidentally include these older JARs, 
> it could break the ability of Spark to communicate with HMS 1.x
> I see two solutions to this:
> *(A) Remove the separate classloader entirely when using "builtin"*
> Starting from 3.0.0, due to SPARK-26839, when using Java 9+, we don't even 
> create a new classloader when using "builtin". This makes sense, as [called 
> out in this 
> comment|https://github.com/apache/spark/pull/24057#discussion_r265142878], 
> since the point of "builtin" is to use the existing JARs on the classpath 
> anyway. This proposes simply extending the changes from SPARK-26839 to all 
> Java versions, instead of restricting to Java 9+ only.
> *(B) Reverse the ordering of 

[jira] [Updated] (SPARK-42539) User-provided JARs can override Spark's Hive metadata client JARs when using "builtin"

2023-02-24 Thread Erik Krogen (Jira)


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

Erik Krogen updated SPARK-42539:

Description: 
Recently we observed that on version 3.2.0 and Java 8, it is possible for 
user-provided Hive JARs to break the ability for Spark, via the Hive metadata 
client / {{IsolatedClientLoader}}, to communicate with Hive Metastore, when 
using the default behavior of the "builtin" Hive version. After SPARK-35321, 
when Spark is compiled against Hive >= 2.3.9 and the "builtin" Hive client 
version is used, we will call the method {{Hive.getWithoutRegisterFns()}} (from 
HIVE-21563) instead of {{Hive.get()}}. If the user has included, for example, 
{{hive-exec-2.3.8.jar}} on their classpath, the client will break with a 
{{NoSuchMethodError}}. This particular failure mode was resolved in 3.2.1 by 
SPARK-37446, but while investigating, we found a general issue that it's 
possible for user JARs to override Spark's own JARs -- but only inside of the 
IsolatedClientLoader when using "builtin". This happens because even when Spark 
is configured to use the "builtin" Hive classes, it still creates a separate 
URLClassLoader for the HiveClientImpl used for HMS communication. To get the 
set of JAR URLs to use for this classloader, Spark [collects all of the JARs 
used by the user classloader (and its parent, and that classloader's parent, 
and so 
on)|https://github.com/apache/spark/blob/87e3d5625e76bb734b8dd753bfb25002822c8585/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala#L412-L438].
 Thus the newly created classloader will have all of the same JARs as the user 
classloader, but the ordering has been reversed! User JARs get prioritized 
ahead of system JARs, because the classloader hierarchy is traversed from 
bottom-to-top. For example let's say we have user JARs "foo.jar" and 
"hive-exec-2.3.8.jar". The user classloader will look like this:
{code}
MutableURLClassLoader
-- foo.jar
-- hive-exec-2.3.8.jar
-- parent: URLClassLoader
- spark-core_2.12-3.2.0.jar
- ...
- hive-exec-2.3.9.jar
- ...
{code}

This setup provides the expected behavior within the user classloader; it will 
first check the parent, so hive-exec-2.3.9.jar takes precedence, and the 
MutableURLClassLoader is only checked if the class doesn't exist in the parent. 
But when a JAR list is constructed for the IsolatedClientLoader, it traverses 
the URLs from MutableURLClassLoader first, then it's parent, so the final list 
looks like (in order):
{code}
URLClassLoader [IsolatedClientLoader]
-- foo.jar
-- hive-exec-2.3.8.jar
-- spark-core_2.12-3.2.0.jar
-- ...
-- hive-exec-2.3.9.jar
-- ...
-- parent: boot classloader (JVM classes)
{code}
Now when a lookup happens, all of the JARs are within the same URLClassLoader, 
and the user JARs are in front of the Spark ones, so the user JARs get 
prioritized. This is the opposite of the expected behavior when using the 
default user/application classloader in Spark, which has parent-first behavior, 
prioritizing the Spark/system classes over the user classes. (Note that this 
behavior is correct when using the {{ChildFirstURLClassLoader}}.)

After SPARK-37446, the NoSuchMethodError is no longer an issue, but this still 
breaks assumptions about how user JARs should be treated vs. system JARs, and 
presents the ability for the client to break in other ways. For example in 
SPARK-37446 it describes a scenario whereby Hive 2.3.8 JARs have been included; 
the changes in Hive 2.3.9 were needed to improve compatibility with older HMS, 
so if a user were to accidentally include these older JARs, it could break the 
ability of Spark to communicate with HMS 1.x

I see two solutions to this:

*(A) Remove the separate classloader entirely when using "builtin"*
Starting from 3.0.0, due to SPARK-26839, when using Java 9+, we don't even 
create a new classloader when using "builtin". This makes sense, as [called out 
in this 
comment|https://github.com/apache/spark/pull/24057#discussion_r265142878], 
since the point of "builtin" is to use the existing JARs on the classpath 
anyway. This proposes simply extending the changes from SPARK-26839 to all Java 
versions, instead of restricting to Java 9+ only.

*(B) Reverse the ordering of parent/child JARs when constructing the URL list*
The most targeted fix that can be made is to simply reverse the ordering on 
[this line in 
HiveUtils|https://github.com/apache/spark/blob/87e3d5625e76bb734b8dd753bfb25002822c8585/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala#L419],
 which prioritizes child-classloader JARs over parent-classloader JARs, 
reversing the expected ordering. There is already special handling for 
{{ChildFirstURLClassLoader}}, so all that needs to be done is to reverse this 
order.

I prefer (A) because I think it is a clean solution in that it both simplifies 
the classloader setup, and reduces divergence / special 

[jira] [Updated] (SPARK-42539) User-provided JARs can override Spark's Hive metadata client JARs when using "builtin"

2023-02-23 Thread Erik Krogen (Jira)


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

Erik Krogen updated SPARK-42539:

Description: 
Recently we observed that on version 3.2.0 and Java 8, it is possible for 
user-provided Hive JARs to break the ability for Spark, via the Hive metadata 
client / {{IsolatedClientLoader}}, to communicate with Hive Metastore, when 
using the default behavior of the "builtin" Hive version. After SPARK-35321, 
when Spark is compiled against Hive >= 2.3.9 and the "builtin" Hive client 
version is used, we will call the method {{Hive.getWithoutRegisterFns()}} (from 
HIVE-21563) instead of {{Hive.get()}}. If the user has included, for example, 
{{hive-exec-2.3.8.jar}} on their classpath, the client will break with a 
{{NoSuchMethodError}}. This particular failure mode was resolved in 3.2.1 by 
SPARK-37446, but while investigating, we found a general issue that it's 
possible for user JARs to override Spark's own JARs -- but only inside of the 
IsolatedClientLoader when using "builtin". This happens because even when Spark 
is configured to use the "builtin" Hive classes, it still creates a separate 
URLClassLoader for the HiveClientImpl used for HMS communication. To get the 
set of JAR URLs to use for this classloader, Spark [collects all of the JARs 
used by the user classloader (and its parent, and that classloader's parent, 
and so 
on)](https://github.com/apache/spark/blob/87e3d5625e76bb734b8dd753bfb25002822c8585/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala#L412-L438).
 Thus the newly created classloader will have all of the same JARs as the user 
classloader, but the ordering has been reversed! User JARs get prioritized 
ahead of system JARs, because the classloader hierarchy is traversed from 
bottom-to-top. For example let's say we have user JARs "foo.jar" and 
"hive-exec-2.3.8.jar". The user classloader will look like this:
{code}
MutableURLClassLoader
-- foo.jar
-- hive-exec-2.3.8.jar
-- parent: URLClassLoader
- spark-core_2.12-3.2.0.jar
- ...
- hive-exec-2.3.9.jar
- ...
{code}

This setup provides the expected behavior within the user classloader; it will 
first check the parent, so hive-exec-2.3.9.jar takes precedence, and the 
MutableURLClassLoader is only checked if the class doesn't exist in the parent. 
But when a JAR list is constructed for the IsolatedClientLoader, it traverses 
the URLs from MutableURLClassLoader first, then it's parent, so the final list 
looks like (in order):
{code}
URLClassLoader [IsolatedClientLoader]
-- foo.jar
-- hive-exec-2.3.8.jar
-- spark-core_2.12-3.2.0.jar
-- ...
-- hive-exec-2.3.9.jar
-- ...
-- parent: boot classloader (JVM classes)
{code}
Now when a lookup happens, all of the JARs are within the same URLClassLoader, 
and the user JARs are in front of the Spark ones, so the user JARs get 
prioritized. This is the opposite of the expected behavior when using the 
default user/application classloader in Spark, which has parent-first behavior, 
prioritizing the Spark/system classes over the user classes. (Note that this 
behavior is correct when using the {{ChildFirstURLClassLoader}}.)

After SPARK-37446, the NoSuchMethodError is no longer an issue, but this still 
breaks assumptions about how user JARs should be treated vs. system JARs, and 
presents the ability for the client to break in other ways. For example in 
SPARK-37446 it describes a scenario whereby Hive 2.3.8 JARs have been included; 
the changes in Hive 2.3.9 were needed to improve compatibility with older HMS, 
so if a user were to accidentally include these older JARs, it could break the 
ability of Spark to communicate with HMS 1.x

I see two solutions to this:

*(A) Remove the separate classloader entirely when using "builtin"*
Starting from 3.0.0, due to SPARK-26839, when using Java 9+, we don't even 
create a new classloader when using "builtin". This makes sense, as [called out 
in this 
comment|https://github.com/apache/spark/pull/24057#discussion_r265142878], 
since the point of "builtin" is to use the existing JARs on the classpath 
anyway. This proposes simply extending the changes from SPARK-26839 to all Java 
versions, instead of restricting to Java 9+ only.

*(B) Reverse the ordering of parent/child JARs when constructing the URL list*
The most targeted fix that can be made is to simply reverse the ordering on 
[this line in 
HiveUtils|https://github.com/apache/spark/blob/87e3d5625e76bb734b8dd753bfb25002822c8585/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala#L419],
 which prioritizes child-classloader JARs over parent-classloader JARs, 
reversing the expected ordering. There is already special handling for 
{{ChildFirstURLClassLoader}}, so all that needs to be done is to reverse this 
order.

I prefer (A) because I think it is a clean solution in that it both simplifies 
the classloader setup, and reduces divergence / 

[jira] [Created] (SPARK-42539) User-provided JARs can override Spark's Hive metadata client JARs when using "builtin"

2023-02-23 Thread Erik Krogen (Jira)
Erik Krogen created SPARK-42539:
---

 Summary: User-provided JARs can override Spark's Hive metadata 
client JARs when using "builtin"
 Key: SPARK-42539
 URL: https://issues.apache.org/jira/browse/SPARK-42539
 Project: Spark
  Issue Type: Bug
  Components: SQL
Affects Versions: 3.3.2, 3.2.3, 3.1.3
Reporter: Erik Krogen


Recently we observed that on version 3.2.0, it is possible for user-provided 
Hive JARs to break the ability for Spark, via the Hive metadata client / 
{{IsolatedClientLoader}}, to communicate with Hive Metastore, when using the 
default behavior of the "builtin" Hive version. After SPARK-35321, when Spark 
is compiled against Hive >= 2.3.9 and the "builtin" Hive client version is 
used, we will call the method {{Hive.getWithoutRegisterFns()}} (from 
HIVE-21563) instead of {{Hive.get()}}. If the user has included, for example, 
{{hive-exec-2.3.8.jar}} on their classpath, the client will break with a 
{{NoSuchMethodError}}. This particular failure mode was resolved in 3.2.1 by 
SPARK-37446, but while investigating, we found a general issue that it's 
possible for user JARs to override Spark's own JARs -- but only inside of the 
IsolatedClientLoader when using "builtin". This happens because even when Spark 
is configured to use the "builtin" Hive classes, it still creates a separate 
URLClassLoader for the HiveClientImpl used for HMS communication. To get the 
set of JAR URLs to use for this classloader, Spark [collects all of the JARs 
used by the user classloader (and its parent, and that classloader's parent, 
and so 
on)](https://github.com/apache/spark/blob/87e3d5625e76bb734b8dd753bfb25002822c8585/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala#L412-L438).
 Thus the newly created classloader will have all of the same JARs as the user 
classloader, but the ordering has been reversed! User JARs get prioritized 
ahead of system JARs, because the classloader hierarchy is traversed from 
bottom-to-top. For example let's say we have user JARs "foo.jar" and 
"hive-exec-2.3.8.jar". The user classloader will look like this:
{code}
MutableURLClassLoader
-- foo.jar
-- hive-exec-2.3.8.jar
-- parent: URLClassLoader
- spark-core_2.12-3.2.0.jar
- ...
- hive-exec-2.3.9.jar
- ...
{code}

This setup provides the expected behavior within the user classloader; it will 
first check the parent, so hive-exec-2.3.9.jar takes precedence, and the 
MutableURLClassLoader is only checked if the class doesn't exist in the parent. 
But when a JAR list is constructed for the IsolatedClientLoader, it traverses 
the URLs from MutableURLClassLoader first, then it's parent, so the final list 
looks like (in order):
{code}
URLClassLoader [IsolatedClientLoader]
-- foo.jar
-- hive-exec-2.3.8.jar
-- spark-core_2.12-3.2.0.jar
-- ...
-- hive-exec-2.3.9.jar
-- ...
-- parent: boot classloader (JVM classes)
{code}
Now when a lookup happens, all of the JARs are within the same URLClassLoader, 
and the user JARs are in front of the Spark ones, so the user JARs get 
prioritized. This is the opposite of the expected behavior when using the 
default user/application classloader in Spark, which has parent-first behavior, 
prioritizing the Spark/system classes over the user classes. (Note that this 
behavior is correct when using the {{ChildFirstURLClassLoader}}.)

After SPARK-37446, the NoSuchMethodError is no longer an issue, but this still 
breaks assumptions about how user JARs should be treated vs. system JARs, and 
presents the ability for the client to break in other ways. For example in 
SPARK-37446 it describes a scenario whereby Hive 2.3.8 JARs have been included; 
the changes in Hive 2.3.9 were needed to improve compatibility with older HMS, 
so if a user were to accidentally include these older JARs, it could break the 
ability of Spark to communicate with HMS 1.x

I see two solutions to this:

*(A) Remove the separate classloader entirely when using "builtin"*
Starting from 3.0.0, due to SPARK-26839, when using Java 9+, we don't even 
create a new classloader when using "builtin". This makes sense, as [called out 
in this 
comment|https://github.com/apache/spark/pull/24057#discussion_r265142878], 
since the point of "builtin" is to use the existing JARs on the classpath 
anyway. This proposes simply extending the changes from SPARK-26839 to all Java 
versions, instead of restricting to Java 9+ only.

*(B) Reverse the ordering of parent/child JARs when constructing the URL list*
The most targeted fix that can be made is to simply reverse the ordering on 
[this line in 
HiveUtils|https://github.com/apache/spark/blob/87e3d5625e76bb734b8dd753bfb25002822c8585/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala#L419],
 which prioritizes child-classloader JARs over parent-classloader JARs, 
reversing the expected ordering. There is already special handling for 

[jira] [Commented] (SPARK-39375) SPIP: Spark Connect - A client and server interface for Apache Spark

2023-02-07 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-39375:
-

UDFs are a complex space, e.g. for Scala the current impl completed in 
SPARK-42283 cannot handle externally defined classes, which are a common 
requirement in UDFs. It's also a notable design decision that we are choosing 
to process UDFs in the Spark Connect server session, vs. a sidecar process like 
a UDF server that can provide isolation between different UDFs (e.g. as 
[supported by Presto|https://github.com/prestodb/presto/issues/14053] and 
[leveraged heavily by 
Meta|https://www.databricks.com/session_na21/portable-udfs-write-once-run-anywhere]).
 It would be nice to see more discussion on the merits of various approaches to 
UDFs in the Spark Connect framework and a clear plan, rather than pushing 
forward with them piecemeal. It's of course reasonable that UDFs were left out 
of scope for the original SPIP, but based on that omission I was expecting we 
would have a subsequent discussion on UDFs for Spark Connect before starting 
implementation for them.

> SPIP: Spark Connect - A client and server interface for Apache Spark
> 
>
> Key: SPARK-39375
> URL: https://issues.apache.org/jira/browse/SPARK-39375
> Project: Spark
>  Issue Type: Epic
>  Components: Connect
>Affects Versions: 3.4.0
>Reporter: Martin Grund
>Assignee: Martin Grund
>Priority: Critical
>  Labels: SPIP
>
> Please find the full document for discussion here: [Spark Connect 
> SPIP|https://docs.google.com/document/d/1Mnl6jmGszixLW4KcJU5j9IgpG9-UabS0dcM6PM2XGDc/edit#heading=h.wmsrrfealhrj]
>  Below, we have just referenced the introduction.
> h2. What are you trying to do?
> While Spark is used extensively, it was designed nearly a decade ago, which, 
> in the age of serverless computing and ubiquitous programming language use, 
> poses a number of limitations. Most of the limitations stem from the tightly 
> coupled Spark driver architecture and fact that clusters are typically shared 
> across users: (1) {*}Lack of built-in remote connectivity{*}: the Spark 
> driver runs both the client application and scheduler, which results in a 
> heavyweight architecture that requires proximity to the cluster. There is no 
> built-in capability to  remotely connect to a Spark cluster in languages 
> other than SQL and users therefore rely on external solutions such as the 
> inactive project [Apache Livy|https://livy.apache.org/]. (2) {*}Lack of rich 
> developer experience{*}: The current architecture and APIs do not cater for 
> interactive data exploration (as done with Notebooks), or allow for building 
> out rich developer experience common in modern code editors. (3) 
> {*}Stability{*}: with the current shared driver architecture, users causing 
> critical exceptions (e.g. OOM) bring the whole cluster down for all users. 
> (4) {*}Upgradability{*}: the current entangling of platform and client APIs 
> (e.g. first and third-party dependencies in the classpath) does not allow for 
> seamless upgrades between Spark versions (and with that, hinders new feature 
> adoption).
>  
> We propose to overcome these challenges by building on the DataFrame API and 
> the underlying unresolved logical plans. The DataFrame API is widely used and 
> makes it very easy to iteratively express complex logic. We will introduce 
> {_}Spark Connect{_}, a remote option of the DataFrame API that separates the 
> client from the Spark server. With Spark Connect, Spark will become 
> decoupled, allowing for built-in remote connectivity: The decoupled client 
> SDK can be used to run interactive data exploration and connect to the server 
> for DataFrame operations. 
>  
> Spark Connect will benefit Spark developers in different ways: The decoupled 
> architecture will result in improved stability, as clients are separated from 
> the driver. From the Spark Connect client perspective, Spark will be (almost) 
> versionless, and thus enable seamless upgradability, as server APIs can 
> evolve without affecting the client API. The decoupled client-server 
> architecture can be leveraged to build close integrations with local 
> developer tooling. Finally, separating the client process from the Spark 
> server process will improve Spark’s overall security posture by avoiding the 
> tight coupling of the client inside the Spark runtime environment.
>  
> Spark Connect will strengthen Spark’s position as the modern unified engine 
> for large-scale data analytics and expand applicability to use cases and 
> developers we could not reach with the current setup: Spark will become 
> ubiquitously usable as the DataFrame API can be used with (almost) any 
> 

[jira] [Commented] (SPARK-39375) SPIP: Spark Connect - A client and server interface for Apache Spark

2023-02-01 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-39375:
-

I see some work being done on UDFs (SPARK-42246 for PySpark UDFs, SPARK-42283 
for the start of Scala UDFs). In the [design doc for Spark 
Connect|https://docs.google.com/document/d/17X6-P5H2522SnE-gF1BVwyildp_PDX8oXD-4l9vqQmA/edit#]
 UDFs were left as a later problem. Do we have a design/approach documented 
anywhere for UDFs? The design of these is a crucial part of the future/success 
of Spark Connect and it's a bit concerning to me that we're making 
implementation progres in this direction without an agreed-upon design (AFAICT 
-- please let me know if I missed something).

> SPIP: Spark Connect - A client and server interface for Apache Spark
> 
>
> Key: SPARK-39375
> URL: https://issues.apache.org/jira/browse/SPARK-39375
> Project: Spark
>  Issue Type: Epic
>  Components: Connect
>Affects Versions: 3.4.0
>Reporter: Martin Grund
>Assignee: Martin Grund
>Priority: Critical
>  Labels: SPIP
>
> Please find the full document for discussion here: [Spark Connect 
> SPIP|https://docs.google.com/document/d/1Mnl6jmGszixLW4KcJU5j9IgpG9-UabS0dcM6PM2XGDc/edit#heading=h.wmsrrfealhrj]
>  Below, we have just referenced the introduction.
> h2. What are you trying to do?
> While Spark is used extensively, it was designed nearly a decade ago, which, 
> in the age of serverless computing and ubiquitous programming language use, 
> poses a number of limitations. Most of the limitations stem from the tightly 
> coupled Spark driver architecture and fact that clusters are typically shared 
> across users: (1) {*}Lack of built-in remote connectivity{*}: the Spark 
> driver runs both the client application and scheduler, which results in a 
> heavyweight architecture that requires proximity to the cluster. There is no 
> built-in capability to  remotely connect to a Spark cluster in languages 
> other than SQL and users therefore rely on external solutions such as the 
> inactive project [Apache Livy|https://livy.apache.org/]. (2) {*}Lack of rich 
> developer experience{*}: The current architecture and APIs do not cater for 
> interactive data exploration (as done with Notebooks), or allow for building 
> out rich developer experience common in modern code editors. (3) 
> {*}Stability{*}: with the current shared driver architecture, users causing 
> critical exceptions (e.g. OOM) bring the whole cluster down for all users. 
> (4) {*}Upgradability{*}: the current entangling of platform and client APIs 
> (e.g. first and third-party dependencies in the classpath) does not allow for 
> seamless upgrades between Spark versions (and with that, hinders new feature 
> adoption).
>  
> We propose to overcome these challenges by building on the DataFrame API and 
> the underlying unresolved logical plans. The DataFrame API is widely used and 
> makes it very easy to iteratively express complex logic. We will introduce 
> {_}Spark Connect{_}, a remote option of the DataFrame API that separates the 
> client from the Spark server. With Spark Connect, Spark will become 
> decoupled, allowing for built-in remote connectivity: The decoupled client 
> SDK can be used to run interactive data exploration and connect to the server 
> for DataFrame operations. 
>  
> Spark Connect will benefit Spark developers in different ways: The decoupled 
> architecture will result in improved stability, as clients are separated from 
> the driver. From the Spark Connect client perspective, Spark will be (almost) 
> versionless, and thus enable seamless upgradability, as server APIs can 
> evolve without affecting the client API. The decoupled client-server 
> architecture can be leveraged to build close integrations with local 
> developer tooling. Finally, separating the client process from the Spark 
> server process will improve Spark’s overall security posture by avoiding the 
> tight coupling of the client inside the Spark runtime environment.
>  
> Spark Connect will strengthen Spark’s position as the modern unified engine 
> for large-scale data analytics and expand applicability to use cases and 
> developers we could not reach with the current setup: Spark will become 
> ubiquitously usable as the DataFrame API can be used with (almost) any 
> programming language.



--
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-41049) Nondeterministic expressions have unstable values if they are children of CodegenFallback expressions

2023-01-11 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-41049:
-

Thanks for the input [~viirya]! [~cloud_fan], will you work on putting together 
a PR for the other branches? I would guess that it's not a trivial cherry-pick 
given the size of the change.

> Nondeterministic expressions have unstable values if they are children of 
> CodegenFallback expressions
> -
>
> Key: SPARK-41049
> URL: https://issues.apache.org/jira/browse/SPARK-41049
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.1.2
>Reporter: Guy Boo
>Assignee: Wenchen Fan
>Priority: Major
>  Labels: correctness
> Fix For: 3.4.0
>
>
> h2. Expectation
> For a given row, Nondeterministic expressions are expected to have stable 
> values.
> {code:scala}
> import org.apache.spark.sql.functions._
> val df = sparkContext.parallelize(1 to 5).toDF("x")
> val v1 = rand().*(lit(1)).cast(IntegerType)
> df.select(v1, v1).collect{code}
> Returns a set like this:
> |8777|8777|
> |1357|1357|
> |3435|3435|
> |9204|9204|
> |3870|3870|
> where both columns always have the same value, but what that value is changes 
> from row to row. This is different from the following:
> {code:scala}
> df.select(rand(), rand()).collect{code}
> In this case, because the rand() calls are distinct, the values in both 
> columns should be different.
> h2. Problem
> This expectation does not appear to be stable in the event that any 
> subsequent expression is a CodegenFallback. This program:
> {code:scala}
> import org.apache.spark.sql._
> import org.apache.spark.sql.types._
> import org.apache.spark.sql.functions._
> val sparkSession = SparkSession.builder().getOrCreate()
> val df = sparkSession.sparkContext.parallelize(1 to 5).toDF("x")
> val v1 = rand().*(lit(1)).cast(IntegerType)
> val v2 = to_csv(struct(v1.as("a"))) // to_csv is CodegenFallback
> df.select(v1, v1, v2, v2).collect {code}
> produces output like this:
> |8159|8159|8159|{color:#ff}2028{color}|
> |8320|8320|8320|{color:#ff}1640{color}|
> |7937|7937|7937|{color:#ff}769{color}|
> |436|436|436|{color:#ff}8924{color}|
> |8924|8924|2827|{color:#ff}2731{color}|
> Not sure why the first call via the CodegenFallback path should be correct 
> while subsequent calls aren't.
> h2. Workaround
> If the Nondeterministic expression is moved to a separate, earlier select() 
> call, so the CodegenFallback instead only refers to a column reference, then 
> the problem seems to go away. But this workaround may not be reliable if 
> optimization is ever able to restructure adjacent select()s.



--
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-40588) Sorting issue with partitioned-writing and AQE turned on

2023-01-10 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-40588:
-

Labeling with 'correctness' since this breaks correctness of output by breaking 
the sort ordering.

> Sorting issue with partitioned-writing and AQE turned on
> 
>
> Key: SPARK-40588
> URL: https://issues.apache.org/jira/browse/SPARK-40588
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 3.1.3
> Environment: Spark v3.1.3
> Scala v2.12.13
>Reporter: Swetha Baskaran
>Assignee: Enrico Minack
>Priority: Major
>  Labels: correctness
> Fix For: 3.2.3, 3.3.2
>
> Attachments: image-2022-10-16-22-05-47-159.png
>
>
> We are attempting to partition data by a few columns, sort by a particular 
> _sortCol_ and write out one file per partition. 
> {code:java}
> df
>     .repartition(col("day"), col("month"), col("year"))
>     .withColumn("partitionId",spark_partition_id)
>     .withColumn("monotonicallyIncreasingIdUnsorted",monotonicallyIncreasingId)
>     .sortWithinPartitions("year", "month", "day", "sortCol")
>     .withColumn("monotonicallyIncreasingIdSorted",monotonicallyIncreasingId)
>     .write
>     .partitionBy("year", "month", "day")
>     .parquet(path){code}
> When inspecting the results, we observe one file per partition, however we 
> see an _alternating_ pattern of unsorted rows in some files.
> {code:java}
> {"sortCol":10,"partitionId":732,"monotonicallyIncreasingIdUnsorted":6287832121344,"monotonicallyIncreasingIdSorted":6287832121344}
> {"sortCol":1303413,"partitionId":732,"monotonicallyIncreasingIdUnsorted":6287877022389,"monotonicallyIncreasingIdSorted":6287876860586}
> {"sortCol":10,"partitionId":732,"monotonicallyIncreasingIdUnsorted":6287877567881,"monotonicallyIncreasingIdSorted":6287832121345}
> {"sortCol":1303413,"partitionId":732,"monotonicallyIncreasingIdUnsorted":6287835105553,"monotonicallyIncreasingIdSorted":6287876860587}
> {"sortCol":10,"partitionId":732,"monotonicallyIncreasingIdUnsorted":6287832570127,"monotonicallyIncreasingIdSorted":6287832121346}
> {"sortCol":1303413,"partitionId":732,"monotonicallyIncreasingIdUnsorted":6287879965760,"monotonicallyIncreasingIdSorted":6287876860588}
> {"sortCol":10,"partitionId":732,"monotonicallyIncreasingIdUnsorted":6287878762347,"monotonicallyIncreasingIdSorted":6287832121347}
> {"sortCol":1303413,"partitionId":732,"monotonicallyIncreasingIdUnsorted":6287837165012,"monotonicallyIncreasingIdSorted":6287876860589}
> {"sortCol":10,"partitionId":732,"monotonicallyIncreasingIdUnsorted":6287832910545,"monotonicallyIncreasingIdSorted":6287832121348}
> {"sortCol":1303413,"partitionId":732,"monotonicallyIncreasingIdUnsorted":6287881244758,"monotonicallyIncreasingIdSorted":6287876860590}
> {"sortCol":10,"partitionId":732,"monotonicallyIncreasingIdUnsorted":6287880041345,"monotonicallyIncreasingIdSorted":6287832121349}{code}
> Here is a 
> [gist|https://gist.github.com/Swebask/543030748a768be92d3c0ae343d2ae89] to 
> reproduce the issue. 
> Turning off AQE with spark.conf.set("spark.sql.adaptive.enabled", false) 
> fixes the issue.
> I'm working on identifying why AQE affects the sort order. Any leads or 
> thoughts would be appreciated!



--
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-40588) Sorting issue with partitioned-writing and AQE turned on

2023-01-10 Thread Erik Krogen (Jira)


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

Erik Krogen updated SPARK-40588:

Labels: correctness  (was: )

> Sorting issue with partitioned-writing and AQE turned on
> 
>
> Key: SPARK-40588
> URL: https://issues.apache.org/jira/browse/SPARK-40588
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 3.1.3
> Environment: Spark v3.1.3
> Scala v2.12.13
>Reporter: Swetha Baskaran
>Assignee: Enrico Minack
>Priority: Major
>  Labels: correctness
> Fix For: 3.2.3, 3.3.2
>
> Attachments: image-2022-10-16-22-05-47-159.png
>
>
> We are attempting to partition data by a few columns, sort by a particular 
> _sortCol_ and write out one file per partition. 
> {code:java}
> df
>     .repartition(col("day"), col("month"), col("year"))
>     .withColumn("partitionId",spark_partition_id)
>     .withColumn("monotonicallyIncreasingIdUnsorted",monotonicallyIncreasingId)
>     .sortWithinPartitions("year", "month", "day", "sortCol")
>     .withColumn("monotonicallyIncreasingIdSorted",monotonicallyIncreasingId)
>     .write
>     .partitionBy("year", "month", "day")
>     .parquet(path){code}
> When inspecting the results, we observe one file per partition, however we 
> see an _alternating_ pattern of unsorted rows in some files.
> {code:java}
> {"sortCol":10,"partitionId":732,"monotonicallyIncreasingIdUnsorted":6287832121344,"monotonicallyIncreasingIdSorted":6287832121344}
> {"sortCol":1303413,"partitionId":732,"monotonicallyIncreasingIdUnsorted":6287877022389,"monotonicallyIncreasingIdSorted":6287876860586}
> {"sortCol":10,"partitionId":732,"monotonicallyIncreasingIdUnsorted":6287877567881,"monotonicallyIncreasingIdSorted":6287832121345}
> {"sortCol":1303413,"partitionId":732,"monotonicallyIncreasingIdUnsorted":6287835105553,"monotonicallyIncreasingIdSorted":6287876860587}
> {"sortCol":10,"partitionId":732,"monotonicallyIncreasingIdUnsorted":6287832570127,"monotonicallyIncreasingIdSorted":6287832121346}
> {"sortCol":1303413,"partitionId":732,"monotonicallyIncreasingIdUnsorted":6287879965760,"monotonicallyIncreasingIdSorted":6287876860588}
> {"sortCol":10,"partitionId":732,"monotonicallyIncreasingIdUnsorted":6287878762347,"monotonicallyIncreasingIdSorted":6287832121347}
> {"sortCol":1303413,"partitionId":732,"monotonicallyIncreasingIdUnsorted":6287837165012,"monotonicallyIncreasingIdSorted":6287876860589}
> {"sortCol":10,"partitionId":732,"monotonicallyIncreasingIdUnsorted":6287832910545,"monotonicallyIncreasingIdSorted":6287832121348}
> {"sortCol":1303413,"partitionId":732,"monotonicallyIncreasingIdUnsorted":6287881244758,"monotonicallyIncreasingIdSorted":6287876860590}
> {"sortCol":10,"partitionId":732,"monotonicallyIncreasingIdUnsorted":6287880041345,"monotonicallyIncreasingIdSorted":6287832121349}{code}
> Here is a 
> [gist|https://gist.github.com/Swebask/543030748a768be92d3c0ae343d2ae89] to 
> reproduce the issue. 
> Turning off AQE with spark.conf.set("spark.sql.adaptive.enabled", false) 
> fixes the issue.
> I'm working on identifying why AQE affects the sort order. Any leads or 
> thoughts would be appreciated!



--
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-41049) Nondeterministic expressions have unstable values if they are children of CodegenFallback expressions

2023-01-06 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-41049:
-

Thanks! [~cloud_fan]  [~viirya]  shall we backport this to branch-3.3 and 
branch-3.2, given it is a correctness bug?

> Nondeterministic expressions have unstable values if they are children of 
> CodegenFallback expressions
> -
>
> Key: SPARK-41049
> URL: https://issues.apache.org/jira/browse/SPARK-41049
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.1.2
>Reporter: Guy Boo
>Assignee: Wenchen Fan
>Priority: Major
>  Labels: correctness
> Fix For: 3.4.0
>
>
> h2. Expectation
> For a given row, Nondeterministic expressions are expected to have stable 
> values.
> {code:scala}
> import org.apache.spark.sql.functions._
> val df = sparkContext.parallelize(1 to 5).toDF("x")
> val v1 = rand().*(lit(1)).cast(IntegerType)
> df.select(v1, v1).collect{code}
> Returns a set like this:
> |8777|8777|
> |1357|1357|
> |3435|3435|
> |9204|9204|
> |3870|3870|
> where both columns always have the same value, but what that value is changes 
> from row to row. This is different from the following:
> {code:scala}
> df.select(rand(), rand()).collect{code}
> In this case, because the rand() calls are distinct, the values in both 
> columns should be different.
> h2. Problem
> This expectation does not appear to be stable in the event that any 
> subsequent expression is a CodegenFallback. This program:
> {code:scala}
> import org.apache.spark.sql._
> import org.apache.spark.sql.types._
> import org.apache.spark.sql.functions._
> val sparkSession = SparkSession.builder().getOrCreate()
> val df = sparkSession.sparkContext.parallelize(1 to 5).toDF("x")
> val v1 = rand().*(lit(1)).cast(IntegerType)
> val v2 = to_csv(struct(v1.as("a"))) // to_csv is CodegenFallback
> df.select(v1, v1, v2, v2).collect {code}
> produces output like this:
> |8159|8159|8159|{color:#ff}2028{color}|
> |8320|8320|8320|{color:#ff}1640{color}|
> |7937|7937|7937|{color:#ff}769{color}|
> |436|436|436|{color:#ff}8924{color}|
> |8924|8924|2827|{color:#ff}2731{color}|
> Not sure why the first call via the CodegenFallback path should be correct 
> while subsequent calls aren't.
> h2. Workaround
> If the Nondeterministic expression is moved to a separate, earlier select() 
> call, so the CodegenFallback instead only refers to a column reference, then 
> the problem seems to go away. But this workaround may not be reliable if 
> optimization is ever able to restructure adjacent select()s.



--
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-41049) Nondeterministic expressions have unstable values if they are children of CodegenFallback expressions

2023-01-03 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-41049:
-

[~cloud_fan]  [~gboo]  I think this should be tagged as a 'correctness' issue, 
do you agree?

> Nondeterministic expressions have unstable values if they are children of 
> CodegenFallback expressions
> -
>
> Key: SPARK-41049
> URL: https://issues.apache.org/jira/browse/SPARK-41049
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.1.2
>Reporter: Guy Boo
>Assignee: Wenchen Fan
>Priority: Major
> Fix For: 3.4.0
>
>
> h2. Expectation
> For a given row, Nondeterministic expressions are expected to have stable 
> values.
> {code:scala}
> import org.apache.spark.sql.functions._
> val df = sparkContext.parallelize(1 to 5).toDF("x")
> val v1 = rand().*(lit(1)).cast(IntegerType)
> df.select(v1, v1).collect{code}
> Returns a set like this:
> |8777|8777|
> |1357|1357|
> |3435|3435|
> |9204|9204|
> |3870|3870|
> where both columns always have the same value, but what that value is changes 
> from row to row. This is different from the following:
> {code:scala}
> df.select(rand(), rand()).collect{code}
> In this case, because the rand() calls are distinct, the values in both 
> columns should be different.
> h2. Problem
> This expectation does not appear to be stable in the event that any 
> subsequent expression is a CodegenFallback. This program:
> {code:scala}
> import org.apache.spark.sql._
> import org.apache.spark.sql.types._
> import org.apache.spark.sql.functions._
> val sparkSession = SparkSession.builder().getOrCreate()
> val df = sparkSession.sparkContext.parallelize(1 to 5).toDF("x")
> val v1 = rand().*(lit(1)).cast(IntegerType)
> val v2 = to_csv(struct(v1.as("a"))) // to_csv is CodegenFallback
> df.select(v1, v1, v2, v2).collect {code}
> produces output like this:
> |8159|8159|8159|{color:#ff}2028{color}|
> |8320|8320|8320|{color:#ff}1640{color}|
> |7937|7937|7937|{color:#ff}769{color}|
> |436|436|436|{color:#ff}8924{color}|
> |8924|8924|2827|{color:#ff}2731{color}|
> Not sure why the first call via the CodegenFallback path should be correct 
> while subsequent calls aren't.
> h2. Workaround
> If the Nondeterministic expression is moved to a separate, earlier select() 
> call, so the CodegenFallback instead only refers to a column reference, then 
> the problem seems to go away. But this workaround may not be reliable if 
> optimization is ever able to restructure adjacent select()s.



--
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-41589) PyTorch Distributor

2022-12-19 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-41589:
-

Nope :(

> PyTorch Distributor
> ---
>
> Key: SPARK-41589
> URL: https://issues.apache.org/jira/browse/SPARK-41589
> Project: Spark
>  Issue Type: Umbrella
>  Components: ML, PySpark
>Affects Versions: 3.4.0
>Reporter: Rithwik Ediga Lakhamsani
>Priority: Major
>
> This is a project to make it easier for PySpark users to distribute PyTorch 
> code using PySpark. The corresponding [Design 
> Document|https://docs.google.com/document/d/1_nhUP46cHnYmnZoyirySXvuY1KDMU3vdHRx9MngSVtA/edit]
>  and 
> [PRD|https://docs.google.com/document/d/1KprHkzx9r3lv47TLgO6FnkYZT92xOx6OeKvTJPxqpfk/edit]
>  can give more context. This was a project determined by the Databricks ML 
> Training Team; please reach out to [~gurwls223] (Spark-side proxy) or 
> [~erithwik] for more context.



--
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-41589) PyTorch Distributor

2022-12-19 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-41589:
-

[~erithwik]  can you make the linked documents world-viewable? I get access 
denied.

> PyTorch Distributor
> ---
>
> Key: SPARK-41589
> URL: https://issues.apache.org/jira/browse/SPARK-41589
> Project: Spark
>  Issue Type: Umbrella
>  Components: ML
>Affects Versions: 3.4.0
>Reporter: Rithwik Ediga Lakhamsani
>Priority: Major
>
> This is a project to make it easier for PySpark users to distribute PyTorch 
> code using PySpark. The corresponding [Design 
> Document|https://docs.google.com/document/d/1_nhUP46cHnYmnZoyirySXvuY1KDMU3vdHRx9MngSVtA/edit]
>  and 
> [PRD|https://docs.google.com/document/d/1KprHkzx9r3lv47TLgO6FnkYZT92xOx6OeKvTJPxqpfk/edit]
>  can give more context. 



--
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-41313) AM shutdown hook fails with IllegalStateException if AM crashes on startup (recurrence of SPARK-3900)

2022-12-02 Thread Erik Krogen (Jira)


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

Erik Krogen updated SPARK-41313:

Summary: AM shutdown hook fails with IllegalStateException if AM crashes on 
startup (recurrence of SPARK-3900)  (was: Combine fixes for SPARK-3900 and 
SPARK-21138)

> AM shutdown hook fails with IllegalStateException if AM crashes on startup 
> (recurrence of SPARK-3900)
> -
>
> Key: SPARK-41313
> URL: https://issues.apache.org/jira/browse/SPARK-41313
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, YARN
>Affects Versions: 3.4.0
>Reporter: Xing Lin
>Priority: Minor
>
> SPARK-3900 fixed the {{IllegalStateException}} in cleanupStagingDir in 
> ApplicationMaster's shutdownhook. However, SPARK-21138 accidentally 
> reverted/undid that change when fixing the "Wrong FS" bug. Now, we are seeing 
> SPARK-3900 reported by our users at Linkedin. We need to bring back the fix 
> for SPARK-3900.
> The illegalStateException when creating a new filesystem object is due to the 
> limitation in Hadoop that we can not register a shutdownhook during shutdown. 
> So, when a spark job fails during pre-launch, as part of shutdown, 
> cleanupStagingDir would be called. Then, if we attempt to create a new 
> filesystem object for the first time, HDFS would try to register a hook to 
> shutdown KeyProviderCache when creating a ClientContext for DFSClient. As a 
> result, we hit the {{IllegalStateException}}. We should avoid the creation of 
> a new filesystem object in cleanupStagingDir() when it is called in a 
> shutdown hook. This was introduced in SPARK-3900. However, SPARK-21138 
> accidentally reverted/undid that change. We need to bring back that fix to 
> Spark to avoid the {{IllegalStateException}}.
>   



--
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-41313) Combine fixes for SPARK-3900 and SPARK-21138

2022-12-02 Thread Erik Krogen (Jira)


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

Erik Krogen updated SPARK-41313:

Description: 
SPARK-3900 fixed the {{IllegalStateException}} in cleanupStagingDir in 
ApplicationMaster's shutdownhook. However, SPARK-21138 accidentally 
reverted/undid that change when fixing the "Wrong FS" bug. Now, we are seeing 
SPARK-3900 reported by our users at Linkedin. We need to bring back the fix for 
SPARK-3900.

The illegalStateException when creating a new filesystem object is due to the 
limitation in Hadoop that we can not register a shutdownhook during shutdown. 
So, when a spark job fails during pre-launch, as part of shutdown, 
cleanupStagingDir would be called. Then, if we attempt to create a new 
filesystem object for the first time, HDFS would try to register a hook to 
shutdown KeyProviderCache when creating a ClientContext for DFSClient. As a 
result, we hit the {{IllegalStateException}}. We should avoid the creation of a 
new filesystem object in cleanupStagingDir() when it is called in a shutdown 
hook. This was introduced in SPARK-3900. However, SPARK-21138 accidentally 
reverted/undid that change. We need to bring back that fix to Spark to avoid 
the {{IllegalStateException}}.

  

  was:
spark-3900 fixed the illegalStateException in cleanupStagingDir in 
ApplicationMaster's shutdownhook. However, spark-21138 accidentally 
reverted/undid that change when fixing the "Wrong FS" bug. Now, we are seeing 
spark-3900 reported by our users at Linkedin. We need to bring back the fix for 
spark-3900.

The illegalStateException when creating a new filesystem object is due to the 
limitation in hadoop that we can not register a shutdownhook during shutdown. 
So, when a spark job fails during pre-launch, as part of shutdown, 
cleanupStagingDir would be called. Then, if we attempt to create a new 
filesystem object for the first time, hadoop would try to register a hook to 
shutdown KeyProviderCache when creating a ClientContext for DFSClient. As a 
result, we hit the illegalStateException. We should avoid the creation of a new 
filesystem object in cleanupStagingDir() when it is called in a shutdown hook. 
This was introduced in spark-3900. However, spark-21138 accidentally 
reverted/undid that change. We need to bring back that fix to Spark to avoid 
the illegalStateException.

  


> Combine fixes for SPARK-3900 and SPARK-21138
> 
>
> Key: SPARK-41313
> URL: https://issues.apache.org/jira/browse/SPARK-41313
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, YARN
>Affects Versions: 3.4.0
>Reporter: Xing Lin
>Priority: Minor
>
> SPARK-3900 fixed the {{IllegalStateException}} in cleanupStagingDir in 
> ApplicationMaster's shutdownhook. However, SPARK-21138 accidentally 
> reverted/undid that change when fixing the "Wrong FS" bug. Now, we are seeing 
> SPARK-3900 reported by our users at Linkedin. We need to bring back the fix 
> for SPARK-3900.
> The illegalStateException when creating a new filesystem object is due to the 
> limitation in Hadoop that we can not register a shutdownhook during shutdown. 
> So, when a spark job fails during pre-launch, as part of shutdown, 
> cleanupStagingDir would be called. Then, if we attempt to create a new 
> filesystem object for the first time, HDFS would try to register a hook to 
> shutdown KeyProviderCache when creating a ClientContext for DFSClient. As a 
> result, we hit the {{IllegalStateException}}. We should avoid the creation of 
> a new filesystem object in cleanupStagingDir() when it is called in a 
> shutdown hook. This was introduced in SPARK-3900. However, SPARK-21138 
> accidentally reverted/undid that change. We need to bring back that fix to 
> Spark to avoid the {{IllegalStateException}}.
>   



--
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-41241) Use Hive and Spark SQL to modify table field comment, the modified results of Hive cannot be queried using Spark SQL

2022-11-28 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-41241:
-

I think, as a general rule, it's not safe to modify Spark datasource tables 
using Hive. Spark stores a bunch of extra (Spark-specific) metadata in the 
properties of the table, and Hive will not modify this metadata since it is not 
aware of it. So I would consider this behavior to be expected?

> Use Hive and Spark SQL to modify table field comment, the modified results of 
> Hive cannot be queried using Spark SQL
> 
>
> Key: SPARK-41241
> URL: https://issues.apache.org/jira/browse/SPARK-41241
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.0.0, 3.1.0, 3.2.0, 3.3.0
>Reporter: weiliang hao
>Priority: Major
>
> ---HIVE---
> > create table table_test(id int);
> > alter table table_test change column id id int comment "hive comment";
> > desc formatted table_test;
> {code:java}
> +---+++
> |           col_name            |                     data_type               
>        |                      comment                       |
> +---+++
> | # col_name                    | data_type                                   
>        | comment                                            |
> | id                            | int                                         
>        | hive comment                                        |
> |                               | NULL                                        
>        | NULL                                               |
> | # Detailed Table Information  | NULL                                        
>        | NULL                                               |
> | Database:                     | default                                     
>        | NULL                                               |
> | OwnerType:                    | USER                                        
>        | NULL                                               |
> | Owner:                        | anonymous                                   
>        | NULL                                               |
> | CreateTime:                   | Wed Nov 23 23:06:41 CST 2022                
>        | NULL                                               |
> | LastAccessTime:               | UNKNOWN                                     
>        | NULL                                               |
> | Retention:                    | 0                                           
>        | NULL                                               |
> | Location:                     | 
> hdfs://localhost:8020/warehouse/tablespace/managed/hive/table_test | NULL     
>                                           |
> | Table Type:                   | MANAGED_TABLE                               
>        | NULL                                               |
> | Table Parameters:             | NULL                                        
>        | NULL                                               |
> |                               | COLUMN_STATS_ACCURATE                       
>        | {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"id\":\"true\"}} |
> |                               | bucketing_version                           
>        | 2                                                  |
> |                               | last_modified_by                            
>        | anonymous                                          |
> |                               | last_modified_time                          
>        | 1669216665                                         |
> |                               | numFiles                                    
>        | 0                                                  |
> |                               | numRows                                     
>        | 0                                                  |
> |                               | rawDataSize                                 
>        | 0                                                  |
> |                               | totalSize                                   
>        | 0                                                  |
> |                               | transactional                               
>        | true                                               |
> |                               | transactional_properties                    
>        | 

[jira] [Updated] (SPARK-41226) Refactor Spark types by introducing physical types

2022-11-22 Thread Erik Krogen (Jira)


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

Erik Krogen updated SPARK-41226:

Description: 
I am creating this one for Desmond Cheong since he can't signup for an account 
because of 
[https://infra.apache.org/blog/jira-public-signup-disabled.html.|https://infra.apache.org/blog/jira-public-signup-disabled.html]
 
His description for this improvement:
The Spark type system currently supports multiple data types with the same 
physical representation in memory. For example {{DateType}} and 
{{YearMonthIntervalType}} are both implemented using {{{}IntegerType{}}}. 
Because of this, operations on data types often involve case matching where 
multiple data types match to the same effects.To simplify this case matching 
logic, we can introduce the notion of logical and physical data types where 
multiple logical data types can be implemented with the same physical data 
type, then perform case matching on physical data types.Some areas that can 
utilize this logical/physical type separation are:
 * {{SpecializedGettersReader}} in {{SpecializedGettersReader.java}}
 * {{copy}} in {{ColumnarBatchRow.java}} and {{ColumnarRow.java}}
 * {{getAccessor}} in {{InternalRow.scala}}
 * {{externalDataTypeFor}} in {{RowEncoder.scala}}
 * {{unsafeWriter}} in {{InterpretedUnsafeProjection.scala}}
 * {{getValue}} and {{javaType}} in {{CodeGenerator.scala}}
 * {{doValidate}}  in {{literals.scala}}

  was:
I am creating this one for Desmond Cheong since he can't signup for an account 
because of 
[https://infra.apache.org/blog/jira-public-signup-disabled.html.|https://infra.apache.org/blog/jira-public-signup-disabled.html]
 
His description for this improvement:
The Spark type system currently supports multiple data types with the same 
physical representation in memory. For example {{DateType}} and 
{{YearMonthIntervalType}} are both implemented using {{{}IntegerType{}}}. 
Because of this, operations on data types often involve case matching where 
multiple data types match to the same effects.To simplify this case matching 
logic, we can introduce the notion of logical and physical data types where 
multiple logical data types can be implemented with the same physical data 
type, then perform case matching on physical data types.Some areas that can 
utilize this logical/physical type separation are: * 
{{SpecializedGettersReader}} in {{SpecializedGettersReader.java}}
 * {{copy}} in {{ColumnarBatchRow.java}} and {{ColumnarRow.java}}
 * {{getAccessor}} in {{InternalRow.scala}}
 * {{externalDataTypeFor}} in {{RowEncoder.scala}}
 * {{unsafeWriter}} in {{InterpretedUnsafeProjection.scala}}
 * {{getValue}} and {{javaType}} in {{CodeGenerator.scala}}
 * {{doValidate}}  in {{literals.scala}}


> Refactor Spark types by introducing physical types
> --
>
> Key: SPARK-41226
> URL: https://issues.apache.org/jira/browse/SPARK-41226
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 3.4.0
>Reporter: Gengliang Wang
>Priority: Major
>
> I am creating this one for Desmond Cheong since he can't signup for an 
> account because of 
> [https://infra.apache.org/blog/jira-public-signup-disabled.html.|https://infra.apache.org/blog/jira-public-signup-disabled.html]
>  
> His description for this improvement:
> The Spark type system currently supports multiple data types with the same 
> physical representation in memory. For example {{DateType}} and 
> {{YearMonthIntervalType}} are both implemented using {{{}IntegerType{}}}. 
> Because of this, operations on data types often involve case matching where 
> multiple data types match to the same effects.To simplify this case matching 
> logic, we can introduce the notion of logical and physical data types where 
> multiple logical data types can be implemented with the same physical data 
> type, then perform case matching on physical data types.Some areas that can 
> utilize this logical/physical type separation are:
>  * {{SpecializedGettersReader}} in {{SpecializedGettersReader.java}}
>  * {{copy}} in {{ColumnarBatchRow.java}} and {{ColumnarRow.java}}
>  * {{getAccessor}} in {{InternalRow.scala}}
>  * {{externalDataTypeFor}} in {{RowEncoder.scala}}
>  * {{unsafeWriter}} in {{InterpretedUnsafeProjection.scala}}
>  * {{getValue}} and {{javaType}} in {{CodeGenerator.scala}}
>  * {{doValidate}}  in {{literals.scala}}



--
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-41073) Spark ThriftServer generate huge amounts of DelegationToken

2022-11-14 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-41073:
-

I'm not very familiar with this area, but it seems you are trying to solve the 
same problem that is already discussed in SPARK-36328

> Spark ThriftServer generate huge amounts of DelegationToken
> ---
>
> Key: SPARK-41073
> URL: https://issues.apache.org/jira/browse/SPARK-41073
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.0.1
>Reporter: zhengchenyu
>Priority: Major
> Attachments: SPARK-41073.proposal.A.draft.001.patch
>
>
> In our cluster, zookeeper nearly crashed. I found the znodes of 
> /zkdtsm/ZKDTSMRoot/ZKDTSMTokensRoot increased quickly. 
> After some research, I found some sql running on spark-thriftserver obtain 
> huge amounts of DelegationToken.
> The reason is that in these spark-sql, every hive parition acquire a 
> different delegation token. 
> And HadoopRDDs in thriftserver can't share credentials from 
> CoarseGrainedSchedulerBackend::delegationTokens, we must share 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] [Updated] (SPARK-40916) udf could not filter null value cause npe

2022-10-31 Thread Erik Krogen (Jira)


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

Erik Krogen updated SPARK-40916:

Description: 
{code:sql}
select
t22.uid,
from
(
SELECT
code,
count(distinct uid) cnt
FROM
(
SELECT
uid,
code,
lng,
lat
FROM
(
select
 
riskmanage_dw.GEOHASH_ENCODE(manhattan_dw.aes_decode(lng),manhattan_dw.aes_decode(lat),8)
 as code,
uid,
lng,
lat,
dt as event_time 
from
(
select
param['timestamp'] as dt,

get_json_object(get_json_object(param['input'],'$.baseInfo'),'$.uid') uid,

get_json_object(get_json_object(param['input'],'$.envInfo'),'$.lng') lng,

get_json_object(get_json_object(param['input'],'$.envInfo'),'$.lat') lat 
from manhattan_ods.ods_log_manhattan_fbi_workflow_result_log
and 
get_json_object(get_json_object(param['input'],'$.bizExtents'),'$.productId')='2001'
 
)a
and lng is not null
and lat is not null
) t2
group by uid,code,lng,lat
) t1
GROUP BY code having count(DISTINCT uid)>=10
)t11
join
(
SELECT
uid,
code,
lng,
lat
FROM
(
select

riskmanage_dw.GEOHASH_ENCODE(manhattan_dw.aes_decode(lng),manhattan_dw.aes_decode(lat),8)
 as code,
uid,
lng,
lat,
dt as event_time
from
(
select
param['timestamp'] as dt,

get_json_object(get_json_object(param['input'],'$.baseInfo'),'$.uid') uid,

get_json_object(get_json_object(param['input'],'$.envInfo'),'$.lng') lng, 

get_json_object(get_json_object(param['input'],'$.envInfo'),'$.lat') lat 
from manhattan_ods.ods_log_manhattan_fbi_workflow_result_log 
and 
get_json_object(get_json_object(param['input'],'$.bizExtents'),'$.productId')='2001'
 
)a
and lng is not null
and lat is not null
) t2
where substr(code,0,6)<>'wx4ey3'
group by uid,code,lng,lat
) t22 on t11.code=t22.code
group by t22.uid
{code}
this sql can't run because 
`riskmanage_dw.GEOHASH_ENCODE(manhattan_dw.aes_decode(lng),manhattan_dw.aes_decode(lat),8)`
 will throw npe(`Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: 
Unable to execute method public java.lang.String 
com.xiaoju.automarket.GeohashEncode.evaluate(java.lang.Double,java.lang.Double,java.lang.Integer)
 with arguments {null,null,8}:null`), but I have filter null in my condition, 
the udf of manhattan_dw.aes_decode will return null if lng or lat is null, *but 
after I remove `where substr(code,0,6)<>'wx4ey3' `this condition, it can run 
normally.* 


complete :
Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Unable to execute 
method public java.lang.String 
com.xiaoju.automarket.GeohashEncode.evaluate(java.lang.Double,java.lang.Double,java.lang.Integer)
 with arguments {null,null,8}:null
at 
org.apache.hadoop.hive.ql.exec.FunctionRegistry.invoke(FunctionRegistry.java:1049)
at org.apache.spark.sql.hive.HiveSimpleUDF.eval(hiveUDFs.scala:102)
at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificPredicate.subExpr_3$(Unknown
 Source)
at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificPredicate.eval(Unknown
 Source)
at 
org.apache.spark.sql.execution.FilterExec.$anonfun$doExecute$3(basicPhysicalOperators.scala:275)
at 
org.apache.spark.sql.execution.FilterExec.$anonfun$doExecute$3$adapted(basicPhysicalOperators.scala:274)
at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:515)
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.agg_doAggregateWithKeys_0$(Unknown
 Source)
at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown
 Source)

  was:
```
select
t22.uid,
from
(
SELECT
code,
count(distinct uid) cnt
FROM
(
SELECT
uid,
code,
lng,
lat
FROM
(
select
 
riskmanage_dw.GEOHASH_ENCODE(manhattan_dw.aes_decode(lng),manhattan_dw.aes_decode(lat),8)
 as code,
uid,
lng,
lat,
dt as event_time 
from
(
select
param['timestamp'] as dt,

get_json_object(get_json_object(param['input'],'$.baseInfo'),'$.uid') uid,
 

[jira] [Commented] (SPARK-40939) Release a shaded version of Apache Spark / shade jars on main jar

2022-10-28 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-40939:
-

As a reference for prior work, there is also HADOOP-11656, in which Hadoop 
began publishing a new {{hadoop-client-runtime}} JAR into which all of the 
transitive dependencies are shaded. In the 
[proposal|https://issues.apache.org/jira/secure/attachment/12709266/HADOOP-11656_proposal.md]
 a technique similar to Flink's was proposed and eventually rejected due to 
higher maintenance burden to publish separate artifacts for each downstream 
library that is shaded.

There are some pitfalls that come with Spark being a Scala project, unlike 
Hadoop/Flink which are Java based. Most shading tools cannot handle certain 
Scala language elements, specifically {{ScalaSig}} causes problems because 
shading tools that are not Scala-aware do not perform relocations within the 
{{ScalaSig}} (see examples 
[one|https://github.com/coursier/coursier/issues/454#issuecomment-288969207] 
and [two|https://lists.apache.org/thread/x7b4z0os9zbzzprb5scft7b4wnr7c3mv] and 
[this previous Spark PR that tried to shade 
Jackson|https://github.com/apache/spark/pull/10931]). That being said, 
{{sbt}}'s [assembly plugin has had support for this since 
2020|https://github.com/sbt/sbt-assembly/pull/393], and this functionality was 
subsequently pulled out into a standalone library, [Jar Jar 
Abrams|http://eed3si9n.com/jarjar-abrams/]. So there is hope that this should 
be more achievable now than it was back in 2016 when that PR was filed. There's 
also been [interest in shading all of Spark's dependencies on the Spark 
dev-list|https://lists.apache.org/thread/vkkx8s2zv0ln7j7oo46k30x084mn163p].

I would love to hear what the community thinks of pursuing this earnestly with 
the tools available in 2022, though [~almogtavor] I'll note that this type of 
large change is better discussed on the dev mailing list (and probably an 
accompanying SPIP).

> Release a shaded version of Apache Spark / shade jars on main jar
> -
>
> Key: SPARK-40939
> URL: https://issues.apache.org/jira/browse/SPARK-40939
> Project: Spark
>  Issue Type: Improvement
>  Components: Deploy
>Affects Versions: 3.4.0
>Reporter: Almog Tavor
>Priority: Major
>
> I suggest shading in Apache Spark, to resolve the dependency hell that may 
> occur when building / deploying Apache Spark. This mainly occurs on Java 
> projects and on Hadoop environments, but shading will help for using Spark 
> with Scala & even Python either.
> Flink has a similar solution, delivering 
> [flink-shaded|https://github.com/apache/flink-shaded/blob/master/README.md].
> The dependencies I think that are relevant for shading are Jackson, Guava, 
> Netty & any of the Hadoop ecosystems if possible.
> As for releasing sources for the shaded version, I think the [issue that has 
> been raised in Flink|https://github.com/apache/flink-shaded/issues/25] is 
> relevant and unanswered here too, hence I don't think that's an option 
> currently (personally I don't see any value for it either).



--
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-40407) Repartition of DataFrame can result in severe data skew in some special case

2022-09-12 Thread Erik Krogen (Jira)


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

Erik Krogen updated SPARK-40407:

Description: 
{code:scala}
_val df = spark.range(0, 100, 1, 50).repartition(4)_
_val v = df.rdd.mapPartitions { iter => {_
        _Iterator.single(iter.length)_
{_}}{_}{_}.collect(){_}
_println(v.mkString(","))_
{code}

The above simple code outputs `50,0,0,50`, which means there is no data in 
partition 1 and partition 2.

I just debugged it and found the RoundRobin seems to ensure to distribute the 
records evenly **in the same partition**, and not guarantee it between 
partitions.

Below is the code to generate the key

{code:scala}
      case RoundRobinPartitioning(numPartitions) =>
        // Distributes elements evenly across output partitions, starting from 
a random partition.
        var position = new 
Random(TaskContext.get().partitionId()).nextInt(numPartitions)  
        (row: InternalRow) => {
          // The HashPartitioner will handle the `mod` by the number of 
partitions
          position += 1
          position
        }
{code}

In this case, There are 50 partitions, each partition will only compute 2 
elements. The issue for RoundRobin here is it always starts with *position=2* 
to do the Roundrobin.

See the output of Random
{code:scala}
scala> (1 to 200).foreach(partitionId => print(new 
Random(partitionId).nextInt(4) + " "))  // the position is always 2.
2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 
2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 
2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 
2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 
2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 
{code}

Similarly, the below Random code also outputs the same value, 

{code:scala}
(1 to 200).foreach(partitionId => print(new Random(partitionId).nextInt(2) + " 
"))

(1 to 200).foreach(partitionId => print(new Random(partitionId).nextInt(4) + " 
"))

(1 to 200).foreach(partitionId => print(new Random(partitionId).nextInt(8) + " 
"))

(1 to 200).foreach(partitionId => print(new Random(partitionId).nextInt(16) + " 
"))

(1 to 200).foreach(partitionId => print(new Random(partitionId).nextInt(32) + " 
"))
{code}

Let's go back to this case,

Consider partition 0, the total elements are [0, 1], so when shuffle writes, 
for element 0, the key will be (position + 1) = 2 + 1 = 3%4=3, the element 1, 
the key will be (position + 1)=(3+1)=4%4 = 0
consider partition 1, the total elements are [2, 3], so when shuffle writes, 
for element 2, the key will be (position + 1) = 2 + 1 = 3%4=3, the element 3, 
the key will be (position + 1)=(3+1)=4%4 = 0

 

The calculation is also applied for other left partitions since the starting 
position is always 2 for this case.

So, as you can see, each partition will write its elements to Partition [0, 3], 
which results in Partition [1, 2] without any data.

 

I will try to provide the patch to fix this issue.

  was:
_val df = spark.range(0, 100, 1, 50).repartition(4)_
_val v = df.rdd.mapPartitions { iter => {_
        _Iterator.single(iter.length)_
{_}}{_}{_}.collect(){_}
_println(v.mkString(","))_


The above simple code outputs `50,0,0,50`, which means there is no data in 
partition 1 and partition 2.

 

I just debugged it and found the RoundRobin seems to ensure to distribute the 
records evenly **in the same partition**, and not guarantee it between 
partitions.


Below is the code to generate the key

``` scala
      case RoundRobinPartitioning(numPartitions) =>
        // Distributes elements evenly across output partitions, starting from 
a random partition.
        var position = new 
Random(TaskContext.get().partitionId()).nextInt(numPartitions)  
        (row: InternalRow) => {
          // The HashPartitioner will handle the `mod` by the number of 
partitions
          position += 1
          position
        }
```

In this case, There are 50 partitions, each partition will only compute 2 
elements. The issue for RoundRobin here is it always starts with *position=2* 
to do the Roundrobin.

See the output of Random
``` scala
scala> (1 to 200).foreach(partitionId => print(new 
Random(partitionId).nextInt(4) + " "))  // the position is always 2.
2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 
2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 
2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 
2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 
2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2 
```

Similarly, the below Random code also outputs the same value, 

 

(1 to 200).foreach(partitionId => print(new Random(partitionId).nextInt(2) + " 
"))


[jira] [Commented] (SPARK-40262) Expensive UDF evaluation pushed down past a join leads to performance issues

2022-08-31 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-40262:
-

Good find and thanks for sharing the investigation [~shardulm]! I agree that a 
similar strategy as SPARK-37392 makes sense here. cc also [~viirya] and 
[~joshrosen] who were involved in SPARK-37392.

> Expensive UDF evaluation pushed down past a join leads to performance issues 
> -
>
> Key: SPARK-40262
> URL: https://issues.apache.org/jira/browse/SPARK-40262
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.4.0
>Reporter: Shardul Mahadik
>Priority: Major
>
> Consider a Spark job with an expensive UDF which looks like follows:
> {code:scala}
> val expensive_udf = spark.udf.register("expensive_udf", (i: Int) => Option(i))
> spark.range(10).write.format("orc").save("/tmp/orc")
> val df = spark.read.format("orc").load("/tmp/orc").as("a")
> .join(spark.range(10).as("b"), "id")
> .withColumn("udf_op", expensive_udf($"a.id"))
> .join(spark.range(10).as("c"), $"udf_op" === $"c.id")
> {code}
> This creates a physical plan as follows:
> {code:java}
> == Physical Plan ==
> AdaptiveSparkPlan isFinalPlan=false
> +- BroadcastHashJoin [cast(udf_op#338 as bigint)], [id#344L], Inner, 
> BuildRight, false
>:- Project [id#330L, if (isnull(cast(id#330L as int))) null else 
> expensive_udf(knownnotnull(cast(id#330L as int))) AS udf_op#338]
>:  +- BroadcastHashJoin [id#330L], [id#332L], Inner, BuildRight, false
>: :- Filter ((isnotnull(id#330L) AND isnotnull(cast(id#330L as int))) 
> AND isnotnull(expensive_udf(knownnotnull(cast(id#330L as int)
>: :  +- FileScan orc [id#330L] Batched: true, DataFilters: 
> [isnotnull(id#330L), isnotnull(cast(id#330L as int)), 
> isnotnull(expensive_udf(knownnotnull(cast(i..., Format: ORC, Location: 
> InMemoryFileIndex(1 paths)[file:/tmp/orc], PartitionFilters: [], 
> PushedFilters: [IsNotNull(id)], ReadSchema: struct
>: +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, 
> bigint, false]),false), [plan_id=416]
>:+- Range (0, 10, step=1, splits=16)
>+- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, 
> false]),false), [plan_id=420]
>   +- Range (0, 10, step=1, splits=16)
> {code}
> In this case, the expensive UDF call is duplicated thrice. Since the UDF 
> output is used in a future join, `InferFiltersFromConstraints` adds an `IS 
> NOT NULL` filter on the UDF output. But the pushdown rules duplicate this UDF 
> call and push the UDF past a previous join. The duplication behaviour [is 
> documented|https://github.com/apache/spark/blob/c95ed826e23fdec6e1a779cfebde7b3364594fb5/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala#L196]
>  and in itself is not a huge issue. But given a highly restrictive join, the 
> UDF gets evaluated on many orders of magnitude more rows than it should have 
> slowing down the job.
> Can we avoid this duplication of UDF calls? In SPARK-37392, we made a 
> [similar change|https://github.com/apache/spark/pull/34823/files] where we 
> decided to only add inferred filters if the input is an attribute. Should we 
> use a similar strategy for `InferFiltersFromConstraints`?



--
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-40199) Spark throws NPE without useful message when NULL value appears in non-null schema

2022-08-23 Thread Erik Krogen (Jira)
Erik Krogen created SPARK-40199:
---

 Summary: Spark throws NPE without useful message when NULL value 
appears in non-null schema
 Key: SPARK-40199
 URL: https://issues.apache.org/jira/browse/SPARK-40199
 Project: Spark
  Issue Type: Improvement
  Components: SQL
Affects Versions: 3.2.2
Reporter: Erik Krogen


Currently in some cases, if Spark encounters a NULL value where the schema 
indicates that the column/field should be non-null, it will throw a 
{{NullPointerException}} with no message and thus no way to debug further. This 
can happen, for example, if you have a UDF which is erroneously marked as 
{{asNonNullable()}}, or if you read input data where the actual values don't 
match the schema (which could happen e.g. with Avro if the reader provides a 
schema declaring non-null although the data was written with null values).

As an example of how to reproduce:
{code:scala}
val badUDF = spark.udf.register[String, Int]("bad_udf", in => 
null).asNonNullable()
Seq(1, 2).toDF("c1").select(badUDF($"c1")).collect()
{code}

This throws an exception like:
{code}
Driver stacktrace:
org.apache.spark.SparkException: Job aborted due to stage failure: Task 1 in 
stage 0.0 failed 1 times, most recent failure: Lost task 1.0 in stage 0.0 (TID 
1) (xx executor driver): java.lang.NullPointerException
at 
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeWriter.write(UnsafeWriter.java:110)
at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown
 Source)
at 
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
at 
org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:760)
at 
org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:364)
at 
org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:890)
at 
org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:890)
at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:92)
at org.apache.spark.scheduler.Task.run(Task.scala:139)
at 
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:548)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1490)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:551)
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)
{code}

As a user, it is very confusing -- it looks like there is a bug in Spark. We 
have had many users report such problems, and though we can guide them to a 
schema-data mismatch, there is no indication of what field might contain the 
bad values, so a laborious data exploration process is required to find and 
remedy it.

We should provide a better error message in such cases.



--
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-39802) Support recursive references in Avro schemas in Spark

2022-07-29 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-39802:
-

The Spark type system doesn't support recursive types, so it's not possible to 
support without substantial changes to the type system.

> Support recursive references in Avro schemas in Spark
> -
>
> Key: SPARK-39802
> URL: https://issues.apache.org/jira/browse/SPARK-39802
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 3.4.0
>Reporter: Ivan Sadikov
>Priority: Major
>
> This is a follow-up for https://issues.apache.org/jira/browse/SPARK-25718. 
> It would be great if Spark could support recursive references in Avro schemas 
> as currently this is disabled and the following exception is thrown: 
> {code:java}
> org.apache.spark.sql.avro.IncompatibleSchemaException: Found recursive 
> reference in Avro schema, which can not be processed by Spark: {
>   "type": "record",
>   "name": "Struct",
>   "fields": [
>     {
>       "name": "fields",
>       "type": [
>         "null",
>         {
>           "type": "array",
>           "items": {
>             "type": "record",
>             "name": "fields",
>             "fields": [
>               {
>                 "name": "value",
>                 "type": [
>                   "null",
>                   "string"
>                 ],
>                 "default": null
>               }
>             ]
>           }
>         }
>       ],
>       "default": null
>     }
>   ]
> }{code}
> I would appreciate it if someone could comment on the overall effort to make 
> recursive Avro schemas work in Spark and whether or not there are any API 
> restrictions that could potentially hinder the implementation, thanks.



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

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



[jira] [Updated] (SPARK-25718) Detect recursive reference in Avro schema and throw exception

2022-07-29 Thread Erik Krogen (Jira)


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

Erik Krogen updated SPARK-25718:

Description: 
Avro schema allows recursive reference, e.g. the example schema in 
https://avro.apache.org/docs/1.8.2/spec.html#schema_record
{code}
{
  "type": "record",
  "name": "LongList",
  "aliases": ["LinkedLongs"],  // old name for this
  "fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["null", "LongList"]} // optional next element
  ]
}
{code}

However, it is impossible to convert the schema as `StructType` in current 
Spark SQL. Running `SchemaConverters.toSqlType(avroSchema: Schema)` and we will 
get stack overflow exception.

We should detect the recursive reference and throw exception for it.


  was:
Avro schema allows recursive reference, e.g. the example schema in 
https://avro.apache.org/docs/1.8.2/spec.html#schema_record
```
{
  "type": "record",
  "name": "LongList",
  "aliases": ["LinkedLongs"],  // old name for this
  "fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["null", "LongList"]} // optional next element
  ]
}
```

However, it is impossible to convert the schema as `StructType` in current 
Spark SQL. Running `SchemaConverters.toSqlType(avroSchema: Schema)` and we will 
get stack overflow exception.

We should detect the recursive reference and throw exception for it.



> Detect recursive reference in Avro schema and throw exception
> -
>
> Key: SPARK-25718
> URL: https://issues.apache.org/jira/browse/SPARK-25718
> Project: Spark
>  Issue Type: Sub-task
>  Components: SQL
>Affects Versions: 2.4.0
>Reporter: Gengliang Wang
>Assignee: Gengliang Wang
>Priority: Major
> Fix For: 2.4.0
>
>
> Avro schema allows recursive reference, e.g. the example schema in 
> https://avro.apache.org/docs/1.8.2/spec.html#schema_record
> {code}
> {
>   "type": "record",
>   "name": "LongList",
>   "aliases": ["LinkedLongs"],  // old name for this
>   "fields" : [
> {"name": "value", "type": "long"}, // each element has a long
> {"name": "next", "type": ["null", "LongList"]} // optional next element
>   ]
> }
> {code}
> However, it is impossible to convert the schema as `StructType` in current 
> Spark SQL. Running `SchemaConverters.toSqlType(avroSchema: Schema)` and we 
> will get stack overflow exception.
> We should detect the recursive reference and throw exception for 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] [Commented] (SPARK-39722) Make Dataset.showString() public

2022-07-27 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-39722:
-

General +1 from me. We have some internal code that does exactly the 
{{Console.out}} redirection hack you described.

> Make Dataset.showString() public
> 
>
> Key: SPARK-39722
> URL: https://issues.apache.org/jira/browse/SPARK-39722
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 2.4.8, 3.3.0
>Reporter: Jatin Sharma
>Priority: Trivial
>
> Currently, we have {{.show}} APIs on a Dataset, but they print directly to 
> stdout.
> But there are a lot of cases where we might need to get a String 
> representation of the show output. For example
>  * We have a logging framework to which we need to push the representation of 
> a df
>  * We have to send the string over a REST call from the driver
>  * We want to send the string to stderr instead of stdout
> For such cases, currently one needs to do a hack by changing the Console.out 
> temporarily and catching the representation in a ByteArrayOutputStream or 
> similar, then extracting the string from it.
> Strictly only printing to stdout seems like a limiting choice. 
>  
> Solution:
> We expose APIs to return the String representation back. We already have the 
> .{{{}showString{}}} method internally.
>  
> We could mirror the current {{.show}} APIS with a corresponding 
> {{.showString}} (and rename the internal private function to something else 
> if required)



--
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-39500) Ivy doesn't work correctly on IPv6-only environment

2022-06-21 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-39500:
-

Thanks for clarifying!

> Ivy doesn't work correctly on IPv6-only environment
> ---
>
> Key: SPARK-39500
> URL: https://issues.apache.org/jira/browse/SPARK-39500
> Project: Spark
>  Issue Type: Sub-task
>  Components: Spark Core
>Affects Versions: 3.4.0
>Reporter: Dongjoon Hyun
>Priority: Major
>
> Ivy doesn't work correctly on IPv6.
> {code}
>   SparkSubmitUtils.resolveMavenCoordinates(
> "org.apache.logging.log4j:log4j-api:2.17.2",
> SparkSubmitUtils.buildIvySettings(None, Some("/tmp/ivy")),
> transitive = true)
> {code}
> {code}
> % bin/spark-shell
> 22/06/16 22:22:12 WARN Utils: Your hostname, m1ipv6.local resolves to a 
> loopback address: 127.0.0.1; using 2600:1700:232e:3de0:0:0:0:b instead (on 
> interface en0)
> 22/06/16 22:22:12 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to 
> another address
> = https://ipv6.repo1.maven.org/maven2/
> =https://maven-central.storage-download.googleapis.com/maven2/
> Setting default log level to "WARN".
> To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use 
> setLogLevel(newLevel).
> 22/06/16 22:22:14 WARN NativeCodeLoader: Unable to load native-hadoop library 
> for your platform... using builtin-java classes where applicable
> Spark context Web UI available at http://unknown1498776019fa.attlocal.net:4040
> Spark context available as 'sc' (master = local[*], app id = 
> local-1655443334687).
> Spark session available as 'spark'.
> Welcome to
>     __
>  / __/__  ___ _/ /__
> _\ \/ _ \/ _ `/ __/  '_/
>/___/ .__/\_,_/_/ /_/\_\   version 3.4.0-SNAPSHOT
>   /_/
> Using Scala version 2.12.16 (OpenJDK 64-Bit Server VM, Java 17.0.3)
> Type in expressions to have them evaluated.
> Type :help for more information.
> scala> :paste -raw
> // Entering paste mode (ctrl-D to finish)
> package org.apache.spark.deploy
> object Download {
>   SparkSubmitUtils.resolveMavenCoordinates(
> "org.apache.logging.log4j:log4j-api:2.17.2",
> SparkSubmitUtils.buildIvySettings(None, Some("/tmp/ivy")),
> transitive = true)
> }
> // Exiting paste mode, now interpreting.
> scala> org.apache.spark.deploy.Download
> = https://ipv6.repo1.maven.org/maven2/
> =https://maven-central.storage-download.googleapis.com/maven2/
> :: loading settings :: url = 
> jar:file:/Users/dongjoon/APACHE/spark/assembly/target/scala-2.12/jars/ivy-2.5.0.jar!/org/apache/ivy/core/settings/ivysettings.xml
> Ivy Default Cache set to: /tmp/ivy/cache
> The jars for the packages stored in: /tmp/ivy/jars
> org.apache.logging.log4j#log4j-api added as a dependency
> :: resolving dependencies :: 
> org.apache.spark#spark-submit-parent-f47b503f-897e-4b92-95da-3806c32c220f;1.0
> confs: [default]
> :: resolution report :: resolve 95ms :: artifacts dl 0ms
> :: modules in use:
> -
> |  |modules||   artifacts   |
> |   conf   | number| search|dwnlded|evicted|| number|dwnlded|
> -
> |  default |   1   |   0   |   0   |   0   ||   0   |   0   |
> -
> :: problems summary ::
>  WARNINGS
> module not found: org.apache.logging.log4j#log4j-api;2.17.2
>  local-m2-cache: tried
>   
> file:/Users/dongjoon/.m2/repository/org/apache/logging/log4j/log4j-api/2.17.2/log4j-api-2.17.2.pom
>   -- artifact org.apache.logging.log4j#log4j-api;2.17.2!log4j-api.jar:
>   
> file:/Users/dongjoon/.m2/repository/org/apache/logging/log4j/log4j-api/2.17.2/log4j-api-2.17.2.jar
>  local-ivy-cache: tried
>   
> /tmp/ivy/local/org.apache.logging.log4j/log4j-api/2.17.2/ivys/ivy.xml
>   -- artifact org.apache.logging.log4j#log4j-api;2.17.2!log4j-api.jar:
>   
> /tmp/ivy/local/org.apache.logging.log4j/log4j-api/2.17.2/jars/log4j-api.jar
>  ipv6: tried
>   
> https://ipv6.repo1.maven.org/maven2/org/apache/logging/log4j/log4j-api/2.17.2/log4j-api-2.17.2.pom
>   -- artifact org.apache.logging.log4j#log4j-api;2.17.2!log4j-api.jar:
>   
> https://ipv6.repo1.maven.org/maven2/org/apache/logging/log4j/log4j-api/2.17.2/log4j-api-2.17.2.jar
>  central: tried
>   
> https://maven-central.storage-download.googleapis.com/maven2/org/apache/logging/log4j/log4j-api/2.17.2/log4j-api-2.17.2.pom
>   -- artifact 

[jira] [Commented] (SPARK-39500) Ivy doesn't work correctly on IPv6-only environment

2022-06-17 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-39500:
-

I see this is resolved, is there a fix? Or is it not actually an issue?

> Ivy doesn't work correctly on IPv6-only environment
> ---
>
> Key: SPARK-39500
> URL: https://issues.apache.org/jira/browse/SPARK-39500
> Project: Spark
>  Issue Type: Sub-task
>  Components: Spark Core
>Affects Versions: 3.4.0
>Reporter: Dongjoon Hyun
>Priority: Major
>
> Ivy doesn't work correctly on IPv6.
> {code}
>   SparkSubmitUtils.resolveMavenCoordinates(
> "org.apache.logging.log4j:log4j-api:2.17.2",
> SparkSubmitUtils.buildIvySettings(None, Some("/tmp/ivy")),
> transitive = true)
> {code}
> {code}
> % bin/spark-shell
> 22/06/16 22:22:12 WARN Utils: Your hostname, m1ipv6.local resolves to a 
> loopback address: 127.0.0.1; using 2600:1700:232e:3de0:0:0:0:b instead (on 
> interface en0)
> 22/06/16 22:22:12 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to 
> another address
> = https://ipv6.repo1.maven.org/maven2/
> =https://maven-central.storage-download.googleapis.com/maven2/
> Setting default log level to "WARN".
> To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use 
> setLogLevel(newLevel).
> 22/06/16 22:22:14 WARN NativeCodeLoader: Unable to load native-hadoop library 
> for your platform... using builtin-java classes where applicable
> Spark context Web UI available at http://unknown1498776019fa.attlocal.net:4040
> Spark context available as 'sc' (master = local[*], app id = 
> local-1655443334687).
> Spark session available as 'spark'.
> Welcome to
>     __
>  / __/__  ___ _/ /__
> _\ \/ _ \/ _ `/ __/  '_/
>/___/ .__/\_,_/_/ /_/\_\   version 3.4.0-SNAPSHOT
>   /_/
> Using Scala version 2.12.16 (OpenJDK 64-Bit Server VM, Java 17.0.3)
> Type in expressions to have them evaluated.
> Type :help for more information.
> scala> :paste -raw
> // Entering paste mode (ctrl-D to finish)
> package org.apache.spark.deploy
> object Download {
>   SparkSubmitUtils.resolveMavenCoordinates(
> "org.apache.logging.log4j:log4j-api:2.17.2",
> SparkSubmitUtils.buildIvySettings(None, Some("/tmp/ivy")),
> transitive = true)
> }
> // Exiting paste mode, now interpreting.
> scala> org.apache.spark.deploy.Download
> = https://ipv6.repo1.maven.org/maven2/
> =https://maven-central.storage-download.googleapis.com/maven2/
> :: loading settings :: url = 
> jar:file:/Users/dongjoon/APACHE/spark/assembly/target/scala-2.12/jars/ivy-2.5.0.jar!/org/apache/ivy/core/settings/ivysettings.xml
> Ivy Default Cache set to: /tmp/ivy/cache
> The jars for the packages stored in: /tmp/ivy/jars
> org.apache.logging.log4j#log4j-api added as a dependency
> :: resolving dependencies :: 
> org.apache.spark#spark-submit-parent-f47b503f-897e-4b92-95da-3806c32c220f;1.0
> confs: [default]
> :: resolution report :: resolve 95ms :: artifacts dl 0ms
> :: modules in use:
> -
> |  |modules||   artifacts   |
> |   conf   | number| search|dwnlded|evicted|| number|dwnlded|
> -
> |  default |   1   |   0   |   0   |   0   ||   0   |   0   |
> -
> :: problems summary ::
>  WARNINGS
> module not found: org.apache.logging.log4j#log4j-api;2.17.2
>  local-m2-cache: tried
>   
> file:/Users/dongjoon/.m2/repository/org/apache/logging/log4j/log4j-api/2.17.2/log4j-api-2.17.2.pom
>   -- artifact org.apache.logging.log4j#log4j-api;2.17.2!log4j-api.jar:
>   
> file:/Users/dongjoon/.m2/repository/org/apache/logging/log4j/log4j-api/2.17.2/log4j-api-2.17.2.jar
>  local-ivy-cache: tried
>   
> /tmp/ivy/local/org.apache.logging.log4j/log4j-api/2.17.2/ivys/ivy.xml
>   -- artifact org.apache.logging.log4j#log4j-api;2.17.2!log4j-api.jar:
>   
> /tmp/ivy/local/org.apache.logging.log4j/log4j-api/2.17.2/jars/log4j-api.jar
>  ipv6: tried
>   
> https://ipv6.repo1.maven.org/maven2/org/apache/logging/log4j/log4j-api/2.17.2/log4j-api-2.17.2.pom
>   -- artifact org.apache.logging.log4j#log4j-api;2.17.2!log4j-api.jar:
>   
> https://ipv6.repo1.maven.org/maven2/org/apache/logging/log4j/log4j-api/2.17.2/log4j-api-2.17.2.jar
>  central: tried
>   
> https://maven-central.storage-download.googleapis.com/maven2/org/apache/logging/log4j/log4j-api/2.17.2/log4j-api-2.17.2.pom
>   -- 

[jira] [Updated] (SPARK-39334) Change to exclude `slf4j-reload4j` for `hadoop-minikdc`

2022-05-31 Thread Erik Krogen (Jira)


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

Erik Krogen updated SPARK-39334:

Description: 
HADOOP-18088 Replace log4j 1.x with reload4j in Hadoop 3.3.3 and there will be 
waring log when in Spark when test

{code}
Testing started at 10:11 ...
SLF4J: Class path contains multiple SLF4J bindings.
SLF4J: Found binding in 
[jar:file:/Users/yangjie01/.m2/repository/org/apache/logging/log4j/log4j-slf4j-impl/2.17.2/log4j-slf4j-impl-2.17.2.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: Found binding in 
[jar:file:/Users/yangjie01/.m2/repository/org/slf4j/slf4j-reload4j/1.7.36/slf4j-reload4j-1.7.36.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings for an explanation.
SLF4J: Actual binding is of type [org.apache.logging.slf4j.Log4jLoggerFactory]
{code}


  was:
HADOOP-18088 Replace log4j 1.x with reload4j in Hadoop 3.3.3 and there will be 
waring log when in Spark when test

```

Testing started at 10:11 ...
SLF4J: Class path contains multiple SLF4J bindings.
SLF4J: Found binding in 
[jar:file:/Users/yangjie01/.m2/repository/org/apache/logging/log4j/log4j-slf4j-impl/2.17.2/log4j-slf4j-impl-2.17.2.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: Found binding in 
[jar:file:/Users/yangjie01/.m2/repository/org/slf4j/slf4j-reload4j/1.7.36/slf4j-reload4j-1.7.36.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings for an explanation.
SLF4J: Actual binding is of type [org.apache.logging.slf4j.Log4jLoggerFactory]

```


> Change to exclude `slf4j-reload4j` for `hadoop-minikdc`
> ---
>
> Key: SPARK-39334
> URL: https://issues.apache.org/jira/browse/SPARK-39334
> Project: Spark
>  Issue Type: Test
>  Components: Build, Tests
>Affects Versions: 3.4.0
>Reporter: Yang Jie
>Assignee: Yang Jie
>Priority: Minor
> Fix For: 3.3.0
>
>
> HADOOP-18088 Replace log4j 1.x with reload4j in Hadoop 3.3.3 and there will 
> be waring log when in Spark when test
> {code}
> Testing started at 10:11 ...
> SLF4J: Class path contains multiple SLF4J bindings.
> SLF4J: Found binding in 
> [jar:file:/Users/yangjie01/.m2/repository/org/apache/logging/log4j/log4j-slf4j-impl/2.17.2/log4j-slf4j-impl-2.17.2.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> SLF4J: Found binding in 
> [jar:file:/Users/yangjie01/.m2/repository/org/slf4j/slf4j-reload4j/1.7.36/slf4j-reload4j-1.7.36.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings for an 
> explanation.
> SLF4J: Actual binding is of type [org.apache.logging.slf4j.Log4jLoggerFactory]
> {code}



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

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



[jira] [Resolved] (SPARK-39145) CLONE - SPIP: Public APIs for extended Columnar Processing Support

2022-05-12 Thread Erik Krogen (Jira)


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

Erik Krogen resolved SPARK-39145.
-
Resolution: Duplicate

Closing this as a duplicate..

> CLONE - SPIP: Public APIs for extended Columnar Processing Support
> --
>
> Key: SPARK-39145
> URL: https://issues.apache.org/jira/browse/SPARK-39145
> Project: Spark
>  Issue Type: Epic
>  Components: SQL
>Affects Versions: 3.0.0
>Reporter: Abhi Shah
>Assignee: Robert Joseph Evans
>Priority: Major
>
> *strong text**SPIP: Columnar Processing Without Arrow Formatting Guarantees.*
>  
> *Q1.* What are you trying to do? Articulate your objectives using absolutely 
> no jargon.
> The Dataset/DataFrame API in Spark currently only exposes to users one row at 
> a time when processing data.  The goals of this are to
>  # Add to the current sql extensions mechanism so advanced users can have 
> access to the physical SparkPlan and manipulate it to provide columnar 
> processing for existing operators, including shuffle.  This will allow them 
> to implement their own cost based optimizers to decide when processing should 
> be columnar and when it should not.
>  # Make any transitions between the columnar memory layout and a row based 
> layout transparent to the users so operations that are not columnar see the 
> data as rows, and operations that are columnar see the data as columns.
>  
> Not Requirements, but things that would be nice to have.
>  # Transition the existing in memory columnar layouts to be compatible with 
> Apache Arrow.  This would make the transformations to Apache Arrow format a 
> no-op. The existing formats are already very close to those layouts in many 
> cases.  This would not be using the Apache Arrow java library, but instead 
> being compatible with the memory 
> [layout|https://arrow.apache.org/docs/format/Layout.html] and possibly only a 
> subset of that layout.
>  
> *Q2.* What problem is this proposal NOT designed to solve? 
> The goal of this is not for ML/AI but to provide APIs for accelerated 
> computing in Spark primarily targeting SQL/ETL like workloads.  ML/AI already 
> have several mechanisms to get data into/out of them. These can be improved 
> but will be covered in a separate SPIP.
> This is not trying to implement any of the processing itself in a columnar 
> way, with the exception of examples for documentation.
> This does not cover exposing the underlying format of the data.  The only way 
> to get at the data in a ColumnVector is through the public APIs.  Exposing 
> the underlying format to improve efficiency will be covered in a separate 
> SPIP.
> This is not trying to implement new ways of transferring data to external 
> ML/AI applications.  That is covered by separate SPIPs already.
> This is not trying to add in generic code generation for columnar processing. 
>  Currently code generation for columnar processing is only supported when 
> translating columns to rows.  We will continue to support this, but will not 
> extend it as a general solution. That will be covered in a separate SPIP if 
> we find it is helpful.  For now columnar processing will be interpreted.
> This is not trying to expose a way to get columnar data into Spark through 
> DataSource V2 or any other similar API.  That would be covered by a separate 
> SPIP if we find it is needed.
>  
> *Q3.* How is it done today, and what are the limits of current practice?
> The current columnar support is limited to 3 areas.
>  # Internal implementations of FileFormats, optionally can return a 
> ColumnarBatch instead of rows.  The code generation phase knows how to take 
> that columnar data and iterate through it as rows for stages that wants rows, 
> which currently is almost everything.  The limitations here are mostly 
> implementation specific. The current standard is to abuse Scala’s type 
> erasure to return ColumnarBatches as the elements of an RDD[InternalRow]. The 
> code generation can handle this because it is generating java code, so it 
> bypasses scala’s type checking and just casts the InternalRow to the desired 
> ColumnarBatch.  This makes it difficult for others to implement the same 
> functionality for different processing because they can only do it through 
> code generation. There really is no clean separate path in the code 
> generation for columnar vs row based. Additionally, because it is only 
> supported through code generation if for any reason code generation would 
> fail there is no backup.  This is typically fine for input formats but can be 
> problematic when we get into more extensive processing.
>  # When caching data it can optionally be cached in a columnar format if the 
> input is also columnar.  This is similar to the first area and has 

[jira] [Commented] (SPARK-39145) CLONE - SPIP: Public APIs for extended Columnar Processing Support

2022-05-11 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-39145:
-

Looks like this is a duplicate of SPARK-27396, is that intentional? Why does 
this Jira exist?

> CLONE - SPIP: Public APIs for extended Columnar Processing Support
> --
>
> Key: SPARK-39145
> URL: https://issues.apache.org/jira/browse/SPARK-39145
> Project: Spark
>  Issue Type: Epic
>  Components: SQL
>Affects Versions: 3.0.0
>Reporter: Abhi Shah
>Assignee: Robert Joseph Evans
>Priority: Major
>
> *strong text**SPIP: Columnar Processing Without Arrow Formatting Guarantees.*
>  
> *Q1.* What are you trying to do? Articulate your objectives using absolutely 
> no jargon.
> The Dataset/DataFrame API in Spark currently only exposes to users one row at 
> a time when processing data.  The goals of this are to
>  # Add to the current sql extensions mechanism so advanced users can have 
> access to the physical SparkPlan and manipulate it to provide columnar 
> processing for existing operators, including shuffle.  This will allow them 
> to implement their own cost based optimizers to decide when processing should 
> be columnar and when it should not.
>  # Make any transitions between the columnar memory layout and a row based 
> layout transparent to the users so operations that are not columnar see the 
> data as rows, and operations that are columnar see the data as columns.
>  
> Not Requirements, but things that would be nice to have.
>  # Transition the existing in memory columnar layouts to be compatible with 
> Apache Arrow.  This would make the transformations to Apache Arrow format a 
> no-op. The existing formats are already very close to those layouts in many 
> cases.  This would not be using the Apache Arrow java library, but instead 
> being compatible with the memory 
> [layout|https://arrow.apache.org/docs/format/Layout.html] and possibly only a 
> subset of that layout.
>  
> *Q2.* What problem is this proposal NOT designed to solve? 
> The goal of this is not for ML/AI but to provide APIs for accelerated 
> computing in Spark primarily targeting SQL/ETL like workloads.  ML/AI already 
> have several mechanisms to get data into/out of them. These can be improved 
> but will be covered in a separate SPIP.
> This is not trying to implement any of the processing itself in a columnar 
> way, with the exception of examples for documentation.
> This does not cover exposing the underlying format of the data.  The only way 
> to get at the data in a ColumnVector is through the public APIs.  Exposing 
> the underlying format to improve efficiency will be covered in a separate 
> SPIP.
> This is not trying to implement new ways of transferring data to external 
> ML/AI applications.  That is covered by separate SPIPs already.
> This is not trying to add in generic code generation for columnar processing. 
>  Currently code generation for columnar processing is only supported when 
> translating columns to rows.  We will continue to support this, but will not 
> extend it as a general solution. That will be covered in a separate SPIP if 
> we find it is helpful.  For now columnar processing will be interpreted.
> This is not trying to expose a way to get columnar data into Spark through 
> DataSource V2 or any other similar API.  That would be covered by a separate 
> SPIP if we find it is needed.
>  
> *Q3.* How is it done today, and what are the limits of current practice?
> The current columnar support is limited to 3 areas.
>  # Internal implementations of FileFormats, optionally can return a 
> ColumnarBatch instead of rows.  The code generation phase knows how to take 
> that columnar data and iterate through it as rows for stages that wants rows, 
> which currently is almost everything.  The limitations here are mostly 
> implementation specific. The current standard is to abuse Scala’s type 
> erasure to return ColumnarBatches as the elements of an RDD[InternalRow]. The 
> code generation can handle this because it is generating java code, so it 
> bypasses scala’s type checking and just casts the InternalRow to the desired 
> ColumnarBatch.  This makes it difficult for others to implement the same 
> functionality for different processing because they can only do it through 
> code generation. There really is no clean separate path in the code 
> generation for columnar vs row based. Additionally, because it is only 
> supported through code generation if for any reason code generation would 
> fail there is no backup.  This is typically fine for input formats but can be 
> problematic when we get into more extensive processing.
>  # When caching data it can optionally be cached in a columnar format 

[jira] [Commented] (SPARK-39075) IncompatibleSchemaException when selecting data from table stored from a DataFrame in Avro format with BYTE/SHORT

2022-05-03 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-39075:
-

I looked more closely at the code snippet you shared and I now understand that 
the issue is that the data gets written out with an Avro schema of {{INT}}, but 
the Spark schema associated with the table (e.g. {{spark.sources.schema}} 
property) contains a {{ShortType}}, which is then unreadable. I agree that this 
is definitely a bug. The question, then, is which side is wrong: the reader or 
the writer?

In this case I actually think the best thing to do would be to annotate the 
output Avro schema with additional information that, although the type is 
{{INT}}, it actually contains shorts. Avro allows for attaching arbitrary 
properties to schemas, which are used internally by Avro for defining logical 
types. We could do a similar thing, storing a property like "actualType = 
short", which could then be used by the reader to understand that the downcast 
is safe.

> IncompatibleSchemaException when selecting data from table stored from a 
> DataFrame in Avro format with BYTE/SHORT
> -
>
> Key: SPARK-39075
> URL: https://issues.apache.org/jira/browse/SPARK-39075
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.2.1
>Reporter: xsys
>Priority: Major
>
> h3. Describe the bug
> We are trying to save a table constructed through a DataFrame with the 
> {{Avro}} data format. The table contains {{ByteType}} or {{ShortType}} as 
> part of the schema.
> When we {{INSERT}} some valid values (e.g. {{{}-128{}}}) and {{SELECT}} from 
> the table, we expect it to give back the inserted value. However, we instead 
> get an {{IncompatibleSchemaException}} from the {{{}AvroDeserializer{}}}.
> This appears to be caused by a missing case statement handling the {{(INT, 
> ShortType)}} and {{(INT, ByteType)}} cases in [{{AvroDeserializer 
> newWriter}}|https://github.com/apache/spark/blob/4f25b3f71238a00508a356591553f2dfa89f8290/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala#L321].
> h3. To Reproduce
> On Spark 3.2.1 (commit {{{}4f25b3f712{}}}), using {{spark-shell}} with the 
> Avro package:
> {code:java}
> ./bin/spark-shell --packages org.apache.spark:spark-avro_2.12:3.2.1{code}
> Execute the following:
> {code:java}
> import org.apache.spark.sql.Row
> import org.apache.spark.sql.types._
> val schema = new StructType().add(StructField("c1", ShortType, true))
> val rdd = sc.parallelize(Seq(Row("-128".toShort)))
> val df = spark.createDataFrame(rdd, schema)
> df.write.mode("overwrite").format("avro").saveAsTable("t0")
> spark.sql("select * from t0;").show(false){code}
> Resulting error:
> {code:java}
> 22/04/27 18:04:14 ERROR Executor: Exception in task 0.0 in stage 2.0 (TID 32) 
> org.apache.spark.sql.avro.IncompatibleSchemaException: Cannot convert Avro 
> type {"type":"record","name":"topLevelRecord","fields":[
> {"name":"c1","type":["int","null"]}
> ]} to SQL type STRUCT<`c1`: SMALLINT>. 
> at 
> org.apache.spark.sql.avro.AvroDeserializer.liftedTree1$1(AvroDeserializer.scala:102)
>  
> at 
> org.apache.spark.sql.avro.AvroDeserializer.(AvroDeserializer.scala:74) 
> at 
> org.apache.spark.sql.avro.AvroFileFormat$$anon$1.(AvroFileFormat.scala:143)
>  
> at 
> org.apache.spark.sql.avro.AvroFileFormat.$anonfun$buildReader$1(AvroFileFormat.scala:136)
>  
> at 
> org.apache.spark.sql.execution.datasources.FileFormat$$anon$1.apply(FileFormat.scala:148)
>  
> at 
> org.apache.spark.sql.execution.datasources.FileFormat$$anon$1.apply(FileFormat.scala:133)
>  
> at 
> org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.org$apache$spark$sql$execution$datasources$FileScanRDD$$anon$$readCurrentFile(FileScanRDD.scala:127)
>  
> at 
> org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:187)
>  
> at 
> org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:104)
>  
> at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460) 
> at 
> org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown
>  Source) 
> at 
> org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
>  
> at 
> org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:759)
> at 
> org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:349)
>  
> at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:898) 
> at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:898)
>  
> at 

[jira] [Commented] (SPARK-39075) IncompatibleSchemaException when selecting data from table stored from a DataFrame in Avro format with BYTE/SHORT

2022-04-29 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-39075:
-

I'm not sure of the history here, but this might be intentional.

In AvroSerializer, going from a Catalyst {{ShortType}} to an Avro INT is an 
upcast, and therefore safe (a short will always fit inside of an int). But in 
AvroDeserializer, going from an Avro INT to a Catalyst {{ShortType}} is a 
downcast, so it's not a safe conversion.

We can see that similarly, we don't support double-to-float or long-to-int 
(though we also don't support float-to-double or int-to-long, for that matter).

I personally wouldn't have a concern with adding the int-to-short/byte 
conversion as long as overflow detection is handled gracefully.

cc [~cloud_fan] [~Gengliang.Wang]

> IncompatibleSchemaException when selecting data from table stored from a 
> DataFrame in Avro format with BYTE/SHORT
> -
>
> Key: SPARK-39075
> URL: https://issues.apache.org/jira/browse/SPARK-39075
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.2.1
>Reporter: xsys
>Priority: Major
>
> h3. Describe the bug
> We are trying to save a table constructed through a DataFrame with the 
> {{Avro}} data format. The table contains {{ByteType}} or {{ShortType}} as 
> part of the schema.
> When we {{INSERT}} some valid values (e.g. {{{}-128{}}}) and {{SELECT}} from 
> the table, we expect it to give back the inserted value. However, we instead 
> get an {{IncompatibleSchemaException}} from the {{{}AvroDeserializer{}}}.
> This appears to be caused by a missing case statement handling the {{(INT, 
> ShortType)}} and {{(INT, ByteType)}} cases in [{{AvroDeserializer 
> newWriter}}|https://github.com/apache/spark/blob/4f25b3f71238a00508a356591553f2dfa89f8290/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala#L321].
> h3. To Reproduce
> On Spark 3.2.1 (commit {{{}4f25b3f712{}}}), using {{spark-shell}} with the 
> Avro package:
> {code:java}
> ./bin/spark-shell --packages org.apache.spark:spark-avro_2.12:3.2.1{code}
> Execute the following:
> {code:java}
> import org.apache.spark.sql.Row
> import org.apache.spark.sql.types._
> val schema = new StructType().add(StructField("c1", ShortType, true))
> val rdd = sc.parallelize(Seq(Row("-128".toShort)))
> val df = spark.createDataFrame(rdd, schema)
> df.write.mode("overwrite").format("avro").saveAsTable("t0")
> spark.sql("select * from t0;").show(false){code}
> Resulting error:
> {code:java}
> 22/04/27 18:04:14 ERROR Executor: Exception in task 0.0 in stage 2.0 (TID 32) 
> org.apache.spark.sql.avro.IncompatibleSchemaException: Cannot convert Avro 
> type {"type":"record","name":"topLevelRecord","fields":[
> {"name":"c1","type":["int","null"]}
> ]} to SQL type STRUCT<`c1`: SMALLINT>. 
> at 
> org.apache.spark.sql.avro.AvroDeserializer.liftedTree1$1(AvroDeserializer.scala:102)
>  
> at 
> org.apache.spark.sql.avro.AvroDeserializer.(AvroDeserializer.scala:74) 
> at 
> org.apache.spark.sql.avro.AvroFileFormat$$anon$1.(AvroFileFormat.scala:143)
>  
> at 
> org.apache.spark.sql.avro.AvroFileFormat.$anonfun$buildReader$1(AvroFileFormat.scala:136)
>  
> at 
> org.apache.spark.sql.execution.datasources.FileFormat$$anon$1.apply(FileFormat.scala:148)
>  
> at 
> org.apache.spark.sql.execution.datasources.FileFormat$$anon$1.apply(FileFormat.scala:133)
>  
> at 
> org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.org$apache$spark$sql$execution$datasources$FileScanRDD$$anon$$readCurrentFile(FileScanRDD.scala:127)
>  
> at 
> org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:187)
>  
> at 
> org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:104)
>  
> at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460) 
> at 
> org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown
>  Source) 
> at 
> org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
>  
> at 
> org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:759)
> at 
> org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:349)
>  
> at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:898) 
> at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:898)
>  
> at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) 
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373) 
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:337) 
> at 

[jira] [Updated] (SPARK-39075) IncompatibleSchemaException when selecting data from table stored from a DataFrame in Avro format with BYTE/SHORT

2022-04-29 Thread Erik Krogen (Jira)


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

Erik Krogen updated SPARK-39075:

Description: 
h3. Describe the bug

We are trying to save a table constructed through a DataFrame with the {{Avro}} 
data format. The table contains {{ByteType}} or {{ShortType}} as part of the 
schema.

When we {{INSERT}} some valid values (e.g. {{{}-128{}}}) and {{SELECT}} from 
the table, we expect it to give back the inserted value. However, we instead 
get an {{IncompatibleSchemaException}} from the {{{}AvroDeserializer{}}}.

This appears to be caused by a missing case statement handling the {{(INT, 
ShortType)}} and {{(INT, ByteType)}} cases in [{{AvroDeserializer 
newWriter}}|https://github.com/apache/spark/blob/4f25b3f71238a00508a356591553f2dfa89f8290/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala#L321].
h3. To Reproduce

On Spark 3.2.1 (commit {{{}4f25b3f712{}}}), using {{spark-shell}} with the Avro 
package:
{code:java}
./bin/spark-shell --packages org.apache.spark:spark-avro_2.12:3.2.1{code}
Execute the following:
{code:java}
import org.apache.spark.sql.Row
import org.apache.spark.sql.types._
val schema = new StructType().add(StructField("c1", ShortType, true))
val rdd = sc.parallelize(Seq(Row("-128".toShort)))
val df = spark.createDataFrame(rdd, schema)
df.write.mode("overwrite").format("avro").saveAsTable("t0")
spark.sql("select * from t0;").show(false){code}
Resulting error:
{code:java}
22/04/27 18:04:14 ERROR Executor: Exception in task 0.0 in stage 2.0 (TID 32) 
org.apache.spark.sql.avro.IncompatibleSchemaException: Cannot convert Avro type 
{"type":"record","name":"topLevelRecord","fields":[
{"name":"c1","type":["int","null"]}
]} to SQL type STRUCT<`c1`: SMALLINT>. 
at 
org.apache.spark.sql.avro.AvroDeserializer.liftedTree1$1(AvroDeserializer.scala:102)
 
at org.apache.spark.sql.avro.AvroDeserializer.(AvroDeserializer.scala:74) 
at 
org.apache.spark.sql.avro.AvroFileFormat$$anon$1.(AvroFileFormat.scala:143)
 
at 
org.apache.spark.sql.avro.AvroFileFormat.$anonfun$buildReader$1(AvroFileFormat.scala:136)
 
at 
org.apache.spark.sql.execution.datasources.FileFormat$$anon$1.apply(FileFormat.scala:148)
 
at 
org.apache.spark.sql.execution.datasources.FileFormat$$anon$1.apply(FileFormat.scala:133)
 
at 
org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.org$apache$spark$sql$execution$datasources$FileScanRDD$$anon$$readCurrentFile(FileScanRDD.scala:127)
 
at 
org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:187)
 
at 
org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:104)
 
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460) 
at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown
 Source) 
at 
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
 
at 
org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:759)
at 
org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:349)
 
at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:898) 
at 
org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:898)
 
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) 
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373) 
at org.apache.spark.rdd.RDD.iterator(RDD.scala:337) 
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90) 
at org.apache.spark.scheduler.Task.run(Task.scala:131) 
at 
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506)
 
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462) 
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509) 
at 
java.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: org.apache.spark.sql.avro.IncompatibleSchemaException: Cannot 
convert Avro field 'c1' to SQL field 'c1' because schema is incompatible 
(avroType = "int", sqlType = SMALLINT) 
at 
org.apache.spark.sql.avro.AvroDeserializer.newWriter(AvroDeserializer.scala:321)
at 
org.apache.spark.sql.avro.AvroDeserializer.getRecordWriter(AvroDeserializer.scala:356)
 
at 
org.apache.spark.sql.avro.AvroDeserializer.liftedTree1$1(AvroDeserializer.scala:84)
... 26 more
{code}
h3. Expected behavior & Possible Solution

We expect the output to successfully select {{-128}}. We tried other formats 
like Parquet and the outcome is consistent with this expectation.

In the [{{AvroSerializer 

[jira] [Commented] (SPARK-38812) when i clean data ,I hope one rdd spill two rdd according clean data rule

2022-04-13 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-38812:
-

You may want to check the discussion on SPARK-2373 and SPARK-6664

> when i clean data ,I hope one rdd spill two rdd according clean data rule
> -
>
> Key: SPARK-38812
> URL: https://issues.apache.org/jira/browse/SPARK-38812
> Project: Spark
>  Issue Type: New Feature
>  Components: Spark Core
>Affects Versions: 3.2.1
>Reporter: gaokui
>Priority: Major
>
> when id do clean data,one rdd according one value(>or <) filter data, and 
> then generate two different set,one is error data file, another is errorless 
> data file.
> Now I use filter, but this filter must have two spark dag job, that cost too 
> much.
> exactly some code like iterator.span(preidicate) and then return one 
> tuple(iter1,iter2)



--
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-38788) More comprehensive DSV2 push down capabilities

2022-04-13 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-38788:
-

Yeah, I got that, but isn't SPARK-38852 trying to do the same thing? Or are 
they targeting different functionality..? The descriptions seem the same to me.

> More comprehensive DSV2 push down capabilities
> --
>
> Key: SPARK-38788
> URL: https://issues.apache.org/jira/browse/SPARK-38788
> Project: Spark
>  Issue Type: Epic
>  Components: SQL
>Affects Versions: 3.3.0
>Reporter: Max Gekk
>Assignee: Max Gekk
>Priority: Major
>
> Get together all tickets related to push down (filters) via Datasource V2 
> APIs.



--
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-38788) More comprehensive DSV2 push down capabilities

2022-04-12 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-38788:
-

What's the relationship between this and SPARK-38852? Seems like they are 
laying out the same goal?

> More comprehensive DSV2 push down capabilities
> --
>
> Key: SPARK-38788
> URL: https://issues.apache.org/jira/browse/SPARK-38788
> Project: Spark
>  Issue Type: Epic
>  Components: SQL
>Affects Versions: 3.3.0
>Reporter: Max Gekk
>Assignee: Max Gekk
>Priority: Major
>
> Get together all tickets related to push down (filters) via Datasource V2 
> APIs.



--
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-38852) Better Data Source V2 operator pushdown framework

2022-04-12 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-38852:
-

What's the relationship between this and SPARK-38788? Seems like they are 
laying out the same goal?

> Better Data Source V2 operator pushdown framework
> -
>
> Key: SPARK-38852
> URL: https://issues.apache.org/jira/browse/SPARK-38852
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 3.3.0
>Reporter: jiaan.geng
>Priority: Major
>
> Currently, Spark supports push down Filters and Aggregates to data source.
> However, the Data Source V2 operator pushdown framework has the following 
> shortcomings:
> # Only simple filter and aggregate are supported, which makes it impossible 
> to apply in most scenarios
> # The incompatibility of SQL syntax makes it impossible to apply in most 
> scenarios
> # Aggregate push down does not support multiple partitions of data sources
> # Spark's additional aggregate will cause some overhead
> # Limit push down is not supported
> # Top n push down is not supported
> # Offset push down is not supported



--
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] [Updated] (SPARK-35066) Spark 3.1.1 is slower than 3.0.2 by 4-5 times

2022-03-17 Thread Erik Krogen (Jira)


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

Erik Krogen updated SPARK-35066:

Description: 
Hi,

The following snippet code runs 4-5 times slower when it's used in Apache Spark 
or PySpark 3.1.1 compare to Apache Spark or PySpark 3.0.2:

 
{code:java}
spark = SparkSession.builder \
        .master("local[*]") \
        .config("spark.driver.memory", "16G") \
        .config("spark.driver.maxResultSize", "0") \
        .config("spark.serializer",
"org.apache.spark.serializer.KryoSerializer") \
        .config("spark.kryoserializer.buffer.max", "2000m") \
        .getOrCreate()

Toys = spark.read \
  .parquet('./toys-cleaned').repartition(12)

# tokenize the text
regexTokenizer = RegexTokenizer(inputCol="reviewText",
outputCol="all_words", pattern="\\W")
toys_with_words = regexTokenizer.transform(Toys)

# remove stop words
remover = StopWordsRemover(inputCol="all_words", outputCol="words")
toys_with_tokens = remover.transform(toys_with_words).drop("all_words")

all_words = toys_with_tokens.select(explode("words").alias("word"))
# group by, sort and limit to 50k
top50k =
all_words.groupBy("word").agg(count("*").alias("total")).sort(col("total").desc()).limit(5)

top50k.show()
{code}
 

Some debugging on my side revealed that in Spark/PySpark 3.0.2 the 12 
partitions are respected in a way that all 12 tasks are being processed 
altogether. However, in Spark/PySpark 3.1.1 even though we have 12 tasks, 10 of 
them finish immediately and only 2 are being processed. (I've tried to disable 
a couple of configs related to something similar, but none of them worked)

Screenshot of spark 3.1.1 task:

!image-2022-03-17-17-18-36-793.png|width=1073,height=652!

!image-2022-03-17-17-19-11-655.png!

 

Screenshot of spark 3.0.2 task:

 

!image-2022-03-17-17-19-34-906.png!

For a longer discussion: [Spark User List 
|https://lists.apache.org/thread/1hlg9fpxnw8dzx8bd2fvffmk7yozoszf]

 

You can reproduce this big difference of performance between Spark 3.1.1 and 
Spark 3.0.2 by using the shared code with any dataset that is large enough to 
take longer than a minute. Not sure if this is related to SQL, any Spark config 
being enabled in 3.x but not really into action before 3.1.1, or it's about 
.transform in Spark ML.

  was:
Hi,

The following snippet code runs 4-5 times slower when it's used in Apache Spark 
or PySpark 3.1.1 compare to Apache Spark or PySpark 3.0.2:

 
{code:java}
spark = SparkSession.builder \
        .master("local[*]") \
        .config("spark.driver.memory", "16G") \
        .config("spark.driver.maxResultSize", "0") \
        .config("spark.serializer",
"org.apache.spark.serializer.KryoSerializer") \
        .config("spark.kryoserializer.buffer.max", "2000m") \
        .getOrCreate()

Toys = spark.read \
  .parquet('./toys-cleaned').repartition(12)

# tokenize the text
regexTokenizer = RegexTokenizer(inputCol="reviewText",
outputCol="all_words", pattern="\\W")
toys_with_words = regexTokenizer.transform(Toys)

# remove stop words
remover = StopWordsRemover(inputCol="all_words", outputCol="words")
toys_with_tokens = remover.transform(toys_with_words).drop("all_words")

all_words = toys_with_tokens.select(explode("words").alias("word"))
# group by, sort and limit to 50k
top50k =
all_words.groupBy("word").agg(count("*").alias("total")).sort(col("total").desc()).limit(5)

top50k.show()
{code}
 

Some debugging on my side revealed that in Spark/PySpark 3.0.2 the 12 
partitions are respected in a way that all 12 tasks are being processed 
altogether. However, in Spark/PySpark 3.1.1 even though we have 12 tasks, 10 of 
them finish immediately and only 2 are being processed. (I've tried to disable 
a couple of configs related to something similar, but none of them worked)

Screenshot of spark 3.1.1 task:

!image-2022-03-17-17-18-36-793.png|width=1073,height=652!

!image-2022-03-17-17-19-11-655.png!

 

Screenshot of spark 3.0.2 task:

 

!image-2022-03-17-17-19-34-906.png!

For a longer discussion: [Spark User List 
|https://lists.apache.org/thread/1bslwjdwnr5tw7wjkv0672vj41x4g2f1]

 

You can reproduce this big difference of performance between Spark 3.1.1 and 
Spark 3.0.2 by using the shared code with any dataset that is large enough to 
take longer than a minute. Not sure if this is related to SQL, any Spark config 
being enabled in 3.x but not really into action before 3.1.1, or it's about 
.transform in Spark ML.


> Spark 3.1.1 is slower than 3.0.2 by 4-5 times
> -
>
> Key: SPARK-35066
> URL: https://issues.apache.org/jira/browse/SPARK-35066
> Project: Spark
>  Issue Type: Bug
>  Components: ML, SQL
>Affects Versions: 3.1.1
> Environment: Spark/PySpark: 3.1.1
> Language: Python 3.7.x / Scala 12
> OS: macOS, Linux, and Windows
> Cloud: Databricks 7.3 for 3.0.1 

[jira] [Updated] (SPARK-35066) Spark 3.1.1 is slower than 3.0.2 by 4-5 times

2022-03-17 Thread Erik Krogen (Jira)


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

Erik Krogen updated SPARK-35066:

Description: 
Hi,

The following snippet code runs 4-5 times slower when it's used in Apache Spark 
or PySpark 3.1.1 compare to Apache Spark or PySpark 3.0.2:

 
{code:java}
spark = SparkSession.builder \
        .master("local[*]") \
        .config("spark.driver.memory", "16G") \
        .config("spark.driver.maxResultSize", "0") \
        .config("spark.serializer",
"org.apache.spark.serializer.KryoSerializer") \
        .config("spark.kryoserializer.buffer.max", "2000m") \
        .getOrCreate()

Toys = spark.read \
  .parquet('./toys-cleaned').repartition(12)

# tokenize the text
regexTokenizer = RegexTokenizer(inputCol="reviewText",
outputCol="all_words", pattern="\\W")
toys_with_words = regexTokenizer.transform(Toys)

# remove stop words
remover = StopWordsRemover(inputCol="all_words", outputCol="words")
toys_with_tokens = remover.transform(toys_with_words).drop("all_words")

all_words = toys_with_tokens.select(explode("words").alias("word"))
# group by, sort and limit to 50k
top50k =
all_words.groupBy("word").agg(count("*").alias("total")).sort(col("total").desc()).limit(5)

top50k.show()
{code}
 

Some debugging on my side revealed that in Spark/PySpark 3.0.2 the 12 
partitions are respected in a way that all 12 tasks are being processed 
altogether. However, in Spark/PySpark 3.1.1 even though we have 12 tasks, 10 of 
them finish immediately and only 2 are being processed. (I've tried to disable 
a couple of configs related to something similar, but none of them worked)

Screenshot of spark 3.1.1 task:

!image-2022-03-17-17-18-36-793.png|width=1073,height=652!

!image-2022-03-17-17-19-11-655.png!

 

Screenshot of spark 3.0.2 task:

 

!image-2022-03-17-17-19-34-906.png!

For a longer discussion: [Spark User List 
|https://lists.apache.org/thread/1bslwjdwnr5tw7wjkv0672vj41x4g2f1]

 

You can reproduce this big difference of performance between Spark 3.1.1 and 
Spark 3.0.2 by using the shared code with any dataset that is large enough to 
take longer than a minute. Not sure if this is related to SQL, any Spark config 
being enabled in 3.x but not really into action before 3.1.1, or it's about 
.transform in Spark ML.

  was:
Hi,

The following snippet code runs 4-5 times slower when it's used in Apache Spark 
or PySpark 3.1.1 compare to Apache Spark or PySpark 3.0.2:

 
{code:java}
spark = SparkSession.builder \
        .master("local[*]") \
        .config("spark.driver.memory", "16G") \
        .config("spark.driver.maxResultSize", "0") \
        .config("spark.serializer",
"org.apache.spark.serializer.KryoSerializer") \
        .config("spark.kryoserializer.buffer.max", "2000m") \
        .getOrCreate()

Toys = spark.read \
  .parquet('./toys-cleaned').repartition(12)

# tokenize the text
regexTokenizer = RegexTokenizer(inputCol="reviewText",
outputCol="all_words", pattern="\\W")
toys_with_words = regexTokenizer.transform(Toys)

# remove stop words
remover = StopWordsRemover(inputCol="all_words", outputCol="words")
toys_with_tokens = remover.transform(toys_with_words).drop("all_words")

all_words = toys_with_tokens.select(explode("words").alias("word"))
# group by, sort and limit to 50k
top50k =
all_words.groupBy("word").agg(count("*").alias("total")).sort(col("total").desc()).limit(5)

top50k.show()
{code}
 

Some debugging on my side revealed that in Spark/PySpark 3.0.2 the 12 
partitions are respected in a way that all 12 tasks are being processed 
altogether. However, in Spark/PySpark 3.1.1 even though we have 12 tasks, 10 of 
them finish immediately and only 2 are being processed. (I've tried to disable 
a couple of configs related to something similar, but none of them worked)

Screenshot of spark 3.1.1 task:

!image-2022-03-17-17-18-36-793.png|width=1073,height=652!

!image-2022-03-17-17-19-11-655.png!

 

Screenshot of spark 3.0.2 task:

 

!image-2022-03-17-17-19-34-906.png!

For a longer discussion: [Spark User List 
|http://apache-spark-user-list.1001560.n3.nabble.com/Why-is-Spark-3-0-x-faster-than-Spark-3-1-x-td39979.html](no
 longer exists!)

 

You can reproduce this big difference of performance between Spark 3.1.1 and 
Spark 3.0.2 by using the shared code with any dataset that is large enough to 
take longer than a minute. Not sure if this is related to SQL, any Spark config 
being enabled in 3.x but not really into action before 3.1.1, or it's about 
.transform in Spark ML.


> Spark 3.1.1 is slower than 3.0.2 by 4-5 times
> -
>
> Key: SPARK-35066
> URL: https://issues.apache.org/jira/browse/SPARK-35066
> Project: Spark
>  Issue Type: Bug
>  Components: ML, SQL
>Affects Versions: 3.1.1
> Environment: Spark/PySpark: 3.1.1
> Language: Python 3.7.x / Scala 12
> 

[jira] [Commented] (SPARK-38408) Add option for partial nested field writes with nullable fields.

2022-03-03 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-38408:
-

Sounds very similar to what was done in SPARK-34378 for the Avro datasource 
specifically. Is what you're discussing a general feature, or specific to a 
datasource? I don't quite follow the use-case.

> Add option for partial nested field writes with nullable fields.
> 
>
> Key: SPARK-38408
> URL: https://issues.apache.org/jira/browse/SPARK-38408
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 3.3.0
>Reporter: Holden Karau
>Priority: Major
>
> If a nested field has nullable entries a user shouldn't have to specify all 
> of the fields, instead the nullable fields can have the default value of 
> null. This will allow for more effective schema migration. We might want to 
> put this behind a feature flag though.



--
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] [Updated] (SPARK-38334) Implement support for DEFAULT values for columns in tables

2022-02-28 Thread Erik Krogen (Jira)


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

Erik Krogen updated SPARK-38334:

Description: 
This story tracks the implementation of DEFAULT values for columns in tables.

CREATE TABLE and ALTER TABLE invocations will support setting column default 
values for future operations. Following INSERT, UPDATE, MERGE statements may 
then reference the value using the DEFAULT keyword as needed.

Examples:
{code:sql}
CREATE TABLE T(a INT, b INT NOT NULL);

-- The default default is NULL
INSERT INTO T VALUES (DEFAULT, 0);
INSERT INTO T(b)  VALUES (1);
SELECT * FROM T;
(NULL, 0)
(NULL, 1)

-- Adding a default to a table with rows, sets the values for the
-- existing rows (exist default) and new rows (current default).
ALTER TABLE T ADD COLUMN c INT DEFAULT 5;
INSERT INTO T VALUES (1, 2, DEFAULT);
SELECT * FROM T;
(NULL, 0, 5)
(NULL, 1, 5)
(1, 2, 5) {code}

  was:
This story tracks the implementation of DEFAULT values for columns in tables.

CREATE TABLE and ALTER TABLE invocations will support setting column default 
values for future operations. Following INSERT, UPDATE, MERGE statements may 
then reference the value using the DEFAULT keyword as needed.

Examples:
{code:sql}
CREATE TABLE T(a INT, b INT NOT NULL);

– The default default is NULL
INSERT INTO T VALUES (DEFAULT, 0);
INSERT INTO T(b)  VALUES (1);
SELECT * FROM T;
(NULL, 0)
(NULL, 1)

– Adding a default to a table with rows, sets the values for the
-- existing rows (exist default) and new rows (current default).
ALTER TABLE T ADD COLUMN c INT DEFAULT 5;
INSERT INTO T VALUES (1, 2, DEFAULT);
SELECT * FROM T;
(NULL, 0, 5)
(NULL, 1, 5)
(1, 2, 5) {code}


> Implement support for DEFAULT values for columns in tables 
> ---
>
> Key: SPARK-38334
> URL: https://issues.apache.org/jira/browse/SPARK-38334
> Project: Spark
>  Issue Type: Story
>  Components: SQL
>Affects Versions: 3.2.1
>Reporter: Daniel
>Priority: Major
>
> This story tracks the implementation of DEFAULT values for columns in tables.
> CREATE TABLE and ALTER TABLE invocations will support setting column default 
> values for future operations. Following INSERT, UPDATE, MERGE statements may 
> then reference the value using the DEFAULT keyword as needed.
> Examples:
> {code:sql}
> CREATE TABLE T(a INT, b INT NOT NULL);
> -- The default default is NULL
> INSERT INTO T VALUES (DEFAULT, 0);
> INSERT INTO T(b)  VALUES (1);
> SELECT * FROM T;
> (NULL, 0)
> (NULL, 1)
> -- Adding a default to a table with rows, sets the values for the
> -- existing rows (exist default) and new rows (current default).
> ALTER TABLE T ADD COLUMN c INT DEFAULT 5;
> INSERT INTO T VALUES (1, 2, DEFAULT);
> SELECT * FROM T;
> (NULL, 0, 5)
> (NULL, 1, 5)
> (1, 2, 5) {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-37318) Make FallbackStorageSuite robust in terms of DNS

2022-02-24 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-37318:
-

Great point [~dongjoon], thanks for pointing it out!

> Make FallbackStorageSuite robust in terms of DNS
> 
>
> Key: SPARK-37318
> URL: https://issues.apache.org/jira/browse/SPARK-37318
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, Tests
>Affects Versions: 3.2.0, 3.3.0
>Reporter: Dongjoon Hyun
>Assignee: Dongjoon Hyun
>Priority: Major
> Fix For: 3.2.1, 3.3.0
>
>
> Usually, the test case expects the hostname doesn't exist.
> {code}
> $ ping remote
> ping: cannot resolve remote: Unknown host
> {code}
> In some DNS environments, it returns always.
> {code}
> $ ping remote
> PING remote (23.217.138.110): 56 data bytes
> 64 bytes from 23.217.138.110: icmp_seq=0 ttl=57 time=8.660 ms
> {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-37318) Make FallbackStorageSuite robust in terms of DNS

2022-02-23 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-37318:
-

Note that the changes in this PR were reverted in SPARK-38062, in favor of a 
solution which fixes the production code rather than disabling the test case in 
certain environments.

> Make FallbackStorageSuite robust in terms of DNS
> 
>
> Key: SPARK-37318
> URL: https://issues.apache.org/jira/browse/SPARK-37318
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, Tests
>Affects Versions: 3.2.0, 3.3.0
>Reporter: Dongjoon Hyun
>Assignee: Dongjoon Hyun
>Priority: Major
> Fix For: 3.2.1, 3.3.0
>
>
> Usually, the test case expects the hostname doesn't exist.
> {code}
> $ ping remote
> ping: cannot resolve remote: Unknown host
> {code}
> In some DNS environments, it returns always.
> {code}
> $ ping remote
> PING remote (23.217.138.110): 56 data bytes
> 64 bytes from 23.217.138.110: icmp_seq=0 ttl=57 time=8.660 ms
> {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-38245) Avro Complex Union Type return `member$I`

2022-02-22 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-38245:
-

FWIW, though I don't have context for when this logic was added to Spark, my 
guess is that the {{member#}} is inspired by the [extract_union Hive UDF which 
performs a similar 
operation|https://github.com/apache/hive/blob/1aa6ce84798e78ea53c3bec2beedb5f55b6c/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFExtractUnion.java#L50]
 (though using {{tag_}} as the prefix instead of {{member}}):
{code:java}
name = "extract_union",
value = "_FUNC_(union[, tag])" + " - Recursively explodes unions into 
structs or simply extracts the given tag.",
extended = "  > SELECT _FUNC_({0:\"foo\"}).tag_0 FROM src;\n  foo\n"
+ "  > SELECT _FUNC_({0:\"foo\"}).tag_1 FROM src;\n  null\n"
+ "  > SELECT _FUNC_({0:\"foo\"}, 0) FROM src;\n  foo\n"
+ "  > SELECT _FUNC_({0:\"foo\"}, 1) FROM src;\n  null")
{code}
But there are competing union-conversion schema definitions, e.g. Trino uses 
{{STRUCT}} (see [PR #3483 · 
trinodb/trino|https://github.com/trinodb/trino/pull/3483]).

It might be useful to add logic which makes the union-to-struct conversion 
process configurable, e.g. something like {{spark.sql.uniontype.schema}} which 
can take values like {{member}} (matches current format), {{tag}} (matches hive 
format), {{typename}} (uses types for field names, or something similar, to 
address your use case), {{tag_field}} (matches Trino format), etc.

cc folks who implemented this logically initially or reviewed the 
implementation (from SPARK-24768): [~Gengliang.Wang] [~hyukjin.kwon] [~maxgekk] 
[~viirya] [~cloud_fan]

> Avro Complex Union Type return `member$I`
> -
>
> Key: SPARK-38245
> URL: https://issues.apache.org/jira/browse/SPARK-38245
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.2.1
> Environment: +OS+
>  * Debian GNU/Linux 10 (Docker Container)
> +packages & others+
>  * spark-avro_2.12-3.2.1
>  * python 3.7.3
>  * pyspark 3.2.1
>  * spark-3.2.1-bin-hadoop3.2
>  * Docker version 20.10.12
>Reporter: Teddy Crepineau
>Priority: Major
>  Labels: avro, newbie
>
> *Short Description*
> When reading complex union types from Avro files, there seems to be some 
> information lost as the name of the record is omitted and {{member$i}} is 
> instead returned.
> *Long Description*
> +Error+
> Given the Avro schema {{{}schema.avsc{}}}, I would expected the schema when 
> reading the avro file using {{read_avro.py}} to be as {{{}expected.txt{}}}. 
> Instead, I get the schema output in {{reality.txt}} where {{RecordOne}} 
> became {{{}member0{}}}, etc.
> This causes information lost and makes the DataFrame unusable.
> From my understanding this behavior was implemented 
> [here.|https://github.com/databricks/spark-avro/pull/117]
>  
> {code:java|title=read_avro.py}
> df = spark.read.format("avro").load("path/to/my/file.avro")
> df.printSchema()
>  {code}
> {code:java|title=schema.avsc}
>  {
>  "type": "record",
>  "name": "SomeData",
>  "namespace": "my.name.space",
>  "fields": [
>   {
>    "name": "ts",
>    "type": {
>     "type": "long",
>     "logicalType": "timestamp-millis"
>    }
>   },
>   {
>    "name": "field_id",
>    "type": [
>     "null",
>     "string"
>    ],
>    "default": null
>   },
>   {
>    "name": "values",
>    "type": [
>     {
>      "type": "record",
>      "name": "RecordOne",
>      "fields": [
>       {
>        "name": "field_a",
>        "type": "long"
>       },
>       {
>        "name": "field_b",
>        "type": {
>         "type": "enum",
>         "name": "FieldB",
>         "symbols": [
>             "..."
>         ],
>        }
>       },
>       {
>        "name": "field_C",
>        "type": {
>         "type": "array",
>         "items": "long"
>        }
>       }
>      ]
>     },
>     {
>      "type": "record",
>      "name": "RecordTwo",
>      "fields": [
>       {
>        "name": "field_a",
>        "type": "long"
>       }
>      ]
>     }
>    ]
>   }
>  ]
> }{code}
> {code:java|title=expected.txt}
> root
>  |-- ts: timestamp (nullable = true)
>  |-- field_id: string (nullable = true)
>  |-- values: struct (nullable = true)
>  ||-- RecordOne: struct (nullable = true)
>  |||-- field_a: long (nullable = true)
>  |||-- field_b: string (nullable = true)
>  |||-- field_c: array (nullable = true)
>  ||||-- element: long (containsNull = true)
>  ||-- RecordTwo: struct (nullable = true)
>  |||-- field_a: long (nullable = true)
> {code}
> {code:java|title=reality.txt}
> root
>  |-- ts: timestamp (nullable = true)
>  |-- field_id: string (nullable = true)
>  |-- values: struct 

[jira] [Comment Edited] (SPARK-38245) Avro Complex Union Type return `member$I`

2022-02-18 Thread Erik Krogen (Jira)


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

Erik Krogen edited comment on SPARK-38245 at 2/18/22, 11:26 PM:


Well, yes, this is true, but it's not related to the union conversion process. 
Converting an Avro schema to a Spark schema _always_ loses record 
name/namespace information, since Spark does not have such a concept. The type 
information is still there, it is just structural rather than named.


was (Author: xkrogen):
Well, yes, this is true, but it's not related to the union conversion process. 
Converting an Avro schema to a Spark schema _always_ loses record 
name/namespace information, since Spark does not have such a concept. 

> Avro Complex Union Type return `member$I`
> -
>
> Key: SPARK-38245
> URL: https://issues.apache.org/jira/browse/SPARK-38245
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.2.1
> Environment: +OS+
>  * Debian GNU/Linux 10 (Docker Container)
> +packages & others+
>  * spark-avro_2.12-3.2.1
>  * python 3.7.3
>  * pyspark 3.2.1
>  * spark-3.2.1-bin-hadoop3.2
>  * Docker version 20.10.12
>Reporter: Teddy Crepineau
>Priority: Major
>  Labels: avro, newbie
>
> *Short Description*
> When reading complex union types from Avro files, there seems to be some 
> information lost as the name of the record is omitted and {{member$i}} is 
> instead returned.
> *Long Description*
> +Error+
> Given the Avro schema {{{}schema.avsc{}}}, I would expected the schema when 
> reading the avro file using {{read_avro.py}} to be as {{{}expected.txt{}}}. 
> Instead, I get the schema output in {{reality.txt}} where {{RecordOne}} 
> became {{{}member0{}}}, etc.
> This causes information lost and makes the DataFrame unusable.
> From my understanding this behavior was implemented 
> [here.|https://github.com/databricks/spark-avro/pull/117]
>  
> {code:java|title=read_avro.py}
> df = spark.read.format("avro").load("path/to/my/file.avro")
> df.printSchema()
>  {code}
> {code:java|title=schema.avsc}
>  {
>  "type": "record",
>  "name": "SomeData",
>  "namespace": "my.name.space",
>  "fields": [
>   {
>    "name": "ts",
>    "type": {
>     "type": "long",
>     "logicalType": "timestamp-millis"
>    }
>   },
>   {
>    "name": "field_id",
>    "type": [
>     "null",
>     "string"
>    ],
>    "default": null
>   },
>   {
>    "name": "values",
>    "type": [
>     {
>      "type": "record",
>      "name": "RecordOne",
>      "fields": [
>       {
>        "name": "field_a",
>        "type": "long"
>       },
>       {
>        "name": "field_b",
>        "type": {
>         "type": "enum",
>         "name": "FieldB",
>         "symbols": [
>             "..."
>         ],
>        }
>       },
>       {
>        "name": "field_C",
>        "type": {
>         "type": "array",
>         "items": "long"
>        }
>       }
>      ]
>     },
>     {
>      "type": "record",
>      "name": "RecordTwo",
>      "fields": [
>       {
>        "name": "field_a",
>        "type": "long"
>       }
>      ]
>     }
>    ]
>   }
>  ]
> }{code}
> {code:java|title=expected.txt}
> root
>  |-- ts: timestamp (nullable = true)
>  |-- field_id: string (nullable = true)
>  |-- values: struct (nullable = true)
>  ||-- RecordOne: struct (nullable = true)
>  |||-- field_a: long (nullable = true)
>  |||-- field_b: string (nullable = true)
>  |||-- field_c: array (nullable = true)
>  ||||-- element: long (containsNull = true)
>  ||-- RecordTwo: struct (nullable = true)
>  |||-- field_a: long (nullable = true)
> {code}
> {code:java|title=reality.txt}
> root
>  |-- ts: timestamp (nullable = true)
>  |-- field_id: string (nullable = true)
>  |-- values: struct (nullable = true)
>  ||-- member0: struct (nullable = true)
>  |||-- field_a: long (nullable = true)
>  |||-- field_b: string (nullable = true)
>  |||-- field_c: array (nullable = true)
>  ||||-- element: long (containsNull = true)
>  ||-- member1: struct (nullable = true)
>  |||-- field_a: long (nullable = true)
> {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-38245) Avro Complex Union Type return `member$I`

2022-02-18 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-38245:
-

Well, yes, this is true, but it's not related to the union conversion process. 
Converting an Avro schema to a Spark schema _always_ loses record 
name/namespace information, since Spark does not have such a concept. 

> Avro Complex Union Type return `member$I`
> -
>
> Key: SPARK-38245
> URL: https://issues.apache.org/jira/browse/SPARK-38245
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.2.1
> Environment: +OS+
>  * Debian GNU/Linux 10 (Docker Container)
> +packages & others+
>  * spark-avro_2.12-3.2.1
>  * python 3.7.3
>  * pyspark 3.2.1
>  * spark-3.2.1-bin-hadoop3.2
>  * Docker version 20.10.12
>Reporter: Teddy Crepineau
>Priority: Major
>  Labels: avro, newbie
>
> *Short Description*
> When reading complex union types from Avro files, there seems to be some 
> information lost as the name of the record is omitted and {{member$i}} is 
> instead returned.
> *Long Description*
> +Error+
> Given the Avro schema {{{}schema.avsc{}}}, I would expected the schema when 
> reading the avro file using {{read_avro.py}} to be as {{{}expected.txt{}}}. 
> Instead, I get the schema output in {{reality.txt}} where {{RecordOne}} 
> became {{{}member0{}}}, etc.
> This causes information lost and makes the DataFrame unusable.
> From my understanding this behavior was implemented 
> [here.|https://github.com/databricks/spark-avro/pull/117]
>  
> {code:java|title=read_avro.py}
> df = spark.read.format("avro").load("path/to/my/file.avro")
> df.printSchema()
>  {code}
> {code:java|title=schema.avsc}
>  {
>  "type": "record",
>  "name": "SomeData",
>  "namespace": "my.name.space",
>  "fields": [
>   {
>    "name": "ts",
>    "type": {
>     "type": "long",
>     "logicalType": "timestamp-millis"
>    }
>   },
>   {
>    "name": "field_id",
>    "type": [
>     "null",
>     "string"
>    ],
>    "default": null
>   },
>   {
>    "name": "values",
>    "type": [
>     {
>      "type": "record",
>      "name": "RecordOne",
>      "fields": [
>       {
>        "name": "field_a",
>        "type": "long"
>       },
>       {
>        "name": "field_b",
>        "type": {
>         "type": "enum",
>         "name": "FieldB",
>         "symbols": [
>             "..."
>         ],
>        }
>       },
>       {
>        "name": "field_C",
>        "type": {
>         "type": "array",
>         "items": "long"
>        }
>       }
>      ]
>     },
>     {
>      "type": "record",
>      "name": "RecordTwo",
>      "fields": [
>       {
>        "name": "field_a",
>        "type": "long"
>       }
>      ]
>     }
>    ]
>   }
>  ]
> }{code}
> {code:java|title=expected.txt}
> root
>  |-- ts: timestamp (nullable = true)
>  |-- field_id: string (nullable = true)
>  |-- values: struct (nullable = true)
>  ||-- RecordOne: struct (nullable = true)
>  |||-- field_a: long (nullable = true)
>  |||-- field_b: string (nullable = true)
>  |||-- field_c: array (nullable = true)
>  ||||-- element: long (containsNull = true)
>  ||-- RecordTwo: struct (nullable = true)
>  |||-- field_a: long (nullable = true)
> {code}
> {code:java|title=reality.txt}
> root
>  |-- ts: timestamp (nullable = true)
>  |-- field_id: string (nullable = true)
>  |-- values: struct (nullable = true)
>  ||-- member0: struct (nullable = true)
>  |||-- field_a: long (nullable = true)
>  |||-- field_b: string (nullable = true)
>  |||-- field_c: array (nullable = true)
>  ||||-- element: long (containsNull = true)
>  ||-- member1: struct (nullable = true)
>  |||-- field_a: long (nullable = true)
> {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-38245) Avro Complex Union Type return `member$I`

2022-02-18 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-38245:
-

{quote}
now I am curious to know why including the namespace would "get messy quickly"
{quote}
Namespaces (in my experience) are often fairly long, so let's say we have 
records like {{org.apache.spark.avrotest.RecordOne}}. This is a very long and 
unwieldy field name! Plus, I don't think we allow periods in field names, so we 
would probably replace them with underscores. Personally I feel I would rather 
refer to a field like {{member1}} than a field like 
{{org_apache_spark_avrotest_RecordOne}} -- but I suppose that there are valid 
arguments for the longer one.

{quote}
I am wondering if other solutions could help make this decoding of avro files 
to DataFrame loss-less (e.g. keeping a mapping of positional name to field 
names, etc.)
{quote}
I am not really seeing how the conversion is lossy, currently. Taking the 
example I shared above with an int/long union, the current schema will be:
{code}
root
 |-- foo: struct
 ||-- member0: int
 ||-- member1: long
{code}
If you want to derive the original union branches, you can just check the types 
in the schema. The mapping you described is already maintained implicitly in 
the mapping of field names to their types.

> Avro Complex Union Type return `member$I`
> -
>
> Key: SPARK-38245
> URL: https://issues.apache.org/jira/browse/SPARK-38245
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.2.1
> Environment: +OS+
>  * Debian GNU/Linux 10 (Docker Container)
> +packages & others+
>  * spark-avro_2.12-3.2.1
>  * python 3.7.3
>  * pyspark 3.2.1
>  * spark-3.2.1-bin-hadoop3.2
>  * Docker version 20.10.12
>Reporter: Teddy Crepineau
>Priority: Major
>  Labels: avro, newbie
>
> *Short Description*
> When reading complex union types from Avro files, there seems to be some 
> information lost as the name of the record is omitted and {{member$i}} is 
> instead returned.
> *Long Description*
> +Error+
> Given the Avro schema {{{}schema.avsc{}}}, I would expected the schema when 
> reading the avro file using {{read_avro.py}} to be as {{{}expected.txt{}}}. 
> Instead, I get the schema output in {{reality.txt}} where {{RecordOne}} 
> became {{{}member0{}}}, etc.
> This causes information lost and makes the DataFrame unusable.
> From my understanding this behavior was implemented 
> [here.|https://github.com/databricks/spark-avro/pull/117]
>  
> {code:java|title=read_avro.py}
> df = spark.read.format("avro").load("path/to/my/file.avro")
> df.printSchema()
>  {code}
> {code:java|title=schema.avsc}
>  {
>  "type": "record",
>  "name": "SomeData",
>  "namespace": "my.name.space",
>  "fields": [
>   {
>    "name": "ts",
>    "type": {
>     "type": "long",
>     "logicalType": "timestamp-millis"
>    }
>   },
>   {
>    "name": "field_id",
>    "type": [
>     "null",
>     "string"
>    ],
>    "default": null
>   },
>   {
>    "name": "values",
>    "type": [
>     {
>      "type": "record",
>      "name": "RecordOne",
>      "fields": [
>       {
>        "name": "field_a",
>        "type": "long"
>       },
>       {
>        "name": "field_b",
>        "type": {
>         "type": "enum",
>         "name": "FieldB",
>         "symbols": [
>             "..."
>         ],
>        }
>       },
>       {
>        "name": "field_C",
>        "type": {
>         "type": "array",
>         "items": "long"
>        }
>       }
>      ]
>     },
>     {
>      "type": "record",
>      "name": "RecordTwo",
>      "fields": [
>       {
>        "name": "field_a",
>        "type": "long"
>       }
>      ]
>     }
>    ]
>   }
>  ]
> }{code}
> {code:java|title=expected.txt}
> root
>  |-- ts: timestamp (nullable = true)
>  |-- field_id: string (nullable = true)
>  |-- values: struct (nullable = true)
>  ||-- RecordOne: struct (nullable = true)
>  |||-- field_a: long (nullable = true)
>  |||-- field_b: string (nullable = true)
>  |||-- field_c: array (nullable = true)
>  ||||-- element: long (containsNull = true)
>  ||-- RecordTwo: struct (nullable = true)
>  |||-- field_a: long (nullable = true)
> {code}
> {code:java|title=reality.txt}
> root
>  |-- ts: timestamp (nullable = true)
>  |-- field_id: string (nullable = true)
>  |-- values: struct (nullable = true)
>  ||-- member0: struct (nullable = true)
>  |||-- field_a: long (nullable = true)
>  |||-- field_b: string (nullable = true)
>  |||-- field_c: array (nullable = true)
>  ||||-- element: long (containsNull = true)
>  ||-- member1: struct (nullable = true)
>  |||-- field_a: long (nullable = true)
> {code}



--

[jira] [Comment Edited] (SPARK-38245) Avro Complex Union Type return `member$I`

2022-02-17 Thread Erik Krogen (Jira)


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

Erik Krogen edited comment on SPARK-38245 at 2/17/22, 11:45 PM:


This behavior is expected. The fields of the union are expanded based on their 
position.

I guess you're proposing that the name of the type be used as the name of the 
field, instead of a positional name? This will get pretty confusing for unions 
of primitives, e.g. the following type:
{code:json}
{
  "name": "foo"
  "type": ["int", "long"]
} {code}
will have Spark type:
{code}
root
 |-- foo: struct
 ||-- int: int
 ||-- long: long {code}
Names of types being used as the name of a field looks very confusing, at least 
to me.

Another problem, you could end up with duplicate field names like:
{code:json}
{
  "name": "foo"
  "type": [{
  "type": "record",
  "name": "RecordOne",
  "namespace": "foo"
 }, {
  "type": "record",
  "name": "RecordOne",
  "namespace": "bar"
}
  ]
} {code}
Since namespaces are different, they are different types, and this is a valid 
union. But in your proposal, they will result in the same field name. Unless we 
include the namespace in the field name as well, but this will get messy 
quickly.


was (Author: xkrogen):
This behavior is expected. The fields of the union are expanded based on their 
position.

 

I guess you're proposing that the name of the type be used as the name of the 
field, instead of a positional name? This will get pretty confusing for unions 
of primitives, e.g. the following type:
{code:java}
{
  "name": "foo"
  "type": ["int", "long"]
} {code}
will have Spark type:
{code:java}
root
 |-- foo: struct
 ||-- int: int
 ||-- long: long {code}
Names of types being used as the name of a field looks very confusing, at least 
to me.

Another problem, you could end up with duplicate field names like:
{code:java}
{
  "name": "foo"
  "type": [{
  "type": "record",
  "name": "RecordOne",
  "namespace": "foo"
 }, {
  "type": "record",
  "name": "RecordOne",
  "namespace": "bar"
}
  ]
} {code}
Since namespaces are different, they are different types, and this is a valid 
union. But in your proposal, they will result in the same field name. Unless we 
include the namespace in the field name as well, but this will get messy 
quickly.

> Avro Complex Union Type return `member$I`
> -
>
> Key: SPARK-38245
> URL: https://issues.apache.org/jira/browse/SPARK-38245
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 3.2.1
> Environment: +OS+
>  * Debian GNU/Linux 10 (Docker Container)
> +packages & others+
>  * spark-avro_2.12-3.2.1
>  * python 3.7.3
>  * pyspark 3.2.1
>  * spark-3.2.1-bin-hadoop3.2
>  * Docker version 20.10.12
>Reporter: Teddy Crepineau
>Priority: Major
>  Labels: avro, newbie
>
> *Short Description*
> When reading complex union types from Avro files, there seems to be some 
> information lost as the name of the record is omitted and {{member$i}} is 
> instead returned.
> *Long Description*
> +Error+
> Given the Avro schema {{{}schema.avsc{}}}, I would expected the schema when 
> reading the avro file using {{read_avro.py}} to be as {{{}expected.txt{}}}. 
> Instead, I get the schema output in {{reality.txt}} where {{RecordOne}} 
> became {{{}member0{}}}, etc.
> This causes information lost and makes the DataFrame unusable.
> From my understanding this behavior was implemented 
> [here.|https://github.com/databricks/spark-avro/pull/117]
>  
> {code:java|title=read_avro.py}
> df = spark.read.format("avro").load("path/to/my/file.avro")
> df.printSchema()
>  {code}
> {code:java|title=schema.avsc}
>  {
>  "type": "record",
>  "name": "SomeData",
>  "namespace": "my.name.space",
>  "fields": [
>   {
>    "name": "ts",
>    "type": {
>     "type": "long",
>     "logicalType": "timestamp-millis"
>    }
>   },
>   {
>    "name": "field_id",
>    "type": [
>     "null",
>     "string"
>    ],
>    "default": null
>   },
>   {
>    "name": "values",
>    "type": [
>     {
>      "type": "record",
>      "name": "RecordOne",
>      "fields": [
>       {
>        "name": "field_a",
>        "type": "long"
>       },
>       {
>        "name": "field_b",
>        "type": {
>         "type": "enum",
>         "name": "FieldB",
>         "symbols": [
>             "..."
>         ],
>        }
>       },
>       {
>        "name": "field_C",
>        "type": {
>         "type": "array",
>         "items": "long"
>        }
>       }
>      ]
>     },
>     {
>      "type": "record",
>      "name": "RecordTwo",
>      "fields": [
>       {
>        "name": "field_a",
>        "type": "long"
>       }
>      ]
>     }
>    ]
>   }
>  ]
> 

[jira] [Commented] (SPARK-38245) Avro Complex Union Type return `member$I`

2022-02-17 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-38245:
-

This behavior is expected. The fields of the union are expanded based on their 
position.

 

I guess you're proposing that the name of the type be used as the name of the 
field, instead of a positional name? This will get pretty confusing for unions 
of primitives, e.g. the following type:
{code:java}
{
  "name": "foo"
  "type": ["int", "long"]
} {code}
will have Spark type:
{code:java}
root
 |-- foo: struct
 ||-- int: int
 ||-- long: long {code}
Names of types being used as the name of a field looks very confusing, at least 
to me.

Another problem, you could end up with duplicate field names like:
{code:java}
{
  "name": "foo"
  "type": [{
  "type": "record",
  "name": "RecordOne",
  "namespace": "foo"
 }, {
  "type": "record",
  "name": "RecordOne",
  "namespace": "bar"
}
  ]
} {code}
Since namespaces are different, they are different types, and this is a valid 
union. But in your proposal, they will result in the same field name. Unless we 
include the namespace in the field name as well, but this will get messy 
quickly.

> Avro Complex Union Type return `member$I`
> -
>
> Key: SPARK-38245
> URL: https://issues.apache.org/jira/browse/SPARK-38245
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 3.2.1
> Environment: +OS+
>  * Debian GNU/Linux 10 (Docker Container)
> +packages & others+
>  * spark-avro_2.12-3.2.1
>  * python 3.7.3
>  * pyspark 3.2.1
>  * spark-3.2.1-bin-hadoop3.2
>  * Docker version 20.10.12
>Reporter: Teddy Crepineau
>Priority: Major
>  Labels: avro, newbie
>
> *Short Description*
> When reading complex union types from Avro files, there seems to be some 
> information lost as the name of the record is omitted and {{member$i}} is 
> instead returned.
> *Long Description*
> +Error+
> Given the Avro schema {{{}schema.avsc{}}}, I would expected the schema when 
> reading the avro file using {{read_avro.py}} to be as {{{}expected.txt{}}}. 
> Instead, I get the schema output in {{reality.txt}} where {{RecordOne}} 
> became {{{}member0{}}}, etc.
> This causes information lost and makes the DataFrame unusable.
> From my understanding this behavior was implemented 
> [here.|https://github.com/databricks/spark-avro/pull/117]
>  
> {code:java|title=read_avro.py}
> df = spark.read.format("avro").load("path/to/my/file.avro")
> df.printSchema()
>  {code}
> {code:java|title=schema.avsc}
>  {
>  "type": "record",
>  "name": "SomeData",
>  "namespace": "my.name.space",
>  "fields": [
>   {
>    "name": "ts",
>    "type": {
>     "type": "long",
>     "logicalType": "timestamp-millis"
>    }
>   },
>   {
>    "name": "field_id",
>    "type": [
>     "null",
>     "string"
>    ],
>    "default": null
>   },
>   {
>    "name": "values",
>    "type": [
>     {
>      "type": "record",
>      "name": "RecordOne",
>      "fields": [
>       {
>        "name": "field_a",
>        "type": "long"
>       },
>       {
>        "name": "field_b",
>        "type": {
>         "type": "enum",
>         "name": "FieldB",
>         "symbols": [
>             "..."
>         ],
>        }
>       },
>       {
>        "name": "field_C",
>        "type": {
>         "type": "array",
>         "items": "long"
>        }
>       }
>      ]
>     },
>     {
>      "type": "record",
>      "name": "RecordTwo",
>      "fields": [
>       {
>        "name": "field_a",
>        "type": "long"
>       }
>      ]
>     }
>    ]
>   }
>  ]
> }{code}
> {code:java|title=expected.txt}
> root
>  |-- ts: timestamp (nullable = true)
>  |-- field_id: string (nullable = true)
>  |-- values: struct (nullable = true)
>  ||-- RecordOne: struct (nullable = true)
>  |||-- field_a: long (nullable = true)
>  |||-- field_b: string (nullable = true)
>  |||-- field_c: array (nullable = true)
>  ||||-- element: long (containsNull = true)
>  ||-- RecordTwo: struct (nullable = true)
>  |||-- field_a: long (nullable = true)
> {code}
> {code:java|title=reality.txt}
> root
>  |-- ts: timestamp (nullable = true)
>  |-- field_id: string (nullable = true)
>  |-- values: struct (nullable = true)
>  ||-- member0: struct (nullable = true)
>  |||-- field_a: long (nullable = true)
>  |||-- field_b: string (nullable = true)
>  |||-- field_c: array (nullable = true)
>  ||||-- element: long (containsNull = true)
>  ||-- member1: struct (nullable = true)
>  |||-- field_a: long (nullable = true)
> {code}



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


[jira] [Commented] (SPARK-38030) Query with cast containing non-nullable columns fails with AQE on Spark 3.1.1

2022-02-08 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-38030:
-

Ahh... Canonicalization is getting rid of the name, I see. That's tricky. 
Thanks.

> Query with cast containing non-nullable columns fails with AQE on Spark 3.1.1
> -
>
> Key: SPARK-38030
> URL: https://issues.apache.org/jira/browse/SPARK-38030
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.1.1
>Reporter: Shardul Mahadik
>Assignee: Shardul Mahadik
>Priority: Major
> Fix For: 3.3.0
>
>
> One of our user queries failed in Spark 3.1.1 when using AQE with the 
> following stacktrace mentioned below (some parts of the plan have been 
> redacted, but the structure is preserved).
> Debugging this issue, we found that the failure was within AQE calling 
> [QueryPlan.canonicalized|https://github.com/apache/spark/blob/91db9a36a9ed74845908f14d21227d5267591653/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala#L402].
> The query contains a cast over a column with non-nullable struct fields. 
> Canonicalization [removes nullability 
> information|https://github.com/apache/spark/blob/91db9a36a9ed74845908f14d21227d5267591653/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala#L45]
>  from the child {{AttributeReference}} of the Cast, however it does not 
> remove nullability information from the Cast's target dataType. This causes 
> the 
> [checkInputDataTypes|https://github.com/apache/spark/blob/branch-3.1/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala#L290]
>  to return false because the child is now nullable and cast target data type 
> is not, leading to {{resolved=false}} and hence the {{UnresolvedException}}.
> {code:java}
> org.apache.spark.sql.catalyst.errors.package$TreeNodeException: makeCopy, 
> tree:
> Exchange RoundRobinPartitioning(1), REPARTITION_BY_NUM, [id=#232]
> +- Union
>:- Project [cast(columnA#30) as struct<...>]
>:  +- BatchScan[columnA#30] hive.tbl 
>+- Project [cast(columnA#35) as struct<...>]
>   +- BatchScan[columnA#35] hive.tbl
>   at 
> org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:56)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.makeCopy(TreeNode.scala:475)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.makeCopy(TreeNode.scala:464)
>   at org.apache.spark.sql.execution.SparkPlan.makeCopy(SparkPlan.scala:87)
>   at org.apache.spark.sql.execution.SparkPlan.makeCopy(SparkPlan.scala:58)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.withNewChildren(TreeNode.scala:301)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.doCanonicalize(QueryPlan.scala:405)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.canonicalized$lzycompute(QueryPlan.scala:373)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.canonicalized(QueryPlan.scala:372)
>   at 
> org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.createQueryStages(AdaptiveSparkPlanExec.scala:404)
>   at 
> org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.$anonfun$createQueryStages$2(AdaptiveSparkPlanExec.scala:447)
>   at 
> scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
>   at scala.collection.immutable.List.foreach(List.scala:392)
>   at scala.collection.TraversableLike.map(TraversableLike.scala:238)
>   at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
>   at scala.collection.immutable.List.map(List.scala:298)
>   at 
> org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.createQueryStages(AdaptiveSparkPlanExec.scala:447)
>   at 
> org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.$anonfun$createQueryStages$2(AdaptiveSparkPlanExec.scala:447)
>   at 
> scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
>   at scala.collection.immutable.List.foreach(List.scala:392)
>   at scala.collection.TraversableLike.map(TraversableLike.scala:238)
>   at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
>   at scala.collection.immutable.List.map(List.scala:298)
>   at 
> org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.createQueryStages(AdaptiveSparkPlanExec.scala:447)
>   at 
> org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.$anonfun$getFinalPhysicalPlan$1(AdaptiveSparkPlanExec.scala:184)
>   at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:772)
>   at 
> org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.getFinalPhysicalPlan(AdaptiveSparkPlanExec.scala:179)
>   at 
> 

[jira] [Commented] (SPARK-38091) AvroSerializer can cause java.lang.ClassCastException at run time

2022-02-02 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-38091:
-

[~Zhen-hao] for formatting you need to use the Atlassian markup: 
[https://jira.atlassian.com/secure/WikiRendererHelpAction.jspa?section=all]
Basically replace ` ... ` with \{{ ... }} and replace ``` ... ``` with \{code} 
... \{code}

> AvroSerializer can cause java.lang.ClassCastException at run time
> -
>
> Key: SPARK-38091
> URL: https://issues.apache.org/jira/browse/SPARK-38091
> 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, 
> 3.2.1
>Reporter: Zhenhao Li
>Priority: Major
>  Labels: Avro, serializers
>
> `AvroSerializer`'s implementation, at least in `newConverter`, was not 100% 
> based on the `InternalRow` and `SpecializedGetters` interface. It assumes 
> many implementation details of the interface. 
> For example, in 
> ```scala
>       case (TimestampType, LONG) => avroType.getLogicalType match {
>           // For backward compatibility, if the Avro type is Long and it is 
> not logical type
>           // (the `null` case), output the timestamp value as with 
> millisecond precision.
>           case null | _: TimestampMillis => (getter, ordinal) =>
>             
> DateTimeUtils.microsToMillis(timestampRebaseFunc(getter.getLong(ordinal)))
>           case _: TimestampMicros => (getter, ordinal) =>
>             timestampRebaseFunc(getter.getLong(ordinal))
>           case other => throw new IncompatibleSchemaException(errorPrefix +
>             s"SQL type ${TimestampType.sql} cannot be converted to Avro 
> logical type $other")
>         }
> ```
> it assumes the `InternalRow` instance encodes `TimestampType` as 
> `java.lang.Long`. That's true for `Unsaferow` but not for 
> `GenericInternalRow`. 
> Hence the above code will end up with runtime exceptions when used on an 
> instance of `GenericInternalRow`, which is the case for Python UDF. 
> I didn't get time to dig deeper than that. I got the impression that Spark's 
> optimizer(s) will turn a row into a `UnsafeRow` and Python UDF doesn't 
> involve the optimizer(s) and hence each row is a `GenericInternalRow`. 
> It would be great if someone can correct me or offer a better explanation. 
>  
> To reproduce the issue, 
> `git checkout master` and `git cherry-pick --no-commit` 
> [this-commit|https://github.com/Zhen-hao/spark/commit/1ffe8e8f35273b2f3529f6c2d004822f480e4c88]
> and run the test `org.apache.spark.sql.avro.AvroSerdeSuite`.
>  
> You will see runtime exceptions like the following one
> ```
> - Serialize DecimalType to Avro BYTES with logical type decimal *** FAILED ***
>   java.lang.ClassCastException: class java.math.BigDecimal cannot be cast to 
> class org.apache.spark.sql.types.Decimal (java.math.BigDecimal is in module 
> java.base of loader 'bootstrap'; org.apache.spark.sql.types.Decimal is in 
> unnamed module of loader 'app')
>   at 
> org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow.getDecimal(rows.scala:45)
>   at 
> org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow.getDecimal$(rows.scala:45)
>   at 
> org.apache.spark.sql.catalyst.expressions.GenericInternalRow.getDecimal(rows.scala:195)
>   at 
> org.apache.spark.sql.avro.AvroSerializer.$anonfun$newConverter$10(AvroSerializer.scala:136)
>   at 
> org.apache.spark.sql.avro.AvroSerializer.$anonfun$newConverter$10$adapted(AvroSerializer.scala:135)
>   at 
> org.apache.spark.sql.avro.AvroSerializer.$anonfun$newStructConverter$2(AvroSerializer.scala:283)
>   at 
> org.apache.spark.sql.avro.AvroSerializer.serialize(AvroSerializer.scala:60)
>   at 
> org.apache.spark.sql.avro.AvroSerdeSuite.$anonfun$new$5(AvroSerdeSuite.scala:82)
>   at 
> org.apache.spark.sql.avro.AvroSerdeSuite.$anonfun$new$5$adapted(AvroSerdeSuite.scala:67)
>   at 
> org.apache.spark.sql.avro.AvroSerdeSuite.$anonfun$withFieldMatchType$2(AvroSerdeSuite.scala:217)
> ```



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

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



[jira] [Created] (SPARK-38089) Improve assertion failure message in TestUtils.assertExceptionMsg

2022-02-02 Thread Erik Krogen (Jira)
Erik Krogen created SPARK-38089:
---

 Summary: Improve assertion failure message in 
TestUtils.assertExceptionMsg
 Key: SPARK-38089
 URL: https://issues.apache.org/jira/browse/SPARK-38089
 Project: Spark
  Issue Type: Improvement
  Components: Spark Core, Tests
Affects Versions: 3.2.1
Reporter: Erik Krogen


{{TestUtils.assertExceptionMsg}} is great, but when the assertion _doesn't_ 
match, it can be challenging to tell why, because the exception tree that was 
searched isn't printed. Only way I could find to fix it up was to run things in 
a debugger and check the exception tree.
It would be very helpful if {{assertExceptionMsg}} printed out the exception 
tree in which it was searching (upon failure).



--
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-38062) FallbackStorage shouldn't attempt to resolve arbitrary "remote" hostname

2022-01-31 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-38062:
-

Thanks for taking a look btw! Even if we don't agree on everything :)

> FallbackStorage shouldn't attempt to resolve arbitrary "remote" hostname
> 
>
> Key: SPARK-38062
> URL: https://issues.apache.org/jira/browse/SPARK-38062
> Project: Spark
>  Issue Type: Improvement
>  Components: Spark Core
>Affects Versions: 3.3.0
>Reporter: Erik Krogen
>Priority: Major
>
> {{FallbackStorage}} uses a placeholder block manager ID:
> {code:scala}
> private[spark] object FallbackStorage extends Logging {
>   /** We use one block manager id as a place holder. */
>   val FALLBACK_BLOCK_MANAGER_ID: BlockManagerId = BlockManagerId("fallback", 
> "remote", 7337)
> {code}
> That second argument is normally interpreted as a hostname, but is passed as 
> the string "remote" in this case.
> {{BlockManager}} will consider this placeholder as one of the peers in some 
> cases:
> {code:language=scala|title=BlockManager.scala}
>   private[storage] def getPeers(forceFetch: Boolean): Seq[BlockManagerId] = {
> peerFetchLock.synchronized {
>   ...
>   if (cachedPeers.isEmpty &&
>   
> conf.get(config.STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH).isDefined) {
> Seq(FallbackStorage.FALLBACK_BLOCK_MANAGER_ID)
>   } else {
> cachedPeers
>   }
> }
>   }
> {code}
> {{BlockManagerDecommissioner.ShuffleMigrationRunnable}} will then attempt to 
> perform an upload to this placeholder ID:
> {code:scala}
> try {
>   blocks.foreach { case (blockId, buffer) =>
> logDebug(s"Migrating sub-block ${blockId}")
> bm.blockTransferService.uploadBlockSync(
>   peer.host,
>   peer.port,
>   peer.executorId,
>   blockId,
>   buffer,
>   StorageLevel.DISK_ONLY,
>   null) // class tag, we don't need for shuffle
> logDebug(s"Migrated sub-block $blockId")
>   }
>   logInfo(s"Migrated $shuffleBlockInfo to $peer")
> } catch {
>   case e: IOException =>
> ...
> if 
> (bm.migratableResolver.getMigrationBlocks(shuffleBlockInfo).size < 
> blocks.size) {
>   logWarning(s"Skipping block $shuffleBlockInfo, block 
> deleted.")
> } else if (fallbackStorage.isDefined) {
>   fallbackStorage.foreach(_.copy(shuffleBlockInfo, bm))
> } else {
>   logError(s"Error occurred during migrating 
> $shuffleBlockInfo", e)
>   keepRunning = false
> }
> {code}
> Since "remote" is not expected to be a resolvable hostname, an 
> {{IOException}} occurs, and {{fallbackStorage}} is used. But, we shouldn't 
> try to resolve this. First off, it's completely unnecessary and strange to be 
> treating the placeholder ID as a resolvable hostname, relying on an exception 
> to realize that we need to use the {{fallbackStorage}}.
> To make matters worse, in some network environments, "remote" may be a 
> resolvable hostname, completely breaking this functionality. In the 
> particular environment that I use for running automated tests, there is a DNS 
> entry for "remote" which, when you attempt to connect to it, will hang for a 
> long period of time. This essentially hangs the executor decommission 
> process, and in the case of unit tests, breaks {{FallbackStorageSuite}} as it 
> exceeds its timeouts. I'm not sure, but it's possible this is related to 
> SPARK-35584 as well (if sometimes in the GA environment, it takes a long time 
> for the OS to decide that "remote" is not a valid hostname).
> We shouldn't attempt to treat this placeholder ID as a real hostname.



--
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-38062) FallbackStorage shouldn't attempt to resolve arbitrary "remote" hostname

2022-01-31 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-38062:
-

It still seems like a bug to me that the presence of a DNS entry breaks certain 
features in Spark (as opposed to an improvement to support a new environment), 
but yes, I can recognize your logic here.

I still feel that this JIRA is obviously related to SPARK-37318, as they are 
tackling nearly identical problems just with a different approach, but will 
defer to your judgement on how to maintain links on the Spark JIRA project.

> FallbackStorage shouldn't attempt to resolve arbitrary "remote" hostname
> 
>
> Key: SPARK-38062
> URL: https://issues.apache.org/jira/browse/SPARK-38062
> Project: Spark
>  Issue Type: Improvement
>  Components: Spark Core
>Affects Versions: 3.3.0
>Reporter: Erik Krogen
>Priority: Major
>
> {{FallbackStorage}} uses a placeholder block manager ID:
> {code:scala}
> private[spark] object FallbackStorage extends Logging {
>   /** We use one block manager id as a place holder. */
>   val FALLBACK_BLOCK_MANAGER_ID: BlockManagerId = BlockManagerId("fallback", 
> "remote", 7337)
> {code}
> That second argument is normally interpreted as a hostname, but is passed as 
> the string "remote" in this case.
> {{BlockManager}} will consider this placeholder as one of the peers in some 
> cases:
> {code:language=scala|title=BlockManager.scala}
>   private[storage] def getPeers(forceFetch: Boolean): Seq[BlockManagerId] = {
> peerFetchLock.synchronized {
>   ...
>   if (cachedPeers.isEmpty &&
>   
> conf.get(config.STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH).isDefined) {
> Seq(FallbackStorage.FALLBACK_BLOCK_MANAGER_ID)
>   } else {
> cachedPeers
>   }
> }
>   }
> {code}
> {{BlockManagerDecommissioner.ShuffleMigrationRunnable}} will then attempt to 
> perform an upload to this placeholder ID:
> {code:scala}
> try {
>   blocks.foreach { case (blockId, buffer) =>
> logDebug(s"Migrating sub-block ${blockId}")
> bm.blockTransferService.uploadBlockSync(
>   peer.host,
>   peer.port,
>   peer.executorId,
>   blockId,
>   buffer,
>   StorageLevel.DISK_ONLY,
>   null) // class tag, we don't need for shuffle
> logDebug(s"Migrated sub-block $blockId")
>   }
>   logInfo(s"Migrated $shuffleBlockInfo to $peer")
> } catch {
>   case e: IOException =>
> ...
> if 
> (bm.migratableResolver.getMigrationBlocks(shuffleBlockInfo).size < 
> blocks.size) {
>   logWarning(s"Skipping block $shuffleBlockInfo, block 
> deleted.")
> } else if (fallbackStorage.isDefined) {
>   fallbackStorage.foreach(_.copy(shuffleBlockInfo, bm))
> } else {
>   logError(s"Error occurred during migrating 
> $shuffleBlockInfo", e)
>   keepRunning = false
> }
> {code}
> Since "remote" is not expected to be a resolvable hostname, an 
> {{IOException}} occurs, and {{fallbackStorage}} is used. But, we shouldn't 
> try to resolve this. First off, it's completely unnecessary and strange to be 
> treating the placeholder ID as a resolvable hostname, relying on an exception 
> to realize that we need to use the {{fallbackStorage}}.
> To make matters worse, in some network environments, "remote" may be a 
> resolvable hostname, completely breaking this functionality. In the 
> particular environment that I use for running automated tests, there is a DNS 
> entry for "remote" which, when you attempt to connect to it, will hang for a 
> long period of time. This essentially hangs the executor decommission 
> process, and in the case of unit tests, breaks {{FallbackStorageSuite}} as it 
> exceeds its timeouts. I'm not sure, but it's possible this is related to 
> SPARK-35584 as well (if sometimes in the GA environment, it takes a long time 
> for the OS to decide that "remote" is not a valid hostname).
> We shouldn't attempt to treat this placeholder ID as a real hostname.



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

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



[jira] [Comment Edited] (SPARK-38062) FallbackStorage shouldn't attempt to resolve arbitrary "remote" hostname

2022-01-31 Thread Erik Krogen (Jira)


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

Erik Krogen edited comment on SPARK-38062 at 1/31/22, 5:33 PM:
---

I think the issue I'm reporting is very similar to SPARK-37318, is it not? The 
root cause in both cases is that the code within 
{{FallbackStorage}}/{{BlockManagerDecommissioner}} makes an assumption that 
"remote" is an invalid DNS entry. In the case of SPARK-37318, the issue 
reported was that "remote" resolved to a valid, reachable IP. In the case of 
this JIRA, "remote" resolved to an IP that wasn't reachable, but in both cases, 
the core issue was that "remote" was resolvable by DNS -- the behavior after 
the resolution isn't too important IMO. The production code is basically broken 
as soon as "remote" is resolvable by DNS.
I'm also proposing essentially reverting SPARK-37318, because my new proposed 
fix resolves the issue at the source (within the production code), rather than 
working around it with a change to the test code to hide the broken nature of 
the production code.
So it seems to me that these two issues are very closely related. Though, 
admittedly the link to SPARK-35584 may have been misplaced.


was (Author: xkrogen):
I think the issue I'm reporting is very similar to SPARK-37318, is it not? The 
root cause in both cases is that the code within 
{{FallbackStorage}}/{{BlockManagerDecommissioner}} makes an assumption that 
"remote" is an invalid DNS entry. In the case of SPARK-37318, the issue 
reported was that "remote" resolved to a valid, reachable IP. In the case of 
this JIRA, "remote" resolved to an IP that wasn't reachable, but in both cases, 
the core issue was that "remote" was resolvable by DNS -- the behavior after 
the resolution isn't too important IMO. The code is basically broken as soon as 
"remote" is resolvable by DNS.
I'm also proposing essentially reverting SPARK-37318, because my new proposed 
fix resolves the issue at the source (within the production code), rather than 
working around it with a change to the test code to hide the broken nature of 
the production code.
So it seems to me that these two issues are very closely related. Though, 
admittedly the link to SPARK-35584 may have been misplaced.

> FallbackStorage shouldn't attempt to resolve arbitrary "remote" hostname
> 
>
> Key: SPARK-38062
> URL: https://issues.apache.org/jira/browse/SPARK-38062
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 3.1.1
>Reporter: Erik Krogen
>Priority: Major
>
> {{FallbackStorage}} uses a placeholder block manager ID:
> {code:scala}
> private[spark] object FallbackStorage extends Logging {
>   /** We use one block manager id as a place holder. */
>   val FALLBACK_BLOCK_MANAGER_ID: BlockManagerId = BlockManagerId("fallback", 
> "remote", 7337)
> {code}
> That second argument is normally interpreted as a hostname, but is passed as 
> the string "remote" in this case.
> {{BlockManager}} will consider this placeholder as one of the peers in some 
> cases:
> {code:language=scala|title=BlockManager.scala}
>   private[storage] def getPeers(forceFetch: Boolean): Seq[BlockManagerId] = {
> peerFetchLock.synchronized {
>   ...
>   if (cachedPeers.isEmpty &&
>   
> conf.get(config.STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH).isDefined) {
> Seq(FallbackStorage.FALLBACK_BLOCK_MANAGER_ID)
>   } else {
> cachedPeers
>   }
> }
>   }
> {code}
> {{BlockManagerDecommissioner.ShuffleMigrationRunnable}} will then attempt to 
> perform an upload to this placeholder ID:
> {code:scala}
> try {
>   blocks.foreach { case (blockId, buffer) =>
> logDebug(s"Migrating sub-block ${blockId}")
> bm.blockTransferService.uploadBlockSync(
>   peer.host,
>   peer.port,
>   peer.executorId,
>   blockId,
>   buffer,
>   StorageLevel.DISK_ONLY,
>   null) // class tag, we don't need for shuffle
> logDebug(s"Migrated sub-block $blockId")
>   }
>   logInfo(s"Migrated $shuffleBlockInfo to $peer")
> } catch {
>   case e: IOException =>
> ...
> if 
> (bm.migratableResolver.getMigrationBlocks(shuffleBlockInfo).size < 
> blocks.size) {
>   logWarning(s"Skipping block $shuffleBlockInfo, block 
> deleted.")
> } else if (fallbackStorage.isDefined) {
>   fallbackStorage.foreach(_.copy(shuffleBlockInfo, bm))
> } else {
>   logError(s"Error occurred during migrating 
> 

[jira] [Commented] (SPARK-38062) FallbackStorage shouldn't attempt to resolve arbitrary "remote" hostname

2022-01-31 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-38062:
-

I think the issue I'm reporting is very similar to SPARK-37318, is it not? The 
root cause in both cases is that the code within 
{{FallbackStorage}}/{{BlockManagerDecommissioner}} makes an assumption that 
"remote" is an invalid DNS entry. In the case of SPARK-37318, the issue 
reported was that "remote" resolved to a valid, reachable IP. In the case of 
this JIRA, "remote" resolved to an IP that wasn't reachable, but in both cases, 
the core issue was that "remote" was resolvable by DNS -- the behavior after 
the resolution isn't too important IMO. The code is basically broken as soon as 
"remote" is resolvable by DNS.
I'm also proposing essentially reverting SPARK-37318, because my new proposed 
fix resolves the issue at the source (within the production code), rather than 
working around it with a change to the test code to hide the broken nature of 
the production code.
So it seems to me that these two issues are very closely related. Though, 
admittedly the link to SPARK-35584 may have been misplaced.

> FallbackStorage shouldn't attempt to resolve arbitrary "remote" hostname
> 
>
> Key: SPARK-38062
> URL: https://issues.apache.org/jira/browse/SPARK-38062
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 3.1.1
>Reporter: Erik Krogen
>Priority: Major
>
> {{FallbackStorage}} uses a placeholder block manager ID:
> {code:scala}
> private[spark] object FallbackStorage extends Logging {
>   /** We use one block manager id as a place holder. */
>   val FALLBACK_BLOCK_MANAGER_ID: BlockManagerId = BlockManagerId("fallback", 
> "remote", 7337)
> {code}
> That second argument is normally interpreted as a hostname, but is passed as 
> the string "remote" in this case.
> {{BlockManager}} will consider this placeholder as one of the peers in some 
> cases:
> {code:language=scala|title=BlockManager.scala}
>   private[storage] def getPeers(forceFetch: Boolean): Seq[BlockManagerId] = {
> peerFetchLock.synchronized {
>   ...
>   if (cachedPeers.isEmpty &&
>   
> conf.get(config.STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH).isDefined) {
> Seq(FallbackStorage.FALLBACK_BLOCK_MANAGER_ID)
>   } else {
> cachedPeers
>   }
> }
>   }
> {code}
> {{BlockManagerDecommissioner.ShuffleMigrationRunnable}} will then attempt to 
> perform an upload to this placeholder ID:
> {code:scala}
> try {
>   blocks.foreach { case (blockId, buffer) =>
> logDebug(s"Migrating sub-block ${blockId}")
> bm.blockTransferService.uploadBlockSync(
>   peer.host,
>   peer.port,
>   peer.executorId,
>   blockId,
>   buffer,
>   StorageLevel.DISK_ONLY,
>   null) // class tag, we don't need for shuffle
> logDebug(s"Migrated sub-block $blockId")
>   }
>   logInfo(s"Migrated $shuffleBlockInfo to $peer")
> } catch {
>   case e: IOException =>
> ...
> if 
> (bm.migratableResolver.getMigrationBlocks(shuffleBlockInfo).size < 
> blocks.size) {
>   logWarning(s"Skipping block $shuffleBlockInfo, block 
> deleted.")
> } else if (fallbackStorage.isDefined) {
>   fallbackStorage.foreach(_.copy(shuffleBlockInfo, bm))
> } else {
>   logError(s"Error occurred during migrating 
> $shuffleBlockInfo", e)
>   keepRunning = false
> }
> {code}
> Since "remote" is not expected to be a resolvable hostname, an 
> {{IOException}} occurs, and {{fallbackStorage}} is used. But, we shouldn't 
> try to resolve this. First off, it's completely unnecessary and strange to be 
> treating the placeholder ID as a resolvable hostname, relying on an exception 
> to realize that we need to use the {{fallbackStorage}}.
> To make matters worse, in some network environments, "remote" may be a 
> resolvable hostname, completely breaking this functionality. In the 
> particular environment that I use for running automated tests, there is a DNS 
> entry for "remote" which, when you attempt to connect to it, will hang for a 
> long period of time. This essentially hangs the executor decommission 
> process, and in the case of unit tests, breaks {{FallbackStorageSuite}} as it 
> exceeds its timeouts. I'm not sure, but it's possible this is related to 
> SPARK-35584 as well (if sometimes in the GA environment, it takes a long time 
> for the OS to decide that "remote" 

[jira] [Comment Edited] (SPARK-38062) FallbackStorage shouldn't attempt to resolve arbitrary "remote" hostname

2022-01-28 Thread Erik Krogen (Jira)


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

Erik Krogen edited comment on SPARK-38062 at 1/28/22, 11:10 PM:


Sorry, I should have clarified -- the test issues/failures I described were 
observed on 3.1.1.
I did confirm that on the current {{master}} branch, these tests are entirely 
skipped in our environment, so there are no failures because the tests are 
never executed.
After the changes in the PR I put up, all FallbackStorageSuite tests run 
successfully on master and 3.1.1


was (Author: xkrogen):
Sorry, I should have clarified -- the test issues/failures I described were 
observed on 3.1.1.
I did confirm that on the current {{master}} branch, these tests are entirely 
skipped in our environment, so there are no failures.
After the changes in the PR I put up, all FallbackStorageSuite tests run 
successfully on master and 3.1.1

> FallbackStorage shouldn't attempt to resolve arbitrary "remote" hostname
> 
>
> Key: SPARK-38062
> URL: https://issues.apache.org/jira/browse/SPARK-38062
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 3.1.1
>Reporter: Erik Krogen
>Priority: Major
>
> {{FallbackStorage}} uses a placeholder block manager ID:
> {code:scala}
> private[spark] object FallbackStorage extends Logging {
>   /** We use one block manager id as a place holder. */
>   val FALLBACK_BLOCK_MANAGER_ID: BlockManagerId = BlockManagerId("fallback", 
> "remote", 7337)
> {code}
> That second argument is normally interpreted as a hostname, but is passed as 
> the string "remote" in this case.
> {{BlockManager}} will consider this placeholder as one of the peers in some 
> cases:
> {code:language=scala|title=BlockManager.scala}
>   private[storage] def getPeers(forceFetch: Boolean): Seq[BlockManagerId] = {
> peerFetchLock.synchronized {
>   ...
>   if (cachedPeers.isEmpty &&
>   
> conf.get(config.STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH).isDefined) {
> Seq(FallbackStorage.FALLBACK_BLOCK_MANAGER_ID)
>   } else {
> cachedPeers
>   }
> }
>   }
> {code}
> {{BlockManagerDecommissioner.ShuffleMigrationRunnable}} will then attempt to 
> perform an upload to this placeholder ID:
> {code:scala}
> try {
>   blocks.foreach { case (blockId, buffer) =>
> logDebug(s"Migrating sub-block ${blockId}")
> bm.blockTransferService.uploadBlockSync(
>   peer.host,
>   peer.port,
>   peer.executorId,
>   blockId,
>   buffer,
>   StorageLevel.DISK_ONLY,
>   null) // class tag, we don't need for shuffle
> logDebug(s"Migrated sub-block $blockId")
>   }
>   logInfo(s"Migrated $shuffleBlockInfo to $peer")
> } catch {
>   case e: IOException =>
> ...
> if 
> (bm.migratableResolver.getMigrationBlocks(shuffleBlockInfo).size < 
> blocks.size) {
>   logWarning(s"Skipping block $shuffleBlockInfo, block 
> deleted.")
> } else if (fallbackStorage.isDefined) {
>   fallbackStorage.foreach(_.copy(shuffleBlockInfo, bm))
> } else {
>   logError(s"Error occurred during migrating 
> $shuffleBlockInfo", e)
>   keepRunning = false
> }
> {code}
> Since "remote" is not expected to be a resolvable hostname, an 
> {{IOException}} occurs, and {{fallbackStorage}} is used. But, we shouldn't 
> try to resolve this. First off, it's completely unnecessary and strange to be 
> treating the placeholder ID as a resolvable hostname, relying on an exception 
> to realize that we need to use the {{fallbackStorage}}.
> To make matters worse, in some network environments, "remote" may be a 
> resolvable hostname, completely breaking this functionality. In the 
> particular environment that I use for running automated tests, there is a DNS 
> entry for "remote" which, when you attempt to connect to it, will hang for a 
> long period of time. This essentially hangs the executor decommission 
> process, and in the case of unit tests, breaks {{FallbackStorageSuite}} as it 
> exceeds its timeouts. I'm not sure, but it's possible this is related to 
> SPARK-35584 as well (if sometimes in the GA environment, it takes a long time 
> for the OS to decide that "remote" is not a valid hostname).
> We shouldn't attempt to treat this placeholder ID as a real hostname.



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

-
To 

[jira] [Commented] (SPARK-38062) FallbackStorage shouldn't attempt to resolve arbitrary "remote" hostname

2022-01-28 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-38062:
-

Sorry, I should have clarified -- the test issues/failures I described were 
observed on 3.1.1.
I did confirm that on the current {{master}} branch, these tests are entirely 
skipped in our environment, so there are no failures.
After the changes in the PR I put up, all FallbackStorageSuite tests run 
successfully on master and 3.1.1

> FallbackStorage shouldn't attempt to resolve arbitrary "remote" hostname
> 
>
> Key: SPARK-38062
> URL: https://issues.apache.org/jira/browse/SPARK-38062
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 3.1.1
>Reporter: Erik Krogen
>Priority: Major
>
> {{FallbackStorage}} uses a placeholder block manager ID:
> {code:scala}
> private[spark] object FallbackStorage extends Logging {
>   /** We use one block manager id as a place holder. */
>   val FALLBACK_BLOCK_MANAGER_ID: BlockManagerId = BlockManagerId("fallback", 
> "remote", 7337)
> {code}
> That second argument is normally interpreted as a hostname, but is passed as 
> the string "remote" in this case.
> {{BlockManager}} will consider this placeholder as one of the peers in some 
> cases:
> {code:language=scala|title=BlockManager.scala}
>   private[storage] def getPeers(forceFetch: Boolean): Seq[BlockManagerId] = {
> peerFetchLock.synchronized {
>   ...
>   if (cachedPeers.isEmpty &&
>   
> conf.get(config.STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH).isDefined) {
> Seq(FallbackStorage.FALLBACK_BLOCK_MANAGER_ID)
>   } else {
> cachedPeers
>   }
> }
>   }
> {code}
> {{BlockManagerDecommissioner.ShuffleMigrationRunnable}} will then attempt to 
> perform an upload to this placeholder ID:
> {code:scala}
> try {
>   blocks.foreach { case (blockId, buffer) =>
> logDebug(s"Migrating sub-block ${blockId}")
> bm.blockTransferService.uploadBlockSync(
>   peer.host,
>   peer.port,
>   peer.executorId,
>   blockId,
>   buffer,
>   StorageLevel.DISK_ONLY,
>   null) // class tag, we don't need for shuffle
> logDebug(s"Migrated sub-block $blockId")
>   }
>   logInfo(s"Migrated $shuffleBlockInfo to $peer")
> } catch {
>   case e: IOException =>
> ...
> if 
> (bm.migratableResolver.getMigrationBlocks(shuffleBlockInfo).size < 
> blocks.size) {
>   logWarning(s"Skipping block $shuffleBlockInfo, block 
> deleted.")
> } else if (fallbackStorage.isDefined) {
>   fallbackStorage.foreach(_.copy(shuffleBlockInfo, bm))
> } else {
>   logError(s"Error occurred during migrating 
> $shuffleBlockInfo", e)
>   keepRunning = false
> }
> {code}
> Since "remote" is not expected to be a resolvable hostname, an 
> {{IOException}} occurs, and {{fallbackStorage}} is used. But, we shouldn't 
> try to resolve this. First off, it's completely unnecessary and strange to be 
> treating the placeholder ID as a resolvable hostname, relying on an exception 
> to realize that we need to use the {{fallbackStorage}}.
> To make matters worse, in some network environments, "remote" may be a 
> resolvable hostname, completely breaking this functionality. In the 
> particular environment that I use for running automated tests, there is a DNS 
> entry for "remote" which, when you attempt to connect to it, will hang for a 
> long period of time. This essentially hangs the executor decommission 
> process, and in the case of unit tests, breaks {{FallbackStorageSuite}} as it 
> exceeds its timeouts. I'm not sure, but it's possible this is related to 
> SPARK-35584 as well (if sometimes in the GA environment, it takes a long time 
> for the OS to decide that "remote" is not a valid hostname).
> We shouldn't attempt to treat this placeholder ID as a real hostname.



--
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-38062) FallbackStorage shouldn't attempt to resolve arbitrary "remote" hostname

2022-01-28 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-38062:
-

Just found SPARK-37318, which disables all {{FallbackStorageSuite}} test cases 
in an environment like ours where {{remote}} is resolvable. I don't feel that 
this is the right approach, as (a) it disables tests unnecessarily, (b) it does 
not solve the problem that this could affect production deployments as well.

cc [~viirya] [~dongjoon]

> FallbackStorage shouldn't attempt to resolve arbitrary "remote" hostname
> 
>
> Key: SPARK-38062
> URL: https://issues.apache.org/jira/browse/SPARK-38062
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 3.1.1
>Reporter: Erik Krogen
>Priority: Major
>
> {{FallbackStorage}} uses a placeholder block manager ID:
> {code:scala}
> private[spark] object FallbackStorage extends Logging {
>   /** We use one block manager id as a place holder. */
>   val FALLBACK_BLOCK_MANAGER_ID: BlockManagerId = BlockManagerId("fallback", 
> "remote", 7337)
> {code}
> That second argument is normally interpreted as a hostname, but is passed as 
> the string "remote" in this case.
> {{BlockManager}} will consider this placeholder as one of the peers in some 
> cases:
> {code:language=scala|title=BlockManager.scala}
>   private[storage] def getPeers(forceFetch: Boolean): Seq[BlockManagerId] = {
> peerFetchLock.synchronized {
>   ...
>   if (cachedPeers.isEmpty &&
>   
> conf.get(config.STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH).isDefined) {
> Seq(FallbackStorage.FALLBACK_BLOCK_MANAGER_ID)
>   } else {
> cachedPeers
>   }
> }
>   }
> {code}
> {{BlockManagerDecommissioner.ShuffleMigrationRunnable}} will then attempt to 
> perform an upload to this placeholder ID:
> {code:scala}
> try {
>   blocks.foreach { case (blockId, buffer) =>
> logDebug(s"Migrating sub-block ${blockId}")
> bm.blockTransferService.uploadBlockSync(
>   peer.host,
>   peer.port,
>   peer.executorId,
>   blockId,
>   buffer,
>   StorageLevel.DISK_ONLY,
>   null) // class tag, we don't need for shuffle
> logDebug(s"Migrated sub-block $blockId")
>   }
>   logInfo(s"Migrated $shuffleBlockInfo to $peer")
> } catch {
>   case e: IOException =>
> ...
> if 
> (bm.migratableResolver.getMigrationBlocks(shuffleBlockInfo).size < 
> blocks.size) {
>   logWarning(s"Skipping block $shuffleBlockInfo, block 
> deleted.")
> } else if (fallbackStorage.isDefined) {
>   fallbackStorage.foreach(_.copy(shuffleBlockInfo, bm))
> } else {
>   logError(s"Error occurred during migrating 
> $shuffleBlockInfo", e)
>   keepRunning = false
> }
> {code}
> Since "remote" is not expected to be a resolvable hostname, an 
> {{IOException}} occurs, and {{fallbackStorage}} is used. But, we shouldn't 
> try to resolve this. First off, it's completely unnecessary and strange to be 
> treating the placeholder ID as a resolvable hostname, relying on an exception 
> to realize that we need to use the {{fallbackStorage}}.
> To make matters worse, in some network environments, "remote" may be a 
> resolvable hostname, completely breaking this functionality. In the 
> particular environment that I use for running automated tests, there is a DNS 
> entry for "remote" which, when you attempt to connect to it, will hang for a 
> long period of time. This essentially hangs the executor decommission 
> process, and in the case of unit tests, breaks {{FallbackStorageSuite}} as it 
> exceeds its timeouts. I'm not sure, but it's possible this is related to 
> SPARK-35584 as well (if sometimes in the GA environment, it takes a long time 
> for the OS to decide that "remote" is not a valid hostname).
> We shouldn't attempt to treat this placeholder ID as a real hostname.



--
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] [Updated] (SPARK-38062) FallbackStorage shouldn't attempt to resolve arbitrary "remote" hostname

2022-01-28 Thread Erik Krogen (Jira)


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

Erik Krogen updated SPARK-38062:

Affects Version/s: 3.1.1
   (was: 3.2.1)

> FallbackStorage shouldn't attempt to resolve arbitrary "remote" hostname
> 
>
> Key: SPARK-38062
> URL: https://issues.apache.org/jira/browse/SPARK-38062
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 3.1.1
>Reporter: Erik Krogen
>Priority: Major
>
> {{FallbackStorage}} uses a placeholder block manager ID:
> {code:scala}
> private[spark] object FallbackStorage extends Logging {
>   /** We use one block manager id as a place holder. */
>   val FALLBACK_BLOCK_MANAGER_ID: BlockManagerId = BlockManagerId("fallback", 
> "remote", 7337)
> {code}
> That second argument is normally interpreted as a hostname, but is passed as 
> the string "remote" in this case.
> {{BlockManager}} will consider this placeholder as one of the peers in some 
> cases:
> {code:language=scala|title=BlockManager.scala}
>   private[storage] def getPeers(forceFetch: Boolean): Seq[BlockManagerId] = {
> peerFetchLock.synchronized {
>   ...
>   if (cachedPeers.isEmpty &&
>   
> conf.get(config.STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH).isDefined) {
> Seq(FallbackStorage.FALLBACK_BLOCK_MANAGER_ID)
>   } else {
> cachedPeers
>   }
> }
>   }
> {code}
> {{BlockManagerDecommissioner.ShuffleMigrationRunnable}} will then attempt to 
> perform an upload to this placeholder ID:
> {code:scala}
> try {
>   blocks.foreach { case (blockId, buffer) =>
> logDebug(s"Migrating sub-block ${blockId}")
> bm.blockTransferService.uploadBlockSync(
>   peer.host,
>   peer.port,
>   peer.executorId,
>   blockId,
>   buffer,
>   StorageLevel.DISK_ONLY,
>   null) // class tag, we don't need for shuffle
> logDebug(s"Migrated sub-block $blockId")
>   }
>   logInfo(s"Migrated $shuffleBlockInfo to $peer")
> } catch {
>   case e: IOException =>
> ...
> if 
> (bm.migratableResolver.getMigrationBlocks(shuffleBlockInfo).size < 
> blocks.size) {
>   logWarning(s"Skipping block $shuffleBlockInfo, block 
> deleted.")
> } else if (fallbackStorage.isDefined) {
>   fallbackStorage.foreach(_.copy(shuffleBlockInfo, bm))
> } else {
>   logError(s"Error occurred during migrating 
> $shuffleBlockInfo", e)
>   keepRunning = false
> }
> {code}
> Since "remote" is not expected to be a resolvable hostname, an 
> {{IOException}} occurs, and {{fallbackStorage}} is used. But, we shouldn't 
> try to resolve this. First off, it's completely unnecessary and strange to be 
> treating the placeholder ID as a resolvable hostname, relying on an exception 
> to realize that we need to use the {{fallbackStorage}}.
> To make matters worse, in some network environments, "remote" may be a 
> resolvable hostname, completely breaking this functionality. In the 
> particular environment that I use for running automated tests, there is a DNS 
> entry for "remote" which, when you attempt to connect to it, will hang for a 
> long period of time. This essentially hangs the executor decommission 
> process, and in the case of unit tests, breaks {{FallbackStorageSuite}} as it 
> exceeds its timeouts. I'm not sure, but it's possible this is related to 
> SPARK-35584 as well (if sometimes in the GA environment, it takes a long time 
> for the OS to decide that "remote" is not a valid hostname).
> We shouldn't attempt to treat this placeholder ID as a real hostname.



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

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



[jira] [Created] (SPARK-38062) FallbackStorage shouldn't attempt to resolve arbitrary "remote" hostname

2022-01-28 Thread Erik Krogen (Jira)
Erik Krogen created SPARK-38062:
---

 Summary: FallbackStorage shouldn't attempt to resolve arbitrary 
"remote" hostname
 Key: SPARK-38062
 URL: https://issues.apache.org/jira/browse/SPARK-38062
 Project: Spark
  Issue Type: Bug
  Components: Spark Core
Affects Versions: 3.2.1
Reporter: Erik Krogen


{{FallbackStorage}} uses a placeholder block manager ID:
{code:scala}
private[spark] object FallbackStorage extends Logging {
  /** We use one block manager id as a place holder. */
  val FALLBACK_BLOCK_MANAGER_ID: BlockManagerId = BlockManagerId("fallback", 
"remote", 7337)
{code}
That second argument is normally interpreted as a hostname, but is passed as 
the string "remote" in this case.

{{BlockManager}} will consider this placeholder as one of the peers in some 
cases:
{code:language=scala|title=BlockManager.scala}
  private[storage] def getPeers(forceFetch: Boolean): Seq[BlockManagerId] = {
peerFetchLock.synchronized {
  ...
  if (cachedPeers.isEmpty &&
  
conf.get(config.STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH).isDefined) {
Seq(FallbackStorage.FALLBACK_BLOCK_MANAGER_ID)
  } else {
cachedPeers
  }
}
  }
{code}

{{BlockManagerDecommissioner.ShuffleMigrationRunnable}} will then attempt to 
perform an upload to this placeholder ID:
{code:scala}
try {
  blocks.foreach { case (blockId, buffer) =>
logDebug(s"Migrating sub-block ${blockId}")
bm.blockTransferService.uploadBlockSync(
  peer.host,
  peer.port,
  peer.executorId,
  blockId,
  buffer,
  StorageLevel.DISK_ONLY,
  null) // class tag, we don't need for shuffle
logDebug(s"Migrated sub-block $blockId")
  }
  logInfo(s"Migrated $shuffleBlockInfo to $peer")
} catch {
  case e: IOException =>
...
if 
(bm.migratableResolver.getMigrationBlocks(shuffleBlockInfo).size < blocks.size) 
{
  logWarning(s"Skipping block $shuffleBlockInfo, block 
deleted.")
} else if (fallbackStorage.isDefined) {
  fallbackStorage.foreach(_.copy(shuffleBlockInfo, bm))
} else {
  logError(s"Error occurred during migrating 
$shuffleBlockInfo", e)
  keepRunning = false
}
{code}
Since "remote" is not expected to be a resolvable hostname, an {{IOException}} 
occurs, and {{fallbackStorage}} is used. But, we shouldn't try to resolve this. 
First off, it's completely unnecessary and strange to be treating the 
placeholder ID as a resolvable hostname, relying on an exception to realize 
that we need to use the {{fallbackStorage}}.

To make matters worse, in some network environments, "remote" may be a 
resolvable hostname, completely breaking this functionality. In the particular 
environment that I use for running automated tests, there is a DNS entry for 
"remote" which, when you attempt to connect to it, will hang for a long period 
of time. This essentially hangs the executor decommission process, and in the 
case of unit tests, breaks {{FallbackStorageSuite}} as it exceeds its timeouts. 
I'm not sure, but it's possible this is related to SPARK-35584 as well (if 
sometimes in the GA environment, it takes a long time for the OS to decide that 
"remote" is not a valid hostname).

We shouldn't attempt to treat this placeholder ID as a real hostname.



--
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-38030) Query with cast containing non-nullable columns fails with AQE on Spark 3.1.1

2022-01-26 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-38030:
-

[~shardulm] can you also comment on why the exception message is so unhelpful?
{quote}
Caused by: org.apache.spark.sql.catalyst.analysis.UnresolvedException: Invalid 
call to nullable on unresolved object, tree: '
{quote}
Why don't we get any information about the path leading to this issue? What is 
"{{tree: '}}" -- is something missing there, or is it actually happening at the 
root?

> Query with cast containing non-nullable columns fails with AQE on Spark 3.1.1
> -
>
> Key: SPARK-38030
> URL: https://issues.apache.org/jira/browse/SPARK-38030
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.1.1
>Reporter: Shardul Mahadik
>Priority: Major
>
> One of our user queries failed in Spark 3.1.1 when using AQE with the 
> following stacktrace mentioned below (some parts of the plan have been 
> redacted, but the structure is preserved).
> Debugging this issue, we found that the failure was within AQE calling 
> [QueryPlan.canonicalized|https://github.com/apache/spark/blob/91db9a36a9ed74845908f14d21227d5267591653/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala#L402].
> The query contains a cast over a column with non-nullable struct fields. 
> Canonicalization [removes nullability 
> information|https://github.com/apache/spark/blob/91db9a36a9ed74845908f14d21227d5267591653/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala#L45]
>  from the child {{AttributeReference}} of the Cast, however it does not 
> remove nullability information from the Cast's target dataType. This causes 
> the 
> [checkInputDataTypes|https://github.com/apache/spark/blob/branch-3.1/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala#L290]
>  to return false because the child is now nullable and cast target data type 
> is not, leading to {{resolved=false}} and hence the {{UnresolvedException}}.
> {code:java}
> org.apache.spark.sql.catalyst.errors.package$TreeNodeException: makeCopy, 
> tree:
> Exchange RoundRobinPartitioning(1), REPARTITION_BY_NUM, [id=#232]
> +- Union
>:- Project [cast(columnA#30) as struct<...>]
>:  +- BatchScan[columnA#30] hive.tbl 
>+- Project [cast(columnA#35) as struct<...>]
>   +- BatchScan[columnA#35] hive.tbl
>   at 
> org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:56)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.makeCopy(TreeNode.scala:475)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.makeCopy(TreeNode.scala:464)
>   at org.apache.spark.sql.execution.SparkPlan.makeCopy(SparkPlan.scala:87)
>   at org.apache.spark.sql.execution.SparkPlan.makeCopy(SparkPlan.scala:58)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.withNewChildren(TreeNode.scala:301)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.doCanonicalize(QueryPlan.scala:405)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.canonicalized$lzycompute(QueryPlan.scala:373)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.canonicalized(QueryPlan.scala:372)
>   at 
> org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.createQueryStages(AdaptiveSparkPlanExec.scala:404)
>   at 
> org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.$anonfun$createQueryStages$2(AdaptiveSparkPlanExec.scala:447)
>   at 
> scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
>   at scala.collection.immutable.List.foreach(List.scala:392)
>   at scala.collection.TraversableLike.map(TraversableLike.scala:238)
>   at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
>   at scala.collection.immutable.List.map(List.scala:298)
>   at 
> org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.createQueryStages(AdaptiveSparkPlanExec.scala:447)
>   at 
> org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.$anonfun$createQueryStages$2(AdaptiveSparkPlanExec.scala:447)
>   at 
> scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
>   at scala.collection.immutable.List.foreach(List.scala:392)
>   at scala.collection.TraversableLike.map(TraversableLike.scala:238)
>   at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
>   at scala.collection.immutable.List.map(List.scala:298)
>   at 
> org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.createQueryStages(AdaptiveSparkPlanExec.scala:447)
>   at 
> org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.$anonfun$getFinalPhysicalPlan$1(AdaptiveSparkPlanExec.scala:184)
>   at 

[jira] [Commented] (SPARK-37994) Unable to build spark3.2 with -Dhadoop.version=3.1.4

2022-01-24 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-37994:
-

cc [~csun]  – is this related to your changes for using the shaded 
hadoop-client artifact?

> Unable to build spark3.2 with -Dhadoop.version=3.1.4
> 
>
> Key: SPARK-37994
> URL: https://issues.apache.org/jira/browse/SPARK-37994
> Project: Spark
>  Issue Type: Bug
>  Components: Build
>Affects Versions: 3.2.0
>Reporter: Vu Tan
>Priority: Minor
>
> I downloaded Spark 3.2 sourcecode from 
> [https://github.com/apache/spark/archive/refs/tags/v3.2.0.zip]
> and try building with the below command 
> {code:java}
> ./dev/make-distribution.sh --name without-hadoop --pip --r --tgz -Psparkr 
> -Phive -Phive-thriftserver -Phadoop-provided -Pyarn -Dhadoop.version=3.1.4 
> -Pkubernetes {code}
> Then it gives the below error 
> {code:java}
> [INFO] --- scala-maven-plugin:4.3.0:compile (scala-compile-first) @ 
> spark-core_2.12 ---
> [INFO] Using incremental compilation using Mixed compile order
> [INFO] Compiler bridge file: 
> /Users/JP28431/.sbt/1.0/zinc/org.scala-sbt/org.scala-sbt-compiler-bridge_2.12-1.3.1-bin_2.12.15__52.0-1.3.1_20191012T045515.jar
> [INFO] compiler plugin: 
> BasicArtifact(com.github.ghik,silencer-plugin_2.12.15,1.7.6,null)
> [INFO] Compiling 567 Scala sources and 104 Java sources to 
> /Users/JP28431/Downloads/spark-3.2.0-github/core/target/scala-2.12/classes ...
> [ERROR] [Error] 
> /Users/JP28431/Downloads/spark-3.2.0-github/core/src/main/scala/org/apache/spark/SparkContext.scala:38:
>  object io is not a member of package org.apache.hadoop
> [ERROR] [Error] 
> /Users/JP28431/Downloads/spark-3.2.0-github/core/src/main/scala/org/apache/spark/SparkContext.scala:2778:
>  not found: type ArrayWritable
> [ERROR] [Error] 
> /Users/JP28431/Downloads/spark-3.2.0-github/core/src/main/scala/org/apache/spark/SparkContext.scala:2777:
>  not found: type Writable
> [ERROR] [Error] 
> /Users/JP28431/Downloads/spark-3.2.0-github/core/src/main/scala/org/apache/spark/SSLOptions.scala:24:
>  object conf is not a member of package org.apache.hadoop
> [ERROR] [Error] 
> /Users/JP28431/Downloads/spark-3.2.0-github/core/src/main/scala/org/apache/spark/SSLOptions.scala:174:
>  not found: type Configuration
> [ERROR] [Error] 
> /Users/JP28431/Downloads/spark-3.2.0-github/core/src/main/scala/org/apache/spark/SecurityManager.scala:25:
>  object io is not a member of package org.apache.hadoop
> [ERROR] [Error] 
> /Users/JP28431/Downloads/spark-3.2.0-github/core/src/main/scala/org/apache/spark/SecurityManager.scala:26:
>  object security is not a member of package org.apache.hadoop
> [ERROR] [Error] 
> /Users/JP28431/Downloads/spark-3.2.0-github/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala:33:
>  object fs is not a member of package org.apache.hadoop
> [ERROR] [Error] 
> /Users/JP28431/Downloads/spark-3.2.0-github/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala:32:
>  object conf is not a member of package org.apache.hadoop
> [ERROR] [Error] 
> /Users/JP28431/Downloads/spark-3.2.0-github/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala:121:
>  not found: type Configuration
> [ERROR] [Error] 
> /Users/JP28431/Downloads/spark-3.2.0-github/core/src/main/scala/org/apache/spark/SecurityManager.scala:284:
>  not found: value UserGroupInformation
> [ERROR] [Error] 
> /Users/JP28431/Downloads/spark-3.2.0-github/core/src/main/scala/org/apache/spark/SparkContext.scala:41:
>  object mapreduce is not a member of package org.apache.hadoop
> [ERROR] [Error] 
> /Users/JP28431/Downloads/spark-3.2.0-github/core/src/main/scala/org/apache/spark/SparkContext.scala:40:
>  object mapreduce is not a member of package org.apache.hadoop
> [ERROR] [Error] 
> /Users/JP28431/Downloads/spark-3.2.0-github/core/src/main/scala/org/apache/spark/SparkContext.scala:39:
>  object mapred is not a member of package org.apache.hadoop
> [ERROR] [Error] 
> /Users/JP28431/Downloads/spark-3.2.0-github/core/src/main/scala/org/apache/spark/SparkContext.scala:37:
>  object fs is not a member of package org.apache.hadoop
> [ERROR] [Error] 
> /Users/JP28431/Downloads/spark-3.2.0-github/core/src/main/scala/org/apache/spark/SparkContext.scala:36:
>  object conf is not a member of package org.apache.hadoop
> [ERROR] [Error] 
> /Users/JP28431/Downloads/spark-3.2.0-github/core/src/main/scala/org/apache/spark/SecurityManager.scala:348:
>  not found: type Credentials
> [ERROR] [Error] 
> /Users/JP28431/Downloads/spark-3.2.0-github/core/src/main/scala/org/apache/spark/SecurityManager.scala:350:
>  not found: value UserGroupInformation
> [ERROR] [Error] 
> 

[jira] [Updated] (SPARK-37621) ClassCastException when trying to persist the result of a join between two Iceberg tables

2021-12-13 Thread Erik Krogen (Jira)


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

Erik Krogen updated SPARK-37621:

Description: 
I am gettin an error when I try to persist the results on a Join operation. 
Note that both tables to be joined and the output table are Iceberg tables.

SQL code to repro. 
{code}
String sqlJoin = String.format(
"SELECT * from " +
"((select %s from %s.%s where %s ) l " +
"join (select %s from %s.%s where %s ) r " +
"using (%s))",
);
spark.sql(sqlJoin).writeTo("ciptest.ttt").option("write-format", 
"parquet").createOrReplace();
{code}

My exception stack is:
{code}
Caused by: java.lang.ClassCastException: 
org.apache.spark.sql.catalyst.expressions.GenericInternalRow cannot be cast to 
org.apache.spark.sql.catalyst.expressions.UnsafeRow
at 
org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$1.writeValue(UnsafeRowSerializer.scala:64)
at 
org.apache.spark.storage.DiskBlockObjectWriter.write(DiskBlockObjectWriter.scala:249)
at 
org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:158)
at 
org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
at 
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
at 
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
at org.apache.spark.scheduler.Task.run(Task.scala:131)
at 
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:497)
at ….
{code}

Explain on the Sql statement gets the following plan:
{code}
== Physical Plan ==
Project [ ... ]
+- SortMergeJoin […], Inner
  :- Sort […], false, 0
  : +- Exchange hashpartitioning(…, 10), ENSURE_REQUIREMENTS, [id=#38]
  :   +- Filter (…)
  :+- BatchScan[... ] left [filters=…]
  +- *(2) Sort […], false, 0
   +- Exchange hashpartitioning(…, 10), ENSURE_REQUIREMENTS, [id=#47]
 +- *(1) Filter (…)
  +- BatchScan[…] right [filters=…] 
{code}

Note that several variations of this fail. Besides the repro code listed above 
I have tried doing CTAS and trying to write the result into parquet files 
without making a table out of it.

  was:
I am gettin an error when I try to persist the results on a Join operation. 
Note that both tables to be joined and the output table are Iceberg tables.

SQL code to repro. 


String sqlJoin = String.format(
"SELECT * from " +
"((select %s from %s.%s where %s ) l " +
"join (select %s from %s.%s where %s ) r " +
"using (%s))",
);
spark.sql(sqlJoin).writeTo("ciptest.ttt").option("write-format", 
"parquet").createOrReplace();

My exception stack is:
{{Caused by: java.lang.ClassCastException: 
org.apache.spark.sql.catalyst.expressions.GenericInternalRow cannot be cast to 
org.apache.spark.sql.catalyst.expressions.UnsafeRow}}
{{  at 
org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$1.writeValue(UnsafeRowSerializer.scala:64)}}
{{  at 
org.apache.spark.storage.DiskBlockObjectWriter.write(DiskBlockObjectWriter.scala:249)}}
{{  at 
org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:158)}}
{{  at 
org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)}}
{{  at 
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)}}
{{  at 
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)}}
{{  at org.apache.spark.scheduler.Task.run(Task.scala:131)}}
{{  at 
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:497)}}
{{  at ….}}

Explain on the Sql statement gets the following plan:
{{== Physical Plan ==}}
{{Project [ ... ]}}
{{+- SortMergeJoin […], Inner}}
{{  :- Sort […], false, 0}}
{{  : +- Exchange hashpartitioning(…, 10), ENSURE_REQUIREMENTS, [id=#38]}}
{{  :   +- Filter (…)}}
{{  :+- BatchScan[... ] left [filters=…]}}
{{  +- *(2) Sort […], false, 0}}
{{   +- Exchange hashpartitioning(…, 10), ENSURE_REQUIREMENTS, [id=#47]}}
{{ +- *(1) Filter (…)}}
{{  +- BatchScan[…] right [filters=…] }}


{{Note that several variations of this fail. Besides the repro code listed 
above I have tried doing CTAS and trying to write the result into parquet files 
without making a table out of it.}}


> ClassCastException when trying to persist the result of a join between two 
> Iceberg tables
> -
>
> Key: SPARK-37621
> URL: https://issues.apache.org/jira/browse/SPARK-37621
> Project: Spark
>  Issue Type: Bug
>  Components: Input/Output
>Affects Versions: 3.1.2
>Reporter: 

[jira] [Commented] (SPARK-37166) SPIP: Storage Partitioned Join

2021-11-01 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-37166:
-

[~csun] can you link the doc here?

> SPIP: Storage Partitioned Join
> --
>
> Key: SPARK-37166
> URL: https://issues.apache.org/jira/browse/SPARK-37166
> Project: Spark
>  Issue Type: New Feature
>  Components: SQL
>Affects Versions: 3.3.0
>Reporter: Chao Sun
>Priority: Major
>
> This JIRA tracks the SPIP for storage partitioned join.



--
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-37121) TestUtils.isPythonVersionAtLeast38 returns incorrect results

2021-10-26 Thread Erik Krogen (Jira)
Erik Krogen created SPARK-37121:
---

 Summary: TestUtils.isPythonVersionAtLeast38 returns incorrect 
results
 Key: SPARK-37121
 URL: https://issues.apache.org/jira/browse/SPARK-37121
 Project: Spark
  Issue Type: Bug
  Components: Tests
Affects Versions: 3.2.0
Reporter: Erik Krogen


I was working on {{HiveExternalCatalogVersionsSuite}} recently and noticed that 
it was never running against the Spark 2.x release lines, only the 3.x ones. 
The problem was coming from here, specifically the Python 3.8+ version check:
{code}
versions
  .filter(v => v.startsWith("3") || !TestUtils.isPythonVersionAtLeast38())
  .filter(v => v.startsWith("3") || 
!SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_9))
{code}

I found that {{TestUtils.isPythonVersionAtLeast38()}} was always returning 
true, even when my system installation of Python3 was 3.7. Thinking it was an 
environment issue, I pulled up a debugger to check which version of Python the 
test JVM was seeing, and it was in fact Python 3.7.

Turns out the issue is with the {{isPythonVersionAtLeast38}} method:
{code}
  def isPythonVersionAtLeast38(): Boolean = {
val attempt = if (Utils.isWindows) {
  Try(Process(Seq("cmd.exe", "/C", "python3 --version"))
.run(ProcessLogger(s => s.startsWith("Python 3.8") || 
s.startsWith("Python 3.9")))
.exitValue())
} else {
  Try(Process(Seq("sh", "-c", "python3 --version"))
.run(ProcessLogger(s => s.startsWith("Python 3.8") || 
s.startsWith("Python 3.9")))
.exitValue())
}
attempt.isSuccess && attempt.get == 0
  }
{code}
It's trying to evaluate the version of Python using a {{ProcessLogger}}, but 
the logger accepts a {{String => Unit}} function, i.e., it does not make use of 
the return value in any way (since it's meant for logging). So the result of 
the {{startsWith}} checks are thrown away, and {{attempt.isSuccess && 
attempt.get == 0}} will always be true as long as your system has a {{python3}} 
binary of any version.



--
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-37043) Cancel all running job after AQE plan finished

2021-10-25 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-37043:
-

Converted to subtask of SPARK-37063.

> Cancel all running job after AQE plan finished
> --
>
> Key: SPARK-37043
> URL: https://issues.apache.org/jira/browse/SPARK-37043
> Project: Spark
>  Issue Type: Sub-task
>  Components: SQL
>Affects Versions: 3.3.0
>Reporter: XiDuo You
>Priority: Major
>
> We see stage was still running after AQE plan finished. This is because the 
> plan which contains a empty join has been converted to `LocalTableScanExec` 
> during `AQEOptimizer`, but the other side of this join is still running 
> (shuffle map stage).
>  
> It's no meaning to keep running the stage, It's better to cancel the running 
> stage after AQE plan finished in case wasting the task resource.



--
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-37043) Cancel all running job after AQE plan finished

2021-10-25 Thread Erik Krogen (Jira)


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

Erik Krogen updated SPARK-37043:

Parent: SPARK-37063
Issue Type: Sub-task  (was: Improvement)

> Cancel all running job after AQE plan finished
> --
>
> Key: SPARK-37043
> URL: https://issues.apache.org/jira/browse/SPARK-37043
> Project: Spark
>  Issue Type: Sub-task
>  Components: SQL
>Affects Versions: 3.3.0
>Reporter: XiDuo You
>Priority: Major
>
> We see stage was still running after AQE plan finished. This is because the 
> plan which contains a empty join has been converted to `LocalTableScanExec` 
> during `AQEOptimizer`, but the other side of this join is still running 
> (shuffle map stage).
>  
> It's no meaning to keep running the stage, It's better to cancel the running 
> stage after AQE plan finished in case wasting the task resource.



--
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-37043) Cancel all running job after AQE plan finished

2021-10-19 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-37043:
-

[~ulysses] any concerns if I make this a sub-task of SPARK-37063 to track it 
alongside other AQE fixes?

> Cancel all running job after AQE plan finished
> --
>
> Key: SPARK-37043
> URL: https://issues.apache.org/jira/browse/SPARK-37043
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 3.3.0
>Reporter: XiDuo You
>Priority: Major
>
> We see stage was still running after AQE plan finished. This is because the 
> plan which contains a empty join has been converted to `LocalTableScanExec` 
> during `AQEOptimizer`, but the other side of this join is still running 
> (shuffle map stage).
>  
> It's no meaning to keep running the stage, It's better to cancel the running 
> stage after AQE plan finished in case wasting the task resource.



--
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-33828) SQL Adaptive Query Execution QA

2021-10-19 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-33828:
-

Thanks [~dongjoon]!

> SQL Adaptive Query Execution QA
> ---
>
> Key: SPARK-33828
> URL: https://issues.apache.org/jira/browse/SPARK-33828
> Project: Spark
>  Issue Type: Umbrella
>  Components: SQL
>Affects Versions: 3.2.0
>Reporter: Dongjoon Hyun
>Priority: Critical
>  Labels: releasenotes
> Fix For: 3.2.0
>
>
> Since SPARK-31412 is delivered at 3.0.0, we received and handled many JIRA 
> issues at 3.0.x/3.1.0/3.2.0. This umbrella JIRA issue aims to enable it by 
> default and collect all information in order to do QA for this feature in 
> Apache Spark 3.2.0 timeframe.



--
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-37027) Fix behavior inconsistent in Hive table when ‘path’ is provided in SERDEPROPERTIES

2021-10-18 Thread Erik Krogen (Jira)


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

Erik Krogen edited comment on SPARK-37027 at 10/18/21, 6:10 PM:


[~yuzhousun] actually this is already fixed by SPARK-28266 in 3.1.3 and 3.2.0. 
Can you try compiling from latest {{master}} or using the 3.2.0 binaries (not 
yet officially released but [can be found on Maven 
Central|https://search.maven.org/search?q=g:org.apache.spark%20AND%20v:3.2.0]).


was (Author: xkrogen):
[~yuzhousun] actually this is already fixed by SPARK-28266 in 3.1.3 and 3.2.0. 
Can you try compiling from latest {{master}} or using the 3.2.0 binaries (not 
yet officially released but 
[https://search.maven.org/search?q=g:org.apache.spark%20AND%20v:3.2.0|can be 
found on Maven Central]).

> Fix behavior inconsistent in Hive table when ‘path’ is provided in 
> SERDEPROPERTIES
> --
>
> Key: SPARK-37027
> URL: https://issues.apache.org/jira/browse/SPARK-37027
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 2.4.5, 3.1.2
>Reporter: Yuzhou Sun
>Priority: Trivial
> Attachments: SPARK-37027-test-example.patch
>
>
> If a Hive table is created with both {{WITH SERDEPROPERTIES 
> ('path'='')}} and {{LOCATION }}, Spark can 
> return doubled rows when reading the table. This issue seems to be an 
> extension of SPARK-30507.
>  Reproduce steps:
>  # Create table and insert records via Hive (Spark doesn't allow to insert 
> into table like this)
> {code:sql}
> CREATE TABLE `test_table`(
>   `c1` LONG,
>   `c2` STRING)
> ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
> WITH SERDEPROPERTIES ('path'=''" )
> STORED AS
>   INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
>   OUTPUTFORMAT 
> 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
> LOCATION '';
> INSERT INTO TABLE `test_table`
> VALUES (0, '0');
> SELECT * FROM `test_table`;
> -- will return
> -- 0 0
> {code}
>  # Read above table from Spark
> {code:sql}
> SELECT * FROM `test_table`;
> -- will return
> -- 0 0
> -- 0 0
> {code}
> But if we set {{spark.sql.hive.convertMetastoreParquet=false}}, Spark will 
> return same result as Hive (i.e. single row)
> A similar case is that, if a Hive table is created with both {{WITH 
> SERDEPROPERTIES ('path'='')}} and {{LOCATION }}, 
> Spark will read both rows under {{anotherPath}} and rows under 
> {{tableLocation}}, regardless of {{spark.sql.hive.convertMetastoreParquet}} 
> ‘s value. However, actually Hive seems to return only rows under 
> {{tableLocation}}
> Another similar case is that, if {{path}} is provided in {{TBLPROPERTIES}}, 
> Spark won’t double the rows when {{'path'=''}}. If 
> {{'path'=''}}, Spark will read both rows under {{anotherPath}} 
> and rows under {{tableLocation}}, Hive seems to keep ignoring the {{path}} in 
> {{TBLPROPERTIES}}
> Code examples for the above cases (diff patch wrote in 
> {{HiveParquetMetastoreSuite.scala}}) can be found in Attachments



--
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-37027) Fix behavior inconsistent in Hive table when ‘path’ is provided in SERDEPROPERTIES

2021-10-18 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-37027:
-

[~yuzhousun] actually this is already fixed by SPARK-28266 in 3.1.3 and 3.2.0. 
Can you try compiling from latest {{master}} or using the 3.2.0 binaries (not 
yet officially released but 
[https://search.maven.org/search?q=g:org.apache.spark%20AND%20v:3.2.0|can be 
found on Maven Central]).

> Fix behavior inconsistent in Hive table when ‘path’ is provided in 
> SERDEPROPERTIES
> --
>
> Key: SPARK-37027
> URL: https://issues.apache.org/jira/browse/SPARK-37027
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 2.4.5, 3.1.2
>Reporter: Yuzhou Sun
>Priority: Trivial
> Attachments: SPARK-37027-test-example.patch
>
>
> If a Hive table is created with both {{WITH SERDEPROPERTIES 
> ('path'='')}} and {{LOCATION }}, Spark can 
> return doubled rows when reading the table. This issue seems to be an 
> extension of SPARK-30507.
>  Reproduce steps:
>  # Create table and insert records via Hive (Spark doesn't allow to insert 
> into table like this)
> {code:sql}
> CREATE TABLE `test_table`(
>   `c1` LONG,
>   `c2` STRING)
> ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
> WITH SERDEPROPERTIES ('path'=''" )
> STORED AS
>   INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
>   OUTPUTFORMAT 
> 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
> LOCATION '';
> INSERT INTO TABLE `test_table`
> VALUES (0, '0');
> SELECT * FROM `test_table`;
> -- will return
> -- 0 0
> {code}
>  # Read above table from Spark
> {code:sql}
> SELECT * FROM `test_table`;
> -- will return
> -- 0 0
> -- 0 0
> {code}
> But if we set {{spark.sql.hive.convertMetastoreParquet=false}}, Spark will 
> return same result as Hive (i.e. single row)
> A similar case is that, if a Hive table is created with both {{WITH 
> SERDEPROPERTIES ('path'='')}} and {{LOCATION }}, 
> Spark will read both rows under {{anotherPath}} and rows under 
> {{tableLocation}}, regardless of {{spark.sql.hive.convertMetastoreParquet}} 
> ‘s value. However, actually Hive seems to return only rows under 
> {{tableLocation}}
> Another similar case is that, if {{path}} is provided in {{TBLPROPERTIES}}, 
> Spark won’t double the rows when {{'path'=''}}. If 
> {{'path'=''}}, Spark will read both rows under {{anotherPath}} 
> and rows under {{tableLocation}}, Hive seems to keep ignoring the {{path}} in 
> {{TBLPROPERTIES}}
> Code examples for the above cases (diff patch wrote in 
> {{HiveParquetMetastoreSuite.scala}}) can be found in Attachments



--
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-33828) SQL Adaptive Query Execution QA

2021-10-18 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-33828:
-

[~dongjoon] as you mentioned above, this epic was initially for collecting 
issues to be resolved in 3.2.0. Now that release has been finalized, but we 
still have a few open issues here, and there are still new AQE issues being 
created (e.g. SPARK-37043 just today). Shall we keep this epic open and 
continue to use it, or create a new one targeted for 3.3.0? Or any other 
suggestions?

> SQL Adaptive Query Execution QA
> ---
>
> Key: SPARK-33828
> URL: https://issues.apache.org/jira/browse/SPARK-33828
> Project: Spark
>  Issue Type: Umbrella
>  Components: SQL
>Affects Versions: 3.2.0
>Reporter: Dongjoon Hyun
>Priority: Critical
>  Labels: releasenotes
>
> Since SPARK-31412 is delivered at 3.0.0, we received and handled many JIRA 
> issues at 3.0.x/3.1.0/3.2.0. This umbrella JIRA issue aims to enable it by 
> default and collect all information in order to do QA for this feature in 
> Apache Spark 3.2.0 timeframe.



--
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-36905) Reading Hive view without explicit column names fails in Spark

2021-10-04 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-36905:
-

cc also [~maropu] [~viirya] [~csun]

> Reading Hive view without explicit column names fails in Spark 
> ---
>
> Key: SPARK-36905
> URL: https://issues.apache.org/jira/browse/SPARK-36905
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.2.0
>Reporter: Shardul Mahadik
>Priority: Major
>
> Consider a Hive view in which some columns are not explicitly named
> {code:sql}
> CREATE VIEW test_view AS
> SELECT 1
> FROM some_table
> {code}
> Reading this view in Spark leads to an {{AnalysisException}}
> {code:java}
> org.apache.spark.sql.AnalysisException: cannot resolve '`_c0`' given input 
> columns: [1]
>   at 
> org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
>   at 
> org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$$nestedInanonfun$checkAnalysis$1$2.applyOrElse(CheckAnalysis.scala:188)
>   at 
> org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$$nestedInanonfun$checkAnalysis$1$2.applyOrElse(CheckAnalysis.scala:185)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformUp$2(TreeNode.scala:340)
>   at 
> org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:72)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:340)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformUp$1(TreeNode.scala:337)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:406)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:242)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:404)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:357)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:337)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformUp$1(TreeNode.scala:337)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:406)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:242)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:404)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:357)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:337)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$transformExpressionsUp$1(QueryPlan.scala:104)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$1(QueryPlan.scala:116)
>   at 
> org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:72)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpression$1(QueryPlan.scala:116)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:127)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$3(QueryPlan.scala:132)
>   at 
> scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
>   at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
>   at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
>   at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
>   at scala.collection.TraversableLike.map(TraversableLike.scala:238)
>   at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
>   at scala.collection.AbstractTraversable.map(Traversable.scala:108)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:132)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$4(QueryPlan.scala:137)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:242)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.mapExpressions(QueryPlan.scala:137)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpressionsUp(QueryPlan.scala:104)
>   at 
> org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis$1(CheckAnalysis.scala:185)
>   at 
> org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis$1$adapted(CheckAnalysis.scala:94)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:182)
>   at 
> org.apache.spark.sql.catalyst.analysis.CheckAnalysis.checkAnalysis(CheckAnalysis.scala:94)
>   at 
> org.apache.spark.sql.catalyst.analysis.CheckAnalysis.checkAnalysis$(CheckAnalysis.scala:91)
>   at 
> 

[jira] [Commented] (SPARK-36905) Reading Hive view without explicit column names fails in Spark

2021-09-30 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-36905:
-

[~shardulm] is important here that the view is from Hive? Can this be 
replicated in Spark w/o Hive?

> Reading Hive view without explicit column names fails in Spark 
> ---
>
> Key: SPARK-36905
> URL: https://issues.apache.org/jira/browse/SPARK-36905
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.2.0
>Reporter: Shardul Mahadik
>Priority: Major
>
> Consider a Hive view in which some columns are not explicitly named
> {code:sql}
> CREATE VIEW test_view AS
> SELECT 1
> FROM some_table
> {code}
> Reading this view in Spark leads to an {{AnalysisException}}
> {code:java}
> org.apache.spark.sql.AnalysisException: cannot resolve '`_c0`' given input 
> columns: [1]
>   at 
> org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
>   at 
> org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$$nestedInanonfun$checkAnalysis$1$2.applyOrElse(CheckAnalysis.scala:188)
>   at 
> org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$$nestedInanonfun$checkAnalysis$1$2.applyOrElse(CheckAnalysis.scala:185)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformUp$2(TreeNode.scala:340)
>   at 
> org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:72)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:340)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformUp$1(TreeNode.scala:337)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:406)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:242)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:404)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:357)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:337)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformUp$1(TreeNode.scala:337)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:406)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:242)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:404)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:357)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:337)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$transformExpressionsUp$1(QueryPlan.scala:104)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$1(QueryPlan.scala:116)
>   at 
> org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:72)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpression$1(QueryPlan.scala:116)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:127)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$3(QueryPlan.scala:132)
>   at 
> scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
>   at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
>   at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
>   at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
>   at scala.collection.TraversableLike.map(TraversableLike.scala:238)
>   at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
>   at scala.collection.AbstractTraversable.map(Traversable.scala:108)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:132)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$4(QueryPlan.scala:137)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:242)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.mapExpressions(QueryPlan.scala:137)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpressionsUp(QueryPlan.scala:104)
>   at 
> org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis$1(CheckAnalysis.scala:185)
>   at 
> org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis$1$adapted(CheckAnalysis.scala:94)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:182)
>   at 
> org.apache.spark.sql.catalyst.analysis.CheckAnalysis.checkAnalysis(CheckAnalysis.scala:94)
>   at 
> org.apache.spark.sql.catalyst.analysis.CheckAnalysis.checkAnalysis$(CheckAnalysis.scala:91)

[jira] [Commented] (SPARK-35672) Spark fails to launch executors with very large user classpath lists on YARN

2021-09-27 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-35672:
-

Re-submitted at [PR #34120|https://github.com/apache/spark/pull/34120]

> Spark fails to launch executors with very large user classpath lists on YARN
> 
>
> Key: SPARK-35672
> URL: https://issues.apache.org/jira/browse/SPARK-35672
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, YARN
>Affects Versions: 3.1.2
> Environment: Linux RHEL7
> Spark 3.1.1
>Reporter: Erik Krogen
>Assignee: Erik Krogen
>Priority: Major
>
> When running Spark on YARN, the {{user-class-path}} argument to 
> {{CoarseGrainedExecutorBackend}} is used to pass a list of user JAR URIs to 
> executor processes. The argument is specified once for each JAR, and the URIs 
> are fully-qualified, so the paths can be quite long. With large user JAR 
> lists (say 1000+), this can result in system-level argument length limits 
> being exceeded, typically manifesting as the error message:
> {code}
> /bin/bash: Argument list too long
> {code}
> A [Google 
> search|https://www.google.com/search?q=spark%20%22%2Fbin%2Fbash%3A%20argument%20list%20too%20long%22=spark%20%22%2Fbin%2Fbash%3A%20argument%20list%20too%20long%22]
>  indicates that this is not a theoretical problem and afflicts real users, 
> including ours. This issue was originally observed on Spark 2.3, but has been 
> confirmed to exist in the master branch as well.



--
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-35672) Spark fails to launch executors with very large user classpath lists on YARN

2021-09-24 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-35672:
-

Thanks [~petertoth] [~hyukjin.kwon] [~Gengliang.Wang] for reporting and dealing 
with the issue.

I'll work on submitting a new PR to master with the changes from PRs #31810 
(original) and #34084 (environment variable fix) incorporated.

> Spark fails to launch executors with very large user classpath lists on YARN
> 
>
> Key: SPARK-35672
> URL: https://issues.apache.org/jira/browse/SPARK-35672
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, YARN
>Affects Versions: 3.1.2
> Environment: Linux RHEL7
> Spark 3.1.1
>Reporter: Erik Krogen
>Assignee: Erik Krogen
>Priority: Major
>
> When running Spark on YARN, the {{user-class-path}} argument to 
> {{CoarseGrainedExecutorBackend}} is used to pass a list of user JAR URIs to 
> executor processes. The argument is specified once for each JAR, and the URIs 
> are fully-qualified, so the paths can be quite long. With large user JAR 
> lists (say 1000+), this can result in system-level argument length limits 
> being exceeded, typically manifesting as the error message:
> {code}
> /bin/bash: Argument list too long
> {code}
> A [Google 
> search|https://www.google.com/search?q=spark%20%22%2Fbin%2Fbash%3A%20argument%20list%20too%20long%22=spark%20%22%2Fbin%2Fbash%3A%20argument%20list%20too%20long%22]
>  indicates that this is not a theoretical problem and afflicts real users, 
> including ours. This issue was originally observed on Spark 2.3, but has been 
> confirmed to exist in the master branch as well.



--
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-36810) Handle HDFS read inconsistencies on Spark when observer Namenode is used

2021-09-20 Thread Erik Krogen (Jira)


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

Erik Krogen updated SPARK-36810:

Summary: Handle HDFS read inconsistencies on Spark when observer Namenode 
is used  (was: Handle HDSF read inconsistencies on Spark when observer Namenode 
is used)

> Handle HDFS read inconsistencies on Spark when observer Namenode is used
> 
>
> Key: SPARK-36810
> URL: https://issues.apache.org/jira/browse/SPARK-36810
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, SQL
>Affects Versions: 3.2.0
>Reporter: Venkata krishnan Sowrirajan
>Priority: Major
>
> In short, with HDFS HA and with the use of [Observer 
> Namenode|https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/ObserverNameNode.html]
>  the read-after-write consistency is only available when both the write and 
> the read happens from the same client.
> But if the write happens on executor and the read happens on the driver, then 
> the reads would be inconsistent causing application failure issues. This can 
> be fixed by calling `FileSystem.msync` before making any read calls where the 
> client thinks the write could have possibly happened elsewhere.
> This issue is discussed in greater detail in this 
> [discussion|https://mail-archives.apache.org/mod_mbox/spark-dev/202108.mbox/browser]
>  



--
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-36810) Handle HDSF read inconsistencies on Spark when observer Namenode is used

2021-09-20 Thread Erik Krogen (Jira)


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

Erik Krogen updated SPARK-36810:

Description: 
In short, with HDFS HA and with the use of [Observer 
Namenode|https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/ObserverNameNode.html]
 the read-after-write consistency is only available when both the write and the 
read happens from the same client.

But if the write happens on executor and the read happens on the driver, then 
the reads would be inconsistent causing application failure issues. This can be 
fixed by calling `FileSystem.msync` before making any read calls where the 
client thinks the write could have possibly happened elsewhere.

This issue is discussed in greater detail in this 
[discussion|https://mail-archives.apache.org/mod_mbox/spark-dev/202108.mbox/browser]
 

  was:
In short, with HDFS HA and with the use of [Observer 
Namenode|[https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/ObserverNameNode.html],]
 the read-after-write consistency is only available when both the write and the 
read happens from the same client.

But if the write happens on executor and the read happens on the driver, then 
the reads would be inconsistent causing application failure issues. This can be 
fixed by calling `FileSystem.msync` before making any read calls where the 
client thinks the write could have possibly happened elsewhere.

This issue is discussed in greater detail in this 
[discussion|https://mail-archives.apache.org/mod_mbox/spark-dev/202108.mbox/browser]
 


> Handle HDSF read inconsistencies on Spark when observer Namenode is used
> 
>
> Key: SPARK-36810
> URL: https://issues.apache.org/jira/browse/SPARK-36810
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, SQL
>Affects Versions: 3.2.0
>Reporter: Venkata krishnan Sowrirajan
>Priority: Major
>
> In short, with HDFS HA and with the use of [Observer 
> Namenode|https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/ObserverNameNode.html]
>  the read-after-write consistency is only available when both the write and 
> the read happens from the same client.
> But if the write happens on executor and the read happens on the driver, then 
> the reads would be inconsistent causing application failure issues. This can 
> be fixed by calling `FileSystem.msync` before making any read calls where the 
> client thinks the write could have possibly happened elsewhere.
> This issue is discussed in greater detail in this 
> [discussion|https://mail-archives.apache.org/mod_mbox/spark-dev/202108.mbox/browser]
>  



--
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-36673) Incorrect Unions of struct with mismatched field name case

2021-09-09 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-36673:
-

>From the Scaladoc for {{union}}:
{code}
   * Also as standard in SQL, this function resolves columns by position (not 
by name):
{code}
So it seems to me we should be ignoring the names altogether and just doing 
positional matching.

{quote}
AFAIK, in SQL the names in the struct are case sensitive, while the name of the 
normal fields are not.
{quote}
I was under the impression that nested fields are also case-insensitive. Do we 
have any documentation around this?

In any case, when we fix this, we need to be careful about {{unionByName}}, 
including the new-ish {{allowMissingColumns}} option...

> Incorrect Unions of struct with mismatched field name case
> --
>
> Key: SPARK-36673
> URL: https://issues.apache.org/jira/browse/SPARK-36673
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.1.1, 3.2.0
>Reporter: Shardul Mahadik
>Priority: Major
>
> If a nested field has different casing on two sides of the union, the 
> resultant schema of the union will both fields in its schemaa
> {code:java}
> scala> val df1 = spark.range(2).withColumn("nested", struct(expr("id * 5 AS 
> INNER")))
> df1: org.apache.spark.sql.DataFrame = [id: bigint, nested: struct bigint>]
> val df2 = spark.range(2).withColumn("nested", struct(expr("id * 5 AS inner")))
> df2: org.apache.spark.sql.DataFrame = [id: bigint, nested: struct bigint>]
> scala> df1.union(df2).printSchema
> root
>  |-- id: long (nullable = false)
>  |-- nested: struct (nullable = false)
>  ||-- INNER: long (nullable = false)
>  ||-- inner: long (nullable = false)
>  {code}
> This seems like a bug. I would expect that Spark SQL would either just union 
> by index or if the user has requested {{unionByName}}, then it should matched 
> fields case insensitively if {{spark.sql.caseSensitive}} is {{false}}.
> However the output data only has one nested column
> {code:java}
> scala> df1.union(df2).show()
> +---+--+
> | id|nested|
> +---+--+
> |  0|   {0}|
> |  1|   {5}|
> |  0|   {0}|
> |  1|   {5}|
> +---+--+
> {code}
> Trying to project fields of {{nested}} throws an error:
> {code:java}
> scala> df1.union(df2).select("nested.*").show()
> java.lang.ArrayIndexOutOfBoundsException: 1
>   at org.apache.spark.sql.types.StructType.apply(StructType.scala:414)
>   at 
> org.apache.spark.sql.catalyst.expressions.GetStructField.dataType(complexTypeExtractors.scala:108)
>   at 
> org.apache.spark.sql.catalyst.expressions.Alias.toAttribute(namedExpressions.scala:192)
>   at 
> org.apache.spark.sql.catalyst.plans.logical.Project.$anonfun$output$1(basicLogicalOperators.scala:63)
>   at 
> scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
>   at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
>   at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
>   at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
>   at scala.collection.TraversableLike.map(TraversableLike.scala:238)
>   at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
>   at scala.collection.AbstractTraversable.map(Traversable.scala:108)
>   at 
> org.apache.spark.sql.catalyst.plans.logical.Project.output(basicLogicalOperators.scala:63)
>   at 
> org.apache.spark.sql.catalyst.plans.logical.Union.$anonfun$output$3(basicLogicalOperators.scala:260)
>   at 
> scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
>   at scala.collection.immutable.List.foreach(List.scala:392)
>   at scala.collection.TraversableLike.map(TraversableLike.scala:238)
>   at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
>   at scala.collection.immutable.List.map(List.scala:298)
>   at 
> org.apache.spark.sql.catalyst.plans.logical.Union.output(basicLogicalOperators.scala:260)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.outputSet$lzycompute(QueryPlan.scala:49)
>   at 
> org.apache.spark.sql.catalyst.plans.QueryPlan.outputSet(QueryPlan.scala:49)
>   at 
> org.apache.spark.sql.catalyst.optimizer.ColumnPruning$$anonfun$apply$8.applyOrElse(Optimizer.scala:747)
>   at 
> org.apache.spark.sql.catalyst.optimizer.ColumnPruning$$anonfun$apply$8.applyOrElse(Optimizer.scala:695)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$1(TreeNode.scala:316)
>   at 
> org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:72)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:316)
>   at 
> 

[jira] [Commented] (SPARK-32333) Drop references to Master

2021-08-27 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-32333:
-

Personally, tackling publicly-facing things is my highest priority. This blocks 
other projects/users from being able to pursue their own efforts in this area. 
For example, internally we have code with references to the {{--master}} 
argument, and can't drop these references because of lack of an alternative. So 
this would be (1) above, though I'll note that things like 
{{SparkConf.setMaster}} are used beyond the context of standalone mode.

For (2) / (3), I'm actually kind of surprised to see there's not _too_ many:
{code}
. find . -name "*.scala" -or -name "*.java" | xargs grep -o -i -E "class 
[[:alpha:][:digit:]]*master[[:alpha:][:digit:]]*"
./resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ApplicationMasterSuite.scala:class
 ApplicationMasterSuite
./resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala:class
 ApplicationMasterArguments
./resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala:class
 ApplicationMaster
./resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterSource.scala:class
 ApplicationMasterSource
./core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala:class
 MasterWebUISuite
./core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala:class 
MasterSuite
./core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala:class
 DummyMaster
./core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala:class
 SmarterMaster
./core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:class 
MyBlockManagerMaster
./core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:class 
MyMapOutputTrackerMaster
./core/src/test/scala/org/apache/spark/storage/BlockManagerMasterSuite.scala:class
 BlockManagerMasterSuite
./core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala:class 
MasterChangeAcknowledged
./core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala:class 
MasterChanged
./core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala:class 
MasterStateResponse
./core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala:class 
TestMasterInfo
./core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala:class 
MasterPage
./core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala:class 
MasterWebUI
./core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala:class
 MasterArguments
./core/src/main/scala/org/apache/spark/deploy/master/Master.scala:class Master
./core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala:class 
MasterSource
./core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala:class 
BlockManagerMaster
./core/src/main/scala/org/apache/spark/storage/BlockManagerMasterHeartbeatEndpoint.scala:class
 BlockManagerMasterHeartbeatEndpoint
./core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala:class
 BlockManagerMasterEndpoint
./core/src/main/scala/org/apache/spark/MapOutputTracker.scala:class 
MapOutputTrackerMasterEndpoint
./core/src/main/scala/org/apache/spark/MapOutputTracker.scala:class 
MapOutputTrackerMaster
{code}
So it seems that it's likely reasonable to tackle given the breakdown you've 
outlined. Though the entire {{org.apache.spark.deploy.master}} package will 
also have to be renamed as part of (2).

> Drop references to Master
> -
>
> Key: SPARK-32333
> URL: https://issues.apache.org/jira/browse/SPARK-32333
> Project: Spark
>  Issue Type: Improvement
>  Components: Spark Core, SQL
>Affects Versions: 3.0.0
>Reporter: Thomas Graves
>Priority: Major
>
> We have a lot of references to "master" in the code base. It will be 
> beneficial to remove references to problematic language that can alienate 
> potential community members. 
> SPARK-32004 removed references to slave
>  
> Here is a IETF draft to fix up some of the most egregious examples
> (master/slave, whitelist/backlist) with proposed alternatives.
> https://tools.ietf.org/id/draft-knodel-terminology-00.html#rfc.section.1.1.1



--
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-36416) Add SQL metrics to AdaptiveSparkPlanExec for BHJs and Skew joins

2021-08-04 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-36416:
-

+1 this would be very helpful!

> Add SQL metrics to AdaptiveSparkPlanExec for BHJs and Skew joins
> 
>
> Key: SPARK-36416
> URL: https://issues.apache.org/jira/browse/SPARK-36416
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 3.1.2
>Reporter: Eugene Koifman
>Priority: Major
>
> Add {{"num broadcast joins conversions"}} and {{"num skew join conversions"}}
>  metrics to {{AdaptiveSparkPlanExec}} so that it's easy to get a sense of how 
> much impact AQE had on an a complex query.
> It's also useful for systems that collect metrics later analysis of AQE 
> effectiveness in large production deployment.



--
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-36134) jackson-databind RCE vulnerability

2021-07-16 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-36134:
-

Whoops, must have missed the 3.1.2 release :) Thanks for correcting me.

Still, 3.1.2 is using Jackson 2.10.0, so I don't see where the CVE report is 
coming from. Can you elaborate?

> jackson-databind RCE vulnerability
> --
>
> Key: SPARK-36134
> URL: https://issues.apache.org/jira/browse/SPARK-36134
> Project: Spark
>  Issue Type: Task
>  Components: Java API
>Affects Versions: 3.1.2, 3.1.3
>Reporter: Sumit
>Priority: Major
> Attachments: Screenshot 2021-07-15 at 1.00.55 PM.png
>
>
> Need to upgrade   jackson-databind version to *2.9.3.1*
> At the beginning of 2018, jackson-databind was reported to contain another 
> remote code execution (RCE) vulnerability (CVE-2017-17485) that affects 
> versions 2.9.3 and earlier, 2.7.9.1 and earlier, and 2.8.10 and earlier. This 
> vulnerability is caused by jackson-dababind’s incomplete blacklist. An 
> application that uses jackson-databind will become vulnerable when the 
> enableDefaultTyping method is called via the ObjectMapper object within the 
> application. An attacker can thus compromise the application by sending 
> maliciously crafted JSON input to gain direct control over a server. 
> Currently, a proof of concept (POC) exploit for this vulnerability has been 
> publicly available. All users who are affected by this vulnerability should 
> upgrade to the latest versions as soon as possible to fix this issue.



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

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



[jira] [Commented] (SPARK-36134) jackson-databind RCE vulnerability

2021-07-15 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-36134:
-

3.1.2 doesn't exist yet, the only release in the 3.1 line is 3.1.1. How are you 
using 3.1.2?

Regardless, the jackson-databind version used by 3.1.1 (and the 3.1 line 
generally) is 2.10.0: 
https://github.com/apache/spark/blob/1d550c4e90275ab418b9161925049239227f3dc9/pom.xml#L172

> jackson-databind RCE vulnerability
> --
>
> Key: SPARK-36134
> URL: https://issues.apache.org/jira/browse/SPARK-36134
> Project: Spark
>  Issue Type: Task
>  Components: Java API
>Affects Versions: 3.1.2, 3.1.3
>Reporter: Sumit
>Priority: Major
> Attachments: Screenshot 2021-07-15 at 1.00.55 PM.png
>
>
> Need to upgrade   jackson-databind version to *2.9.3.1*
> At the beginning of 2018, jackson-databind was reported to contain another 
> remote code execution (RCE) vulnerability (CVE-2017-17485) that affects 
> versions 2.9.3 and earlier, 2.7.9.1 and earlier, and 2.8.10 and earlier. This 
> vulnerability is caused by jackson-dababind’s incomplete blacklist. An 
> application that uses jackson-databind will become vulnerable when the 
> enableDefaultTyping method is called via the ObjectMapper object within the 
> application. An attacker can thus compromise the application by sending 
> maliciously crafted JSON input to gain direct control over a server. 
> Currently, a proof of concept (POC) exploit for this vulnerability has been 
> publicly available. All users who are affected by this vulnerability should 
> upgrade to the latest versions as soon as possible to fix this issue.



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

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



[jira] [Commented] (SPARK-36134) jackson-databind RCE vulnerability [Need to upgrade to 2.9.3.1]

2021-07-14 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-36134:
-

Jackson is already 2.12.3 (from 
[pom.xml|https://github.com/apache/spark/blob/fd06cc211d7d1579067ad717da9976aabd71b70d/pom.xml#L170]):
{code}
2.12.3
{code}
So what's the issue?

> jackson-databind RCE vulnerability [Need to upgrade to 2.9.3.1]
> ---
>
> Key: SPARK-36134
> URL: https://issues.apache.org/jira/browse/SPARK-36134
> Project: Spark
>  Issue Type: Task
>  Components: Java API
>Affects Versions: 3.1.2, 3.1.3
>Reporter: Sumit
>Priority: Major
>
> Need to upgrade   jackson-databind version to *2.9.3.1*
> At the beginning of 2018, jackson-databind was reported to contain another 
> remote code execution (RCE) vulnerability (CVE-2017-17485) that affects 
> versions 2.9.3 and earlier, 2.7.9.1 and earlier, and 2.8.10 and earlier. This 
> vulnerability is caused by jackson-dababind’s incomplete blacklist. An 
> application that uses jackson-databind will become vulnerable when the 
> enableDefaultTyping method is called via the ObjectMapper object within the 
> application. An attacker can thus compromise the application by sending 
> maliciously crafted JSON input to gain direct control over a server. 
> Currently, a proof of concept (POC) exploit for this vulnerability has been 
> publicly available. All users who are affected by this vulnerability should 
> upgrade to the latest versions as soon as possible to fix this issue.



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

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



[jira] [Reopened] (SPARK-28266) data duplication when `path` serde property is present

2021-07-13 Thread Erik Krogen (Jira)


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

Erik Krogen reopened SPARK-28266:
-

Re-opening this issue based on [~shardulm]'s example above demonstrating that 
this is indeed a real-world issue. We have faced this issue internally on a 
number of occasions and have been operating a fix, on which [~shardulm]'s PR is 
based, for a few years.

> data duplication when `path` serde property is present
> --
>
> Key: SPARK-28266
> URL: https://issues.apache.org/jira/browse/SPARK-28266
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 2.2.0, 2.2.1, 2.2.2
>Reporter: Ruslan Dautkhanov
>Priority: Major
>  Labels: correctness
>
> Spark duplicates returned datasets when `path` serde is present in a parquet 
> table. 
> Confirmed versions affected: Spark 2.2, Spark 2.3, Spark 2.4.
> Confirmed unaffected versions: Spark 2.1 and earlier (tested with Spark 1.6 
> at least).
> Reproducer:
> {code:python}
> >>> spark.sql("create table ruslan_test.test55 as select 1 as id")
> DataFrame[]
> >>> spark.table("ruslan_test.test55").explain()
> == Physical Plan ==
> HiveTableScan [id#16], HiveTableRelation `ruslan_test`.`test55`, 
> org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [id#16]
> >>> spark.table("ruslan_test.test55").count()
> 1
> {code}
> (all is good at this point, now exist session and run in Hive for example - )
> {code:sql}
> ALTER TABLE ruslan_test.test55 SET SERDEPROPERTIES ( 
> 'path'='hdfs://epsdatalake/hivewarehouse/ruslan_test.db/test55' )
> {code}
> So LOCATION and serde `path` property would point to the same location.
> Now see count returns two records instead of one:
> {code:python}
> >>> spark.table("ruslan_test.test55").count()
> 2
> >>> spark.table("ruslan_test.test55").explain()
> == Physical Plan ==
> *(1) FileScan parquet ruslan_test.test55[id#9] Batched: true, Format: 
> Parquet, Location: 
> InMemoryFileIndex[hdfs://epsdatalake/hivewarehouse/ruslan_test.db/test55, 
> hdfs://epsdatalake/hive..., PartitionFilters: [], PushedFilters: [], 
> ReadSchema: struct
> >>>
> {code}
> Also notice that the presence of `path` serde property makes TABLE location 
> show up twice - 
> {quote}
> InMemoryFileIndex[hdfs://epsdatalake/hivewarehouse/ruslan_test.db/test55, 
> hdfs://epsdatalake/hive..., 
> {quote}
> We have some applications that create parquet tables in Hive with `path` 
> serde property
> and it makes data duplicate in query results. 
> Hive, Impala etc and Spark version 2.1 and earlier read such tables fine, but 
> not Spark 2.2 and later releases.



--
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-35957) Cannot convert Avro schema to catalyst type because schema at path is not compatible

2021-07-13 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-35957:
-

[~jkdll] would it be possible for you to try against the latest master? Some of 
this logic has been changing recently and that will help with debugging. A 
minimum schema / example to help reproduce would also be helpful.

> Cannot convert Avro schema to catalyst type because schema at path is not 
> compatible
> 
>
> Key: SPARK-35957
> URL: https://issues.apache.org/jira/browse/SPARK-35957
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, SQL
>Affects Versions: 3.0.0
>Reporter: Jake Dalli
>Priority: Major
>
> * The Apache Avro specification has a *null* primitive type.
>  * Using org.apache.spark:spark-avro_2.12:3.0.3 on Spark 3.0.0 with Scala 2.12
>  * I try to load an avro schema with the a field defined as follows:
>  
> {code:java}
> {
>   "name": "messageKey",
>   "type": "null"
> },
> {code}
>  * I get the following error:
> {code:java}
> ERROR Client: Application diagnostics message: User class threw exception: 
> org.apache.spark.sql.avro.IncompatibleSchemaException: Unsupported type NULL
> {code}
> This issue is experienced when using Apache Hudi 0.7.0.
> Full stack trace:
> {code:java}
> Caused by: org.apache.spark.SparkException: Job aborted due to stage failure: 
> Task 0 in stage 1.0 failed 4 times, most recent failure: Lost task 0.3 in 
> stage 1.0 (TID 4, ip-10-102-8-124.eu-central-1.compute.internal, executor 1): 
> org.apache.spark.sql.avro.IncompatibleSchemaException: Cannot convert Avro 
> schema to catalyst type because schema at path messageKey is not compatible 
> (avroType = NullType, sqlType = NULL).
> Source Avro Schema: ...
> Target Catalyst type: ...
> at 
> org.apache.hudi.AvroConversionHelper$.createConverter$1(AvroConversionHelper.scala:265)
> at 
> org.apache.hudi.AvroConversionHelper$.createConverter$1(AvroConversionHelper.scala:146)
> at 
> org.apache.hudi.AvroConversionHelper$.createConverterToRow(AvroConversionHelper.scala:273)
> at 
> org.apache.hudi.AvroConversionUtils$.$anonfun$createDataFrame$1(AvroConversionUtils.scala:42)
> at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2(RDD.scala:837)
> at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2$adapted(RDD.scala:837)
> at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
> at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
> at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
> at 
> org.apache.spark.sql.execution.SQLExecutionRDD.$anonfun$compute$1(SQLExecutionRDD.scala:52)
> at 
> org.apache.spark.sql.internal.SQLConf$.withExistingConf(SQLConf.scala:100)
> at 
> org.apache.spark.sql.execution.SQLExecutionRDD.compute(SQLExecutionRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
> at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
> at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
> 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)
> Driver stacktrace:
> at 
> org.apache.spark.scheduler.DAGScheduler.failJobAndIndependentStages(DAGScheduler.scala:2175)
> at 
> 

[jira] [Comment Edited] (SPARK-35957) Cannot convert Avro schema to catalyst type because schema at path is not compatible

2021-07-12 Thread Erik Krogen (Jira)


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

Erik Krogen edited comment on SPARK-35957 at 7/12/21, 5:10 PM:
---

Based on the discussion in [the linked Hudi 
issue|https://github.com/apache/hudi/issues/3113], it sounds like you 
encountered two different issues:
{code}
ERROR Client: Application diagnostics message: User class threw exception: 
org.apache.spark.sql.avro.IncompatibleSchemaException: Unsupported type NULL
{code}
This was based on Spark 2.4.8, and is expected, because support for Avro NULL 
types was not added until SPARK-26765 in 3.0.0.

{code}
org.apache.spark.sql.avro.IncompatibleSchemaException: Cannot convert Avro 
schema to catalyst type because schema at path messageKey is not compatible 
(avroType = NullType, sqlType = NULL)
{code}
This error is being thrown by Hudi, not Spark, so I don't understand why you've 
opened a Spark JIRA for it. It also appears that it is a Hudi bug based on the 
[PR opened to fix it|https://github.com/apache/hudi/pull/3195]. Can you 
elaborate if there you actually suspect any issue on the Spark side, and if so, 
provide more information on how to reproduce this within Spark w/o the Hudi 
context?


was (Author: xkrogen):
Based on the discussion in [the linked Hudi 
issue|https://github.com/apache/hudi/issues/3113], it sounds like you've 
creating this JIRA based on two different issues:
{code}
ERROR Client: Application diagnostics message: User class threw exception: 
org.apache.spark.sql.avro.IncompatibleSchemaException: Unsupported type NULL
{code}
This was based on Spark 2.4.8, and is expected, because support for Avro NULL 
types was not added until SPARK-26765 in 3.0.0.

{code}
org.apache.spark.sql.avro.IncompatibleSchemaException: Cannot convert Avro 
schema to catalyst type because schema at path messageKey is not compatible 
(avroType = NullType, sqlType = NULL)
{code}
This error is being thrown by Hudi, not Spark, so I don't understand why you've 
opened a Spark JIRA for it. It also appears that it is a Hudi bug based on the 
[PR opened to fix it|https://github.com/apache/hudi/pull/3195]. Can you 
elaborate if there you actually suspect any issue on the Spark side, and if so, 
provide more information on how to reproduce this within Spark w/o the Hudi 
context?

> Cannot convert Avro schema to catalyst type because schema at path is not 
> compatible
> 
>
> Key: SPARK-35957
> URL: https://issues.apache.org/jira/browse/SPARK-35957
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, SQL
>Affects Versions: 3.0.0
>Reporter: Jake Dalli
>Priority: Major
>
> * The Apache Avro specification has a *null* primitive type.
>  * Using org.apache.spark:spark-avro_2.12:3.0.3 on Spark 3.0.0 with Scala 2.12
>  * I try to load an avro schema with the a field defined as follows:
>  
> {code:java}
> {
>   "name": "messageKey",
>   "type": "null"
> },
> {code}
>  * I get the following error:
> {code:java}
> ERROR Client: Application diagnostics message: User class threw exception: 
> org.apache.spark.sql.avro.IncompatibleSchemaException: Unsupported type NULL
> {code}
> This issue is experienced when using Apache Hudi 0.7.0.
> Full stack trace:
> {code:java}
> Caused by: org.apache.spark.SparkException: Job aborted due to stage failure: 
> Task 0 in stage 1.0 failed 4 times, most recent failure: Lost task 0.3 in 
> stage 1.0 (TID 4, ip-10-102-8-124.eu-central-1.compute.internal, executor 1): 
> org.apache.spark.sql.avro.IncompatibleSchemaException: Cannot convert Avro 
> schema to catalyst type because schema at path messageKey is not compatible 
> (avroType = NullType, sqlType = NULL).
> Source Avro Schema: ...
> Target Catalyst type: ...
> at 
> org.apache.hudi.AvroConversionHelper$.createConverter$1(AvroConversionHelper.scala:265)
> at 
> org.apache.hudi.AvroConversionHelper$.createConverter$1(AvroConversionHelper.scala:146)
> at 
> org.apache.hudi.AvroConversionHelper$.createConverterToRow(AvroConversionHelper.scala:273)
> at 
> org.apache.hudi.AvroConversionUtils$.$anonfun$createDataFrame$1(AvroConversionUtils.scala:42)
> at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2(RDD.scala:837)
> at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2$adapted(RDD.scala:837)
> at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
> at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
>  

[jira] [Comment Edited] (SPARK-35957) Cannot convert Avro schema to catalyst type because schema at path is not compatible

2021-07-12 Thread Erik Krogen (Jira)


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

Erik Krogen edited comment on SPARK-35957 at 7/12/21, 5:10 PM:
---

Based on the discussion in [the linked Hudi 
issue|https://github.com/apache/hudi/issues/3113], it sounds like you 
encountered two different issues:
{code}
ERROR Client: Application diagnostics message: User class threw exception: 
org.apache.spark.sql.avro.IncompatibleSchemaException: Unsupported type NULL
{code}
This was based on Spark 2.4.8, and is expected, because support for Avro NULL 
types was not added until SPARK-26765 in 3.0.0.

Then, after upgrading to Spark 3.0.0:
{code}
org.apache.spark.sql.avro.IncompatibleSchemaException: Cannot convert Avro 
schema to catalyst type because schema at path messageKey is not compatible 
(avroType = NullType, sqlType = NULL)
{code}
This error is being thrown by Hudi, not Spark, so I don't understand why you've 
opened a Spark JIRA for it. It also appears that it is a Hudi bug based on the 
[PR opened to fix it|https://github.com/apache/hudi/pull/3195]. Can you 
elaborate if there you actually suspect any issue on the Spark side, and if so, 
provide more information on how to reproduce this within Spark w/o the Hudi 
context?


was (Author: xkrogen):
Based on the discussion in [the linked Hudi 
issue|https://github.com/apache/hudi/issues/3113], it sounds like you 
encountered two different issues:
{code}
ERROR Client: Application diagnostics message: User class threw exception: 
org.apache.spark.sql.avro.IncompatibleSchemaException: Unsupported type NULL
{code}
This was based on Spark 2.4.8, and is expected, because support for Avro NULL 
types was not added until SPARK-26765 in 3.0.0.

{code}
org.apache.spark.sql.avro.IncompatibleSchemaException: Cannot convert Avro 
schema to catalyst type because schema at path messageKey is not compatible 
(avroType = NullType, sqlType = NULL)
{code}
This error is being thrown by Hudi, not Spark, so I don't understand why you've 
opened a Spark JIRA for it. It also appears that it is a Hudi bug based on the 
[PR opened to fix it|https://github.com/apache/hudi/pull/3195]. Can you 
elaborate if there you actually suspect any issue on the Spark side, and if so, 
provide more information on how to reproduce this within Spark w/o the Hudi 
context?

> Cannot convert Avro schema to catalyst type because schema at path is not 
> compatible
> 
>
> Key: SPARK-35957
> URL: https://issues.apache.org/jira/browse/SPARK-35957
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, SQL
>Affects Versions: 3.0.0
>Reporter: Jake Dalli
>Priority: Major
>
> * The Apache Avro specification has a *null* primitive type.
>  * Using org.apache.spark:spark-avro_2.12:3.0.3 on Spark 3.0.0 with Scala 2.12
>  * I try to load an avro schema with the a field defined as follows:
>  
> {code:java}
> {
>   "name": "messageKey",
>   "type": "null"
> },
> {code}
>  * I get the following error:
> {code:java}
> ERROR Client: Application diagnostics message: User class threw exception: 
> org.apache.spark.sql.avro.IncompatibleSchemaException: Unsupported type NULL
> {code}
> This issue is experienced when using Apache Hudi 0.7.0.
> Full stack trace:
> {code:java}
> Caused by: org.apache.spark.SparkException: Job aborted due to stage failure: 
> Task 0 in stage 1.0 failed 4 times, most recent failure: Lost task 0.3 in 
> stage 1.0 (TID 4, ip-10-102-8-124.eu-central-1.compute.internal, executor 1): 
> org.apache.spark.sql.avro.IncompatibleSchemaException: Cannot convert Avro 
> schema to catalyst type because schema at path messageKey is not compatible 
> (avroType = NullType, sqlType = NULL).
> Source Avro Schema: ...
> Target Catalyst type: ...
> at 
> org.apache.hudi.AvroConversionHelper$.createConverter$1(AvroConversionHelper.scala:265)
> at 
> org.apache.hudi.AvroConversionHelper$.createConverter$1(AvroConversionHelper.scala:146)
> at 
> org.apache.hudi.AvroConversionHelper$.createConverterToRow(AvroConversionHelper.scala:273)
> at 
> org.apache.hudi.AvroConversionUtils$.$anonfun$createDataFrame$1(AvroConversionUtils.scala:42)
> at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2(RDD.scala:837)
> at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2$adapted(RDD.scala:837)
> at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
> at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at 

[jira] [Commented] (SPARK-35957) Cannot convert Avro schema to catalyst type because schema at path is not compatible

2021-07-12 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-35957:
-

Based on the discussion in [the linked Hudi 
issue|https://github.com/apache/hudi/issues/3113], it sounds like you've 
creating this JIRA based on two different issues:
{code}
ERROR Client: Application diagnostics message: User class threw exception: 
org.apache.spark.sql.avro.IncompatibleSchemaException: Unsupported type NULL
{code}
This was based on Spark 2.4.8, and is expected, because support for Avro NULL 
types was not added until SPARK-26765 in 3.0.0.

{code}
org.apache.spark.sql.avro.IncompatibleSchemaException: Cannot convert Avro 
schema to catalyst type because schema at path messageKey is not compatible 
(avroType = NullType, sqlType = NULL)
{code}
This error is being thrown by Hudi, not Spark, so I don't understand why you've 
opened a Spark JIRA for it. It also appears that it is a Hudi bug based on the 
[PR opened to fix it|https://github.com/apache/hudi/pull/3195]. Can you 
elaborate if there you actually suspect any issue on the Spark side, and if so, 
provide more information on how to reproduce this within Spark w/o the Hudi 
context?

> Cannot convert Avro schema to catalyst type because schema at path is not 
> compatible
> 
>
> Key: SPARK-35957
> URL: https://issues.apache.org/jira/browse/SPARK-35957
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, SQL
>Affects Versions: 3.0.0
>Reporter: Jake Dalli
>Priority: Major
>
> * The Apache Avro specification has a *null* primitive type.
>  * Using org.apache.spark:spark-avro_2.12:3.0.3 on Spark 3.0.0 with Scala 2.12
>  * I try to load an avro schema with the a field defined as follows:
>  
> {code:java}
> {
>   "name": "messageKey",
>   "type": "null"
> },
> {code}
>  * I get the following error:
> {code:java}
> ERROR Client: Application diagnostics message: User class threw exception: 
> org.apache.spark.sql.avro.IncompatibleSchemaException: Unsupported type NULL
> {code}
> This issue is experienced when using Apache Hudi 0.7.0.
> Full stack trace:
> {code:java}
> Caused by: org.apache.spark.SparkException: Job aborted due to stage failure: 
> Task 0 in stage 1.0 failed 4 times, most recent failure: Lost task 0.3 in 
> stage 1.0 (TID 4, ip-10-102-8-124.eu-central-1.compute.internal, executor 1): 
> org.apache.spark.sql.avro.IncompatibleSchemaException: Cannot convert Avro 
> schema to catalyst type because schema at path messageKey is not compatible 
> (avroType = NullType, sqlType = NULL).
> Source Avro Schema: ...
> Target Catalyst type: ...
> at 
> org.apache.hudi.AvroConversionHelper$.createConverter$1(AvroConversionHelper.scala:265)
> at 
> org.apache.hudi.AvroConversionHelper$.createConverter$1(AvroConversionHelper.scala:146)
> at 
> org.apache.hudi.AvroConversionHelper$.createConverterToRow(AvroConversionHelper.scala:273)
> at 
> org.apache.hudi.AvroConversionUtils$.$anonfun$createDataFrame$1(AvroConversionUtils.scala:42)
> at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2(RDD.scala:837)
> at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2$adapted(RDD.scala:837)
> at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
> at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
> at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
> at 
> org.apache.spark.sql.execution.SQLExecutionRDD.$anonfun$compute$1(SQLExecutionRDD.scala:52)
> at 
> org.apache.spark.sql.internal.SQLConf$.withExistingConf(SQLConf.scala:100)
> at 
> org.apache.spark.sql.execution.SQLExecutionRDD.compute(SQLExecutionRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
> at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
> at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
> 

[jira] [Commented] (SPARK-32333) Drop references to Master

2021-07-12 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-32333:
-

+1 on leader from my side

> Drop references to Master
> -
>
> Key: SPARK-32333
> URL: https://issues.apache.org/jira/browse/SPARK-32333
> Project: Spark
>  Issue Type: Improvement
>  Components: Spark Core, SQL
>Affects Versions: 3.0.0
>Reporter: Thomas Graves
>Priority: Major
>
> We have a lot of references to "master" in the code base. It will be 
> beneficial to remove references to problematic language that can alienate 
> potential community members. 
> SPARK-32004 removed references to slave
>  
> Here is a IETF draft to fix up some of the most egregious examples
> (master/slave, whitelist/backlist) with proposed alternatives.
> https://tools.ietf.org/id/draft-knodel-terminology-00.html#rfc.section.1.1.1



--
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-35918) Consolidate logic between AvroSerializer/AvroDeserializer for schema mismatch handling and error messages

2021-06-28 Thread Erik Krogen (Jira)
Erik Krogen created SPARK-35918:
---

 Summary: Consolidate logic between AvroSerializer/AvroDeserializer 
for schema mismatch handling and error messages
 Key: SPARK-35918
 URL: https://issues.apache.org/jira/browse/SPARK-35918
 Project: Spark
  Issue Type: Improvement
  Components: SQL
Affects Versions: 3.1.2
Reporter: Erik Krogen


While working on [PR #31490|https://github.com/apache/spark/pull/31490] for 
SPARK-34365, we discussed that there is room for improvement in how schema 
mismatch errors are reported 
([comment1|https://github.com/apache/spark/pull/31490#discussion_r659970793], 
[comment2|https://github.com/apache/spark/pull/31490#issuecomment-869866848]). 
We can also consolidate more of the logic between AvroSerializer and 
AvroDeserializer to avoid some duplication of error handling and consolidate 
how these error messages are generated.

This will essentially be taking the [logic from the initial proposal from PR 
#31490|https://github.com/apache/spark/pull/31490/commits/83a922fdff08528e59233f67930ac78bfb3fa178],
 but applied separately from the current set of proposed changes to cut down on 
PR size.



--
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-35259) ExternalBlockHandler metrics have misleading unit in the name

2021-06-28 Thread Erik Krogen (Jira)


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

Erik Krogen updated SPARK-35259:

Description: 
Today {{ExternalBlockHandler}} exposes a few {{Timer}} metrics:
{code}
// Time latency for open block request in ms
private final Timer openBlockRequestLatencyMillis = new Timer();
// Time latency for executor registration latency in ms
private final Timer registerExecutorRequestLatencyMillis = new Timer();
// Time latency for processing fetch merged blocks meta request latency in 
ms
private final Timer fetchMergedBlocksMetaLatencyMillis = new Timer();
// Time latency for processing finalize shuffle merge request latency in ms
private final Timer finalizeShuffleMergeLatencyMillis = new Timer();
{code}
However these Dropwizard Timers by default use nanoseconds 
([documentation|https://metrics.dropwizard.io/3.2.3/getting-started.html#timers]).
 It's certainly possible to extract milliseconds from them, but it seems 
misleading to have millis in the name here.

This causes {{YarnShuffleServiceMetrics}} to expose confusingly-named metrics 
like {{openBlockRequestLatencyMillis_count}} and 
{{openBlockRequestLatencyMillis_nanos}}. It should be up to the metrics 
exporter, like {{YarnShuffleServiceMetrics}}, to decide the unit and adjust the 
name accordingly, so the unit shouldn't be included in the name of the metric 
itself.

  was:
Today {{ExternalBlockHandler}} exposes a few {{Timer}} metrics:
{code}
// Time latency for open block request in ms
private final Timer openBlockRequestLatencyMillis = new Timer();
// Time latency for executor registration latency in ms
private final Timer registerExecutorRequestLatencyMillis = new Timer();
// Time latency for processing finalize shuffle merge request latency in ms
private final Timer finalizeShuffleMergeLatencyMillis = new Timer();
{code}
However these Dropwizard Timers by default use nanoseconds 
([documentation|https://metrics.dropwizard.io/3.2.3/getting-started.html#timers]).
 It's certainly possible to extract milliseconds from them, but it seems 
misleading to have millis in the name here.

{{YarnShuffleServiceMetrics}} currently doesn't expose any incorrectly-named 
metrics since it doesn't export any timing information from these metrics 
(which I am trying to address in SPARK-35258), but these names still result in 
kind of misleading metric names like 
{{finalizeShuffleMergeLatencyMillis_count}} -- a count doesn't have a unit. It 
should be up to the metrics exporter, like {{YarnShuffleServiceMetrics}}, to 
decide the unit and adjust the name accordingly.


> ExternalBlockHandler metrics have misleading unit in the name
> -
>
> Key: SPARK-35259
> URL: https://issues.apache.org/jira/browse/SPARK-35259
> Project: Spark
>  Issue Type: Bug
>  Components: Shuffle
>Affects Versions: 3.1.1
>Reporter: Erik Krogen
>Priority: Major
>
> Today {{ExternalBlockHandler}} exposes a few {{Timer}} metrics:
> {code}
> // Time latency for open block request in ms
> private final Timer openBlockRequestLatencyMillis = new Timer();
> // Time latency for executor registration latency in ms
> private final Timer registerExecutorRequestLatencyMillis = new Timer();
> // Time latency for processing fetch merged blocks meta request latency 
> in ms
> private final Timer fetchMergedBlocksMetaLatencyMillis = new Timer();
> // Time latency for processing finalize shuffle merge request latency in 
> ms
> private final Timer finalizeShuffleMergeLatencyMillis = new Timer();
> {code}
> However these Dropwizard Timers by default use nanoseconds 
> ([documentation|https://metrics.dropwizard.io/3.2.3/getting-started.html#timers]).
>  It's certainly possible to extract milliseconds from them, but it seems 
> misleading to have millis in the name here.
> This causes {{YarnShuffleServiceMetrics}} to expose confusingly-named metrics 
> like {{openBlockRequestLatencyMillis_count}} and 
> {{openBlockRequestLatencyMillis_nanos}}. It should be up to the metrics 
> exporter, like {{YarnShuffleServiceMetrics}}, to decide the unit and adjust 
> the name accordingly, so the unit shouldn't be included in the name of the 
> metric itself.



--
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-35672) Spark fails to launch executors with very large user classpath lists on YARN

2021-06-25 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-35672:
-

#32810 went into master.

Put up #33090 for branch-3.1

> Spark fails to launch executors with very large user classpath lists on YARN
> 
>
> Key: SPARK-35672
> URL: https://issues.apache.org/jira/browse/SPARK-35672
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core, YARN
>Affects Versions: 3.1.2
> Environment: Linux RHEL7
> Spark 3.1.1
>Reporter: Erik Krogen
>Assignee: Erik Krogen
>Priority: Major
> Fix For: 3.2.0
>
>
> When running Spark on YARN, the {{user-class-path}} argument to 
> {{CoarseGrainedExecutorBackend}} is used to pass a list of user JAR URIs to 
> executor processes. The argument is specified once for each JAR, and the URIs 
> are fully-qualified, so the paths can be quite long. With large user JAR 
> lists (say 1000+), this can result in system-level argument length limits 
> being exceeded, typically manifesting as the error message:
> {code}
> /bin/bash: Argument list too long
> {code}
> A [Google 
> search|https://www.google.com/search?q=spark%20%22%2Fbin%2Fbash%3A%20argument%20list%20too%20long%22=spark%20%22%2Fbin%2Fbash%3A%20argument%20list%20too%20long%22]
>  indicates that this is not a theoretical problem and afflicts real users, 
> including ours. This issue was originally observed on Spark 2.3, but has been 
> confirmed to exist in the master branch as well.



--
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-35744) Performance degradation in avro SpecificRecordBuilders

2021-06-21 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-35744:
-

[~steven.aerts] going a bit off topic from this JIRA, but out of curiosity -- 
is your work based off of SPARK-25789 / [PR 
#22878|https://github.com/apache/spark/pull/22878]? We (LinkedIn) also maintain 
an {{AvroEncoder}} for {{SpecificRecord}} classes which is based off of that 
PR. We've also been planning to make another effort to push this upstream since 
the attempt in #22878 eventually stalled. I'd be interested in learning more 
about your work and potentially collaborating here.

> Performance degradation in avro SpecificRecordBuilders
> --
>
> Key: SPARK-35744
> URL: https://issues.apache.org/jira/browse/SPARK-35744
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 3.2.0
>Reporter: Steven Aerts
>Priority: Minor
>
> Creating this bug to let you know that when we tested out spark 3.2.0 we saw 
> a significant performance degradation where our code was handling Avro 
> Specific Record objects.  This slowed down some of our jobs with a factor 4.
> Spark 3.2.0 upsteps the avro version from 1.8.2 to 1.10.2.
> The degradation was caused by a change introduced in avro 1.9.0.  This change 
> degrades performance when creating avro specific records in certain 
> classloader topologies, like the ones used in spark.
> We notified and [proposed|https://github.com/apache/avro/pull/1253] a simple 
> fix upstream in the avro project.  (Links contain more details)
> It is unclear for us how many other projects are using avro specific records 
> in a spark context and will be impacted by this degradation.
>  Feel free to close this issue if you think this issue is too much of a 
> corner case.



--
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-35817) Queries against wide Avro tables can be slow

2021-06-18 Thread Erik Krogen (Jira)


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

Erik Krogen edited comment on SPARK-35817 at 6/18/21, 10:09 PM:


Thanks for catching this [~bersprockets]! I will be happy to contribute as 
well, given I'm the original author of SPARK-34133. Let me know if there's 
anything I can do or provide input on. I guess we should create a map of one 
sides' fields (with lowercased names for case-insensitivity) and do lookups, to 
reduce the O(N^2) complexity to O(N) (N for map creation and N constant-time 
lookups during resolution).


was (Author: xkrogen):
Thanks for catching this [~bersprockets]! I will be happy to contribute as 
well, given I'm the original author of SPARK-34133. Let me know if there's 
anything I can do or provide input on.

> Queries against wide Avro tables can be slow
> 
>
> Key: SPARK-35817
> URL: https://issues.apache.org/jira/browse/SPARK-35817
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.2.0
>Reporter: Bruce Robbins
>Priority: Major
>
> A query against an Avro table can be quite slow when all are true:
> - There are many columns in the Avro file
> - The query contains a wide projection
> - There are many splits in the input
> - Some of the splits are read serially (e.g., less executors than there are 
> tasks)
> A write to an Avro table can be quite slow when all are true:
> - There are many columns in the new rows
> - The operation is creating many files
> For example, a single-threaded query against a 6000 column Avro data set with 
> 50K rows and 20 files takes less than a minute with Spark 3.0.1 but over 7 
> minutes with Spark 3.2.0-SNAPSHOT.
> The culprit appears to be this line of code:
> https://github.com/apache/spark/blob/3fb044e043a2feab01d79b30c25b93d4fd166b12/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala#L226
> For each split, AvroDeserializer will call this function once for each column 
> in the projection, resulting in a potential n^2 lookup per split.
> For each file, AvroSerializer will call this function once for each column, 
> resulting in an n^2 lookup per 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-35817) Queries against wide Avro tables can be slow

2021-06-18 Thread Erik Krogen (Jira)


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

Erik Krogen commented on SPARK-35817:
-

Thanks for catching this [~bersprockets]! I will be happy to contribute as 
well, given I'm the original author of SPARK-34133. Let me know if there's 
anything I can do or provide input on.

> Queries against wide Avro tables can be slow
> 
>
> Key: SPARK-35817
> URL: https://issues.apache.org/jira/browse/SPARK-35817
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.2.0
>Reporter: Bruce Robbins
>Priority: Major
>
> A query against an Avro table can be quite slow when all are true:
> - There are many columns in the Avro file
> - The query contains a wide projection
> - There are many splits in the input
> - Some of the splits are read serially (e.g., less executors than there are 
> tasks)
> A write to an Avro table can be quite slow when all are true:
> - There are many columns in the new rows
> - The operation is creating many files
> For example, a single-threaded query against a 6000 column Avro data set with 
> 50K rows and 20 files takes less than a minute with Spark 3.0.1 but over 7 
> minutes with Spark 3.2.0-SNAPSHOT.
> The culprit appears to be this line of code:
> https://github.com/apache/spark/blob/3fb044e043a2feab01d79b30c25b93d4fd166b12/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala#L226
> For each split, AvroDeserializer will call this function once for each column 
> in the projection, resulting in a potential n^2 lookup per split.
> For each file, AvroSerializer will call this function once for each column, 
> resulting in an n^2 lookup per 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



  1   2   >