[jira] [Created] (SPARK-40059) Row indexes can overshadow user-created data

2022-08-12 Thread Ala Luszczak (Jira)
Ala Luszczak created SPARK-40059:


 Summary: Row indexes can overshadow user-created data
 Key: SPARK-40059
 URL: https://issues.apache.org/jira/browse/SPARK-40059
 Project: Spark
  Issue Type: Improvement
  Components: SQL
Affects Versions: 3.4.0
Reporter: Ala Luszczak


https://github.com/apache/spark/pull/37228 introduces ability to compute row 
indexes, which users can access through `_metadata.row_index` column. 
Internally this is achieved with the help of an extra column 
`_tmp_metadata_row_index`. When present in the schema sent to parquet reader, 
the reader populates it with row indexes, and the values are later placed in 
the `_metadata` struct. 

While relatively unlikely, it's still possible, that a user might want to 
include column `_tmp_metadata_row_index` in their data. In such scenario, the 
column will be populated with row indexes, rather than data read from the file.

For repro, search `FileMetadataStructRowIndexSuite.scala` for this Jira ticket 
number.

We could introduce some kind of countermeasure to handle this scenario.



--
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-39806) Queries accessing METADATA struct crash on partitioned tables

2022-07-18 Thread Ala Luszczak (Jira)


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

Ala Luszczak updated SPARK-39806:
-
Description: 
There is a problem with a projection we use in `FileScanRDD` to join the 
metadata row to the row produced by the reader.

https://github.com/apache/spark/blob/e4ca8424474e571d8e137388fe5d54732b68c2f3/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala#L128-L133

The issue is that the projection omits partition columns. As a result, the 
expressions down the line return a malformed row. The errors crash the query, 
but the exact message can vary (for example: failed assertion on number of 
fields in the row, accessing field of incorrect type).

This defect affects only readers producing rows, and only data sets using 
dynamic partitioning.

  was:
There is a problem with a projection we use in `FileScanRDD` to join the 
metadata row to the row produced by the reader.

https://github.com/apache/spark/blob/e4ca8424474e571d8e137388fe5d54732b68c2f3/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala#L128-L133

The issue is that the projection omits partition columns. As a result, the 
expressions down the line return a malformed row. The errors crash the query, 
but the exact message can vary (for example: failed assertion on number of 
fields in the row, accessing field of incorrect type).

This defect affect only readers producing rows, and only data sets using 
dynamic partitioning.


> Queries accessing METADATA struct crash on partitioned tables
> -
>
> Key: SPARK-39806
> URL: https://issues.apache.org/jira/browse/SPARK-39806
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 3.3.0
>Reporter: Ala Luszczak
>Priority: Major
>
> There is a problem with a projection we use in `FileScanRDD` to join the 
> metadata row to the row produced by the reader.
> https://github.com/apache/spark/blob/e4ca8424474e571d8e137388fe5d54732b68c2f3/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala#L128-L133
> The issue is that the projection omits partition columns. As a result, the 
> expressions down the line return a malformed row. The errors crash the query, 
> but the exact message can vary (for example: failed assertion on number of 
> fields in the row, accessing field of incorrect type).
> This defect affects only readers producing rows, and only data sets using 
> dynamic partitioning.



--
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-39806) Queries accessing METADATA struct crash on partitioned tables

2022-07-18 Thread Ala Luszczak (Jira)


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

Ala Luszczak updated SPARK-39806:
-
Description: 
There is a problem with a projection we use in `FileScanRDD` to join the 
metadata row to the row produced by the reader.

https://github.com/apache/spark/blob/e4ca8424474e571d8e137388fe5d54732b68c2f3/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala#L128-L133

The issue is that the projection omits partition columns. As a result, the 
expressions down the line return a malformed row. The errors crash the query, 
but the exact message can vary (for example: failed assertion on number of 
fields in the row, accessing field of incorrect type).

This defect affects only readers producing rows (as opposed to batches), and 
only data sets using dynamic partitioning.

  was:
There is a problem with a projection we use in `FileScanRDD` to join the 
metadata row to the row produced by the reader.

https://github.com/apache/spark/blob/e4ca8424474e571d8e137388fe5d54732b68c2f3/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala#L128-L133

The issue is that the projection omits partition columns. As a result, the 
expressions down the line return a malformed row. The errors crash the query, 
but the exact message can vary (for example: failed assertion on number of 
fields in the row, accessing field of incorrect type).

This defect affects only readers producing rows, and only data sets using 
dynamic partitioning.


> Queries accessing METADATA struct crash on partitioned tables
> -
>
> Key: SPARK-39806
> URL: https://issues.apache.org/jira/browse/SPARK-39806
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 3.3.0
>Reporter: Ala Luszczak
>Priority: Major
>
> There is a problem with a projection we use in `FileScanRDD` to join the 
> metadata row to the row produced by the reader.
> https://github.com/apache/spark/blob/e4ca8424474e571d8e137388fe5d54732b68c2f3/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala#L128-L133
> The issue is that the projection omits partition columns. As a result, the 
> expressions down the line return a malformed row. The errors crash the query, 
> but the exact message can vary (for example: failed assertion on number of 
> fields in the row, accessing field of incorrect type).
> This defect affects only readers producing rows (as opposed to batches), and 
> only data sets using dynamic partitioning.



--
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-39806) Queries accessing METADATA struct crash on partitioned tables

2022-07-18 Thread Ala Luszczak (Jira)
Ala Luszczak created SPARK-39806:


 Summary: Queries accessing METADATA struct crash on partitioned 
tables
 Key: SPARK-39806
 URL: https://issues.apache.org/jira/browse/SPARK-39806
 Project: Spark
  Issue Type: Improvement
  Components: SQL
Affects Versions: 3.3.0
Reporter: Ala Luszczak


There is a problem with a projection we use in `FileScanRDD` to join the 
metadata row to the row produced by the reader.

https://github.com/apache/spark/blob/e4ca8424474e571d8e137388fe5d54732b68c2f3/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala#L128-L133

The issue is that the projection omits partition columns. As a result, the 
expressions down the line return a malformed row. The errors crash the query, 
but the exact message can vary (for example: failed assertion on number of 
fields in the row, accessing field of incorrect type).

This defect affect only readers producing rows, and only data sets using 
dynamic partitioning.



--
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-39634) Allow file splitting in combination with row index generation

2022-06-29 Thread Ala Luszczak (Jira)
Title: Message Title


 
 
 
 

 
 
 

 
   
 Ala Luszczak created an issue  
 

  
 
 
 
 

 
 
  
 
 
 
 

 
 Spark /  SPARK-39634  
 
 
  Allow file splitting in combination with row index generation   
 

  
 
 
 
 

 
Issue Type: 
  Improvement  
 
 
Affects Versions: 
 3.3.0  
 
 
Assignee: 
 Unassigned  
 
 
Components: 
 SQL  
 
 
Created: 
 29/Jun/22 16:10  
 
 
Priority: 
  Major  
 
 
Reporter: 
 Ala Luszczak  
 

  
 
 
 
 

 
 This issue is a follow up for SPARK-37980 Because of a bug in parquet-mr https://issues.apache.org/jira/browse/PARQUET-2161 it is currently impossible to generate row indexes for parquet files if they are split into multiple pieces. Instead, each file must be read in a single task.  Once the version of parquet-mr with the fix is included in Spark, we should remove the workarounds from the code (marked with this ticket number) from the code, so that parquet files are splittable even when the row indexes need to be generated.  
 

  
 
 
 
 

 
 
 

 
 
 Add Comment  
 

  

[jira] [Commented] (SPARK-33594) Forbid binary type as partition column

2020-12-21 Thread Ala Luszczak (Jira)


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

Ala Luszczak commented on SPARK-33594:
--

Big :+1: here. Having binary column as partition-by is a terrible idea.
I've seen at least two really bad scenarios result from this.

(1) When reading the data with the vectorized reader, I've seen segmentation 
faults.
(2) When reading the same data with the non-vectorized (parquet-mr) reader, the 
segmentation faults disappear, but instead incorrect values are returned for 
the binary columns.

I would like to point out that just covering the CREATE TABLE statement might 
not be enough. I think we should bail in the read path as well. After all the 
user can jest do spark.read.parquet("my/path") without creating a table first.

> Forbid binary type as partition column
> --
>
> Key: SPARK-33594
> URL: https://issues.apache.org/jira/browse/SPARK-33594
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 3.1.0
>Reporter: angerszhu
>Priority: Major
>
> Forbid binary type as partition column



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

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



[jira] [Created] (SPARK-26878) QueryTest.compare does not handle maps with array keys correctly

2019-02-14 Thread Ala Luszczak (JIRA)
Ala Luszczak created SPARK-26878:


 Summary: QueryTest.compare does not handle maps with array keys 
correctly
 Key: SPARK-26878
 URL: https://issues.apache.org/jira/browse/SPARK-26878
 Project: Spark
  Issue Type: Bug
  Components: SQL, Tests
Affects Versions: 2.4.0
Reporter: Ala Luszczak


The current strategy for comparing Maps is sorting the (key, value) tuples by 
_.toString, zipping tuples from both maps together, and then comparing tuples 
within each of the pairs separately.

See: 
https://github.com/apache/spark/blob/ac9c0536bc518f173f2ff53bee42b7a89d28ee20/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala#L344-L346

This is not ideal for byte arrays. The string representations of byte arrays 
looks like “[B@7d263ddc” and has nothing to do with values actually contained 
within the array.

Hence, if a map has byte array keys, then random values get compared with each 
other, which can result in false negatives.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Created] (SPARK-26175) PySpark cannot terminate worker process if user program reads from stdin

2018-11-26 Thread Ala Luszczak (JIRA)
Ala Luszczak created SPARK-26175:


 Summary: PySpark cannot terminate worker process if user program 
reads from stdin
 Key: SPARK-26175
 URL: https://issues.apache.org/jira/browse/SPARK-26175
 Project: Spark
  Issue Type: Bug
  Components: PySpark
Affects Versions: 2.4.0
Reporter: Ala Luszczak


PySpark worker daemon reads from stdin the worker PIDs to kill. 
https://github.com/apache/spark/blob/1bb60ab8392adf8b896cc04fb1d060620cf09d8a/python/pyspark/daemon.py#L127

However, the worker process is a forked process from the worker daemon process 
and we didn't close stdin on the child after fork. This means the child and 
user program can read stdin as well, which blocks daemon from receiving the PID 
to kill. This can cause issues because the task reaper might detect the task 
was not terminated and eventually kill the JVM.

Possible fix could be:
* Closing stdin of the worker process right after fork.
* Creating a new socket to receive PIDs to kill instead of using stdin.

h4. Steps to reproduce

# Paste the following code in pyspark:
{code}
import subprocess
def task(_):
  subprocess.check_output(["cat"])

sc.parallelize(range(1), 1).mapPartitions(task).count()
{code}
# Press CTRL+C to cancel the job.
# The following message is displayed:
{code}
18/11/26 17:52:51 WARN PythonRunner: Incomplete task 0.0 in stage 0 (TID 0) 
interrupted: Attempting to kill Python Worker
18/11/26 17:52:52 WARN TaskSetManager: Lost task 0.0 in stage 0.0 (TID 0, 
localhost, executor driver): TaskKilled (Stage cancelled)
{code}
# Run {{ps -xf}} to see that {{cat}} process was in fact not killed:
{code}
19773 pts/2Sl+0:00  |   |   \_ python
19803 pts/2Sl+0:11  |   |   \_ /usr/lib/jvm/java-8-oracle/bin/java 
-cp 
/home/ala/Repos/apache-spark-GOOD-2/conf/:/home/ala/Repos/apache-spark-GOOD-2/assembly/target/scala-2.12/jars/*
 -Xmx1g org.apache.spark.deploy.SparkSubmit --name PySparkShell pyspark-shell
19879 pts/2S  0:00  |   |   \_ python -m pyspark.daemon
19895 pts/2S  0:00  |   |   \_ python -m pyspark.daemon
19898 pts/2S  0:00  |   |   \_ cat
{code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Created] (SPARK-26147) Python UDFs in join condition fail even when using columns from only one side of join

2018-11-22 Thread Ala Luszczak (JIRA)
Ala Luszczak created SPARK-26147:


 Summary: Python UDFs in join condition fail even when using 
columns from only one side of join
 Key: SPARK-26147
 URL: https://issues.apache.org/jira/browse/SPARK-26147
 Project: Spark
  Issue Type: Bug
  Components: PySpark
Affects Versions: 2.4.0
Reporter: Ala Luszczak


The rule {{PullOutPythonUDFInJoinCondition}} was implemented in 
[https://github.com/apache/spark/commit/2a8cbfddba2a59d144b32910c68c22d0199093fe]
 As far as I understand, this rule was intended to prevent the use of Python 
UDFs in join condition if they take arguments from both sides of the join, and 
this doesn't make sense in combination with the join type.

The rule {{PullOutPythonUDFInJoinCondition}} seems to make an assumption, that 
if a given UDF is only using columns from a single side of the join, it will be 
already pushed down under the join before this rule is executed.

However, this is not always the case. Here's a simple example that fails, even 
though it looks like it should run just fine (and it does in earlier versions 
of Spark):
{code:java}
from pyspark.sql import Row
from pyspark.sql.types import StringType
from pyspark.sql.functions import udf

cars_list = [ Row("NL", "1234AB"), Row("UK", "987654") ]
insurance_list = [ Row("NL-1234AB"), Row("BE-112233") ]

spark.createDataFrame(data = cars_list, schema = ["country", 
"plate_nr"]).createOrReplaceTempView("cars")
spark.createDataFrame(data = insurance_list, schema = 
["insurance_code"]).createOrReplaceTempView("insurance")

to_insurance_code = udf(lambda x, y: x + "-" + y, StringType()) 
sqlContext.udf.register('to_insurance_code', to_insurance_code)

spark.conf.set("spark.sql.crossJoin.enabled", "true")

# This query runs just fine.
sql("""
  SELECT country, plate_nr, insurance_code
  FROM cars LEFT OUTER JOIN insurance
  ON CONCAT(country, '-', plate_nr) = insurance_code
""").show()

# This equivalent query fails with:
# pyspark.sql.utils.AnalysisException: u'Using PythonUDF in join condition of 
join type LeftOuter is not supported.;'
sql("""
  SELECT country, plate_nr, insurance_code
  FROM cars LEFT OUTER JOIN insurance
  ON to_insurance_code(country, plate_nr) = insurance_code
""").show()
{code}
[~cloud_fan] [~XuanYuan] fyi



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Created] (SPARK-24133) Reading Parquet files containing large strings can fail with java.lang.ArrayIndexOutOfBoundsException

2018-05-01 Thread Ala Luszczak (JIRA)
Ala Luszczak created SPARK-24133:


 Summary: Reading Parquet files containing large strings can fail 
with java.lang.ArrayIndexOutOfBoundsException
 Key: SPARK-24133
 URL: https://issues.apache.org/jira/browse/SPARK-24133
 Project: Spark
  Issue Type: Bug
  Components: SQL
Affects Versions: 2.3.0
Reporter: Ala Luszczak


ColumnVectors store string data in one big byte array. Since the array size is 
capped at just under Integer.MAX_VALUE, a single ColumnVector cannot store more 
than 2GB of string data.

However, since the Parquet files commonly contain large blobs stored as 
strings, and ColumnVectors by default carry 4096 values, it's entirely possible 
to go past that limit.

In such cases a negative capacity is requested from 
WritableColumnVector.reserve(). The call succeeds (requested capacity is 
smaller than already allocated), and consequently  
java.lang.ArrayIndexOutOfBoundsException is thrown when the reader actually 
attempts to put the data into the array.

This behavior is hard to troubleshoot for the users. Spark should instead check 
for negative requested capacity in WritableColumnVector.reserve() and throw 
more informative error, instructing the user to tweak ColumnarBatch size.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (SPARK-23496) Locality of coalesced partitions can be severely skewed by the order of input partitions

2018-02-23 Thread Ala Luszczak (JIRA)

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

Ala Luszczak commented on SPARK-23496:
--

I agree that this solution is merely making the problem unlikely to occur, 
instead of really solving it.

But the code in {{DefaultPartitionsCoalescer.setGroups()}} ([see 
comment|https://github.com/ala/spark/blob/master/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala#L234-L240])
 is deliberately written so that it's fast (O(n log n) with respect to number 
of coalesced partitions, which is assumed order of magnitude smaller than the 
number of input partitions), but not necessarily accurate. The same applies to 
other algorithms there.

Enforcing an even data distribution is not trivial. For example:
 * We merely look at the number of partitions, not on the number of rows in 
each of the partitions. There might be a severe skew across the partitions to 
begin with.
 * It's not clear how to treat partitions with multiple preferred location.
 * It's not clear if it's more important for every input location to find some 
matching coalesced partition, or if it's more important to keep the partition 
size even.
 * It's not clear how best to deal with a mix of partitions with and without 
locality preferences.

I think it's better to have a very simple fix that will work well vast majority 
of the time now, and maybe have a follow-up ticked for revisiting the design of 
{{DefaultPartitionCoalescer}} for later.

> Locality of coalesced partitions can be severely skewed by the order of input 
> partitions
> 
>
> Key: SPARK-23496
> URL: https://issues.apache.org/jira/browse/SPARK-23496
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 3.0.0
>Reporter: Ala Luszczak
>Priority: Major
>
> Example:
> Consider RDD "R" with 100 partitions, half of which have locality preference 
> "hostA" and half have "hostB".
>  * Assume odd-numbered input partitions of R prefer "hostA" and even-numbered 
> prefer "hostB". Then R.coalesce(50) will have 25 partitions with preference 
> "hostA" and 25 with "hostB" (even distribution).
>  * Assume partitions with index 0-49 of R prefer "hostA" and partitions with 
> index 50-99 prefer "hostB". Then R.coalesce(50) will have 49 partitions with 
> "hostA" and 1 with "hostB" (extremely skewed distribution).
>  
> The algorithm in {{DefaultPartitionCoalescer.setupGroups}} is responsible for 
> picking preferred locations for coalesced partitions. It analyzes the 
> preferred locations of input partitions. It starts by trying to create one 
> partition for each unique location in the input. However, if the the 
> requested number of coalesced partitions is higher that the number of unique 
> locations, it has to pick duplicate locations.
> Currently, the duplicate locations are picked by iterating over the input 
> partitions in order, and copying their preferred locations to coalesced 
> partitions. If the input partitions are clustered by location, this can 
> result in severe skew.
> Instead of iterating over the list of input partitions in order, we should 
> pick them at random.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Created] (SPARK-23496) Locality of coalesced partitions can be severely skewed by the order of input partitions

2018-02-23 Thread Ala Luszczak (JIRA)
Ala Luszczak created SPARK-23496:


 Summary: Locality of coalesced partitions can be severely skewed 
by the order of input partitions
 Key: SPARK-23496
 URL: https://issues.apache.org/jira/browse/SPARK-23496
 Project: Spark
  Issue Type: Bug
  Components: Spark Core
Affects Versions: 3.0.0
Reporter: Ala Luszczak


Example:

Consider RDD "R" with 100 partitions, half of which have locality preference 
"hostA" and half have "hostB".
 * Assume odd-numbered input partitions of R prefer "hostA" and even-numbered 
prefer "hostB". Then R.coalesce(50) will have 25 partitions with preference 
"hostA" and 25 with "hostB" (even distribution).
 * Assume partitions with index 0-49 of R prefer "hostA" and partitions with 
index 50-99 prefer "hostB". Then R.coalesce(50) will have 49 partitions with 
"hostA" and 1 with "hostB" (extremely skewed distribution).

 

The algorithm in {{DefaultPartitionCoalescer.setupGroups}} is responsible for 
picking preferred locations for coalesced partitions. It analyzes the preferred 
locations of input partitions. It starts by trying to create one partition for 
each unique location in the input. However, if the the requested number of 
coalesced partitions is higher that the number of unique locations, it has to 
pick duplicate locations.

Currently, the duplicate locations are picked by iterating over the input 
partitions in order, and copying their preferred locations to coalesced 
partitions. If the input partitions are clustered by location, this can result 
in severe skew.

Instead of iterating over the list of input partitions in order, we should pick 
them at random.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (SPARK-22251) Metric "aggregate time" is incorrect when codegen is off

2017-10-11 Thread Ala Luszczak (JIRA)

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

Ala Luszczak commented on SPARK-22251:
--

I checked that if you type spark.conf.set("spark.sql.codegen.wholeStage", 
false) in Spark shell, it does take effect immediately.

> Metric "aggregate time" is incorrect when codegen is off
> 
>
> Key: SPARK-22251
> URL: https://issues.apache.org/jira/browse/SPARK-22251
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.2.0
>Reporter: Ala Luszczak
>Priority: Minor
>
> When whole-stage codegen is off, metric "aggregate time" is not set correctly.
> Repro:
> # spark.conf.set("spark.sql.codegen.wholeStage", false)
> # 
> spark.range(5).crossJoin(spark.range(5)).toDF("a","b").groupBy("a").agg(sum("b")).show
> # In Spark UI > SQL you can see that "aggregate time total" is 0 in 
> "HashAggregate" boxes



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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



[jira] [Created] (SPARK-22251) Metric "aggregate time" is incorrect when codegen is off

2017-10-11 Thread Ala Luszczak (JIRA)
Ala Luszczak created SPARK-22251:


 Summary: Metric "aggregate time" is incorrect when codegen is off
 Key: SPARK-22251
 URL: https://issues.apache.org/jira/browse/SPARK-22251
 Project: Spark
  Issue Type: Bug
  Components: SQL
Affects Versions: 2.2.0
Reporter: Ala Luszczak
Priority: Minor


When whole-stage codegen is off, metric "aggregate time" is not set correctly.

Repro:
# spark.conf.set("spark.sql.codegen.wholeStage", false)
# 
spark.range(5).crossJoin(spark.range(5)).toDF("a","b").groupBy("a").agg(sum("b")).show
# In Spark UI > SQL you can see that "aggregate time total" is 0 in 
"HashAggregate" boxes



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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



[jira] [Created] (SPARK-22092) Reallocation in OffHeapColumnVector.reserveInternal corrupts array data

2017-09-21 Thread Ala Luszczak (JIRA)
Ala Luszczak created SPARK-22092:


 Summary: Reallocation in OffHeapColumnVector.reserveInternal 
corrupts array data
 Key: SPARK-22092
 URL: https://issues.apache.org/jira/browse/SPARK-22092
 Project: Spark
  Issue Type: Bug
  Components: SQL
Affects Versions: 2.2.0
Reporter: Ala Luszczak


OffHeapColumnVector.reserveInternal() will only copy already inserted values 
during reallocation if this.data != null. However, for vectors containing 
arrays, field data is disused and always equals null. Hence, the reallocation 
of array vector always causes data loss.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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



[jira] [Updated] (SPARK-20798) GenerateUnsafeProjection should check if value is null before calling the getter

2017-05-18 Thread Ala Luszczak (JIRA)

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

Ala Luszczak updated SPARK-20798:
-
Description: 
GenerateUnsafeProjection.writeStructToBuffer() does not honor the assumption 
that one should first make sure the value is not null before calling the 
getter. This can lead to errors.

An example of generated code:
{noformat}
/* 059 */ final UTF8String fieldName = value.getUTF8String(0);
/* 060 */ if (value.isNullAt(0)) {
/* 061 */   rowWriter1.setNullAt(0);
/* 062 */ } else {
/* 063 */   rowWriter1.write(0, fieldName);
/* 064 */ }
{noformat}


  was:
GenerateUnsafeProjection.writeStructToBuffer() does not honor the assumption 
that one should first make sure the value is not null before calling the getter.

An example of generated code:
{noformat}
/* 059 */ final UTF8String fieldName = value.getUTF8String(0);
/* 060 */ if (value.isNullAt(0)) {
/* 061 */   rowWriter1.setNullAt(0);
/* 062 */ } else {
/* 063 */   rowWriter1.write(0, fieldName);
/* 064 */ }
{noformat}



> GenerateUnsafeProjection should check if value is null before calling the 
> getter
> 
>
> Key: SPARK-20798
> URL: https://issues.apache.org/jira/browse/SPARK-20798
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.2.0
>Reporter: Ala Luszczak
>
> GenerateUnsafeProjection.writeStructToBuffer() does not honor the assumption 
> that one should first make sure the value is not null before calling the 
> getter. This can lead to errors.
> An example of generated code:
> {noformat}
> /* 059 */ final UTF8String fieldName = value.getUTF8String(0);
> /* 060 */ if (value.isNullAt(0)) {
> /* 061 */   rowWriter1.setNullAt(0);
> /* 062 */ } else {
> /* 063 */   rowWriter1.write(0, fieldName);
> /* 064 */ }
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

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



[jira] [Updated] (SPARK-20798) GenerateUnsafeProjection should check if value is null before calling the getter

2017-05-18 Thread Ala Luszczak (JIRA)

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

Ala Luszczak updated SPARK-20798:
-
Description: 
GenerateUnsafeProjection.writeStructToBuffer() does not honor the assumption 
that one should first make sure the value is not null before calling the getter.

An example of generated code:
{noformat}
/* 059 */ final UTF8String fieldName = value.getUTF8String(0);
/* 060 */ if (value.isNullAt(0)) {
/* 061 */   rowWriter1.setNullAt(0);
/* 062 */ } else {
/* 063 */   rowWriter1.write(0, fieldName);
/* 064 */ }
{noformat}


> GenerateUnsafeProjection should check if value is null before calling the 
> getter
> 
>
> Key: SPARK-20798
> URL: https://issues.apache.org/jira/browse/SPARK-20798
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.2.0
>Reporter: Ala Luszczak
>
> GenerateUnsafeProjection.writeStructToBuffer() does not honor the assumption 
> that one should first make sure the value is not null before calling the 
> getter.
> An example of generated code:
> {noformat}
> /* 059 */ final UTF8String fieldName = value.getUTF8String(0);
> /* 060 */ if (value.isNullAt(0)) {
> /* 061 */   rowWriter1.setNullAt(0);
> /* 062 */ } else {
> /* 063 */   rowWriter1.write(0, fieldName);
> /* 064 */ }
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

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



[jira] [Created] (SPARK-20798) GenerateUnsafeProjection should check if value is null before calling the getter

2017-05-18 Thread Ala Luszczak (JIRA)
Ala Luszczak created SPARK-20798:


 Summary: GenerateUnsafeProjection should check if value is null 
before calling the getter
 Key: SPARK-20798
 URL: https://issues.apache.org/jira/browse/SPARK-20798
 Project: Spark
  Issue Type: Bug
  Components: SQL
Affects Versions: 2.2.0
Reporter: Ala Luszczak






--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

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



[jira] [Created] (SPARK-19607) Finding QueryExecution that matches provided executionId

2017-02-15 Thread Ala Luszczak (JIRA)
Ala Luszczak created SPARK-19607:


 Summary: Finding QueryExecution that matches provided executionId
 Key: SPARK-19607
 URL: https://issues.apache.org/jira/browse/SPARK-19607
 Project: Spark
  Issue Type: New Feature
  Components: SQL
Affects Versions: 2.1.0
Reporter: Ala Luszczak


Create a method for finding QueryExecution that matches provided executionId 
for future use.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

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



[jira] [Created] (SPARK-19549) Allow providing reasons for stage/job cancelling

2017-02-10 Thread Ala Luszczak (JIRA)
Ala Luszczak created SPARK-19549:


 Summary: Allow providing reasons for stage/job cancelling
 Key: SPARK-19549
 URL: https://issues.apache.org/jira/browse/SPARK-19549
 Project: Spark
  Issue Type: New Feature
  Components: Spark Core
Affects Versions: 2.1.0
Reporter: Ala Luszczak
Priority: Minor


Currently it is not possible to pass a cancellation reason to  
SparkContext.cancelStage() and SparkContext.cancelJob(). In many situations 
having such reason included in the exception message would be useful for the 
user.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

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



[jira] [Created] (SPARK-19514) Range is not interruptible

2017-02-08 Thread Ala Luszczak (JIRA)
Ala Luszczak created SPARK-19514:


 Summary: Range is not interruptible
 Key: SPARK-19514
 URL: https://issues.apache.org/jira/browse/SPARK-19514
 Project: Spark
  Issue Type: Bug
  Components: SQL
Affects Versions: 2.1.0
Reporter: Ala Luszczak


Currently Range cannot be interrupted.

For example, if you start executing

spark.range(0, A_LOT, 1).crossJoin(spark.range(0, A_LOT, 1)).count()

and then call

DAGScheduler.cancellStage(...)

the execution won't stop.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

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