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

[GitHub] spark pull request #22014: [SPARK-25036][SQL] avoid match may not be exhaust...

GitHub user kiszk opened a pull request:

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

    [SPARK-25036][SQL] avoid match may not be exhaustive in Scala-2.12

    ## What changes were proposed in this pull request?
    
    The PR remove the following compilation error using scala-2.12 with sbt by adding a default case to `match`.
    
    ```
    /home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/ValueInterval.scala:63: match may not be exhaustive.
    [error] It would fail on the following inputs: (NumericValueInterval(_, _), _), (_, NumericValueInterval(_, _)), (_, _)
    [error] [warn]   def isIntersected(r1: ValueInterval, r2: ValueInterval): Boolean = (r1, r2) match {
    [error] [warn] 
    [error] [warn] /home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/ValueInterval.scala:79: match may not be exhaustive.
    [error] It would fail on the following inputs: (NumericValueInterval(_, _), _), (_, NumericValueInterval(_, _)), (_, _)
    [error] [warn]     (r1, r2) match {
    [error] [warn] 
    [error] [warn] /home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproxCountDistinctForIntervals.scala:67: match may not be exhaustive.
    [error] It would fail on the following inputs: (ArrayType(_, _), _), (_, ArrayData()), (_, _)
    [error] [warn]     (endpointsExpression.dataType, endpointsExpression.eval()) match {
    [error] [warn] 
    [error] [warn] /home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala:470: match may not be exhaustive.
    [error] It would fail on the following inputs: NewFunctionSpec(_, None, Some(_)), NewFunctionSpec(_, Some(_), None)
    [error] [warn]     newFunction match {
    [error] [warn] 
    [error] [warn] [error] [warn] /home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala:709: match may not be exhaustive.
    [error] It would fail on the following input: Schema((x: org.apache.spark.sql.types.DataType forSome x not in org.apache.spark.sql.types.StructType), _)
    [error] [warn]   def attributesFor[T: TypeTag]: Seq[Attribute] = schemaFor[T] match {
    [error] [warn] 
    ```
    
    ## How was this patch tested?
    
    Existing UTs with Scala-2.11.
    Manually build with Scala-2.12

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

    $ git pull https://github.com/kiszk/spark SPARK-25036b

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

    https://github.com/apache/spark/pull/22014.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 #22014
    
----
commit 9cc0c60611d413b363718066f246926f47e03ffd
Author: Kazuaki Ishizaki <is...@...>
Date:   2018-08-06T19:24:08Z

    add default case to match

----


---

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


[GitHub] spark pull request #22014: [SPARK-25036][SQL] avoid match may not be exhaust...

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

    https://github.com/apache/spark/pull/22014#discussion_r208091824
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala ---
    @@ -709,6 +709,7 @@ object ScalaReflection extends ScalaReflection {
       def attributesFor[T: TypeTag]: Seq[Attribute] = schemaFor[T] match {
         case Schema(s: StructType, _) =>
           s.toAttributes
    +    case _ => throw new RuntimeException(s"$schemaFor is not supported at attributesFor()")
    --- End diff --
    
    How about this:
    
    ```scala
        case other =>
          throw new UnsupportedOperationException(s"Attributes for type $other is not supported")
    ```


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

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

    https://github.com/apache/spark/pull/22014
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1923/
    Test PASSed.


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

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

    https://github.com/apache/spark/pull/22014
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1925/
    Test PASSed.


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

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

    https://github.com/apache/spark/pull/22014
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

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

    https://github.com/apache/spark/pull/22014
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

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

    https://github.com/apache/spark/pull/22014
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/22014
  
    LGTM except those rather nits.


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

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

    https://github.com/apache/spark/pull/22014
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

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

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


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

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

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


---

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


[GitHub] spark pull request #22014: [SPARK-25036][SQL] avoid match may not be exhaust...

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

    https://github.com/apache/spark/pull/22014#discussion_r208436646
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala ---
    @@ -87,7 +87,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro
           // For top level row writer, it always writes to the beginning of the global buffer holder,
           // which means its fixed-size region always in the same position, so we don't need to call
           // `reset` to set up its fixed-size region every time.
    -      if (inputs.map(_.isNull).forall(_ == "false")) {
    --- End diff --
    
    Sorry, I made a mistake...


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

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

    https://github.com/apache/spark/pull/22014
  
    **[Test build #94390 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94390/testReport)** for PR 22014 at commit [`3cfbcfc`](https://github.com/apache/spark/commit/3cfbcfc5fd0099ded2a5bd2b5ff1ef9278135285).


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

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

    https://github.com/apache/spark/pull/22014
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

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

    https://github.com/apache/spark/pull/22014
  
    **[Test build #94314 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94314/testReport)** for PR 22014 at commit [`9cc0c60`](https://github.com/apache/spark/commit/9cc0c60611d413b363718066f246926f47e03ffd).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

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

    https://github.com/apache/spark/pull/22014
  
    **[Test build #94388 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94388/testReport)** for PR 22014 at commit [`3cfbcfc`](https://github.com/apache/spark/commit/3cfbcfc5fd0099ded2a5bd2b5ff1ef9278135285).


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/22014
  
    Retest this please.


---

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


[GitHub] spark pull request #22014: [SPARK-25036][SQL] avoid match may not be exhaust...

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

    https://github.com/apache/spark/pull/22014#discussion_r208091445
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproxCountDistinctForIntervals.scala ---
    @@ -67,6 +67,7 @@ case class ApproxCountDistinctForIntervals(
         (endpointsExpression.dataType, endpointsExpression.eval()) match {
           case (ArrayType(elementType, _), arrayData: ArrayData) =>
             arrayData.toObjectArray(elementType).map(_.toString.toDouble)
    +      case _ => throw new RuntimeException("not found at endpoints")
    --- End diff --
    
    Can we do this like:
    
    ```scala
        val endpointsType = endpointsExpression.dataType.asInstanceOf[ArrayType]
        val endpoints = endpointsExpression.eval().asInstanceOf[ArrayData]
        endpoints.toObjectArray(endpointsType.elementType).map(_.toString.toDouble)
    ```



---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

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

    https://github.com/apache/spark/pull/22014
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1870/
    Test PASSed.


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

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

    https://github.com/apache/spark/pull/22014
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

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

    https://github.com/apache/spark/pull/22014
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1934/
    Test PASSed.


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

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

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


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

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

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


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

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

    https://github.com/apache/spark/pull/22014
  
    **[Test build #94402 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94402/testReport)** for PR 22014 at commit [`b9c11d5`](https://github.com/apache/spark/commit/b9c11d5de01f13258e2c41b78396dd21a269c0d2).


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

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

    https://github.com/apache/spark/pull/22014
  
    **[Test build #94390 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94390/testReport)** for PR 22014 at commit [`3cfbcfc`](https://github.com/apache/spark/commit/3cfbcfc5fd0099ded2a5bd2b5ff1ef9278135285).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on the issue:

    https://github.com/apache/spark/pull/22014
  
    cc @srowen 


---

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


[GitHub] spark pull request #22014: [SPARK-25036][SQL] avoid match may not be exhaust...

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

    https://github.com/apache/spark/pull/22014#discussion_r208090085
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala ---
    @@ -471,6 +471,7 @@ class CodegenContext {
           case NewFunctionSpec(functionName, None, None) => functionName
           case NewFunctionSpec(functionName, Some(_), Some(innerClassInstance)) =>
             innerClassInstance + "." + functionName
    +      case _ => null // nothing to do since addNewFunctionInteral() must return one of them
    --- End diff --
    
    Shall we throw an `IllegalArgumentException`? 


---

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


[GitHub] spark pull request #22014: [SPARK-25036][SQL] avoid match may not be exhaust...

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

    https://github.com/apache/spark/pull/22014#discussion_r208089613
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/ValueInterval.scala ---
    @@ -86,6 +87,7 @@ object ValueInterval {
             val newMax = if (n1.max <= n2.max) n1.max else n2.max
             (Some(EstimationUtils.fromDouble(newMin, dt)),
               Some(EstimationUtils.fromDouble(newMax, dt)))
    +      case _ => throw new RuntimeException(s"Not supported pair: $r1, $r2 at intersect()")
    --- End diff --
    
    Shall we do `UnsupportedOperationException`?


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

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

    https://github.com/apache/spark/pull/22014
  
    **[Test build #94314 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94314/testReport)** for PR 22014 at commit [`9cc0c60`](https://github.com/apache/spark/commit/9cc0c60611d413b363718066f246926f47e03ffd).


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

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

    https://github.com/apache/spark/pull/22014
  
    **[Test build #94402 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94402/testReport)** for PR 22014 at commit [`b9c11d5`](https://github.com/apache/spark/commit/b9c11d5de01f13258e2c41b78396dd21a269c0d2).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #22014: [SPARK-25036][SQL] avoid match may not be exhaust...

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

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


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

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

    https://github.com/apache/spark/pull/22014
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

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

    https://github.com/apache/spark/pull/22014
  
    **[Test build #94388 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94388/testReport)** for PR 22014 at commit [`3cfbcfc`](https://github.com/apache/spark/commit/3cfbcfc5fd0099ded2a5bd2b5ff1ef9278135285).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #22014: [SPARK-25036][SQL] avoid match may not be exhaust...

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

    https://github.com/apache/spark/pull/22014#discussion_r208406147
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala ---
    @@ -87,7 +87,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro
           // For top level row writer, it always writes to the beginning of the global buffer holder,
           // which means its fixed-size region always in the same position, so we don't need to call
           // `reset` to set up its fixed-size region every time.
    -      if (inputs.map(_.isNull).forall(_ == "false")) {
    --- End diff --
    
    @kiszk  was this intentional?


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/22014
  
    Merged to master


---

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


[GitHub] spark issue #22014: [SPARK-25036][SQL] avoid match may not be exhaustive in ...

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

    https://github.com/apache/spark/pull/22014
  
    Merged build finished. Test PASSed.


---

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