Github user ueshin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21045#discussion_r190181477
  
    --- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
 ---
    @@ -127,6 +127,176 @@ case class MapKeys(child: Expression)
       override def prettyName: String = "map_keys"
     }
     
    +@ExpressionDescription(
    +  usage = """_FUNC_(a1, a2, ...) - Returns a merged array containing in 
the N-th position the
    +  N-th value of each array given.""",
    +  examples = """
    +    Examples:
    +      > SELECT _FUNC_(array(1, 2, 3), array(2, 3, 4));
    +        [[1, 2], [2, 3], [3, 4]]
    +      > SELECT _FUNC_(array(1, 2), array(2, 3), array(3, 4));
    +        [[1, 2, 3], [2, 3, 4]]
    +  """,
    +  since = "2.4.0")
    +case class Zip(children: Seq[Expression]) extends Expression with 
ExpectsInputTypes {
    +
    +  override def inputTypes: Seq[AbstractDataType] = 
Seq.fill(children.length)(ArrayType)
    +
    +  override def dataType: DataType = ArrayType(mountSchema)
    +
    +  override def nullable: Boolean = children.forall(_.nullable)
    +
    +  private lazy val arrayTypes = 
children.map(_.dataType.asInstanceOf[ArrayType])
    +
    +  private lazy val arrayElementTypes = arrayTypes.map(_.elementType)
    +
    +
    +  def mountSchema: StructType = {
    +    val fields = arrayTypes.zipWithIndex.map { case (arr, idx) =>
    +      val fieldName = if (children(idx).isInstanceOf[NamedExpression]) {
    +          children(idx).asInstanceOf[NamedExpression].name
    +        } else {
    +          s"$idx"
    +        }
    +      StructField(fieldName, arr.elementType, children(idx).nullable || 
arr.containsNull)
    +    }
    --- End diff --
    
    How about:
    
    ```scala
    val fields = children.zip(arrayElementTypes).zipWithIndex.map {
      case ((expr: NamedExpression, elementType), _) =>
        StructField(expr.name, elementType, nullable = true)
      case ((_, elementType), idx) =>
        StructField(s"$idx", elementType, nullable = true)
    }
    ```
    
    ?


---

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

Reply via email to