You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by cloud-fan <gi...@git.apache.org> on 2016/06/21 11:35:37 UTC

[GitHub] spark pull request #13807: [SPARK-16097][SQL] Encoders.tuple should handle n...

GitHub user cloud-fan opened a pull request:

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

    [SPARK-16097][SQL] Encoders.tuple should handle null object correctly

    ## What changes were proposed in this pull request?
    
    Although the top level input object can not be null, but when we use `Encoders.tuple` to combine 2 encoders, their input objects are not top level anymore and can be null. We should handle this case.
    
    
    ## How was this patch tested?
    
    new test in DatasetSuite


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

    $ git pull https://github.com/cloud-fan/spark bug

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

    https://github.com/apache/spark/pull/13807.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 #13807
    
----
commit 45f74f624ba8774c3735896915995c4d81785884
Author: Wenchen Fan <we...@databricks.com>
Date:   2016-06-21T11:31:46Z

    Encoders.tuple should handle null object correctly

----


---
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 issue #13807: [SPARK-16097][SQL] Encoders.tuple should handle null obj...

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

    https://github.com/apache/spark/pull/13807
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/60978/
    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 #13807: [SPARK-16097][SQL] Encoders.tuple should handle n...

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

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


---
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 issue #13807: [SPARK-16097][SQL] Encoders.tuple should handle null obj...

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

    https://github.com/apache/spark/pull/13807
  
    **[Test build #60929 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60929/consoleFull)** for PR 13807 at commit [`45f74f6`](https://github.com/apache/spark/commit/45f74f624ba8774c3735896915995c4d81785884).
     * 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 issue #13807: [SPARK-16097][SQL] Encoders.tuple should handle null obj...

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

    https://github.com/apache/spark/pull/13807
  
    cc @yhuai @liancheng @clockfly


---
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 #13807: [SPARK-16097][SQL] Encoders.tuple should handle n...

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

    https://github.com/apache/spark/pull/13807#discussion_r67965175
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala ---
    @@ -110,16 +110,28 @@ object ExpressionEncoder {
     
         val cls = Utils.getContextOrSparkClassLoader.loadClass(s"scala.Tuple${encoders.size}")
     
    -    val serializer = encoders.map {
    -      case e if e.flat => e.serializer.head
    -      case other => CreateStruct(other.serializer)
    -    }.zipWithIndex.map { case (expr, index) =>
    -      expr.transformUp {
    -        case BoundReference(0, t, _) =>
    -          Invoke(
    -            BoundReference(0, ObjectType(cls), nullable = true),
    -            s"_${index + 1}",
    -            t)
    --- End diff --
    
    this line: https://github.com/apache/spark/pull/13463/files#diff-87cabbe4d0c794f02523ecc1764955d0L88
    we create struct directly and doesn't consider the null case


---
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 issue #13807: [SPARK-16097][SQL] Encoders.tuple should handle null obj...

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

    https://github.com/apache/spark/pull/13807
  
    Merging to master and branch-2.0.


---
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 #13807: [SPARK-16097][SQL] Encoders.tuple should handle n...

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

    https://github.com/apache/spark/pull/13807#discussion_r67912070
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala ---
    @@ -830,6 +830,13 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
           ds.dropDuplicates("_1", "_2"),
           ("a", 1), ("a", 2), ("b", 1))
       }
    +
    +  test("SPARK-16097: Encoders.tuple should handle null object correctly") {
    +    val enc = Encoders.tuple(Encoders.tuple(Encoders.STRING, Encoders.STRING), Encoders.STRING)
    +    val data = Seq((("a", "b"), "c"), (null, "d"))
    +    val ds = spark.createDataset(data)(enc)
    +    checkDataset(ds, (("a", "b"), "c"), (null, "d"))
    +  }
    --- End diff --
    
    Is this equivalent with the original failed case? Seems not? For a outer join case, will we drop the null if `ExpressionEncoder` is not used?


---
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 issue #13807: [SPARK-16097][SQL] Encoders.tuple should handle null obj...

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

    https://github.com/apache/spark/pull/13807
  
    **[Test build #60937 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60937/consoleFull)** for PR 13807 at commit [`0c20c51`](https://github.com/apache/spark/commit/0c20c51f4c136885c479f3eb6f72c46c65b45a32).


---
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 issue #13807: [SPARK-16097][SQL] Encoders.tuple should handle null obj...

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

    https://github.com/apache/spark/pull/13807
  
    **[Test build #60978 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60978/consoleFull)** for PR 13807 at commit [`6519139`](https://github.com/apache/spark/commit/6519139593a0a5b2504c25ccf4db36df05a4f505).


---
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 issue #13807: [SPARK-16097][SQL] Encoders.tuple should handle null obj...

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

    https://github.com/apache/spark/pull/13807
  
    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 #13807: [SPARK-16097][SQL] Encoders.tuple should handle n...

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

    https://github.com/apache/spark/pull/13807#discussion_r67968854
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala ---
    @@ -110,16 +110,28 @@ object ExpressionEncoder {
     
         val cls = Utils.getContextOrSparkClassLoader.loadClass(s"scala.Tuple${encoders.size}")
     
    -    val serializer = encoders.map {
    -      case e if e.flat => e.serializer.head
    -      case other => CreateStruct(other.serializer)
    -    }.zipWithIndex.map { case (expr, index) =>
    -      expr.transformUp {
    -        case BoundReference(0, t, _) =>
    -          Invoke(
    -            BoundReference(0, ObjectType(cls), nullable = true),
    -            s"_${index + 1}",
    -            t)
    +    val serializer = encoders.zipWithIndex.map { case (enc, index) =>
    +      val originalInputObject = enc.serializer.head.collect { case b: BoundReference => b }.head
    --- End diff --
    
    yea, https://github.com/apache/spark/pull/13807/files#diff-91c617f2464cea010922328f4cdbbda9R227


---
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 #13807: [SPARK-16097][SQL] Encoders.tuple should handle n...

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

    https://github.com/apache/spark/pull/13807#discussion_r67932528
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala ---
    @@ -110,16 +110,28 @@ object ExpressionEncoder {
     
         val cls = Utils.getContextOrSparkClassLoader.loadClass(s"scala.Tuple${encoders.size}")
     
    -    val serializer = encoders.map {
    -      case e if e.flat => e.serializer.head
    -      case other => CreateStruct(other.serializer)
    -    }.zipWithIndex.map { case (expr, index) =>
    -      expr.transformUp {
    -        case BoundReference(0, t, _) =>
    -          Invoke(
    -            BoundReference(0, ObjectType(cls), nullable = true),
    -            s"_${index + 1}",
    -            t)
    --- End diff --
    
    Can you also comment on the line that caused the problem?


---
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 issue #13807: [SPARK-16097][SQL] Encoders.tuple should handle null obj...

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

    https://github.com/apache/spark/pull/13807
  
    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 #13807: [SPARK-16097][SQL] Encoders.tuple should handle n...

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

    https://github.com/apache/spark/pull/13807#discussion_r67887470
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala ---
    @@ -830,6 +830,13 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
           ds.dropDuplicates("_1", "_2"),
           ("a", 1), ("a", 2), ("b", 1))
       }
    +
    +  test("SPARK-16097: Encoders.tuple should handle null object correctly") {
    +    val enc = Encoders.tuple(Encoders.tuple(Encoders.STRING, Encoders.STRING), Encoders.STRING)
    +    val data = Seq((("a", "b"), "c"), (null, "d"))
    +    val ds = spark.createDataset(data)(enc)
    +    checkDataset(ds, (("a", "b"), "c"), (null, "d"))
    +  }
    --- End diff --
    
    Actually that is not a valid test as the `ExpressionEncoder` is private and should not be used externally. But it does expose this bug and I added this test case which is valid externally.


---
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 issue #13807: [SPARK-16097][SQL] Encoders.tuple should handle null obj...

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

    https://github.com/apache/spark/pull/13807
  
    **[Test build #60929 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60929/consoleFull)** for PR 13807 at commit [`45f74f6`](https://github.com/apache/spark/commit/45f74f624ba8774c3735896915995c4d81785884).


---
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 #13807: [SPARK-16097][SQL] Encoders.tuple should handle n...

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

    https://github.com/apache/spark/pull/13807#discussion_r67932028
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala ---
    @@ -110,16 +110,28 @@ object ExpressionEncoder {
     
         val cls = Utils.getContextOrSparkClassLoader.loadClass(s"scala.Tuple${encoders.size}")
     
    -    val serializer = encoders.map {
    -      case e if e.flat => e.serializer.head
    -      case other => CreateStruct(other.serializer)
    -    }.zipWithIndex.map { case (expr, index) =>
    -      expr.transformUp {
    -        case BoundReference(0, t, _) =>
    -          Invoke(
    -            BoundReference(0, ObjectType(cls), nullable = true),
    -            s"_${index + 1}",
    -            t)
    +    val serializer = encoders.zipWithIndex.map { case (enc, index) =>
    +      val originalInputObject = enc.serializer.head.collect { case b: BoundReference => b }.head
    --- End diff --
    
    Do we have any assumption at 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 #13807: [SPARK-16097][SQL] Encoders.tuple should handle n...

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

    https://github.com/apache/spark/pull/13807#discussion_r67931868
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala ---
    @@ -110,16 +110,28 @@ object ExpressionEncoder {
     
         val cls = Utils.getContextOrSparkClassLoader.loadClass(s"scala.Tuple${encoders.size}")
     
    -    val serializer = encoders.map {
    -      case e if e.flat => e.serializer.head
    -      case other => CreateStruct(other.serializer)
    -    }.zipWithIndex.map { case (expr, index) =>
    -      expr.transformUp {
    -        case BoundReference(0, t, _) =>
    -          Invoke(
    -            BoundReference(0, ObjectType(cls), nullable = true),
    -            s"_${index + 1}",
    -            t)
    +    val serializer = encoders.zipWithIndex.map { case (enc, index) =>
    +      val originalInputObject = enc.serializer.head.collect { case b: BoundReference => b }.head
    --- End diff --
    
    Safe to call `head`?


---
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 #13807: [SPARK-16097][SQL] Encoders.tuple should handle n...

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

    https://github.com/apache/spark/pull/13807#discussion_r67880034
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala ---
    @@ -830,6 +830,13 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
           ds.dropDuplicates("_1", "_2"),
           ("a", 1), ("a", 2), ("b", 1))
       }
    +
    +  test("SPARK-16097: Encoders.tuple should handle null object correctly") {
    +    val enc = Encoders.tuple(Encoders.tuple(Encoders.STRING, Encoders.STRING), Encoders.STRING)
    +    val data = Seq((("a", "b"), "c"), (null, "d"))
    +    val ds = spark.createDataset(data)(enc)
    +    checkDataset(ds, (("a", "b"), "c"), (null, "d"))
    +  }
    --- End diff --
    
    Shall we add the original outer join test case 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 issue #13807: [SPARK-16097][SQL] Encoders.tuple should handle null obj...

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

    https://github.com/apache/spark/pull/13807
  
    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 issue #13807: [SPARK-16097][SQL] Encoders.tuple should handle null obj...

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

    https://github.com/apache/spark/pull/13807
  
    LGTM except for minor issues.


---
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 #13807: [SPARK-16097][SQL] Encoders.tuple should handle n...

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

    https://github.com/apache/spark/pull/13807#discussion_r67933216
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala ---
    @@ -110,16 +110,25 @@ object ExpressionEncoder {
     
         val cls = Utils.getContextOrSparkClassLoader.loadClass(s"scala.Tuple${encoders.size}")
     
    -    val serializer = encoders.map {
    -      case e if e.flat => e.serializer.head
    -      case other => CreateStruct(other.serializer)
    -    }.zipWithIndex.map { case (expr, index) =>
    -      expr.transformUp {
    -        case BoundReference(0, t, _) =>
    -          Invoke(
    -            BoundReference(0, ObjectType(cls), nullable = true),
    -            s"_${index + 1}",
    -            t)
    +    val serializer = encoders.zipWithIndex.map { case (enc, index) =>
    +      val originalInputObject = enc.serializer.head.collect { case b: BoundReference => b }.head
    +      val newInputObject = Invoke(
    +        BoundReference(0, ObjectType(cls), nullable = true),
    +        s"_${index + 1}",
    +        originalInputObject.dataType)
    +
    +      val newSerializer = enc.serializer.map(_.transformUp {
    +        case b: BoundReference if b == originalInputObject => newInputObject
    +      })
    +
    +      if (enc.flat) {
    +        newSerializer.head
    +      } else {
    +        val struct = CreateStruct(newSerializer)
    +        val nullCheck = Or(
    +          IsNull(newInputObject),
    +          Invoke(Literal.fromObject(None), "equals", BooleanType, newInputObject :: Nil))
    +        If(nullCheck, Literal.create(null, struct.dataType), struct)
    --- End diff --
    
    Also, let's put examples in the comment.


---
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 issue #13807: [SPARK-16097][SQL] Encoders.tuple should handle null obj...

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

    https://github.com/apache/spark/pull/13807
  
    **[Test build #60978 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60978/consoleFull)** for PR 13807 at commit [`6519139`](https://github.com/apache/spark/commit/6519139593a0a5b2504c25ccf4db36df05a4f505).
     * 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 issue #13807: [SPARK-16097][SQL] Encoders.tuple should handle null obj...

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

    https://github.com/apache/spark/pull/13807
  
    **[Test build #60937 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60937/consoleFull)** for PR 13807 at commit [`0c20c51`](https://github.com/apache/spark/commit/0c20c51f4c136885c479f3eb6f72c46c65b45a32).
     * 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 #13807: [SPARK-16097][SQL] Encoders.tuple should handle n...

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

    https://github.com/apache/spark/pull/13807#discussion_r67879774
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala ---
    @@ -110,16 +110,25 @@ object ExpressionEncoder {
     
         val cls = Utils.getContextOrSparkClassLoader.loadClass(s"scala.Tuple${encoders.size}")
     
    -    val serializer = encoders.map {
    -      case e if e.flat => e.serializer.head
    -      case other => CreateStruct(other.serializer)
    -    }.zipWithIndex.map { case (expr, index) =>
    -      expr.transformUp {
    -        case BoundReference(0, t, _) =>
    -          Invoke(
    -            BoundReference(0, ObjectType(cls), nullable = true),
    -            s"_${index + 1}",
    -            t)
    +    val serializer = encoders.zipWithIndex.map { case (enc, index) =>
    +      val originalInputObject = enc.serializer.head.collect { case b: BoundReference => b }.head
    +      val newInputObject = Invoke(
    +        BoundReference(0, ObjectType(cls), nullable = true),
    +        s"_${index + 1}",
    +        originalInputObject.dataType)
    +
    +      val newSerializer = enc.serializer.map(_.transformUp {
    +        case b: BoundReference if b == originalInputObject => newInputObject
    +      })
    +
    +      if (enc.flat) {
    +        newSerializer.head
    +      } else {
    +        val struct = CreateStruct(newSerializer)
    +        val nullCheck = Or(
    +          IsNull(newInputObject),
    +          Invoke(Literal.fromObject(None), "equals", BooleanType, newInputObject :: Nil))
    +        If(nullCheck, Literal.create(null, struct.dataType), struct)
    --- End diff --
    
    This part is quite tricky, let's add comment here to explain why we need to substitute the input object and add the extra null check.


---
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 issue #13807: [SPARK-16097][SQL] Encoders.tuple should handle null obj...

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

    https://github.com/apache/spark/pull/13807
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/60929/
    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 issue #13807: [SPARK-16097][SQL] Encoders.tuple should handle null obj...

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

    https://github.com/apache/spark/pull/13807
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/60937/
    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