Skip to content

Commit

Permalink
[SPARK-24042][SQL] Collection function: zip_with_index
Browse files Browse the repository at this point in the history
  • Loading branch information
mn-mikke authored and mn-mikke committed Apr 21, 2018
1 parent 6c4f8d0 commit d34c99c
Show file tree
Hide file tree
Showing 7 changed files with 401 additions and 74 deletions.
18 changes: 18 additions & 0 deletions python/pyspark/sql/functions.py
Original file line number Diff line number Diff line change
Expand Up @@ -2064,6 +2064,24 @@ def reverse(col):
return Column(sc._jvm.functions.reverse(_to_java_column(col)))


@since(2.4)
def zip_with_index(col, indexFirst=False):
"""
Collection function: transforms the input array by encapsulating elements into pairs
with indexes indicating the order.
:param col: name of column or expression
>>> df = spark.createDataFrame([([2, 5, 3],), ([],)], ['data'])
>>> df.select(zip_with_index(df.data).alias('r')).collect()
[Row(r=[[value=2, index=0], [value=5, index=1], [value=3, index=2]]), Row(r=[])]
>>> df.select(zip_with_index(df.data, indexFirst=True).alias('r')).collect()
[Row(r=[[index=0, value=2], [index=1, value=5], [index=2, value=3]]), Row(r=[])]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.zip_with_index(_to_java_column(col), indexFirst))


@since(2.4)
def flatten(col):
"""
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -409,6 +409,7 @@ object FunctionRegistry {
expression[Concat]("concat"),
expression[Flatten]("flatten"),
expression[Reverse]("reverse"),
expression[ZipWithIndex]("zip_with_index"),
CreateStruct.registryEntry,

// misc functions
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions

import java.util.Comparator

import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
import org.apache.spark.sql.catalyst.expressions.codegen._
Expand Down Expand Up @@ -765,3 +766,139 @@ case class Flatten(child: Expression) extends UnaryExpression {

override def prettyName: String = "flatten"
}

/**
* Returns the maximum value in the array.
*/
// scalastyle:off line.size.limit
@ExpressionDescription(
usage = "_FUNC_(array[, indexFirst]) - Transforms the input array by encapsulating elements into pairs with indexes indicating the order.",
examples = """
Examples:
> SELECT _FUNC_(array("d", "a", null, "b"));
[("d",0),("a",1),(null,2),("b",3)]
> SELECT _FUNC_(array("d", "a", null, "b"), true);
[(0,"d"),(1,"a"),(2,null),(3,"b")]
""",
since = "2.4.0")
case class ZipWithIndex(child: Expression, indexFirst: Expression)
extends UnaryExpression with ExpectsInputTypes {

def this(e: Expression) = this(e, Literal.FalseLiteral)

val indexFirstValue: Boolean = indexFirst match {
case Literal(v: Boolean, BooleanType) => v
case _ => throw new AnalysisException("The second argument has to be a boolean constant.")
}

private val MAX_ARRAY_LENGTH: Int = ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH

override def inputTypes: Seq[AbstractDataType] = Seq(ArrayType)

lazy val childArrayType: ArrayType = child.dataType.asInstanceOf[ArrayType]

override def dataType: DataType = {
val elementField = StructField("value", childArrayType.elementType, childArrayType.containsNull)
val indexField = StructField("index", IntegerType, false)

val fields = if (indexFirstValue) Seq(indexField, elementField) else Seq(elementField, indexField)

ArrayType(StructType(fields), false)
}

override protected def nullSafeEval(input: Any): Any = {
val array = input.asInstanceOf[ArrayData].toObjectArray(childArrayType.elementType)

val makeStruct = (v: Any, i: Int) => if (indexFirstValue) InternalRow(i, v) else InternalRow(v, i)
val resultData = array.zipWithIndex.map{case (v, i) => makeStruct(v, i)}

new GenericArrayData(resultData)
}

override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
nullSafeCodeGen(ctx, ev, c => {
if (ctx.isPrimitiveType(childArrayType.elementType)) {
genCodeForPrimitiveElements(ctx, c, ev.value)
} else {
genCodeForNonPrimitiveElements(ctx, c, ev.value)
}
})
}

private def genCodeForPrimitiveElements(
ctx: CodegenContext,
childVariableName: String,
arrayData: String): String = {
val numElements = ctx.freshName("numElements")
val byteArraySize = ctx.freshName("byteArraySize")
val data = ctx.freshName("byteArray")
val unsafeRow = ctx.freshName("unsafeRow")
val structSize = ctx.freshName("structSize")
val unsafeArrayData = ctx.freshName("unsafeArrayData")
val structsOffset = ctx.freshName("structsOffset")
val calculateArraySize = "UnsafeArrayData.calculateSizeOfUnderlyingByteArray"
val calculateHeader = "UnsafeArrayData.calculateHeaderPortionInBytes"

val baseOffset = Platform.BYTE_ARRAY_OFFSET
val longSize = LongType.defaultSize
val primitiveValueTypeName = ctx.primitiveTypeName(childArrayType.elementType)
val valuePosition = if (indexFirstValue) "1" else "0"
val indexPosition = if (indexFirstValue) "0" else "1"
s"""
|final int $numElements = $childVariableName.numElements();
|final int $structSize = ${UnsafeRow.calculateBitSetWidthInBytes(2) + longSize * 2};
|final long $byteArraySize = $calculateArraySize($numElements, $longSize + $structSize);
|final int $structsOffset = $calculateHeader($numElements) + $numElements * $longSize;
|if ($byteArraySize > $MAX_ARRAY_LENGTH) {
| throw new RuntimeException("Unsuccessful try to zip array with index due to exceeding" +
| " the limit $MAX_ARRAY_LENGTH bytes for UnsafeArrayData. " + $byteArraySize +
| " bytes of data are required for performing the operation with the given array.");
|}
|final byte[] $data = new byte[(int)$byteArraySize];
|UnsafeArrayData $unsafeArrayData = new UnsafeArrayData();
|Platform.putLong($data, $baseOffset, $numElements);
|$unsafeArrayData.pointTo($data, $baseOffset, (int)$byteArraySize);
|UnsafeRow $unsafeRow = new UnsafeRow(2);
|for (int z = 0; z < $numElements; z++) {
| long offset = $structsOffset + z * $structSize;
| $unsafeArrayData.setLong(z, (offset << 32) + $structSize);
| $unsafeRow.pointTo($data, $baseOffset + offset, $structSize);
| if ($childVariableName.isNullAt(z)) {
| $unsafeRow.setNullAt($valuePosition);
| } else {
| $unsafeRow.set$primitiveValueTypeName(
| $valuePosition,
| ${ctx.getValue(childVariableName, childArrayType.elementType, "z")}
| );
| }
| $unsafeRow.setInt($indexPosition, z);
|}
|$arrayData = $unsafeArrayData;
""".stripMargin
}

private def genCodeForNonPrimitiveElements(
ctx: CodegenContext,
childVariableName: String,
arrayData: String): String = {
val genericArrayClass = classOf[GenericArrayData].getName
val rowClass = classOf[GenericInternalRow].getName
val numberOfElements = ctx.freshName("numElements")
val data = ctx.freshName("internalRowArray")

val elementValue = ctx.getValue(childVariableName, childArrayType.elementType, "z")
val arguments = if (indexFirstValue) s"z, $elementValue" else s"$elementValue, z"

s"""
|final int $numberOfElements = $childVariableName.numElements();
|final Object[] $data = new Object[$numberOfElements];
|for (int z = 0; z < $numberOfElements; z++) {
| $data[z] = new $rowClass(new Object[]{$arguments});
|}
|$arrayData = new $genericArrayClass($data);
""".stripMargin
}

override def prettyName: String = "zip_with_index"
}

Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.spark.sql.catalyst.expressions

import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.types._

class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
Expand Down Expand Up @@ -286,4 +287,57 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper
checkEvaluation(Reverse(as7), null)
checkEvaluation(Reverse(aa), Seq(Seq("e"), Seq("c", "d"), Seq("a", "b")))
}

test("Zip With Index") {
def r(values: Any*): InternalRow = create_row(values: _*)
val t = Literal.TrueLiteral
val f = Literal.FalseLiteral

// Primitive-type elements
val ai0 = Literal.create(Seq(2, 8, 4, 7), ArrayType(IntegerType))
val ai1 = Literal.create(Seq(null, 4, null, 2), ArrayType(IntegerType))
val ai2 = Literal.create(Seq(null, null, null), ArrayType(IntegerType))
val ai3 = Literal.create(Seq(1), ArrayType(IntegerType))
val ai4 = Literal.create(Seq.empty, ArrayType(IntegerType))
val ai5 = Literal.create(null, ArrayType(IntegerType))

checkEvaluation(ZipWithIndex(ai0, f), Seq(r(2, 0), r(8, 1), r(4, 2), r(7, 3)))
checkEvaluation(ZipWithIndex(ai1, f), Seq(r(null, 0), r(4, 1), r(null, 2), r(2, 3)))
checkEvaluation(ZipWithIndex(ai2, f), Seq(r(null, 0), r(null, 1), r(null, 2)))
checkEvaluation(ZipWithIndex(ai3, f), Seq(r(1, 0)))
checkEvaluation(ZipWithIndex(ai4, f), Seq.empty)
checkEvaluation(ZipWithIndex(ai5, f), null)

checkEvaluation(ZipWithIndex(ai0, t), Seq(r(0, 2), r(1, 8), r(2, 4), r(3, 7)))
checkEvaluation(ZipWithIndex(ai1, t), Seq(r(0, null), r(1, 4), r(2, null), r(3, 2)))
checkEvaluation(ZipWithIndex(ai2, t), Seq(r(0, null), r(1, null), r(2, null)))
checkEvaluation(ZipWithIndex(ai3, t), Seq(r(0, 1)))
checkEvaluation(ZipWithIndex(ai4, t), Seq.empty)
checkEvaluation(ZipWithIndex(ai5, t), null)

// Non-primitive-type elements
val as0 = Literal.create(Seq("b", "a", "y", "z"), ArrayType(StringType))
val as1 = Literal.create(Seq(null, "x", null, "y"), ArrayType(StringType))
val as2 = Literal.create(Seq(null, null, null), ArrayType(StringType))
val as3 = Literal.create(Seq("a"), ArrayType(StringType))
val as4 = Literal.create(Seq.empty, ArrayType(StringType))
val as5 = Literal.create(null, ArrayType(StringType))
val aas = Literal.create(Seq(Seq("e"), Seq("c", "d")), ArrayType(ArrayType(StringType)))

checkEvaluation(ZipWithIndex(as0, f), Seq(r("b", 0), r("a", 1), r("y", 2), r("z", 3)))
checkEvaluation(ZipWithIndex(as1, f), Seq(r(null, 0), r("x", 1), r(null, 2), r("y", 3)))
checkEvaluation(ZipWithIndex(as2, f), Seq(r(null, 0), r(null, 1), r(null, 2)))
checkEvaluation(ZipWithIndex(as3, f), Seq(r("a", 0)))
checkEvaluation(ZipWithIndex(as4, f), Seq.empty)
checkEvaluation(ZipWithIndex(as5, f), null)
checkEvaluation(ZipWithIndex(aas, f), Seq(r(Seq("e"), 0), r(Seq("c", "d"), 1)))

checkEvaluation(ZipWithIndex(as0, t), Seq(r(0, "b"), r(1, "a"), r(2, "y"), r(3, "z")))
checkEvaluation(ZipWithIndex(as1, t), Seq(r(0, null), r(1, "x"), r(2, null), r(3, "y")))
checkEvaluation(ZipWithIndex(as2, t), Seq(r(0, null), r(1, null), r(2, null)))
checkEvaluation(ZipWithIndex(as3, t), Seq(r(0, "a")))
checkEvaluation(ZipWithIndex(as4, t), Seq.empty)
checkEvaluation(ZipWithIndex(as5, t), null)
checkEvaluation(ZipWithIndex(aas, t), Seq(r(0, Seq("e")), r(1, Seq("c", "d"))))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import org.scalatest.prop.GeneratorDrivenPropertyChecks

import org.apache.spark.{SparkConf, SparkFunSuite}
import org.apache.spark.serializer.JavaSerializer
import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
import org.apache.spark.sql.catalyst.analysis.{ResolveTimeZone, SimpleAnalyzer}
import org.apache.spark.sql.catalyst.expressions.codegen._
Expand Down Expand Up @@ -88,6 +89,10 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks {
if (expected.isNaN) result.isNaN else expected == result
case (result: Float, expected: Float) =>
if (expected.isNaN) result.isNaN else expected == result
case (result: InternalRow, expected: InternalRow) =>
val structType = dataType.asInstanceOf[StructType]
result.toSeq(structType) == expected.toSeq(structType)
case (result: Row, expected: InternalRow) => result.toSeq == expected.toSeq(result.schema)
case _ =>
result == expected
}
Expand Down
11 changes: 11 additions & 0 deletions sql/core/src/main/scala/org/apache/spark/sql/functions.scala
Original file line number Diff line number Diff line change
Expand Up @@ -3249,6 +3249,17 @@ object functions {
*/
def flatten(e: Column): Column = withExpr { Flatten(e.expr) }

/**
* Transforms the input array by encapsulating elements into pairs
* with indexes indicating the order.
*
* @group collection_funcs
* @since 2.4.0
*/
def zip_with_index(e: Column, indexFirst: Boolean = false): Column = withExpr {
ZipWithIndex(e.expr, Literal(indexFirst))
}

/**
* Returns an unordered array containing the keys of the map.
* @group collection_funcs
Expand Down
Loading

0 comments on commit d34c99c

Please sign in to comment.