You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by dongjoon-hyun <gi...@git.apache.org> on 2018/01/14 16:54:48 UTC

[GitHub] spark pull request #20266: [SPARK-23072][SQL] Add a Unicode schema test for ...

GitHub user dongjoon-hyun opened a pull request:

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

    [SPARK-23072][SQL] Add a Unicode schema test for file-based data sources

    ## What changes were proposed in this pull request?
    
    After [SPARK-20682](https://github.com/apache/spark/pull/19651), Apache Spark 2.3 is able to read ORC files with Unicode schema. Previously, it raises `org.apache.spark.sql.catalyst.parser.ParseException`.
    
    This PR adds a Unicode schema test for CSV/JSON/ORC/Parquet file-based data sources.
    
    ## How was this patch tested?
    
    Pass the newly added test case.

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

    $ git pull https://github.com/dongjoon-hyun/spark SPARK-23072

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

    https://github.com/apache/spark/pull/20266.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 #20266
    
----
commit f9a35f189c43e9962956ccb6cfae33c803e46b74
Author: Dongjoon Hyun <do...@...>
Date:   2018-01-14T16:49:46Z

    [SPARK-23072][SQL] Add a Unicode schema test for file-based data sources

----


---

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


[GitHub] spark pull request #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema tes...

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/20266#discussion_r161653628
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala ---
    @@ -0,0 +1,66 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql
    +
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext {
    +  import testImplicits._
    +
    +  Seq("orc", "parquet", "csv", "json", "text").foreach { format =>
    +    test(s"Writing empty datasets should not fail - $format") {
    +      withTempDir { dir =>
    +        Seq("str").toDS.limit(0).write.format(format).save(dir.getCanonicalPath + "/tmp")
    +      }
    +    }
    +  }
    +
    +  Seq("orc", "parquet", "csv", "json").foreach { format =>
    +    test(s"Write and read back unicode schema - $format") {
    +      withTempPath { path =>
    +        val dir = path.getCanonicalPath
    +
    +        // scalastyle:off nonascii
    +        val df = Seq("a").toDF("한글")
    +        // scalastyle:on nonascii
    +
    +        df.write.format(format).option("header", "true").save(dir)
    +        val answerDf = spark.read.format(format).option("header", "true").load(dir)
    +
    +        assert(df.schema === answerDf.schema)
    +        checkAnswer(df, answerDf)
    +      }
    +    }
    +  }
    +
    +  // Only New OrcFileFormat supports this
    +  Seq(classOf[org.apache.spark.sql.execution.datasources.orc.OrcFileFormat].getCanonicalName,
    --- End diff --
    
    `spark.sql.orc.impl` is native by default, can we just use "orc" here?


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    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 #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    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 pull request #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema tes...

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/20266#discussion_r161544031
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala ---
    @@ -2773,4 +2773,22 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
           }
         }
       }
    +
    +  Seq("orc", "parquet", "csv", "json").foreach { format =>
    +    test(s"Write and read back unicode schema - $format") {
    --- End diff --
    
    instead of keeping adding test cases in `SQLQuerySuite`, shall we create a dedicate test suite for file based data source now?


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    @mgaido91 . That suite is using SQL Syntax.


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    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 #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    LGTM


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

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


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    **[Test build #86162 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86162/testReport)** for PR 20266 at commit [`5afaa28`](https://github.com/apache/spark/commit/5afaa2836133cfc18a52de38d666817991d62c5d).
     * 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 #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema tes...

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

    https://github.com/apache/spark/pull/20266#discussion_r161668286
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala ---
    @@ -0,0 +1,66 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql
    +
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext {
    +  import testImplicits._
    +
    +  Seq("orc", "parquet", "csv", "json", "text").foreach { format =>
    +    test(s"Writing empty datasets should not fail - $format") {
    +      withTempDir { dir =>
    +        Seq("str").toDS.limit(0).write.format(format).save(dir.getCanonicalPath + "/tmp")
    +      }
    +    }
    +  }
    +
    +  Seq("orc", "parquet", "csv", "json").foreach { format =>
    +    test(s"Write and read back unicode schema - $format") {
    +      withTempPath { path =>
    +        val dir = path.getCanonicalPath
    +
    +        // scalastyle:off nonascii
    +        val df = Seq("a").toDF("한글")
    +        // scalastyle:on nonascii
    +
    +        df.write.format(format).option("header", "true").save(dir)
    +        val answerDf = spark.read.format(format).option("header", "true").load(dir)
    +
    +        assert(df.schema === answerDf.schema)
    +        checkAnswer(df, answerDf)
    +      }
    +    }
    +  }
    +
    +  // Only New OrcFileFormat supports this
    +  Seq(classOf[org.apache.spark.sql.execution.datasources.orc.OrcFileFormat].getCanonicalName,
    --- End diff --
    
    Yep.


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

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


---

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


[GitHub] spark pull request #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema tes...

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

    https://github.com/apache/spark/pull/20266#discussion_r161943935
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala ---
    @@ -0,0 +1,76 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql
    +
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext {
    +  import testImplicits._
    +
    +  Seq("orc", "parquet", "csv", "json", "text").foreach { format =>
    +    test(s"Writing empty datasets should not fail - $format") {
    +      withTempPath { dir =>
    +        Seq("str").toDS().limit(0).write.format(format).save(dir.getCanonicalPath)
    +      }
    +    }
    +  }
    +
    +  Seq("orc", "parquet", "csv", "json").foreach { format =>
    +    test(s"SPARK-23072 Write and read back unicode schema - $format") {
    --- End diff --
    
    Yep.


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    **[Test build #86141 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86141/testReport)** for PR 20266 at commit [`144c596`](https://github.com/apache/spark/commit/144c5965e20ae24d38deb033d7ce136a95212cac).


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    **[Test build #86183 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86183/testReport)** for PR 20266 at commit [`fb708b7`](https://github.com/apache/spark/commit/fb708b70fe4bb5d29ef55ace7fc0aae61e831c03).
     * 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 #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    Anyway, Jenkins seems to be out of order now.


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

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


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    **[Test build #86184 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86184/testReport)** for PR 20266 at commit [`8fec65b`](https://github.com/apache/spark/commit/8fec65b163b32e7592b21b4a6c19c69352f41919).


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

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


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    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 #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    Oh, I thought you mentioned the suite, @mgaido91 . Sorry! I agreee with you.


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    nit: since we are creating a new test suite what about moving also https://github.com/dongjoon-hyun/spark/blob/5afaa2836133cfc18a52de38d666817991d62c5d/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala#L1347 there?


---

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


[GitHub] spark pull request #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema tes...

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

    https://github.com/apache/spark/pull/20266#discussion_r161944475
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala ---
    @@ -0,0 +1,76 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql
    +
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext {
    +  import testImplicits._
    +
    +  Seq("orc", "parquet", "csv", "json", "text").foreach { format =>
    +    test(s"Writing empty datasets should not fail - $format") {
    +      withTempPath { dir =>
    +        Seq("str").toDS().limit(0).write.format(format).save(dir.getCanonicalPath)
    +      }
    +    }
    +  }
    +
    +  Seq("orc", "parquet", "csv", "json").foreach { format =>
    +    test(s"SPARK-23072 Write and read back unicode schema - $format") {
    +      withTempPath { path =>
    +        val dir = path.getCanonicalPath
    +
    +        // scalastyle:off nonascii
    +        val df = Seq("a").toDF("한글")
    +        // scalastyle:on nonascii
    +
    +        df.write.format(format).option("header", "true").save(dir)
    +        val answerDf = spark.read.format(format).option("header", "true").load(dir)
    +
    +        assert(df.schema === answerDf.schema)
    +        checkAnswer(df, answerDf)
    +      }
    +    }
    +  }
    +
    +  Seq("orc", "parquet").foreach { format =>
    --- End diff --
    
    Thanks!
    1. Only two support this. I added comments.
    2. For the other test cases, I did.
    3. I added a global Seq, `allFileBasedDataSources`.


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    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 #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    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 #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    @dongjoon-hyun the test case I referred to (the one related to SPARK-22146) doesn't seem to use either of them to me. It is only about reading files with special chars.


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

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


---

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


[GitHub] spark pull request #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema tes...

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/20266#discussion_r161653251
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala ---
    @@ -0,0 +1,66 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql
    +
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext {
    +  import testImplicits._
    +
    +  Seq("orc", "parquet", "csv", "json", "text").foreach { format =>
    --- End diff --
    
    instead of iterating all data source names and put the name in test name, can we create a base suite and create `ORCDataSourceSuite`, `ParquetDataSourceSuite`, etc.? It will be much easier if we wanna test more data sources in the future.


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/86162/
    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 #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema tes...

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

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


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

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


---

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


[GitHub] spark pull request #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema tes...

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

    https://github.com/apache/spark/pull/20266#discussion_r161940890
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala ---
    @@ -0,0 +1,76 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql
    +
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext {
    +  import testImplicits._
    +
    +  Seq("orc", "parquet", "csv", "json", "text").foreach { format =>
    +    test(s"Writing empty datasets should not fail - $format") {
    +      withTempPath { dir =>
    +        Seq("str").toDS().limit(0).write.format(format).save(dir.getCanonicalPath)
    +      }
    +    }
    +  }
    +
    +  Seq("orc", "parquet", "csv", "json").foreach { format =>
    +    test(s"SPARK-23072 Write and read back unicode schema - $format") {
    +      withTempPath { path =>
    +        val dir = path.getCanonicalPath
    +
    +        // scalastyle:off nonascii
    +        val df = Seq("a").toDF("한글")
    +        // scalastyle:on nonascii
    +
    +        df.write.format(format).option("header", "true").save(dir)
    +        val answerDf = spark.read.format(format).option("header", "true").load(dir)
    +
    +        assert(df.schema === answerDf.schema)
    +        checkAnswer(df, answerDf)
    +      }
    +    }
    +  }
    +
    +  Seq("orc", "parquet").foreach { format =>
    --- End diff --
    
    Only these two formats support it? If so, please add the comments. 
    
    This is the same comment to the other test cases. Otherwise, add all of them for each test case. 
    
    You can define a global Seq to include all the built-in file formats we support. 


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

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


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    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 #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    **[Test build #86227 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86227/testReport)** for PR 20266 at commit [`c67809c`](https://github.com/apache/spark/commit/c67809c9dbe0a21011649dceededa84d73377d1c).
     * 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 #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    Thank you, @cloud-fan , @gatorsmile , and @mgaido91 !


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    cc @gatorsmile and @cloud-fan .


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

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


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

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


---

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


[GitHub] spark pull request #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema tes...

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/20266#discussion_r161653311
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala ---
    @@ -0,0 +1,66 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql
    +
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext {
    +  import testImplicits._
    +
    +  Seq("orc", "parquet", "csv", "json", "text").foreach { format =>
    +    test(s"Writing empty datasets should not fail - $format") {
    +      withTempDir { dir =>
    +        Seq("str").toDS.limit(0).write.format(format).save(dir.getCanonicalPath + "/tmp")
    --- End diff --
    
    nit: why add `/tmp` at the end?


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    **[Test build #86140 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86140/testReport)** for PR 20266 at commit [`60b8e43`](https://github.com/apache/spark/commit/60b8e43a6b60063690485a9061c7709b64adfa43).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext `


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    **[Test build #86184 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86184/testReport)** for PR 20266 at commit [`8fec65b`](https://github.com/apache/spark/commit/8fec65b163b32e7592b21b4a6c19c69352f41919).
     * 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 #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema tes...

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

    https://github.com/apache/spark/pull/20266#discussion_r161668457
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala ---
    @@ -0,0 +1,66 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql
    +
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext {
    +  import testImplicits._
    +
    +  Seq("orc", "parquet", "csv", "json", "text").foreach { format =>
    +    test(s"Writing empty datasets should not fail - $format") {
    +      withTempDir { dir =>
    +        Seq("str").toDS.limit(0).write.format(format).save(dir.getCanonicalPath + "/tmp")
    --- End diff --
    
    Yep. It's fixed by using `withTempPath`.


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    **[Test build #86140 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86140/testReport)** for PR 20266 at commit [`60b8e43`](https://github.com/apache/spark/commit/60b8e43a6b60063690485a9061c7709b64adfa43).


---

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


[GitHub] spark pull request #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema tes...

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

    https://github.com/apache/spark/pull/20266#discussion_r161940666
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala ---
    @@ -0,0 +1,76 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql
    +
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext {
    +  import testImplicits._
    +
    +  Seq("orc", "parquet", "csv", "json", "text").foreach { format =>
    +    test(s"Writing empty datasets should not fail - $format") {
    +      withTempPath { dir =>
    +        Seq("str").toDS().limit(0).write.format(format).save(dir.getCanonicalPath)
    +      }
    +    }
    +  }
    +
    +  Seq("orc", "parquet", "csv", "json").foreach { format =>
    +    test(s"SPARK-23072 Write and read back unicode schema - $format") {
    --- End diff --
    
    `unicode schema` -> `unicode column names`


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    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 pull request #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema tes...

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

    https://github.com/apache/spark/pull/20266#discussion_r161637206
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala ---
    @@ -0,0 +1,66 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql
    +
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext {
    --- End diff --
    
    According to your comment, I added this suite and collected some suitable test cases.


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    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 #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

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


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    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 #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

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


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    LGTM


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    **[Test build #86159 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86159/testReport)** for PR 20266 at commit [`5afaa28`](https://github.com/apache/spark/commit/5afaa2836133cfc18a52de38d666817991d62c5d).
     * 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 #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/86140/
    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 #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema tes...

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

    https://github.com/apache/spark/pull/20266#discussion_r161599692
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala ---
    @@ -2773,4 +2773,22 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
           }
         }
       }
    +
    +  Seq("orc", "parquet", "csv", "json").foreach { format =>
    +    test(s"Write and read back unicode schema - $format") {
    --- End diff --
    
    +1. That's a best idea. I'll update like that.


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

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


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    **[Test build #86141 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86141/testReport)** for PR 20266 at commit [`144c596`](https://github.com/apache/spark/commit/144c5965e20ae24d38deb033d7ce136a95212cac).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext `


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    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 #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

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


---

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


[GitHub] spark issue #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    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 #20266: [SPARK-23072][SQL][TEST] Add a Unicode schema test for f...

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

    https://github.com/apache/spark/pull/20266
  
    **[Test build #86122 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86122/testReport)** for PR 20266 at commit [`f9a35f1`](https://github.com/apache/spark/commit/f9a35f189c43e9962956ccb6cfae33c803e46b74).
     * 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