You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by maropu <gi...@git.apache.org> on 2018/01/10 03:12:27 UTC

[GitHub] spark pull request #20214: [SPARK-23023][SQL] Cast field data to strings in ...

GitHub user maropu opened a pull request:

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

    [SPARK-23023][SQL] Cast field data to strings in showString

    ## What changes were proposed in this pull request?
    The current `Datset.showString` prints rows thru `RowEncoder` deserializers like;
    ```
    scala> Seq(Seq(Seq(1, 2), Seq(3), Seq(4, 5, 6))).toDF("a").show(false)
    +------------------------------------------------------------+
    |a                                                           |
    +------------------------------------------------------------+
    |[WrappedArray(1, 2), WrappedArray(3), WrappedArray(4, 5, 6)]|
    +------------------------------------------------------------+
    ```
    This result is incorrect because the correct one is;
    ```
    scala> Seq(Seq(Seq(1, 2), Seq(3), Seq(4, 5, 6))).toDF("a").show(false)
    +------------------------+
    |a                       |
    +------------------------+
    |[[1, 2], [3], [4, 5, 6]]|
    +------------------------+
    ```
    So, this pr fixed code in `showString` to cast field data to strings before printing.
    
    ## How was this patch tested?
    Added tests in `DataFrameSuite`.


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

    $ git pull https://github.com/maropu/spark SPARK-23023

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

    https://github.com/apache/spark/pull/20214.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 #20214
    
----
commit eb56aff74352a360d1d4b1273be23b670f3c958a
Author: Takeshi Yamamuro <ya...@...>
Date:   2018-01-06T11:05:54Z

    Cast data to strings in showString

----


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

    https://github.com/apache/spark/pull/20214
  
    **[Test build #85910 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85910/testReport)** for PR 20214 at commit [`e393c63`](https://github.com/apache/spark/commit/e393c637b10353a1efaa799a5779ae319bb070f0).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

    https://github.com/apache/spark/pull/20214
  
    **[Test build #86028 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86028/testReport)** for PR 20214 at commit [`18552a4`](https://github.com/apache/spark/commit/18552a4ffe2a06c9abf71d810c883dd6e41ecb3d).


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark pull request #20214: [SPARK-23023][SQL] Cast field data to strings in ...

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/20214#discussion_r161254906
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -237,13 +237,17 @@ class Dataset[T] private[sql](
       private[sql] def showString(
           _numRows: Int, truncate: Int = 20, vertical: Boolean = false): String = {
         val numRows = _numRows.max(0).min(Int.MaxValue - 1)
    -    val takeResult = toDF().take(numRows + 1)
    +    val newDf = toDF()
    +    val castExprs = newDf.schema.map { f => f.dataType match {
    +      // Since binary types in top-level schema fields have a specific format to print,
    +      // so we do not cast them to strings here.
    +      case BinaryType => s"`${f.name}`"
    --- End diff --
    
    can we use dataframe API? which looks more reliable here
    ```
    newDf.logicalPlan.output.map { col =>
      if (col.dataType == BinaryType) {
        Column(col)
      } else {
        Column(col).cast(StringType)
      }
    }
    ```


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

    https://github.com/apache/spark/pull/20214
  
    retest this please


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark pull request #20214: [SPARK-23023][SQL] Cast field data to strings in ...

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

    https://github.com/apache/spark/pull/20214#discussion_r161123864
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -237,13 +237,19 @@ class Dataset[T] private[sql](
       private[sql] def showString(
           _numRows: Int, truncate: Int = 20, vertical: Boolean = false): String = {
         val numRows = _numRows.max(0).min(Int.MaxValue - 1)
    -    val takeResult = toDF().take(numRows + 1)
    +    val newDf = toDF()
    +    val castExprs = newDf.schema.map { f => f.dataType match {
    +      // Since binary types in top-level schema fields have a specific format to print,
    +      // so we do not cast them to strings here.
    +      case BinaryType => s"${f.name}"
    +      case udt: UserDefinedType[_] => s"${f.name}"
    --- End diff --
    
    nit: `_: UserDefinedType[_]`.


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark pull request #20214: [SPARK-23023][SQL] Cast field data to strings in ...

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

    https://github.com/apache/spark/pull/20214#discussion_r161131086
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -237,13 +237,18 @@ class Dataset[T] private[sql](
       private[sql] def showString(
           _numRows: Int, truncate: Int = 20, vertical: Boolean = false): String = {
         val numRows = _numRows.max(0).min(Int.MaxValue - 1)
    -    val takeResult = toDF().take(numRows + 1)
    +    val newDf = toDF()
    +    val castExprs = newDf.schema.map { f => f.dataType match {
    +      // Since binary types in top-level schema fields have a specific format to print,
    +      // so we do not cast them to strings here.
    +      case BinaryType => s"`${f.name}`"
    +      case _: UserDefinedType[_] => s"`${f.name}`"
    --- End diff --
    
    I added this entry for passing the existing tests in pyspark though, we still hit wired behaviours when casting user-defined types into strings;
    ```
    >>> from pyspark.ml.classification import MultilayerPerceptronClassifier
    >>> from pyspark.ml.linalg import Vectors
    >>> df = spark.createDataFrame([(0.0, Vectors.dense([0.0, 0.0])), (1.0, Vectors.dense([0.0, 1.0]))], ["label", "features"])
    >>> df.selectExpr("CAST(features AS STRING)").show(truncate = False)
    +-------------------------------------------+
    |features                                   |
    +-------------------------------------------+
    |[6,1,0,0,2800000020,2,0,0,0]               |
    |[6,1,0,0,2800000020,2,0,0,3ff0000000000000]|
    +-------------------------------------------+
    ```
    This cast shows the internal data structure of user-define types. 
    I just tried to fix this though, I think we easily can't because, in codegen path, spark can't tell a way to convert a given internal data into an user-defined string;
    https://github.com/apache/spark/compare/master...maropu:CastUDTtoString#diff-258b71121d8d168e4d53cb5b6dc53ffeR844
    
    WDYT? cc: @cloud-fan @ueshin 


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

    https://github.com/apache/spark/pull/20214
  
    **[Test build #86028 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86028/testReport)** for PR 20214 at commit [`18552a4`](https://github.com/apache/spark/commit/18552a4ffe2a06c9abf71d810c883dd6e41ecb3d).
     * This patch **fails from timeout after a configured wait of \`250m\`**.
     * 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 #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

    https://github.com/apache/spark/pull/20214
  
    retest this please


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

    https://github.com/apache/spark/pull/20214
  
    **[Test build #86127 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86127/testReport)** for PR 20214 at commit [`022ed32`](https://github.com/apache/spark/commit/022ed327bc7e2fd3a5cbd498d21183f0eabf2a26).


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

    https://github.com/apache/spark/pull/20214
  
    **[Test build #85951 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85951/testReport)** for PR 20214 at commit [`9cf9954`](https://github.com/apache/spark/commit/9cf995461990e46c007405344481cd802a0d6501).
     * This patch **fails due to an unknown error code, -9**.
     * 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 #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark pull request #20214: [SPARK-23023][SQL] Cast field data to strings in ...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

    https://github.com/apache/spark/pull/20214
  
    `org.apache.spark.sql.streaming.StreamingOuterJoinSuite` is flaky? (It seems this pr is not related to the test).


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

    https://github.com/apache/spark/pull/20214
  
    **[Test build #85948 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85948/testReport)** for PR 20214 at commit [`cbccb1b`](https://github.com/apache/spark/commit/cbccb1b3a4f4220730c8e6260dda0e552c6c044b).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #20214: [SPARK-23023][SQL] Cast field data to strings in ...

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

    https://github.com/apache/spark/pull/20214#discussion_r161130049
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -237,13 +237,19 @@ class Dataset[T] private[sql](
       private[sql] def showString(
           _numRows: Int, truncate: Int = 20, vertical: Boolean = false): String = {
         val numRows = _numRows.max(0).min(Int.MaxValue - 1)
    -    val takeResult = toDF().take(numRows + 1)
    +    val newDf = toDF()
    +    val castExprs = newDf.schema.map { f => f.dataType match {
    +      // Since binary types in top-level schema fields have a specific format to print,
    +      // so we do not cast them to strings here.
    +      case BinaryType => s"${f.name}"
    --- End diff --
    
    oops, I forgot `


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

    https://github.com/apache/spark/pull/20214
  
    Please check #20246 first? Thanks! @ueshin @cloud-fan 


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

    https://github.com/apache/spark/pull/20214
  
    **[Test build #85905 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85905/testReport)** for PR 20214 at commit [`e393c63`](https://github.com/apache/spark/commit/e393c637b10353a1efaa799a5779ae319bb070f0).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #20214: [SPARK-23023][SQL] Cast field data to strings in ...

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

    https://github.com/apache/spark/pull/20214#discussion_r161358894
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala ---
    @@ -1255,6 +1255,34 @@ class DataFrameSuite extends QueryTest with SharedSQLContext {
         assert(testData.select($"*").showString(1, vertical = true) === expectedAnswer)
       }
     
    +  test("SPARK-23023 Cast rows to strings in showString") {
    +    val df1 = Seq(Seq(1, 2, 3, 4)).toDF("a")
    +    assert(df1.showString(10) ===
    --- End diff --
    
    Since `RowEncoder` deserializer converts nested arrays into `WrappedArray`, `toString` shows do so in `showString`.
    
    https://github.com/apache/spark/blob/55dbfbca37ce4c05f83180777ba3d4fe2d96a02e/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala#L304


---

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


[GitHub] spark pull request #20214: [SPARK-23023][SQL] Cast field data to strings in ...

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

    https://github.com/apache/spark/pull/20214#discussion_r160613124
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala ---
    @@ -1255,6 +1255,34 @@ class DataFrameSuite extends QueryTest with SharedSQLContext {
         assert(testData.select($"*").showString(1, vertical = true) === expectedAnswer)
       }
     
    +  test("SPARK-XXXXX Cast rows to strings in showString") {
    --- End diff --
    
    nit: need to update jira id.


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

    https://github.com/apache/spark/pull/20214
  
    retest this please


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

    https://github.com/apache/spark/pull/20214
  
    **[Test build #86012 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86012/testReport)** for PR 20214 at commit [`afe0af5`](https://github.com/apache/spark/commit/afe0af504b8a799dadfcd8c18ade339432f889b0).
     * 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 #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

    https://github.com/apache/spark/pull/20214
  
    **[Test build #85972 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85972/testReport)** for PR 20214 at commit [`66b06c3`](https://github.com/apache/spark/commit/66b06c37bd2a9ed8461f9572eb3f2081853a8d73).
     * 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 #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark pull request #20214: [SPARK-23023][SQL] Cast field data to strings in ...

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

    https://github.com/apache/spark/pull/20214#discussion_r161123911
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -237,13 +237,19 @@ class Dataset[T] private[sql](
       private[sql] def showString(
           _numRows: Int, truncate: Int = 20, vertical: Boolean = false): String = {
         val numRows = _numRows.max(0).min(Int.MaxValue - 1)
    -    val takeResult = toDF().take(numRows + 1)
    +    val newDf = toDF()
    +    val castExprs = newDf.schema.map { f => f.dataType match {
    +      // Since binary types in top-level schema fields have a specific format to print,
    +      // so we do not cast them to strings here.
    +      case BinaryType => s"${f.name}"
    +      case udt: UserDefinedType[_] => s"${f.name}"
    +      case _ => s"CAST(`${f.name}` AS STRING)"
    +
    --- End diff --
    
    nit: remove an extra line.


---

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


[GitHub] spark pull request #20214: [SPARK-23023][SQL] Cast field data to strings in ...

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

    https://github.com/apache/spark/pull/20214#discussion_r161153123
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -237,13 +237,18 @@ class Dataset[T] private[sql](
       private[sql] def showString(
           _numRows: Int, truncate: Int = 20, vertical: Boolean = false): String = {
         val numRows = _numRows.max(0).min(Int.MaxValue - 1)
    -    val takeResult = toDF().take(numRows + 1)
    +    val newDf = toDF()
    +    val castExprs = newDf.schema.map { f => f.dataType match {
    +      // Since binary types in top-level schema fields have a specific format to print,
    +      // so we do not cast them to strings here.
    +      case BinaryType => s"`${f.name}`"
    +      case _: UserDefinedType[_] => s"`${f.name}`"
    --- End diff --
    
    How about something like:
    
    ```scala
          case udt: UserDefinedType[_] =>
            (c, evPrim, evNull) => {
              val udtTerm = ctx.addReferenceObj("udt", udt)
              s"$evPrim = UTF8String.fromString($udtTerm.deserialize($c).toString());"
            }
    ```



---

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


[GitHub] spark pull request #20214: [SPARK-23023][SQL] Cast field data to strings in ...

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

    https://github.com/apache/spark/pull/20214#discussion_r161123821
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -237,13 +237,19 @@ class Dataset[T] private[sql](
       private[sql] def showString(
           _numRows: Int, truncate: Int = 20, vertical: Boolean = false): String = {
         val numRows = _numRows.max(0).min(Int.MaxValue - 1)
    -    val takeResult = toDF().take(numRows + 1)
    +    val newDf = toDF()
    +    val castExprs = newDf.schema.map { f => f.dataType match {
    +      // Since binary types in top-level schema fields have a specific format to print,
    +      // so we do not cast them to strings here.
    +      case BinaryType => s"${f.name}"
    --- End diff --
    
    Do we need to surround `f.name` with `s""`? Or we need to add \` around `f.name`?


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

    https://github.com/apache/spark/pull/20214
  
    **[Test build #85972 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85972/testReport)** for PR 20214 at commit [`66b06c3`](https://github.com/apache/spark/commit/66b06c37bd2a9ed8461f9572eb3f2081853a8d73).


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

    https://github.com/apache/spark/pull/20214
  
    **[Test build #85911 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85911/testReport)** for PR 20214 at commit [`ae7a807`](https://github.com/apache/spark/commit/ae7a8070bbc3aa2524d24cc61c9c2ce141a4f648).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

    https://github.com/apache/spark/pull/20214
  
    thanks, merging to master/2.3


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

    https://github.com/apache/spark/pull/20214
  
    retest this please.


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

    https://github.com/apache/spark/pull/20214
  
    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 #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

    https://github.com/apache/spark/pull/20214
  
    **[Test build #85933 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85933/testReport)** for PR 20214 at commit [`ae7a807`](https://github.com/apache/spark/commit/ae7a8070bbc3aa2524d24cc61c9c2ce141a4f648).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

    https://github.com/apache/spark/pull/20214
  
    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 #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

    https://github.com/apache/spark/pull/20214
  
    retest this please


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

    https://github.com/apache/spark/pull/20214
  
    **[Test build #86069 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86069/testReport)** for PR 20214 at commit [`022ed32`](https://github.com/apache/spark/commit/022ed327bc7e2fd3a5cbd498d21183f0eabf2a26).


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

    https://github.com/apache/spark/pull/20214
  
    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 #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

    https://github.com/apache/spark/pull/20214
  
    **[Test build #85956 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85956/testReport)** for PR 20214 at commit [`9cf9954`](https://github.com/apache/spark/commit/9cf995461990e46c007405344481cd802a0d6501).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #20214: [SPARK-23023][SQL] Cast field data to strings in ...

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

    https://github.com/apache/spark/pull/20214#discussion_r161156501
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -237,13 +237,18 @@ class Dataset[T] private[sql](
       private[sql] def showString(
           _numRows: Int, truncate: Int = 20, vertical: Boolean = false): String = {
         val numRows = _numRows.max(0).min(Int.MaxValue - 1)
    -    val takeResult = toDF().take(numRows + 1)
    +    val newDf = toDF()
    +    val castExprs = newDf.schema.map { f => f.dataType match {
    +      // Since binary types in top-level schema fields have a specific format to print,
    +      // so we do not cast them to strings here.
    +      case BinaryType => s"`${f.name}`"
    +      case _: UserDefinedType[_] => s"`${f.name}`"
    --- End diff --
    
    oh, yea. I missed that. Thanks, I'll make a separate pr.


---

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


[GitHub] spark pull request #20214: [SPARK-23023][SQL] Cast field data to strings in ...

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/20214#discussion_r161255253
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala ---
    @@ -1255,6 +1255,34 @@ class DataFrameSuite extends QueryTest with SharedSQLContext {
         assert(testData.select($"*").showString(1, vertical = true) === expectedAnswer)
       }
     
    +  test("SPARK-23023 Cast rows to strings in showString") {
    +    val df1 = Seq(Seq(1, 2, 3, 4)).toDF("a")
    +    assert(df1.showString(10) ===
    --- End diff --
    
    Do you know why it shows `WrappedArray` before?


---

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


[GitHub] spark pull request #20214: [SPARK-23023][SQL] Cast field data to strings in ...

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

    https://github.com/apache/spark/pull/20214#discussion_r160613267
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala ---
    @@ -1255,6 +1255,34 @@ class DataFrameSuite extends QueryTest with SharedSQLContext {
         assert(testData.select($"*").showString(1, vertical = true) === expectedAnswer)
       }
     
    +  test("SPARK-XXXXX Cast rows to strings in showString") {
    --- End diff --
    
    oh... thx..


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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


[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

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

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


---

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