stackfun opened a new issue #2657:
URL: https://github.com/apache/hudi/issues/2657


   **Describe the problem you faced**
   
   If the data ingested into a hudi MOR table contains two or more 
record/struct array fields, then hive queries fail on the _rt table.
   I believe hudi is attempting to construct the avro schema from the schema 
stored in hive, but hudi re-uses the same "array" record name multiple times. 
In avro, record names have to be unique.
   
   **To Reproduce**
   
   ```
   def array_field_rt_defect(spark, database_name, table_name, destination):
       def gen_data(start, stop):
           return [
               {
                   "uuid": str(i),
                   "partitionpath": "partition",
                   "array_1": [],  # array does not need to be populated to 
reproduce issue
                   "array_2": [],  # two arrays need to be defined in the 
schema to reproduce
                   "ts": str(i),
               }
               for i in range(start, stop)
           ]
       schema = StructType(
           [
               StructField("uuid", StringType(), True),
               StructField("partitionpath", StringType(), True),
               StructField("array_1", ArrayType(
                   StructType(
                       [
                           StructField("field_1", StringType(), True),
                           StructField("field_2", StringType(), True),
                       ]
                   )
               ), True),
               StructField("array_2", ArrayType(
                   StructType(
                       [
                           StructField("field_3", StringType(), True),
                           StructField("field_4", StringType(), True),
                       ]
                   )
               ), True),
               StructField("ts", StringType(), True)
           ]
       )
       hudi_write_options = {
           "hoodie.table.name": table_name,
           "hoodie.datasource.write.operation": "upsert",
           "hoodie.datasource.write.table.name": table_name,
           "hoodie.datasource.write.table.type": "MERGE_ON_READ",
           "hoodie.datasource.hive_sync.enable": True,
           "hoodie.datasource.hive_sync.database": database_name,
           "hoodie.datasource.hive_sync.table": table_name,
           "hoodie.datasource.hive_sync.partition_fields": "partitionpath",
           "hoodie.datasource.hive_sync.partition_extractor_class": 
"org.apache.hudi.hive.MultiPartKeysValueExtractor"
       }
       df = spark.read.json(spark.sparkContext.parallelize(gen_data(0, 100)), 
schema)
       
df.write.format("hudi").options(**hudi_write_options).mode("overwrite").save(destination)
       df = spark.read.json(spark.sparkContext.parallelize(gen_data(50, 150)), 
schema)
       
df.write.format("hudi").options(**hudi_write_options).mode("append").save(destination)
   
       # avro array error
       spark.sql(f"select * from {database_name}.{table_name}_rt").show()
   ```
   
   **Expected behavior**
   
   SparkSQL Query Succeeds
   
   **Environment Description**
   
   * Hudi version : 0.7.0
   
   * Spark version : 2.4.7
   
   * Hive version : 2.3.7
   
   * Hadoop version : 2.9.2
   
   * Storage (HDFS/S3/GCS..) : GCS
   
   * Running on Docker? (yes/no) : no
   
   * GCP Dataproc 1.4
   
   **Stacktrace**
   
   ```
    org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 
in stage 61.0 failed 4 times, most recent failure: Lost task 0.3 in stage 61.0 
(TID 25547, dataproc-w-4.us-east1-b.c.my-project.internal, executor 10): 
org.apache.avro.SchemaParseException: Can't redefine: array
        at org.apache.avro.Schema$Names.put(Schema.java:1128)
        at org.apache.avro.Schema$NamedSchema.writeNameRef(Schema.java:562)
        at org.apache.avro.Schema$RecordSchema.toJson(Schema.java:690)
        at org.apache.avro.Schema$ArraySchema.toJson(Schema.java:805)
        at org.apache.avro.Schema$UnionSchema.toJson(Schema.java:882)
        at org.apache.avro.Schema$RecordSchema.fieldsToJson(Schema.java:716)
        at org.apache.avro.Schema$RecordSchema.toJson(Schema.java:701)
        at org.apache.avro.Schema.toString(Schema.java:324)
        at 
org.apache.hudi.hadoop.realtime.AbstractRealtimeRecordReader.constructHiveOrderedSchema(AbstractRealtimeRecordReader.java:135)
        at 
org.apache.hudi.hadoop.realtime.AbstractRealtimeRecordReader.init(AbstractRealtimeRecordReader.java:103)
        at 
org.apache.hudi.hadoop.realtime.AbstractRealtimeRecordReader.<init>(AbstractRealtimeRecordReader.java:67)
        at 
org.apache.hudi.hadoop.realtime.RealtimeCompactedRecordReader.<init>(RealtimeCompactedRecordReader.java:53)
        at 
org.apache.hudi.hadoop.realtime.HoodieRealtimeRecordReader.constructRecordReader(HoodieRealtimeRecordReader.java:70)
        at 
org.apache.hudi.hadoop.realtime.HoodieRealtimeRecordReader.<init>(HoodieRealtimeRecordReader.java:47)
        at 
org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat.getRecordReader(HoodieParquetRealtimeInputFormat.java:123)
        at 
org.apache.spark.rdd.HadoopRDD$$anon$1.liftedTree1$1(HadoopRDD.scala:267)
        at org.apache.spark.rdd.HadoopRDD$$anon$1.<init>(HadoopRDD.scala:266)
        at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:224)
        at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:95)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:346)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:310)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:346)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:310)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:346)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:310)
        at org.apache.spark.rdd.UnionRDD.compute(UnionRDD.scala:105)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:346)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:310)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:346)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:310)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:346)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:310)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:346)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:310)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:346)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:310)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
        at org.apache.spark.scheduler.Task.run(Task.scala:123)
        at 
org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)
   
   Driver stacktrace:
        at 
org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1892)
        at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1880)
        at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1879)
        at 
scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
        at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
        at 
org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1879)
        at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:927)
        at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:927)
        at scala.Option.foreach(Option.scala:257)
        at 
org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:927)
        at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:2113)
        at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2062)
        at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2051)
        at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:49)
        at 
org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:738)
        at org.apache.spark.SparkContext.runJob(SparkContext.scala:2061)
        at org.apache.spark.SparkContext.runJob(SparkContext.scala:2082)
        at org.apache.spark.SparkContext.runJob(SparkContext.scala:2101)
        at 
org.apache.spark.sql.execution.SparkPlan.executeTake(SparkPlan.scala:365)
        at 
org.apache.spark.sql.execution.CollectLimitExec.executeCollect(limit.scala:38)
        at 
org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$collectFromPlan(Dataset.scala:3389)
        at 
org.apache.spark.sql.Dataset$$anonfun$head$1.apply(Dataset.scala:2550)
        at 
org.apache.spark.sql.Dataset$$anonfun$head$1.apply(Dataset.scala:2550)
        at org.apache.spark.sql.Dataset$$anonfun$52.apply(Dataset.scala:3370)
        at 
org.apache.spark.sql.execution.SQLExecution$$anonfun$withNewExecutionId$1.apply(SQLExecution.scala:80)
        at 
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:127)
        at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:75)
        at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3369)
        at org.apache.spark.sql.Dataset.head(Dataset.scala:2550)
        at org.apache.spark.sql.Dataset.take(Dataset.scala:2764)
        at org.apache.spark.sql.Dataset.getRows(Dataset.scala:254)
        at org.apache.spark.sql.Dataset.showString(Dataset.scala:291)
        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 py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
        at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
        at py4j.Gateway.invoke(Gateway.java:282)
        at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
        at py4j.commands.CallCommand.execute(CallCommand.java:79)
        at py4j.GatewayConnection.run(GatewayConnection.java:238)
        at java.lang.Thread.run(Thread.java:748)
   Caused by: org.apache.avro.SchemaParseException: Can't redefine: array
        at org.apache.avro.Schema$Names.put(Schema.java:1128)
        at org.apache.avro.Schema$NamedSchema.writeNameRef(Schema.java:562)
        at org.apache.avro.Schema$RecordSchema.toJson(Schema.java:690)
        at org.apache.avro.Schema$ArraySchema.toJson(Schema.java:805)
        at org.apache.avro.Schema$UnionSchema.toJson(Schema.java:882)
        at org.apache.avro.Schema$RecordSchema.fieldsToJson(Schema.java:716)
        at org.apache.avro.Schema$RecordSchema.toJson(Schema.java:701)
        at org.apache.avro.Schema.toString(Schema.java:324)
        at 
org.apache.hudi.hadoop.realtime.AbstractRealtimeRecordReader.constructHiveOrderedSchema(AbstractRealtimeRecordReader.java:135)
        at 
org.apache.hudi.hadoop.realtime.AbstractRealtimeRecordReader.init(AbstractRealtimeRecordReader.java:103)
        at 
org.apache.hudi.hadoop.realtime.AbstractRealtimeRecordReader.<init>(AbstractRealtimeRecordReader.java:67)
        at 
org.apache.hudi.hadoop.realtime.RealtimeCompactedRecordReader.<init>(RealtimeCompactedRecordReader.java:53)
        at 
org.apache.hudi.hadoop.realtime.HoodieRealtimeRecordReader.constructRecordReader(HoodieRealtimeRecordReader.java:70)
        at 
org.apache.hudi.hadoop.realtime.HoodieRealtimeRecordReader.<init>(HoodieRealtimeRecordReader.java:47)
        at 
org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat.getRecordReader(HoodieParquetRealtimeInputFormat.java:123)
        at 
org.apache.spark.rdd.HadoopRDD$$anon$1.liftedTree1$1(HadoopRDD.scala:267)
        at org.apache.spark.rdd.HadoopRDD$$anon$1.<init>(HadoopRDD.scala:266)
        at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:224)
        at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:95)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:346)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:310)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:346)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:310)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:346)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:310)
        at org.apache.spark.rdd.UnionRDD.compute(UnionRDD.scala:105)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:346)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:310)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:346)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:310)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:346)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:310)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:346)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:310)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:346)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:310)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
        at org.apache.spark.scheduler.Task.run(Task.scala:123)
        at 
org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        ... 1 more
   ```


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to