You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by yhuai <gi...@git.apache.org> on 2015/11/15 01:54:58 UTC

[GitHub] spark pull request: [SPARK-11738] [SQL] Making ArrayType orderable

GitHub user yhuai opened a pull request:

    https://github.com/apache/spark/pull/9718

    [SPARK-11738] [SQL] Making ArrayType orderable

    https://issues.apache.org/jira/browse/SPARK-11738

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/yhuai/spark makingArrayOrderable

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/9718.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #9718
    
----
commit d49b21839cf69e91e25412d780239fca47aafcb1
Author: Yin Huai <yh...@databricks.com>
Date:   2015-11-14T22:43:01Z

    Make arrays orderable.

commit 519b5933699c644fa75e6db13aa649f6fc58fda1
Author: Yin Huai <yh...@databricks.com>
Date:   2015-11-14T22:56:49Z

    Allow array type in grouping expression.

commit f5f074d0f3cd0b194e94502e3f89b597700d92d2
Author: Yin Huai <yh...@databricks.com>
Date:   2015-11-14T23:26:58Z

    Array column is allowed in grouping expressions.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11738] [SQL] Making ArrayType orderable

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9718#issuecomment-156768943
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/45940/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11738] [SQL] Making ArrayType orderable

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9718#issuecomment-156829687
  
    **[Test build #45955 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45955/consoleFull)** for PR 9718 at commit [`aadfeab`](https://github.com/apache/spark/commit/aadfeabc37cbdb68e704df39ade42b253cf55348).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11738] [SQL] Making ArrayType orderable

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9718#issuecomment-156789219
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11738] [SQL] Making ArrayType orderable

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9718#issuecomment-156768936
  
    **[Test build #45940 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45940/consoleFull)** for PR 9718 at commit [`f5f074d`](https://github.com/apache/spark/commit/f5f074d0f3cd0b194e94502e3f89b597700d92d2).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11738] [SQL] Making ArrayType orderable

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9718#discussion_r44866037
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala ---
    @@ -29,35 +30,76 @@ class InterpretedOrdering(ordering: Seq[SortOrder]) extends Ordering[InternalRow
       def this(ordering: Seq[SortOrder], inputSchema: Seq[Attribute]) =
         this(ordering.map(BindReferences.bindReference(_, inputSchema)))
     
    +  private def compareValue(
    +      left: Any,
    +      right: Any,
    +      dataType: DataType,
    +      direction: SortDirection): Int = {
    +    if (left == null && right == null) {
    +      return 0
    +    } else if (left == null) {
    +      return if (direction == Ascending) -1 else 1
    +    } else if (right == null) {
    +      return if (direction == Ascending) 1 else -1
    +    } else {
    +      dataType match {
    +        case dt: AtomicType if direction == Ascending =>
    +          return dt.ordering.asInstanceOf[Ordering[Any]].compare(left, right)
    +        case dt: AtomicType if direction == Descending =>
    +          return dt.ordering.asInstanceOf[Ordering[Any]].reverse.compare(left, right)
    +        case s: StructType if direction == Ascending =>
    +          return s.interpretedOrdering.asInstanceOf[Ordering[Any]].compare(left, right)
    +        case s: StructType if direction == Descending =>
    +          return s.interpretedOrdering.asInstanceOf[Ordering[Any]].reverse.compare(left, right)
    +        case a: ArrayType =>
    +          val leftArray = left.asInstanceOf[ArrayData]
    +          val rightArray = right.asInstanceOf[ArrayData]
    +          val minLength = scala.math.min(leftArray.numElements(), rightArray.numElements())
    +          var i = 0
    +          while (i < minLength) {
    +            val isNullLeft = leftArray.isNullAt(i)
    +            val isNullRight = rightArray.isNullAt(i)
    +            if (isNullLeft && isNullRight) {
    +              // Do nothing.
    +            } else if (isNullLeft) {
    +              return if (direction == Ascending) -1 else 1
    +            } else if (isNullRight) {
    +              return if (direction == Ascending) 1 else -1
    +            } else {
    +              val comp =
    +                compareValue(
    +                  leftArray.get(i, a.elementType),
    +                  rightArray.get(i, a.elementType),
    +                  a.elementType,
    +                  direction)
    +              if (comp != 0) {
    +                return comp
    +              }
    +            }
    +            i += 1
    +          }
    +          if (leftArray.numElements() < rightArray.numElements()) {
    +            return if (direction == Ascending) -1 else 1
    +          } else if (leftArray.numElements() > rightArray.numElements()) {
    +            return if (direction == Ascending) 1 else -1
    +          } else {
    +            return 0
    +          }
    +        case other =>
    +          throw new IllegalArgumentException(s"Type $other does not support ordered operations")
    +      }
    +    }
    +  }
    +
       def compare(a: InternalRow, b: InternalRow): Int = {
         var i = 0
         while (i < ordering.size) {
           val order = ordering(i)
           val left = order.child.eval(a)
           val right = order.child.eval(b)
    -
    -      if (left == null && right == null) {
    -        // Both null, continue looking.
    -      } else if (left == null) {
    -        return if (order.direction == Ascending) -1 else 1
    -      } else if (right == null) {
    -        return if (order.direction == Ascending) 1 else -1
    -      } else {
    -        val comparison = order.dataType match {
    -          case dt: AtomicType if order.direction == Ascending =>
    -            dt.ordering.asInstanceOf[Ordering[Any]].compare(left, right)
    -          case dt: AtomicType if order.direction == Descending =>
    -            dt.ordering.asInstanceOf[Ordering[Any]].reverse.compare(left, right)
    -          case s: StructType if order.direction == Ascending =>
    -            s.interpretedOrdering.asInstanceOf[Ordering[Any]].compare(left, right)
    -          case s: StructType if order.direction == Descending =>
    -            s.interpretedOrdering.asInstanceOf[Ordering[Any]].reverse.compare(left, right)
    -          case other =>
    -            throw new IllegalArgumentException(s"Type $other does not support ordered operations")
    -        }
    -        if (comparison != 0) {
    -          return comparison
    -        }
    --- End diff --
    
    These lines have been moved to `compareValue`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11738] [SQL] Making ArrayType orderable

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9718#discussion_r44868778
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -267,6 +267,55 @@ class CodeGenContext {
         case dt: DataType if isPrimitiveType(dt) => s"($c1 > $c2 ? 1 : $c1 < $c2 ? -1 : 0)"
         case BinaryType => s"org.apache.spark.sql.catalyst.util.TypeUtils.compareBinary($c1, $c2)"
         case NullType => "0"
    +    case array: ArrayType =>
    +      val elementType = array.elementType
    +      val elementA = freshName("elementA")
    +      val isNullA = freshName("isNullA")
    +      val elementB = freshName("elementB")
    +      val isNullB = freshName("isNullB")
    +      val compareFunc = freshName("compareArray")
    +      val i = freshName("i")
    +      val minLength = freshName("minLength")
    +      val funcCode: String =
    +        s"""
    +          public int $compareFunc(ArrayData a, ArrayData b) {
    +            int lengthA = a.numElements();
    +            int lengthB = b.numElements();
    +            int $minLength = (lengthA > lengthB) ? lengthB : lengthA;
    +            boolean $isNullA;
    +            boolean $isNullB;
    +            ${javaType(elementType)} $elementA;
    +            ${javaType(elementType)} $elementB;
    --- End diff --
    
    These could be defined in the loop (let compiler to optimize them easily)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11738] [SQL] Making ArrayType orderable

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on the pull request:

    https://github.com/apache/spark/pull/9718#issuecomment-156767526
  
    @davies take a look?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11738] [SQL] Making ArrayType orderable

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9718#issuecomment-156775646
  
    **[Test build #45945 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45945/consoleFull)** for PR 9718 at commit [`f43a7f9`](https://github.com/apache/spark/commit/f43a7f9c92dd4b5dc5e37b1fa41f8f8c00ca3020).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11738] [SQL] Making ArrayType orderable

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9718#issuecomment-156844715
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/45955/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11738] [SQL] Making ArrayType orderable

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9718#issuecomment-156768306
  
    **[Test build #45940 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45940/consoleFull)** for PR 9718 at commit [`f5f074d`](https://github.com/apache/spark/commit/f5f074d0f3cd0b194e94502e3f89b597700d92d2).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11738] [SQL] Making ArrayType orderable

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9718#issuecomment-156783662
  
    **[Test build #45946 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45946/consoleFull)** for PR 9718 at commit [`7228093`](https://github.com/apache/spark/commit/72280936d32a4e5c7fc0171670553caa52938a80).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11738] [SQL] Making ArrayType orderable

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9718#issuecomment-156782998
  
    **[Test build #45945 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45945/consoleFull)** for PR 9718 at commit [`f43a7f9`](https://github.com/apache/spark/commit/f43a7f9c92dd4b5dc5e37b1fa41f8f8c00ca3020).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11738] [SQL] Making ArrayType orderable

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/9718#issuecomment-156789726
  
    LGTM, and some minor comments


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11738] [SQL] Making ArrayType orderable

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9718#issuecomment-156789220
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/45946/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11738] [SQL] Making ArrayType orderable

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9718#discussion_r44866039
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala ---
    @@ -49,40 +47,6 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper {
         futures.foreach(Await.result(_, 10.seconds))
       }
     
    -  // Test GenerateOrdering for all common types. For each type, we construct random input rows that
    -  // contain two columns of that type, then for pairs of randomly-generated rows we check that
    -  // GenerateOrdering agrees with RowOrdering.
    -  (DataTypeTestUtils.atomicTypes ++ Set(NullType)).foreach { dataType =>
    -    test(s"GenerateOrdering with $dataType") {
    -      val rowOrdering = InterpretedOrdering.forSchema(Seq(dataType, dataType))
    -      val genOrdering = GenerateOrdering.generate(
    -        BoundReference(0, dataType, nullable = true).asc ::
    -          BoundReference(1, dataType, nullable = true).asc :: Nil)
    -      val rowType = StructType(
    -        StructField("a", dataType, nullable = true) ::
    -          StructField("b", dataType, nullable = true) :: Nil)
    -      val maybeDataGenerator = RandomDataGenerator.forType(rowType, nullable = false)
    -      assume(maybeDataGenerator.isDefined)
    -      val randGenerator = maybeDataGenerator.get
    -      val toCatalyst = CatalystTypeConverters.createToCatalystConverter(rowType)
    -      for (_ <- 1 to 50) {
    -        val a = toCatalyst(randGenerator()).asInstanceOf[InternalRow]
    -        val b = toCatalyst(randGenerator()).asInstanceOf[InternalRow]
    -        withClue(s"a = $a, b = $b") {
    -          assert(genOrdering.compare(a, a) === 0)
    -          assert(genOrdering.compare(b, b) === 0)
    -          assert(rowOrdering.compare(a, a) === 0)
    -          assert(rowOrdering.compare(b, b) === 0)
    -          assert(signum(genOrdering.compare(a, b)) === -1 * signum(genOrdering.compare(b, a)))
    -          assert(signum(rowOrdering.compare(a, b)) === -1 * signum(rowOrdering.compare(b, a)))
    -          assert(
    -            signum(rowOrdering.compare(a, b)) === signum(genOrdering.compare(a, b)),
    -            "Generated and non-generated orderings should agree")
    -        }
    -      }
    -    }
    -  }
    --- End diff --
    
    These lines are not in `OrderingSuite`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11738] [SQL] Making ArrayType orderable

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9718#discussion_r44868784
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -267,6 +267,55 @@ class CodeGenContext {
         case dt: DataType if isPrimitiveType(dt) => s"($c1 > $c2 ? 1 : $c1 < $c2 ? -1 : 0)"
         case BinaryType => s"org.apache.spark.sql.catalyst.util.TypeUtils.compareBinary($c1, $c2)"
         case NullType => "0"
    +    case array: ArrayType =>
    +      val elementType = array.elementType
    +      val elementA = freshName("elementA")
    +      val isNullA = freshName("isNullA")
    +      val elementB = freshName("elementB")
    +      val isNullB = freshName("isNullB")
    +      val compareFunc = freshName("compareArray")
    +      val i = freshName("i")
    +      val minLength = freshName("minLength")
    +      val funcCode: String =
    +        s"""
    +          public int $compareFunc(ArrayData a, ArrayData b) {
    +            int lengthA = a.numElements();
    +            int lengthB = b.numElements();
    +            int $minLength = (lengthA > lengthB) ? lengthB : lengthA;
    +            boolean $isNullA;
    +            boolean $isNullB;
    +            ${javaType(elementType)} $elementA;
    +            ${javaType(elementType)} $elementB;
    +            for (int $i = 0; $i < $minLength; $i++) {
    --- End diff --
    
    `i` should be enough here


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11738] [SQL] Making ArrayType orderable

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/9718


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11738] [SQL] Making ArrayType orderable

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9718#issuecomment-156783008
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/45945/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11738] [SQL] Making ArrayType orderable

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/9718#issuecomment-156859416
  
    Merged into master and 1.6 branch, thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11738] [SQL] Making ArrayType orderable

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9718#issuecomment-156783007
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11738] [SQL] Making ArrayType orderable

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9718#issuecomment-156844690
  
    **[Test build #45955 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45955/consoleFull)** for PR 9718 at commit [`aadfeab`](https://github.com/apache/spark/commit/aadfeabc37cbdb68e704df39ade42b253cf55348).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11738] [SQL] Making ArrayType orderable

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9718#issuecomment-156789202
  
    **[Test build #45946 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45946/consoleFull)** for PR 9718 at commit [`7228093`](https://github.com/apache/spark/commit/72280936d32a4e5c7fc0171670553caa52938a80).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11738] [SQL] Making ArrayType orderable

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9718#issuecomment-156844714
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11738] [SQL] Making ArrayType orderable

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9718#issuecomment-156768942
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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