Skip to content

Conversation

@dilipbiswal
Copy link
Contributor

@dilipbiswal dilipbiswal commented May 4, 2018

What changes were proposed in this pull request?

Add a new UDTF replicate_rows. This function replicates the values based on the first argument to the function. This will be used in EXCEPT ALL AND INTERSECT ALL transformation (future PR) mainly
to preserve "retain duplicates" semantics. Please refer to Link for design. The transformation code changes are in Code

Example

SELECT replicate_rows(3,  1,  2)

Result

spark-sql> SELECT replicate_rows(3, 1, 2);
3	1	2
3	1	2
3	1	2
Time taken: 0.045 seconds, Fetched 3 row(s)

Returns 3 rows based on the first parameter value.

How was this patch tested?

Added tests in GeneratorFunctionSuite, TypeCoercionSuite, SQLQueryTestSuite

@maropu
Copy link
Member

maropu commented May 5, 2018

Why we need this? I thinks it's ok to add a new rewriting rule for EXCEPT ALL and INTERSECT ALL in analyzer?

@maropu
Copy link
Member

maropu commented May 5, 2018

Also, ISTM this function is less useful for end users.

@gatorsmile
Copy link
Member

https://issues.apache.org/jira/browse/HIVE-14768 did the same thing.

@maropu
Copy link
Member

maropu commented May 5, 2018

Ah, ok.

@SparkQA
Copy link

SparkQA commented May 5, 2018

Test build #90219 has finished for PR 21240 at commit 90efeff.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • case class ReplicateRows(children: Seq[Expression]) extends Generator with CodegenFallback

Row(1, null) :: Row(2, null) :: Nil)
}

test("ReplicateRows generator") {
Copy link
Member

Choose a reason for hiding this comment

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

duplicate tests? I feel udtf_replicate_rows.sql is enough for tests.

)
}

test("type coercion for ReplicateRows") {
Copy link
Member

Choose a reason for hiding this comment

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

Can we move this tests into sql-tests/inputs/typeCoercion/native?

* if necessary.
*/
object ReplicateRowsCoercion extends TypeCoercionRule {
private val acceptedTypes = Seq(LongType, IntegerType, ShortType, ByteType)
Copy link
Member

Choose a reason for hiding this comment

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

nit: LongType seems not necessary be here. Can avoid re-entering the following pattern matching if it is already long type.

import org.apache.spark.sql.catalyst.util.{ArrayData, MapData}
import org.apache.spark.sql.types._


Copy link
Member

Choose a reason for hiding this comment

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

Not need to introduce this breaking line.

}

/**
* Replicate the row based N times. N is specified as the first argument to the function.
Copy link
Member

Choose a reason for hiding this comment

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

nit: Replicate N times the row.?

Copy link
Member

Choose a reason for hiding this comment

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

Btw, using n to match following expression description?

Row(3, "row1") :: Row(3, "row1") :: Row(3, "row1") :: Nil)
checkAnswer(df.selectExpr("replicate_rows(-1, 2.5)"), Nil)

// The data for the same column should have the same type.
Copy link
Member

Choose a reason for hiding this comment

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

This copied comment can be removed.

* }}}
*/
@ExpressionDescription(
usage = "_FUNC_(n, expr1, ..., exprk) - Replicates `expr1`, ..., `exprk` into `n` rows.",
Copy link
Member

Choose a reason for hiding this comment

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

Replicates `n`, `expr1`, ..., `exprk` into `n` rows.?

@dilipbiswal
Copy link
Contributor Author

@maropu @viirya Thanks for the comments. I have made the changes.

private val acceptedTypes = Seq(IntegerType, ShortType, ByteType)
override def coerceTypes(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions {
case s @ ReplicateRows(children) if s.childrenResolved &&
s.children.head.dataType != LongType && acceptedTypes.contains(s.children.head.dataType) =>
Copy link
Member

Choose a reason for hiding this comment

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

We should check if s.children isn't empty.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ok

AS tab1(c1, c2, c3);

-- Requires 2 arguments at minimum.
SELECT replicate_rows(c1) FROM tab1;
Copy link
Member

Choose a reason for hiding this comment

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

Add one case SELECT replicate_rows()?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Sure.

* }}}
*/
@ExpressionDescription(
usage = "_FUNC_(n, expr1, ..., exprk) - Replicates `n`, `expr1`, ..., `exprk` into `n` rows.",
Copy link
Member

@viirya viirya May 6, 2018

Choose a reason for hiding this comment

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

I checked the design doc for INTERSECT ALL and EXCEPT ALL. Looks like the n is always stripped and useless after Generate operation. So why we need to keep n in ReplicateRows outputs? Can we do it like:

> SELECT _FUNC_(2, "val1", "val2");
  val1  val2
  val1  val2

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@viirya I did think about it Simon. But then, i decided to match the output with Hive.

@SparkQA
Copy link

SparkQA commented May 6, 2018

Test build #90262 has finished for PR 21240 at commit 748003a.

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

private val acceptedTypes = Seq(IntegerType, ShortType, ByteType)
override def coerceTypes(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions {
case s @ ReplicateRows(children) if s.children.nonEmpty && s.childrenResolved &&
s.children.head.dataType != LongType && acceptedTypes.contains(s.children.head.dataType) =>
Copy link
Member

Choose a reason for hiding this comment

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

nit: s.children.head.dataType != LongType is redundant because we have acceptedTypes.contains(...).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@viirya Thanks. I will fix.

2 val1 val2
2 val1 val2
""")
case class ReplicateRows(children: Seq[Expression]) extends Generator with CodegenFallback {
Copy link
Member

Choose a reason for hiding this comment

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

This can be easily implemented in codegen so we don't need CodegenFallback. We can deal with it in follow-up if you want.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@viirya If you don't mind, i would like to do it in a follow-up.

@viirya
Copy link
Member

viirya commented May 6, 2018

This generator function implementation itself LGTM. I have other thoughts regarding the rewrite rule but it's better to discuss on JIRA.

cc @cloud-fan @kiszk

@SparkQA
Copy link

SparkQA commented May 6, 2018

Test build #90266 has finished for PR 21240 at commit 1761068.

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

@SparkQA
Copy link

SparkQA commented May 6, 2018

Test build #90265 has finished for PR 21240 at commit 02ed058.

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

@dilipbiswal
Copy link
Contributor Author

retest this please

@SparkQA
Copy link

SparkQA commented May 6, 2018

Test build #90267 has finished for PR 21240 at commit 1761068.

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


override def eval(input: InternalRow): TraversableOnce[InternalRow] = {
val numRows = children.head.eval(input).asInstanceOf[Long]
val values = children.map(_.eval(input)).toArray
Copy link
Member

Choose a reason for hiding this comment

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

children.head seems getting evaluated twice here, can we avoid it?

@SparkQA
Copy link

SparkQA commented May 7, 2018

Test build #90287 has finished for PR 21240 at commit 4ab3af0.

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

@viirya
Copy link
Member

viirya commented May 7, 2018

retest this please.

@SparkQA
Copy link

SparkQA commented May 7, 2018

Test build #90295 has finished for PR 21240 at commit 4ab3af0.

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

@viirya
Copy link
Member

viirya commented May 7, 2018

retest this please.

@SparkQA
Copy link

SparkQA commented May 7, 2018

Test build #90303 has finished for PR 21240 at commit 4ab3af0.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds no public classes.

@viirya
Copy link
Member

viirya commented May 7, 2018

retest this please.

@SparkQA
Copy link

SparkQA commented May 7, 2018

Test build #90305 has finished for PR 21240 at commit 4ab3af0.

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

object ReplicateRowsCoercion extends TypeCoercionRule {
private val acceptedTypes = Seq(IntegerType, ShortType, ByteType)
override def coerceTypes(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions {
case s @ ReplicateRows(children) if s.children.nonEmpty && s.childrenResolved &&
Copy link
Member

Choose a reason for hiding this comment

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

children is not used. How about this?

      case s @ ReplicateRows(children) if children.nonEmpty && s.childrenResolved &&
          acceptedTypes.contains(children.head.dataType) =>
        ReplicateRows(Cast(children.head, LongType) +: children.tail)

if (numColumns < 2) {
TypeCheckResult.TypeCheckFailure(s"$prettyName requires at least 2 arguments.")
} else if (children.head.dataType != LongType) {
TypeCheckResult.TypeCheckFailure("The number of rows must be a positive long value.")
Copy link
Member

Choose a reason for hiding this comment

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

How about this message? The first argument type must be byte, short, int, or long, but ${children.head.dataType} found. BTW, it seems we don't reject negative values? (The current message says the number must be positive though...?)

-- Requires 2 arguments at minimum.
SELECT replicate_rows(c1) FROM tab1;

-- First argument should be a numeric type.
Copy link
Member

@maropu maropu May 8, 2018

Choose a reason for hiding this comment

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

nit: I think numeric generally includes float and double, too. integral type?

(1, 'row1', 1.1),
(2, 'row2', 2.2),
(0, 'row3', 3.3),
(-1,'row4', 4.4),
Copy link
Member

Choose a reason for hiding this comment

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

The current behaviour of the negative value case is the same with the hive one?

@gatorsmile
Copy link
Member

Like what @maropu commented at the beginning, replicate_rows might be too specific. @dilipbiswal Could you provide the other more general built-in functions that can also benefit the other cases too?

@HyukjinKwon
Copy link
Member

ping @dilipbiswal for an update.

@asfgit asfgit closed this in 1a4fda8 Jul 19, 2018
zifeif2 pushed a commit to zifeif2/spark that referenced this pull request Nov 22, 2025
Closes apache#17422
Closes apache#17619
Closes apache#18034
Closes apache#18229
Closes apache#18268
Closes apache#17973
Closes apache#18125
Closes apache#18918
Closes apache#19274
Closes apache#19456
Closes apache#19510
Closes apache#19420
Closes apache#20090
Closes apache#20177
Closes apache#20304
Closes apache#20319
Closes apache#20543
Closes apache#20437
Closes apache#21261
Closes apache#21726
Closes apache#14653
Closes apache#13143
Closes apache#17894
Closes apache#19758
Closes apache#12951
Closes apache#17092
Closes apache#21240
Closes apache#16910
Closes apache#12904
Closes apache#21731
Closes apache#21095

Added:
Closes apache#19233
Closes apache#20100
Closes apache#21453
Closes apache#21455
Closes apache#18477

Added:
Closes apache#21812
Closes apache#21787

Author: hyukjinkwon <[email protected]>

Closes apache#21781 from HyukjinKwon/closing-prs.
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.

6 participants