Oscar Cassetti created SPARK-27695:
--------------------------------------

             Summary: SELECT * returns null column when reading from Hive / ORC 
and spark.sql.hive.convertMetastoreOrc=true
                 Key: SPARK-27695
                 URL: https://issues.apache.org/jira/browse/SPARK-27695
             Project: Spark
          Issue Type: Bug
          Components: Input/Output, Spark Core
    Affects Versions: 2.3.3, 2.3.2, 2.3.1
            Reporter: Oscar Cassetti


If you do 
{code:java}
select * from hive.some_table{code}
and the underlying data does not match exactly the schema the last column is 
returned as null 

Example 
{code:java}
from pyspark import SparkConf
from pyspark.sql import SparkSession
from pyspark.sql.types import *

conf = SparkConf().set('spark.sql.hive.convertMetastoreOrc', 'true')

spark = SparkSession.builder.config(conf=conf).enableHiveSupport().getOrCreate()


data = [{'a':i, 'b':i+10, 'd':{'a':i, 'b':i+10}} for i in range(1, 100)]
data_schema = StructType([StructField('a', LongType(), True),
StructField('b', LongType(), True),
StructField('d', MapType(StringType(), LongType(), True), True)
])
rdd = spark.sparkContext.parallelize(data)

df = rdd.toDF(data_schema)

df.write.format("orc").save("./sample_data/")

spark.sql("""create external table tmp(
a bigint,
b bigint,
d map<string, bigint>)
stored as orc
location 'sample_data/'
""")


spark.sql("select * from tmp").show()
{code}
This return correctl

{noformat}
+---+---+-------------------+
|  a|  b|                  d|
+---+---+-------------------+
| 85| 95| [a -> 85, b -> 95]|
| 86| 96| [a -> 86, b -> 96]|
| 87| 97| [a -> 87, b -> 97]|
| 88| 98| [a -> 88, b -> 98]|
| 89| 99| [a -> 89, b -> 99]|
| 90|100|[a -> 90, b -> 100]|

{noformat}

However if add a new column in the underlying data without altering the hive 
schema 
the last column of the hive schema is set to null

{code}
from pyspark import SparkConf
from pyspark.sql import SparkSession
from pyspark.sql.types import *

conf = SparkConf().set('spark.sql.hive.convertMetastoreOrc', 'true')

spark = SparkSession.builder.config(conf=conf).enableHiveSupport().getOrCreate()


data = [{'a':i, 'b':i+10, 'c':i+5, 'd':{'a':i, 'b':i+10, 'c':i+5}} for i in 
range(1, 100)]
data_schema = StructType([StructField('a', LongType(), True),
                              StructField('b', LongType(), True),
                              StructField('c', LongType(), True),
                              StructField('d', MapType(StringType(), 
LongType(), True), True)
                              ])
rdd = spark.sparkContext.parallelize(data)

df = rdd.toDF(data_schema)

df.write.format("orc").mode("overwrite").save("./sample_data/")


spark.sql("select * from tmp").show()

spark.read.orc("./sample_data/").show()
{code}

The first show() returns 
{noformat}
+---+---+----+
|  a|  b|   d|
+---+---+----+
| 85| 95|null|
| 86| 96|null|
| 87| 97|null|
| 88| 98|null|
| 89| 99|null|
| 90|100|null|
| 91|101|null|
| 92|102|null|
| 93|103|null|
| 94|104|null|
{noformat}

But the data on disk is correct
{noformat}
+---+---+---+--------------------+
|  a|  b|  c|                   d|
+---+---+---+--------------------+
| 85| 95| 90|[a -> 85, b -> 95...|
| 86| 96| 91|[a -> 86, b -> 96...|
| 87| 97| 92|[a -> 87, b -> 97...|
| 88| 98| 93|[a -> 88, b -> 98...|
| 89| 99| 94|[a -> 89, b -> 99...|
| 90|100| 95|[a -> 90, b -> 10...|
| 91|101| 96|[a -> 91, b -> 10...|
{noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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

Reply via email to