You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Mateusz Boryn (JIRA)" <ji...@apache.org> on 2017/02/13 13:57:41 UTC

[jira] [Updated] (SPARK-19580) Support for avro.schema.url while writing to hive table

     [ https://issues.apache.org/jira/browse/SPARK-19580?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Mateusz Boryn updated SPARK-19580:
----------------------------------
    Description: 
Support for writing to Hive table which uses Avro schema pointed to by avro.schema.url is missing. 

I have Hive table with Avro data format. Table is created with query like this:

{code:sql}
CREATE TABLE some_table
  PARTITIONED BY (YEAR int, MONTH int, DAY int)
  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:///user/some_user/some_table'
      TBLPROPERTIES (
        'avro.schema.url'='hdfs:///user/some_user/some_table.avsc'
      )
{code}

Please notice that there is `avro.schema.url` and not `avro.schema.literal` property, as we have to keep schemas in separate files for some reasons.
Trying to write to such table results in NPE.

Tried to find workaround for this, but nothing helps. Tried:
    - setting df.write.option("avroSchema", avroSchema) with explicit schema in string
    - changing TBLPROPERTIES to SERDEPROPERTIES
    - replacing explicit detailed SERDE specification with STORED AS AVRO

I found that this can be solved by adding a couple of lines in `org.apache.spark.sql.hive.HiveShim` next to `AvroSerdeUtils.AvroTableProperties.SCHEMA_LITERAL` is referenced.

  was:
Support for writing to Hive table which uses Avro schema pointed to by avro.schema.url is missing. 

I have Hive table with Avro data format. Table is created with query like this:

{code}
CREATE TABLE some_table
  PARTITIONED BY (YEAR int, MONTH int, DAY int)
  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:///user/some_user/some_table'
      TBLPROPERTIES (
        'avro.schema.url'='hdfs:///user/some_user/some_table.avsc'
      )
{code}

Please notice that there is `avro.schema.url` and not `avro.schema.literal` property, as we have to keep schemas in separate files for some reasons.
Trying to write to such table results in NPE.

Tried to find workaround for this, but nothing helps. Tried:
    - setting df.write.option("avroSchema", avroSchema) with explicit schema in string
    - changing TBLPROPERTIES to SERDEPROPERTIES
    - replacing explicit detailed SERDE specification with STORED AS AVRO

I found that this can be solved by adding a couple of lines in `org.apache.spark.sql.hive.HiveShim` next to `AvroSerdeUtils.AvroTableProperties.SCHEMA_LITERAL` is referenced.


> Support for avro.schema.url while writing to hive table
> -------------------------------------------------------
>
>                 Key: SPARK-19580
>                 URL: https://issues.apache.org/jira/browse/SPARK-19580
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 1.6.3, 2.1.0
>            Reporter: Mateusz Boryn
>            Priority: Critical
>
> Support for writing to Hive table which uses Avro schema pointed to by avro.schema.url is missing. 
> I have Hive table with Avro data format. Table is created with query like this:
> {code:sql}
> CREATE TABLE some_table
>   PARTITIONED BY (YEAR int, MONTH int, DAY int)
>   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:///user/some_user/some_table'
>       TBLPROPERTIES (
>         'avro.schema.url'='hdfs:///user/some_user/some_table.avsc'
>       )
> {code}
> Please notice that there is `avro.schema.url` and not `avro.schema.literal` property, as we have to keep schemas in separate files for some reasons.
> Trying to write to such table results in NPE.
> Tried to find workaround for this, but nothing helps. Tried:
>     - setting df.write.option("avroSchema", avroSchema) with explicit schema in string
>     - changing TBLPROPERTIES to SERDEPROPERTIES
>     - replacing explicit detailed SERDE specification with STORED AS AVRO
> I found that this can be solved by adding a couple of lines in `org.apache.spark.sql.hive.HiveShim` next to `AvroSerdeUtils.AvroTableProperties.SCHEMA_LITERAL` is referenced.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

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