Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

...

Hudi supports writing to non-partitioned datasets. For writing to a non-partitioned Hudi dataset and performing hive table syncing, you need to set the below configurations in the properties passed:

Code Block
hoodie.datasource.write.keygenerator.class=org.apache.hudi.NonpartitionedKeyGenerator

...


hoodie.datasource.hive_sync.partition_extractor_class=org.apache.hudi.hive.NonPartitionedExtractor

Why do we have to set 2 different ways of configuring Spark to work with Hudi?

Non-Hive engines tend to do their own listing of DFS to query datasets. For e.g Spark starts reading the paths direct from the file system (HDFS or S3).

From Spark the calls would be as below: 
- org.apache.spark.rdd.NewHadoopRDD.getPartitions 
- org.apache.parquet.hadoop.ParquetInputFormat.getSplits 
- org.apache.hadoop.mapreduce.lib.input.FileInputFormat.getSplits 

Without understanding of Hudi's file layout, engines would just plainly reading all the parquet files and displaying the data within them, with massive amounts of duplicates in the result.

At a high level, there are two ways of configuring a query engine to properly read Hudi datasets

A) Making them invoke methods in `HoodieParquetInputFormat#getSplits` and `HoodieParquetInputFormat#getRecordReader` 

  • Hive does this natively, since the InputFormat is the abstraction in Hive to plugin new table formats. HoodieParquetInputFormat extends MapredParquetInputFormat which is nothing but a input format for hive and we register Hudi tables to Hive metastore backed by these input formats
  • Presto also falls back to calling the input format when it sees a `UseFileSplitsFromInputFormat` annotation, to just obtain splits, but then goes on to use its own optimized/vectorized parquet reader for queries on Copy-on-Write tables
  • Spark can be forced into falling back to the HoodieParquetInputFormat class, using --conf spark.sql.hive.convertMetastoreParquet=false
     `

B) Making the engine invoke a path filter or other means to directly call Hudi classes to filter the files on DFS and pick out the latest file slice

  • Even though we can force Spark to fallback to using the InputFormat class, we could lose ability to use Spark's optimized parquet reader path by doing so. 
  • To keep benefits of native parquet read performance, we set the  `HoodieROTablePathFilter` as a path filter, explicitly set this in the Spark Hadoop Configuration.There is logic in the file: to ensure that folders (paths) or files for Hoodie related files always ensures that latest file slice is selected. This filters out duplicate entries and shows latest entries for each record. 

Contributing to FAQ 

A good and usable FAQ should be community-driven and crowd source questions/thoughts across everyone. 

...