You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@spark.apache.org by mateo7 <ma...@gmail.com> on 2016/07/14 13:37:20 UTC

DataFrameWriter.insertInto() to table with Avro schema by URL

There is a Hive table with Avro schema specified by URL pointing to file in
HDFS.

CREATE EXTERNAL TABLE IF NOT EXISTS my_table
      ROW FORMAT SERDE
            'org.apache.hadoop.hive.serde2.avro.AvroSerDe'
          STORED AS INPUTFORMAT
            'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat'
          OUTPUTFORMAT
            'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat'
          LOCATION 'hdfs:///tmp/my_table'
          TBLPROPERTIES (
            'avro.schema.url'='hdfs:///tmp/my_table.avsc'
-- contents of hdfs:///tmp/my_table.avsc is the same as below
--            'avro.schema.literal'='{   "type": "record",   "name":
"my_table",   "fields": [     {"type": "string", "name": "KEY"},    
{"type": "string", "name": "VALUE"}   ] }'
          )
    ;

DESCRIBE EXTENDED my_table;


I'm trying to save DataFrame to that table with this line:

    import org.apache.spark.sql.SaveMode
    val location = ...
    var df = sqlContext.read.format("csv").option("delimiter",
"\t").load(location)
    df = df.withColumnRenamed("C0", "KEY").withColumnRenamed("C1", "VALUE")
   
df.write.mode(SaveMode.Append).format("com.databricks.spark.avro").insertInto("my_table")

However, there is NullPointerException thrown from spark/hive internals. I
found that
org.apache.spark.sql.hive.execution.InsertIntoHiveTable.newSerializer(tableDesc:
TableDesc) has call to

    serializer.initialize(null, tableDesc.getProperties)

So Avro Serde is initialized with null configuration. It seems configuration
is sometimes required, like for accessing schema present in HDFS.
Is null passed there on purpose? For comparison there is
org.apache.spark.sql.hive.orc.OrcOutputWriter class which passes non-null
configuration object.
Is there any workaround for this problem? Literal Avro schema is not an
option for me.


Hive 1.2.1, Spark 1.6.2

16/07/14 13:27:04 WARN AvroSerDe: Encountered exception determining schema.
Returning signal schema to indicate problem
java.lang.NullPointerException
        at
org.apache.hadoop.fs.FileSystem.getDefaultUri(FileSystem.java:182)
        at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:363)
        at
org.apache.hadoop.hive.serde2.avro.AvroSerdeUtils.getSchemaFromFS(AvroSerdeUtils.java:131)
        at
org.apache.hadoop.hive.serde2.avro.AvroSerdeUtils.determineSchemaOrThrowException(AvroSerdeUtils.java:112)
        at
org.apache.hadoop.hive.serde2.avro.AvroSerDe.determineSchemaOrReturnErrorSchema(AvroSerDe.java:167)
        at
org.apache.hadoop.hive.serde2.avro.AvroSerDe.initialize(AvroSerDe.java:103)
        at
org.apache.spark.sql.hive.execution.InsertIntoHiveTable.newSerializer(InsertIntoHiveTable.scala:59)
        at
org.apache.spark.sql.hive.execution.InsertIntoHiveTable.outputClass$lzycompute(InsertIntoHiveTable.scala:53)
        at
org.apache.spark.sql.hive.execution.InsertIntoHiveTable.outputClass(InsertIntoHiveTable.scala:53)
        at
org.apache.spark.sql.hive.execution.InsertIntoHiveTable.sideEffectResult$lzycompute(InsertIntoHiveTable.scala:201)
        at
org.apache.spark.sql.hive.execution.InsertIntoHiveTable.sideEffectResult(InsertIntoHiveTable.scala:127)
        at
org.apache.spark.sql.hive.execution.InsertIntoHiveTable.doExecute(InsertIntoHiveTable.scala:276)
        at
org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:132)




--
View this message in context: http://apache-spark-developers-list.1001551.n3.nabble.com/DataFrameWriter-insertInto-to-table-with-Avro-schema-by-URL-tp18315.html
Sent from the Apache Spark Developers List mailing list archive at Nabble.com.

---------------------------------------------------------------------
To unsubscribe e-mail: dev-unsubscribe@spark.apache.org