Details
-
Bug
-
Status: Closed
-
Blocker
-
Resolution: Fixed
-
None
-
2
Description
When metadata table is just getting initialized, but first commit is not yet fully complete, reading from metadata table could fail w/ below stacktrace.
22/08/20 02:56:58 ERROR client.RemoteDriver: Failed to run client job 39d720db-b15d-4823-b8b1-54398b143d6e org.apache.hudi.exception.HoodieException: Error fetching partition paths from metadata table at org.apache.hudi.common.fs.FSUtils.getAllPartitionPaths(FSUtils.java:315) at org.apache.hudi.BaseHoodieTableFileIndex.getAllQueryPartitionPaths(BaseHoodieTableFileIndex.java:176) at org.apache.hudi.BaseHoodieTableFileIndex.loadPartitionPathFiles(BaseHoodieTableFileIndex.java:219) at org.apache.hudi.BaseHoodieTableFileIndex.doRefresh(BaseHoodieTableFileIndex.java:264) at org.apache.hudi.BaseHoodieTableFileIndex.(BaseHoodieTableFileIndex.java:139) at org.apache.hudi.hadoop.HiveHoodieTableFileIndex.(HiveHoodieTableFileIndex.java:49) at org.apache.hudi.hadoop.HoodieCopyOnWriteTableInputFormat.listStatusForSnapshotMode(HoodieCopyOnWriteTableInputFormat.java:234) at org.apache.hudi.hadoop.HoodieCopyOnWriteTableInputFormat.listStatus(HoodieCopyOnWriteTableInputFormat.java:141) at org.apache.hudi.hadoop.HoodieParquetInputFormatBase.listStatus(HoodieParquetInputFormatBase.java:90) at org.apache.hudi.hadoop.hive.HoodieCombineHiveInputFormat$HoodieCombineFileInputFormatShim.listStatus(HoodieCombineHiveInputFormat.java:889) at org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat.getSplits(CombineFileInputFormat.java:217) at org.apache.hadoop.mapred.lib.CombineFileInputFormat.getSplits(CombineFileInputFormat.java:76) at org.apache.hudi.hadoop.hive.HoodieCombineHiveInputFormat$HoodieCombineFileInputFormatShim.getSplits(HoodieCombineHiveInputFormat.java:942) at org.apache.hudi.hadoop.hive.HoodieCombineHiveInputFormat.getCombineSplits(HoodieCombineHiveInputFormat.java:241) at org.apache.hudi.hadoop.hive.HoodieCombineHiveInputFormat.getSplits(HoodieCombineHiveInputFormat.java:363) at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:205) at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:253) at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:251) at scala.Option.getOrElse(Option.scala:121) at org.apache.spark.rdd.RDD.partitions(RDD.scala:251) at org.apache.spark.rdd.RDD.getNumPartitions(RDD.scala:267) at org.apache.spark.api.java.JavaRDDLike$class.getNumPartitions(JavaRDDLike.scala:65) at org.apache.spark.api.java.AbstractJavaRDDLike.getNumPartitions(JavaRDDLike.scala:45) at org.apache.hadoop.hive.ql.exec.spark.SparkPlanGenerator.generateMapInput(SparkPlanGenerator.java:252) at org.apache.hadoop.hive.ql.exec.spark.SparkPlanGenerator.generateParentTran(SparkPlanGenerator.java:179) at org.apache.hadoop.hive.ql.exec.spark.SparkPlanGenerator.generate(SparkPlanGenerator.java:130) at org.apache.hadoop.hive.ql.exec.spark.RemoteHiveSparkClient$JobStatusJob.call(RemoteHiveSparkClient.java:355) at org.apache.hive.spark.client.RemoteDriver$JobWrapper.call(RemoteDriver.java:400) at org.apache.hive.spark.client.RemoteDriver$JobWrapper.call(RemoteDriver.java:365) at java.util.concurrent.FutureTask.run(FutureTask.java:266) 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) Caused by: org.apache.hudi.exception.HoodieMetadataException: Failed to retrieve list of partition from metadata at org.apache.hudi.metadata.BaseTableMetadata.getAllPartitionPaths(BaseTableMetadata.java:113) at org.apache.hudi.common.fs.FSUtils.getAllPartitionPaths(FSUtils.java:313) ... 32 more Caused by: java.util.NoSuchElementException: No value present in Option at org.apache.hudi.common.util.Option.get(Option.java:89) at org.apache.hudi.metadata.HoodieTableMetadataUtil.getPartitionFileSlices(HoodieTableMetadataUtil.java:1057) at org.apache.hudi.metadata.HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices(HoodieTableMetadataUtil.java:1001) at org.apache.hudi.metadata.HoodieBackedTableMetadata.getPartitionFileSliceToKeysMapping(HoodieBackedTableMetadata.java:377) at org.apache.hudi.metadata.HoodieBackedTableMetadata.getRecordsByKeys(HoodieBackedTableMetadata.java:204) at org.apache.hudi.metadata.HoodieBackedTableMetadata.getRecordByKey(HoodieBackedTableMetadata.java:140) at org.apache.hudi.metadata.BaseTableMetadata.fetchAllPartitionPaths(BaseTableMetadata.java:281) at org.apache.hudi.metadata.BaseTableMetadata.getAllPartitionPaths(BaseTableMetadata.java:111) ... 33 more 22/08/20 02:56:59 INFO client.RemoteDriver: Shutting down Spark Remote Driver. 22/08/20 02:56:59 INFO server.AbstractConnector: Stopped Spark@ce7a81b{HTTP/1.1,[http/1.1]}{0.0.0.0:0} 22/08/20 02:56:59 INFO ui.SparkUI: Stopped Spark web UI at http://scsp04097:34219 22/08/20 02:56:59 INFO yarn.YarnAllocator: Driver requested a total number of 0 executor(s). 22/08/20 02:56:59 INFO cluster.YarnClusterSchedulerBackend: Shutting down all executors 22/08/20 02:56:59 INFO cluster.YarnSchedulerBackend$YarnDriverEndpoint: Asking each executor to shut down 22/08/20 02:56:59 INFO cluster.SchedulerExtensionServices: Stopping SchedulerExtensionServices (serviceOption=None, services=List(), started=false) 22/08/20 02:56:59 INFO spark.MapOutputTrackerMasterEndpoint: MapOutputTrackerMasterEndpoint stopped! 22/08/20 02:56:59 INFO memory.MemoryStore: MemoryStore cleared
Attachments
Issue Links
- links to