Skip to content

Conversation

@maropu
Copy link
Member

@maropu maropu commented Sep 21, 2018

What changes were proposed in this pull request?

Since AggregationIterator uses MutableProjection for UnsafeRow, InterpretedMutableProjection needs to handle UnsafeRow as buffer internally for fixed-length types only.

How was this patch tested?

Run 'SQLQueryTestSuite' with the interpreted mode.

@maropu
Copy link
Member Author

maropu commented Sep 21, 2018

I thought we currently had less tests for interpreted projections, so I was checking if we had no bug caused by these projections. Then, I noticed these two issues when the interpreted mode enabled in SQLQueryTestSuite. I'm still digging if we have other bugs about interpreted projections, so I set WIP.

Btw, we'd be better to split this pr into multiple ones, probably. But, I'd like to make all the related bugs clear first.

@SparkQA
Copy link

SparkQA commented Sep 21, 2018

Test build #96395 has finished for PR 22512 at commit 39c5e92.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • s\"but $
  • case class Literal(value: Any, dataType: DataType) extends LeafExpression

@maropu
Copy link
Member Author

maropu commented Sep 21, 2018

retest this please

@SparkQA
Copy link

SparkQA commented Sep 21, 2018

Test build #96405 has finished for PR 22512 at commit 39c5e92.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • s\"but $
  • case class Literal(value: Any, dataType: DataType) extends LeafExpression

@maropu
Copy link
Member Author

maropu commented Sep 21, 2018

This is a simple query to reproduce;

$ SPARK_TESTING=1 ./bin/spark-shell
scala> sql("SET spark.sql.codegen.factoryMode=NO_CODEGEN")
scala> sql("CREATE TABLE desc_col_table (key int COMMENT 'column_comment') USING PARQUET")
scala> sql("""ANALYZE TABLE desc_col_table COMPUTE STATISTICS FOR COLUMNS key""")
org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1.0 (TID 0, localhost, executor driver): java.lang.UnsupportedOperationException
	at org.apache.spark.sql.catalyst.expressions.UnsafeRow.update(UnsafeRow.java:206)
	at org.apache.spark.sql.catalyst.expressions.InterpretedMutableProjection.apply(InterpretedMutableProjection.scala:67)
	at org.apache.spark.sql.catalyst.expressions.InterpretedMutableProjection.apply(InterpretedMutableProjection.scala:31)
	at org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.createNewAggregationBuffer(TungstenAggregationIterator.scala:129)
	at org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.<init>(TungstenAggregationIterator.scala:156)
	at org.apache.spark.sql.execution.aggregate.HashAggregateExec$$anonfun$doExecute$1$$anonfun$4.apply(HashAggregateExec.scala:112)
	at org.apache.spark.sql.execution.aggregate.HashAggregateExec$$anonfun$doExecute$1$$anonfun$4.apply(HashAggregateExec.scala:102)

@SparkQA
Copy link

SparkQA commented Sep 21, 2018

Test build #96430 has finished for PR 22512 at commit bff88ee.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • s\"but $
  • case class Literal(value: Any, dataType: DataType) extends LeafExpression

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not sure though, is it ok for Literal to have a different Scala-typed value for the corresponding dataType? , e.g., new Literal(1 /* int value */, LongType)? In the current master, there are some places to do so, e.g.,


In the codegen path, this is ok because we add a correct literal suffix in Literal.doGenCode (e.g., 1L for new Literal(1, LongType));

But, in the non-codegen path (e.g., spark.sql.codegen.factoryMode=NO_CODEGEN and ConstantFolding), this case throws an exception ;


scala> import org.apache.spark.sql.Column
scala> import org.apache.spark.sql.catalyst.expressions.Literal
scala> import org.apache.spark.sql.types._
scala> val intOne: Int = 1
scala> val lit = Literal.create(intOne, LongType)
scala> spark.range(1).select(struct(new Column(lit))).collect
18/10/04 11:35:56 ERROR Executor: Exception in task 3.0 in stage 0.0 (TID 3)
java.lang.ClassCastException: java.lang.Integer cannot be cast to java.lang.Long
	at scala.runtime.BoxesRunTime.unboxToLong(BoxesRunTime.java:105)
	at org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow$class.getLong(rows.scala:42)
	at org.apache.spark.sql.catalyst.expressions.GenericInternalRow.getLong(rows.scala:195)
	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$$anonfun$11$$anon$1.hasNext(WholeStageCodegenExec.scala:619)
    ...

WDYT? cc: @gatorsmile @cloud-fan

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we should not allow it. Can you send a separated PR for this change?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yea, ok.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

UnsafeRow doesn't support it, right?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

oh, yes. I'll recheck again.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This match should only accept the generic internal rows, so I added code to verify types for the UnsafeRow case;
https://github.com/apache/spark/pull/22512/files#diff-3ed819282d4e4941571dd3b08fc03e37R55

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this API (def create(v: Any, dataType: DataType): Literal) might be a little obscure: create(scalaValue, dataType) vs create(catalystValue, dataType). How about splitting this API into the two below?

@SparkQA
Copy link

SparkQA commented Oct 4, 2018

Test build #96923 has finished for PR 22512 at commit 31c623f.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Oct 4, 2018

Test build #96922 has finished for PR 22512 at commit 78795be.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@maropu
Copy link
Member Author

maropu commented Oct 15, 2018

I made a new pr to address the Literal issue in #22724 (I'll fix the Literal issue first, then I'll resume this pr).

@maropu maropu changed the title [SPARK-25498][SQL][WIP] Fix SQLQueryTestSuite failures when the interpreter mode enabled [SPARK-25498][SQL] InterpretedMutableProjection should handle UnsafeRow Oct 15, 2018
@SparkQA
Copy link

SparkQA commented Oct 15, 2018

Test build #97390 has finished for PR 22512 at commit 8e4f2b8.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Oct 15, 2018

Test build #97391 has finished for PR 22512 at commit 2766bd1.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Oct 23, 2018

Test build #97886 has finished for PR 22512 at commit 79c435a.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

shall we call mutableRow.setNullAt?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We need to take care of e.nullable && e.dataType == NullType here?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the corresponding logic in the codegen version is simply call row.update(null, i).

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we have InternalRow.getAccessor, shall we move this method there too?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

oh, yes! yea, I will.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If wholeStageCodegenEnabled is not used, let's not complicate the code now.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok

@cloud-fan
Copy link
Contributor

LGTM, we also need a unit test

@maropu
Copy link
Member Author

maropu commented Oct 23, 2018

ok, I'll add tests.

@maropu
Copy link
Member Author

maropu commented Oct 25, 2018

I'm looking into the failure reason... (passed in the local, but failed in the jenkins...)

@SparkQA
Copy link

SparkQA commented Oct 26, 2018

Test build #98058 has finished for PR 22512 at commit 5227e42.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@kiszk
Copy link
Member

kiszk commented Nov 23, 2018

retest this please

@SparkQA
Copy link

SparkQA commented Nov 23, 2018

Test build #99217 has finished for PR 22512 at commit 5227e42.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@cloud-fan
Copy link
Contributor

@maropu are you still working on it?

@maropu
Copy link
Member Author

maropu commented Nov 26, 2018

Yea, I'll update in a few days.

@SparkQA
Copy link

SparkQA commented Dec 3, 2018

Test build #99583 has finished for PR 22512 at commit 243fae3.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

while (i < validExprs.length) {
val (_, ordinal) = validExprs(i)
mutableRow(ordinal) = buffer(ordinal)
fieldWriters(i)(buffer(ordinal))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since fieldWriters is accessed via index, we should use IndexedSeq or Array explicitly?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ah, sounds reasonable. I'll update later.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed in 95411c8

@SparkQA
Copy link

SparkQA commented Dec 3, 2018

Test build #99590 has finished for PR 22512 at commit 3553d91.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

// all need to return the same result
if (regenerateGoldenFiles && configs.nonEmpty) {
if (regenerateGoldenFiles) {
configs.take(1)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what if configs is empty? take(1) will fail

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actually, it returns an empty array?

scala> Array.empty.take(1)
res0: Array[Nothing] = Array()

scala> Seq.empty.take(1)
res1: Seq[Nothing] = List()

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For better readability, fixed in 4cdc504

@SparkQA
Copy link

SparkQA commented Dec 3, 2018

Test build #99601 has finished for PR 22512 at commit 4cdc504.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

if (configs.nonEmpty) {
configs.take(1)
} else {
Array.empty[Array[(String, String)]]
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: since configs don't matter when generating result, I think we can just return empty configs here. We can clean it up in a followup PR.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok

@cloud-fan
Copy link
Contributor

thanks, merging to master!

@asfgit asfgit closed this in 04046e5 Dec 3, 2018
asfgit pushed a commit that referenced this pull request Dec 4, 2018
…ating the golden files

## What changes were proposed in this pull request?
This pr is to return an empty config set when regenerating the golden files in `SQLQueryTestSuite`.
This is the follow-up of  #22512.

## How was this patch tested?
N/A

Closes #23212 from maropu/SPARK-25498-FOLLOWUP.

Authored-by: Takeshi Yamamuro <[email protected]>
Signed-off-by: Wenchen Fan <[email protected]>
jackylee-ch pushed a commit to jackylee-ch/spark that referenced this pull request Feb 18, 2019
## What changes were proposed in this pull request?
Since `AggregationIterator` uses `MutableProjection` for `UnsafeRow`, `InterpretedMutableProjection` needs to handle `UnsafeRow` as buffer internally for fixed-length types only.

## How was this patch tested?
Run 'SQLQueryTestSuite' with the interpreted mode.

Closes apache#22512 from maropu/InterpreterTest.

Authored-by: Takeshi Yamamuro <[email protected]>
Signed-off-by: Wenchen Fan <[email protected]>
jackylee-ch pushed a commit to jackylee-ch/spark that referenced this pull request Feb 18, 2019
…ating the golden files

## What changes were proposed in this pull request?
This pr is to return an empty config set when regenerating the golden files in `SQLQueryTestSuite`.
This is the follow-up of  apache#22512.

## How was this patch tested?
N/A

Closes apache#23212 from maropu/SPARK-25498-FOLLOWUP.

Authored-by: Takeshi Yamamuro <[email protected]>
Signed-off-by: Wenchen Fan <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants