Skip to content

Conversation

@wecharyu
Copy link
Contributor

@wecharyu wecharyu commented May 16, 2023

Change Logs

HUDI-4149 fix the drop table error if table directory moved, but it will make the Spark catalog table schema not consistent with Hudi schema if some column types are not Avro data types.
This PR want to ensure the schema consistency when create a hoodie table. Otherwise it will throw exception in hive-sync:

Caused by: org.apache.hudi.hive.HoodieHiveSyncException: Could not convert field Type from TINYINT to int for field has_lowest_price_guarantee
	at org.apache.hudi.hive.util.HiveSchemaUtil.getSchemaDifference(HiveSchemaUtil.java:118)
	at org.apache.hudi.hive.HiveSyncTool.syncSchema(HiveSyncTool.java:369)
	at org.apache.hudi.hive.HiveSyncTool.syncHoodieTable(HiveSyncTool.java:280)
	at org.apache.hudi.hive.HiveSyncTool.doSync(HiveSyncTool.java:198)
	at org.apache.hudi.hive.HiveSyncTool.syncHoodieTable(HiveSyncTool.java:164)
	... 65 more

Impact

None.

Risk level (write none, low medium or high below)

None.

Documentation Update

None.

Contributor's checklist

  • Read through contributor's guide
  • Change Logs and Impact were stated clearly
  • Adequate tests were added if applicable
  • CI passed

Copy link
Contributor

@danny0405 danny0405 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

cc @leesf Can you help the review?

assertTrue(exception.getMessage.contains(s"""$tableName is not a Hudi table"""))
}

test("Test hoodie table schema consistency for non-Avro data types") {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

do you also tested it with hive sync enabled?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, I have test the write operation with hive-sync. It works well because the table schema has been converted to avro types.

@wecharyu
Copy link
Contributor Author

@leesf The failed test seems a bit strange, the report failed test is:

2023-05-19T01:35:52.6353614Z [ERROR] Tests run: 12, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 155.564 s <<< FAILURE! - in org.apache.hudi.functional.TestDataSourceForBootstrap
2023-05-19T01:35:52.6358391Z [ERROR] testMetadataBootstrapMORPartitionedInlineClustering{boolean}[2]  Time elapsed: 8.665 s  <<< ERROR!
2023-05-19T01:35:52.6359417Z org.apache.spark.SparkException: 
2023-05-19T01:35:52.6360746Z Job aborted due to stage failure: Task 2 in stage 66.0 failed 1 times, most recent failure: Lost task 2.0 in stage 66.0 (TID 150) (fv-az1114-420.v3gn4mkxnddubagvm3aoiftxed.cx.internal.cloudapp.net executor driver): org.apache.hudi.exception.HoodieException: Unable to instantiate payload class 
2023-05-19T01:35:52.6361900Z 	at org.apache.hudi.common.util.HoodieRecordUtils.loadPayload(HoodieRecordUtils.java:103)
2023-05-19T01:35:52.6362657Z 	at org.apache.hudi.common.util.SpillableMapUtils.convertToHoodieRecordPayload(SpillableMapUtils.java:153)
2023-05-19T01:35:52.6363382Z 	at org.apache.hudi.common.util.SpillableMapUtils.convertToHoodieRecordPayload(SpillableMapUtils.java:121)
2023-05-19T01:35:52.6364119Z 	at org.apache.hudi.avro.HoodieAvroUtils.createHoodieRecordFromAvro(HoodieAvroUtils.java:1100)
2023-05-19T01:35:52.6438493Z 	at org.apache.hudi.common.model.HoodieAvroIndexedRecord.wrapIntoHoodieRecordPayloadWithParams(HoodieAvroIndexedRecord.java:159)
2023-05-19T01:35:52.6439602Z 	at org.apache.hudi.common.table.log.HoodieFileSliceReader.getFileSliceReader(HoodieFileSliceReader.java:45)
2023-05-19T01:35:52.6440832Z 	at org.apache.hudi.client.clustering.run.strategy.MultipleSparkJobExecutionStrategy.lambda$null$8(MultipleSparkJobExecutionStrategy.java:313)
2023-05-19T01:35:52.6441605Z 	at java.util.Iterator.forEachRemaining(Iterator.java:116)
2023-05-19T01:35:52.6442126Z 	at scala.collection.convert.Wrappers$IteratorWrapper.forEachRemaining(Wrappers.scala:30)
2023-05-19T01:35:52.6443174Z 	at org.apache.hudi.client.clustering.run.strategy.MultipleSparkJobExecutionStrategy.lambda$readRecordsForGroupWithLogs$7e6d03ba$1(MultipleSparkJobExecutionStrategy.java:287)
2023-05-19T01:35:52.6444022Z 	at org.apache.spark.api.java.JavaRDDLike.$anonfun$mapPartitions$1(JavaRDDLike.scala:153)
2023-05-19T01:35:52.6444558Z 	at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2(RDD.scala:863)
2023-05-19T01:35:52.6445056Z 	at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2$adapted(RDD.scala:863)
2023-05-19T01:35:52.6445573Z 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
2023-05-19T01:35:52.6446142Z 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
2023-05-19T01:35:52.6446639Z 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
2023-05-19T01:35:52.6447151Z 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
2023-05-19T01:35:52.6447693Z 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
2023-05-19T01:35:52.6448192Z 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
2023-05-19T01:35:52.6448735Z 	at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
2023-05-19T01:35:52.6449356Z 	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
2023-05-19T01:35:52.6449922Z 	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
2023-05-19T01:35:52.6450449Z 	at org.apache.spark.scheduler.Task.run(Task.scala:131)
2023-05-19T01:35:52.6450951Z 	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506)
2023-05-19T01:35:52.6451467Z 	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1491)
2023-05-19T01:35:52.6451952Z 	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509)
2023-05-19T01:35:52.6452568Z 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
2023-05-19T01:35:52.6453166Z 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
2023-05-19T01:35:52.6453639Z 	at java.lang.Thread.run(Thread.java:750)
2023-05-19T01:35:52.6454160Z Caused by: java.lang.reflect.InvocationTargetException
2023-05-19T01:35:52.6454678Z 	at sun.reflect.GeneratedConstructorAccessor117.newInstance(Unknown Source)
2023-05-19T01:35:52.6455358Z 	at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
2023-05-19T01:35:52.6456124Z 	at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
2023-05-19T01:35:52.6456712Z 	at org.apache.hudi.common.util.HoodieRecordUtils.loadPayload(HoodieRecordUtils.java:101)
2023-05-19T01:35:52.6457162Z 	... 28 more
2023-05-19T01:35:52.6459273Z Caused by: org.apache.hudi.exception.HoodieException: Ordering value is null for record: {"_hoodie_commit_time": "00000000000001", "_hoodie_commit_seqno": "00000000000001_0_0", "_hoodie_record_key": "trip_77", "_hoodie_partition_path": "datestr=2020-04-03", "_hoodie_file_name": "8b831469-fe34-45fe-a711-9d1a2523ddd9_0-7-17_00000000000001.parquet", "timestamp": null, "_row_key": null, "partition_path": null, "rider": null, "driver": null, "begin_lat": null, "begin_lon": null, "end_lat": null, "end_lon": null, "fare": null, "tip_history": null, "_hoodie_is_deleted": null, "datestr": null}

I can not reproduce it in local with command for both Spark3.1 and Spark3.2:

mvn test -Dtest=org.apache.hudi.functional.TestDataSourceForBootstrap -Punit-tests -Dscala-2.12 -Dspark3.2 -pl "hudi-spark-datasource/hudi-spark" -e -ntp -B -V -Pwarn-log -Dorg.slf4j.simpleLogger.log.org.apache.maven.plugins.shade=warn -Dorg.slf4j.simpleLogger.log.org.apache.maven.plugins.dependency=warn -DfailIfNoTests=false

image

@wecharyu
Copy link
Contributor Author

The tests have been fixed. Kindly request to retrigger the pipeline. @leesf

@hudi-bot
Copy link
Collaborator

CI report:

Bot commands @hudi-bot supports the following commands:
  • @hudi-bot run azure re-run the last Azure build

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants