Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-17920

HiveWriterContainer passes null configuration to serde.initialize, causing NullPointerException in AvroSerde when using avro.schema.url

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Minor
    • Resolution: Fixed
    • 1.6.2, 2.0.0
    • 2.2.1, 2.3.0
    • SQL
    • None
    • AWS EMR 5.0.0: Spark 2.0.0, Hive 2.1.0

    Description

      When HiveWriterContainer intializes a serde it explicitly passes null for the Configuration:

      https://github.com/apache/spark/blob/v2.0.0/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala#L161

      When attempting to write to a table stored as Avro with avro.schema.url set, this causes a NullPointerException when it tries to get the FileSystem for the URL:

      https://github.com/apache/hive/blob/release-2.1.0-rc3/serde/src/java/org/apache/hadoop/hive/serde2/avro/AvroSerdeUtils.java#L153

      Reproduction:

      spark-sql> create external table avro_in (a string) stored as avro location '/avro-in/' tblproperties ('avro.schema.url'='/avro-schema/avro.avsc');
      
      spark-sql> create external table avro_out (a string) stored as avro location '/avro-out/' tblproperties ('avro.schema.url'='/avro-schema/avro.avsc');
      
      spark-sql> select * from avro_in;
      hello
      Time taken: 1.986 seconds, Fetched 1 row(s)
      
      spark-sql> insert overwrite table avro_out select * from avro_in;
      
      16/10/13 19:34:47 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:174)
      	at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:359)
      	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.SparkHiveWriterContainer.newSerializer(hiveWriterContainers.scala:161)
      	at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.sideEffectResult$lzycompute(InsertIntoHiveTable.scala:236)
      	at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.sideEffectResult(InsertIntoHiveTable.scala:142)
      	at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.doExecute(InsertIntoHiveTable.scala:313)
      	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:115)
      	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:115)
      	at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:136)
      	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
      	at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:133)
      	at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:114)
      	at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:86)
      	at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:86)
      	at org.apache.spark.sql.Dataset.<init>(Dataset.scala:186)
      	at org.apache.spark.sql.Dataset.<init>(Dataset.scala:167)
      	at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:65)
      	at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:582)
      	at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:682)
      	at org.apache.spark.sql.hive.thriftserver.SparkSQLDriver.run(SparkSQLDriver.scala:62)
      	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:331)
      	at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:376)
      	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:247)
      	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)
      	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
      	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
      	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
      	at java.lang.reflect.Method.invoke(Method.java:498)
      	at org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:729)
      	at org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:185)
      	at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:210)
      	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:124)
      	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
      

      Hive fixed a similar issue in FileSinkOperator in https://issues.apache.org/jira/browse/HIVE-9651

      Attachments

        1. avro_data
          0.2 kB
          James Norvell
        2. avro.avsc
          0.1 kB
          James Norvell

        Issue Links

          Activity

            People

              vinodkc Vinod KC
              norvellj James Norvell
              Votes:
              2 Vote for this issue
              Watchers:
              7 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: