You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by ueshin <gi...@git.apache.org> on 2018/06/01 22:41:32 UTC

[GitHub] spark pull request #21282: [SPARK-23934][SQL] Adding map_from_entries functi...

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

    https://github.com/apache/spark/pull/21282#discussion_r192531374
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala ---
    @@ -118,6 +120,229 @@ case class MapValues(child: Expression)
       override def prettyName: String = "map_values"
     }
     
    +/**
    + * Returns a map created from the given array of entries.
    + */
    +@ExpressionDescription(
    +  usage = "_FUNC_(arrayOfEntries) - Returns a map created from the given array of entries.",
    +  examples = """
    +    Examples:
    +      > SELECT _FUNC_(array(struct(1, 'a'), struct(2, 'b')));
    +       {1:"a",2:"b"}
    +  """,
    +  since = "2.4.0")
    +case class MapFromEntries(child: Expression) extends UnaryExpression
    +{
    +  private lazy val resolvedDataType: Option[MapType] = child.dataType match {
    +    case ArrayType(
    +      StructType(Array(
    +        StructField(_, keyType, false, _),
    +        StructField(_, valueType, valueNullable, _))),
    +      false) => Some(MapType(keyType, valueType, valueNullable))
    +    case _ => None
    +  }
    +
    +  override def dataType: MapType = resolvedDataType.get
    +
    +  override def checkInputDataTypes(): TypeCheckResult = resolvedDataType match {
    +    case Some(_) => TypeCheckResult.TypeCheckSuccess
    +    case None => TypeCheckResult.TypeCheckFailure(s"'${child.sql}' is of " +
    +      s"${child.dataType.simpleString} type. $prettyName accepts only null-free arrays " +
    +      "of pair structs. Values of the first struct field can't contain nulls and produce " +
    +      "duplicates.")
    +  }
    +
    +  override protected def nullSafeEval(input: Any): Any = {
    +    val arrayData = input.asInstanceOf[ArrayData]
    +    val length = arrayData.numElements()
    +    val keyArray = new Array[AnyRef](length)
    +    val keySet = new OpenHashSet[AnyRef]()
    +    val valueArray = new Array[AnyRef](length)
    +    var i = 0;
    +    while (i < length) {
    +      val entry = arrayData.getStruct(i, 2)
    +      val key = entry.get(0, dataType.keyType)
    +      if (key == null) {
    +        throw new RuntimeException("The first field from a struct (key) can't be null.")
    +      }
    +      if (keySet.contains(key)) {
    --- End diff --
    
    I'm sorry for the super delay.
    Let's just ignore the duplicated key like `CreateMap` for now. We will need to discuss map-related topics, such as duplicate keys, equality or ordering, etc.


---

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