[ 
https://issues.apache.org/jira/browse/SPARK-22267?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16274588#comment-16274588
 ] 

Mark Petruska commented on SPARK-22267:
---------------------------------------

All evidence suggests that this is a hive bug.
In https://github.com/apache/spark/pull/19744 I tried a couple of 
configurations/properties for the hive `OrcInputFormat` and `OrcSerde`; none of 
them had any effect, the data was always read in the order as written (and not 
in the order requested).

> Spark SQL incorrectly reads ORC file when column order is different
> -------------------------------------------------------------------
>
>                 Key: SPARK-22267
>                 URL: https://issues.apache.org/jira/browse/SPARK-22267
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 1.6.3, 2.0.2, 2.1.0, 2.2.0
>            Reporter: Dongjoon Hyun
>
> For a long time, Apache Spark SQL returns incorrect results when ORC file 
> schema is different from metastore schema order.
> {code}
> scala> Seq(1 -> 2).toDF("c1", 
> "c2").write.format("parquet").mode("overwrite").save("/tmp/p")
> scala> Seq(1 -> 2).toDF("c1", 
> "c2").write.format("orc").mode("overwrite").save("/tmp/o")
> scala> sql("CREATE EXTERNAL TABLE p(c2 INT, c1 INT) STORED AS parquet 
> LOCATION '/tmp/p'")
> scala> sql("CREATE EXTERNAL TABLE o(c2 INT, c1 INT) STORED AS orc LOCATION 
> '/tmp/o'")
> scala> spark.table("p").show  // Parquet is good.
> +---+---+
> | c2| c1|
> +---+---+
> |  2|  1|
> +---+---+
> scala> spark.table("o").show    // This is wrong.
> +---+---+
> | c2| c1|
> +---+---+
> |  1|  2|
> +---+---+
> scala> spark.read.orc("/tmp/o").show  // This is correct.
> +---+---+
> | c1| c2|
> +---+---+
> |  1|  2|
> +---+---+
> {code}
> *TESTCASE*
> {code}
>   test("SPARK-22267 Spark SQL incorrectly reads ORC files when column order 
> is different") {
>     withTempDir { dir =>
>       val path = dir.getCanonicalPath
>       Seq(1 -> 2).toDF("c1", 
> "c2").write.format("orc").mode("overwrite").save(path)
>       checkAnswer(spark.read.orc(path), Row(1, 2))
>       Seq("true", "false").foreach { value =>
>         withTable("t") {
>           withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> value) {
>             sql(s"CREATE EXTERNAL TABLE t(c2 INT, c1 INT) STORED AS ORC 
> LOCATION '$path'")
>             checkAnswer(spark.table("t"), Row(2, 1))
>           }
>         }
>       }
>     }
>   }
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org

Reply via email to