You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by joseph-torres <gi...@git.apache.org> on 2017/08/25 20:50:40 UTC

[GitHub] spark pull request #19056: [SPARK-21765] Check that optimization doesn't aff...

GitHub user joseph-torres opened a pull request:

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

    [SPARK-21765] Check that optimization doesn't affect isStreaming bit.

    ## What changes were proposed in this pull request?
    
    Add an assert in logical plan optimization that the isStreaming bit stays the same, and fix empty relation rules where that wasn't happening.
    
    ## How was this patch tested?
    
    new and existing unit tests

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

    $ git pull https://github.com/joseph-torres/spark SPARK-21765-followup

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

    https://github.com/apache/spark/pull/19056.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 #19056
    
----
commit b83349567760dd0d33388d3fc68d8db1b648e1f1
Author: Jose Torres <jo...@databricks.com>
Date:   2017-08-25T20:48:49Z

    Check that optimization doesn't affect isStreaming bit.

----


---
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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    **[Test build #81187 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81187/testReport)** for PR 19056 at commit [`feda29f`](https://github.com/apache/spark/commit/feda29ff7d7320882c724b7f597beba816e583ce).
     * 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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    **[Test build #81146 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81146/testReport)** for PR 19056 at commit [`a25534e`](https://github.com/apache/spark/commit/a25534eb2ef7c303ff77dce92aad543ca6c171d7).
     * This patch **fails to build**.
     * 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 #19056: [SPARK-21765] Check that optimization doesn't aff...

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

    https://github.com/apache/spark/pull/19056#discussion_r135360845
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala ---
    @@ -65,11 +66,12 @@ object PropagateEmptyRelation extends Rule[LogicalPlan] with PredicateHelper {
           case _: RepartitionByExpression => empty(p)
           // An aggregate with non-empty group expression will return one output row per group when the
           // input to the aggregate is not empty. If the input to the aggregate is empty then all groups
    -      // will be empty and thus the output will be empty.
    +      // will be empty and thus the output will be empty. If we're working on batch data, we can
    +      // then treat the aggregate as redundant.
           //
           // If the grouping expressions are empty, however, then the aggregate will always produce a
           // single output row and thus we cannot propagate the EmptyRelation.
    -      case Aggregate(ge, _, _) if ge.nonEmpty => empty(p)
    +      case Aggregate(ge, _, _) if ge.nonEmpty and !p.isStreaming => empty(p)
    --- End diff --
    
    also make sure that this exception is covered by the tests. 


---
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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    **[Test build #81145 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81145/testReport)** for PR 19056 at commit [`4036767`](https://github.com/apache/spark/commit/4036767f68770324901ee3edbe01f30fe3bba1b4).
     * 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 #19056: [SPARK-21765] Check that optimization doesn't aff...

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

    https://github.com/apache/spark/pull/19056#discussion_r135851433
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala ---
    @@ -126,16 +128,17 @@ class TextSocketSource(host: String, port: Int, includeTimestamp: Boolean, sqlCo
           batches.slice(sliceStart, sliceEnd)
         }
     
    -    import sqlContext.implicits._
    -    val rawBatch = sqlContext.createDataset(rawList)
    +    val rdd = sqlContext.sparkContext.parallelize(rawList).map(
    +        v => InternalRow(UTF8String.fromString(v._1), v._2.getTime()))
    +    val rawBatch = sqlContext.internalCreateDataFrame(rdd, schema, isStreaming = true)
     
         // Underlying MemoryStream has schema (String, Timestamp); strip out the timestamp
         // if requested.
         if (includeTimestamp) {
           rawBatch.toDF("value", "timestamp")
         } else {
           // Strip out timestamp
    -      rawBatch.select("_1").toDF("value")
    +      rawBatch.select("value").toDF()
    --- End diff --
    
    Done.


---
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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    **[Test build #81222 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81222/testReport)** for PR 19056 at commit [`28aad0c`](https://github.com/apache/spark/commit/28aad0cc21fb93c1d3b0026f5d55b3d953c7bb7d).
     * 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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81146/
    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 issue #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    **[Test build #81415 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81415/testReport)** for PR 19056 at commit [`04d5ede`](https://github.com/apache/spark/commit/04d5ede50a362f40c9b24322c31316da3b1efb29).


---

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


[GitHub] spark issue #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    **[Test build #81434 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81434/testReport)** for PR 19056 at commit [`a3ec0f2`](https://github.com/apache/spark/commit/a3ec0f2cf3ec92aa30327c856820722ae7f22e7c).
     * 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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81147/
    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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81222/
    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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    **[Test build #81222 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81222/testReport)** for PR 19056 at commit [`28aad0c`](https://github.com/apache/spark/commit/28aad0cc21fb93c1d3b0026f5d55b3d953c7bb7d).


---
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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81187/
    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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    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 #19056: [SPARK-21765] Check that optimization doesn't aff...

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

    https://github.com/apache/spark/pull/19056#discussion_r137142093
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala ---
    @@ -128,8 +128,9 @@ class TextSocketSource(host: String, port: Int, includeTimestamp: Boolean, sqlCo
           batches.slice(sliceStart, sliceEnd)
         }
     
    -    val rdd = sqlContext.sparkContext.parallelize(rawList).map(
    -        v => InternalRow(UTF8String.fromString(v._1), v._2.getTime()))
    +    val rdd = sqlContext.sparkContext.
    --- End diff --
    
    we generally put the '.' in the next line. So
    ```
    val rdd = sqlContext.sparkContext
       .parall ...
       .map ....
    ```


---

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


[GitHub] spark issue #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81145/
    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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    (Dummy comment to test JIRA linkage)


---
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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    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 #19056: [SPARK-21765] Check that optimization doesn't aff...

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

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


---

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


[GitHub] spark issue #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

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


---
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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    **[Test build #81415 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81415/testReport)** for PR 19056 at commit [`04d5ede`](https://github.com/apache/spark/commit/04d5ede50a362f40c9b24322c31316da3b1efb29).
     * 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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    **[Test build #81228 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81228/testReport)** for PR 19056 at commit [`9f8b11f`](https://github.com/apache/spark/commit/9f8b11f234c86d6f683285b2f2b7558c417f1cfc).
     * 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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    **[Test build #81147 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81147/testReport)** for PR 19056 at commit [`5c61a13`](https://github.com/apache/spark/commit/5c61a13f53f09673705fcc1baa6c084e593c8b00).


---
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 #19056: [SPARK-21765] Check that optimization doesn't aff...

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

    https://github.com/apache/spark/pull/19056#discussion_r135851647
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala ---
    @@ -126,16 +128,17 @@ class TextSocketSource(host: String, port: Int, includeTimestamp: Boolean, sqlCo
           batches.slice(sliceStart, sliceEnd)
         }
     
    -    import sqlContext.implicits._
    -    val rawBatch = sqlContext.createDataset(rawList)
    +    val rdd = sqlContext.sparkContext.parallelize(rawList).map(
    +        v => InternalRow(UTF8String.fromString(v._1), v._2.getTime()))
    +    val rawBatch = sqlContext.internalCreateDataFrame(rdd, schema, isStreaming = true)
     
         // Underlying MemoryStream has schema (String, Timestamp); strip out the timestamp
         // if requested.
         if (includeTimestamp) {
           rawBatch.toDF("value", "timestamp")
    --- End diff --
    
    Done.


---
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 #19056: [SPARK-21765] Check that optimization doesn't aff...

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

    https://github.com/apache/spark/pull/19056#discussion_r135358635
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala ---
    @@ -86,6 +91,8 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging {
                 val runTime = System.nanoTime() - startTime
                 RuleExecutor.timeMap.addAndGet(rule.ruleName, runTime)
     
    +            checkInvariants(result, plan, rule)
    --- End diff --
    
    Call this only when the plan has changed. So just move this inside the condition below.


---
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 #19056: [SPARK-21765] Check that optimization doesn't aff...

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

    https://github.com/apache/spark/pull/19056#discussion_r135358693
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala ---
    @@ -63,6 +63,11 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging {
       /** Defines a sequence of rule batches, to be overridden by the implementation. */
       protected def batches: Seq[Batch]
     
    +  /** Checks invariants that should hold across rule execution. */
    --- End diff --
    
    nit: rule executions*s*


---
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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

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


---
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 #19056: [SPARK-21765] Check that optimization doesn't aff...

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

    https://github.com/apache/spark/pull/19056#discussion_r135360650
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala ---
    @@ -65,11 +66,12 @@ object PropagateEmptyRelation extends Rule[LogicalPlan] with PredicateHelper {
           case _: RepartitionByExpression => empty(p)
           // An aggregate with non-empty group expression will return one output row per group when the
           // input to the aggregate is not empty. If the input to the aggregate is empty then all groups
    -      // will be empty and thus the output will be empty.
    +      // will be empty and thus the output will be empty. If we're working on batch data, we can
    +      // then treat the aggregate as redundant.
           //
           // If the grouping expressions are empty, however, then the aggregate will always produce a
           // single output row and thus we cannot propagate the EmptyRelation.
    -      case Aggregate(ge, _, _) if ge.nonEmpty => empty(p)
    +      case Aggregate(ge, _, _) if ge.nonEmpty and !p.isStreaming => empty(p)
    --- End diff --
    
    Can you add to the docs above why we are avoiding this when its streaming.


---
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 #19056: [SPARK-21765] Check that optimization doesn't aff...

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

    https://github.com/apache/spark/pull/19056#discussion_r135610992
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala ---
    @@ -126,16 +128,17 @@ class TextSocketSource(host: String, port: Int, includeTimestamp: Boolean, sqlCo
           batches.slice(sliceStart, sliceEnd)
         }
     
    -    import sqlContext.implicits._
    -    val rawBatch = sqlContext.createDataset(rawList)
    +    val rdd = sqlContext.sparkContext.parallelize(rawList).map(
    +        v => InternalRow(UTF8String.fromString(v._1), v._2.getTime()))
    +    val rawBatch = sqlContext.internalCreateDataFrame(rdd, schema, isStreaming = true)
     
         // Underlying MemoryStream has schema (String, Timestamp); strip out the timestamp
         // if requested.
         if (includeTimestamp) {
           rawBatch.toDF("value", "timestamp")
    --- End diff --
    
    I think that the schema will already handle what fields are included in `rawBatch` and this `if` is no longer necessary.


---
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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81144/
    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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

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


---

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


[GitHub] spark issue #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    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 issue #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    LGTM. Will merge after tests pass.


---

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


[GitHub] spark issue #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81228/
    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 #19056: [SPARK-21765] Check that optimization doesn't aff...

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

    https://github.com/apache/spark/pull/19056#discussion_r135358597
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -39,6 +39,15 @@ abstract class Optimizer(sessionCatalog: SessionCatalog)
     
       protected def fixedPoint = FixedPoint(SQLConf.get.optimizerMaxIterations)
     
    +  override protected def checkInvariants(
    +      result: LogicalPlan,
    +      original: LogicalPlan,
    +      rule: Rule[LogicalPlan]): Unit = {
    +    assert(
    +      result.isStreaming == original.isStreaming,
    +      s"Rule ${rule.ruleName} changed isStreaming from original ${original.isStreaming}")
    --- End diff --
    
    Print the original and result plans as well. So that its easy to debug.


---
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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    **[Test build #81144 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81144/testReport)** for PR 19056 at commit [`b833495`](https://github.com/apache/spark/commit/b83349567760dd0d33388d3fc68d8db1b648e1f1).
     * 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 #19056: [SPARK-21765] Check that optimization doesn't aff...

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

    https://github.com/apache/spark/pull/19056#discussion_r135610234
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -39,6 +39,16 @@ abstract class Optimizer(sessionCatalog: SessionCatalog)
     
       protected def fixedPoint = FixedPoint(SQLConf.get.optimizerMaxIterations)
     
    +  override protected def checkInvariants(
    +      result: LogicalPlan,
    +      original: LogicalPlan,
    +      rule: Rule[LogicalPlan]): Unit = {
    +    assert(
    +      result.isStreaming == original.isStreaming,
    +      s"Rule ${rule.ruleName} changed isStreaming from original ${original.isStreaming}:" +
    --- End diff --
    
    Space before the closing `"`"?


---
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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    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 #19056: [SPARK-21765] Check that optimization doesn't aff...

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

    https://github.com/apache/spark/pull/19056#discussion_r135851225
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala ---
    @@ -39,6 +39,16 @@ abstract class Optimizer(sessionCatalog: SessionCatalog)
     
       protected def fixedPoint = FixedPoint(SQLConf.get.optimizerMaxIterations)
     
    +  override protected def checkInvariants(
    +      result: LogicalPlan,
    +      original: LogicalPlan,
    +      rule: Rule[LogicalPlan]): Unit = {
    +    assert(
    +      result.isStreaming == original.isStreaming,
    +      s"Rule ${rule.ruleName} changed isStreaming from original ${original.isStreaming}:" +
    --- End diff --
    
    Done.


---
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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

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


---
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 #19056: [SPARK-21765] Check that optimization doesn't aff...

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

    https://github.com/apache/spark/pull/19056#discussion_r136638327
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala ---
    @@ -130,16 +130,7 @@ class TextSocketSource(host: String, port: Int, includeTimestamp: Boolean, sqlCo
     
         val rdd = sqlContext.sparkContext.parallelize(rawList).map(
             v => InternalRow(UTF8String.fromString(v._1), v._2.getTime()))
    --- End diff --
    
    @joseph-torres this is a nit but a good suggestion. 


---
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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

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


---

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


[GitHub] spark issue #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

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


---

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


[GitHub] spark issue #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    **[Test build #81223 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81223/testReport)** for PR 19056 at commit [`b81553c`](https://github.com/apache/spark/commit/b81553ce2d4f5aa4b86bc0ba9485f925a3a7685c).
     * 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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

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


---
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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    **[Test build #81147 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81147/testReport)** for PR 19056 at commit [`5c61a13`](https://github.com/apache/spark/commit/5c61a13f53f09673705fcc1baa6c084e593c8b00).
     * 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 #19056: [SPARK-21765] Check that optimization doesn't aff...

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

    https://github.com/apache/spark/pull/19056#discussion_r135610632
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala ---
    @@ -126,16 +128,17 @@ class TextSocketSource(host: String, port: Int, includeTimestamp: Boolean, sqlCo
           batches.slice(sliceStart, sliceEnd)
         }
     
    -    import sqlContext.implicits._
    -    val rawBatch = sqlContext.createDataset(rawList)
    +    val rdd = sqlContext.sparkContext.parallelize(rawList).map(
    +        v => InternalRow(UTF8String.fromString(v._1), v._2.getTime()))
    +    val rawBatch = sqlContext.internalCreateDataFrame(rdd, schema, isStreaming = true)
     
         // Underlying MemoryStream has schema (String, Timestamp); strip out the timestamp
         // if requested.
         if (includeTimestamp) {
           rawBatch.toDF("value", "timestamp")
         } else {
           // Strip out timestamp
    -      rawBatch.select("_1").toDF("value")
    +      rawBatch.select("value").toDF()
    --- End diff --
    
    `toDF` is unnecessary since it's already a DataFrame.


---
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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    Addressed all 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 #19056: [SPARK-21765] Check that optimization doesn't aff...

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

    https://github.com/apache/spark/pull/19056#discussion_r137143373
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala ---
    @@ -128,8 +128,9 @@ class TextSocketSource(host: String, port: Int, includeTimestamp: Boolean, sqlCo
           batches.slice(sliceStart, sliceEnd)
         }
     
    -    val rdd = sqlContext.sparkContext.parallelize(rawList).map(
    -        v => InternalRow(UTF8String.fromString(v._1), v._2.getTime()))
    +    val rdd = sqlContext.sparkContext.
    --- End diff --
    
    done


---

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


[GitHub] spark issue #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

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


---
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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81223/
    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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

    https://github.com/apache/spark/pull/19056
  
    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 #19056: [SPARK-21765] Check that optimization doesn't affect isS...

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

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


---
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 #19056: [SPARK-21765] Check that optimization doesn't aff...

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

    https://github.com/apache/spark/pull/19056#discussion_r135989439
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala ---
    @@ -130,16 +130,7 @@ class TextSocketSource(host: String, port: Int, includeTimestamp: Boolean, sqlCo
     
         val rdd = sqlContext.sparkContext.parallelize(rawList).map(
             v => InternalRow(UTF8String.fromString(v._1), v._2.getTime()))
    --- End diff --
    
    May I ask to replace `v` with `case...` as follows? IMHO That would make things easier to read.
    
    ```
        val rdd = sqlContext.sparkContext.
          parallelize(rawList).
          map { case (v, ts) => InternalRow(UTF8String.fromString(v), ts.getTime) }
    ```


---
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 #19056: [SPARK-21765] Check that optimization doesn't aff...

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

    https://github.com/apache/spark/pull/19056#discussion_r137048000
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala ---
    @@ -130,16 +130,7 @@ class TextSocketSource(host: String, port: Int, includeTimestamp: Boolean, sqlCo
     
         val rdd = sqlContext.sparkContext.parallelize(rawList).map(
             v => InternalRow(UTF8String.fromString(v._1), v._2.getTime()))
    --- End diff --
    
    Done


---

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