* [HUDI-3290] Different file formats for the partition metadata file.
Partition metadata files are stored in each partition to help identify the base path of a table. These files are saved in the properties file format. Some query engines do not work when non Parquet/ORC files are found in a partition.
Added a new table config 'hoodie.partition.metafile.use.data.format' which when enabled (default false for backward compatibility) ensures that partition metafiles will be saved in the same format as the base files of a dataset.
For new datasets, the config can be set via hudi-cli. Deltastreamer has a new parameter --partition-metafile-use-data-format which will create a table with this setting.
* Code review comments
- Adding a new command to migrate from text to base file formats for meta file.
- Reimplementing readFromFS() to first read the text format, then base format
- Avoid extra exists() checks in readFromFS()
- Added unit tests, enabled parquet format across hoodie-hadoop-mr
- Code cleanup, restructuring, naming consistency.
* Wiring in all the other Spark code paths to respect this config
- Turned on parquet meta format for COW data source tests
- Removed the deltastreamer command line to keep it shorter
* populate HoodiePartitionMetadata#format after readFromFS()
Co-authored-by: Vinoth Chandar <vinoth@apache.org>
Co-authored-by: Raymond Xu <2701446+xushiyan@users.noreply.github.com>
* Add checks for metadata table init to avoid possible out-of-sync
* Revise the logic to reuse existing table config
* Revise docs and naming
Co-authored-by: yuezhang <yuezhang@freewheel.tv>
Co-authored-by: Y Ethan Guo <ethan.guoyihua@gmail.com>
* [HUDI-2560] introduce id_based schema to support full schema evolution.
* add test for FileBasedInternalSchemaStorageManger and rebase code
* add support for change column type and fix some test case
* fix some bugs encountered in the production env and delete useless code
* fix test error
* rebase code
* fixed some nested schema change bugs
* [HUDI-2429][Stacked On HUDI-2560]Support full schema evolution for spark
* [use dummyInternalSchema instead of null]
* add support for spark3.1.x
* remove support for spark3.1.x , sicne some compile fail
* support spark3.1.x
* rebase and prepare solve all comments
* address all comments
* rebase code
* fixed the count(*) bug
* try to get internalSchema by parser commit file/history file directly, not use metaclient which is time cost
address some comments
* fixed all comments
* fix new comments
* rebase code,fix UT failed
* fixed mistake
* rebase code ,fixed new comments
* rebase code , and prepare for address new comments
* address commits
* address new comments
* fix new issues
* control fallback original write logical
In order to drop any metadata partition (index), we can reuse the DELETE_PARTITION operation in metadata table. Subsequent to this, we can support drop index (with table config update) for async metadata indexer.
- Add a new API in HoodieTableMetadataWriter
- Current only supported for Spark metadata writer
- Add a new action called INDEX, whose state transition is described in the RFC.
- Changes in timeline to support the new action.
- Add an index planner in ScheduleIndexActionExecutor.
- Add index plan executor in RunIndexActionExecutor.
- Add 3 APIs in HoodieTableMetadataWriter; a) scheduleIndex: will generate an index plan based on latest completed instant, initialize file groups and add a requested INDEX instant, b) index: executes the index plan and also takes care of writes that happened after indexing was requested, c) dropIndex: will drop index by removing the given metadata partition.
- Add 2 new table configs to serve as the source of truth for inflight and completed indexes.
- Support upgrade/downgrade taking care of the newly added configs.
- Add tool to trigger indexing in HoodieIndexer.
- Handle corner cases related to partial failures.
- Abort gracefully after deleting partition and instant.
- Handle other actions in timeline to consider before catching up
As of now, delete partitions will ensure all file groups are deleted, but the partition as such is not deleted. So, get all partitions might be returning the deleted partitions as well. but no data will be served since all file groups are deleted. With this patch, we are fixing it. We are letting cleaner take care of deleting the partitions when all file groups pertaining to a partitions are deleted.
- Fixed the CleanPlanActionExecutor to return meta info about list of partitions to be deleted. If there are no valid file groups for a partition, clean planner will include the partition to be deleted.
- Fixed HoodieCleanPlan avro schema to include the list of partitions to be deleted
- CleanActionExecutor is fixed to delete partitions if any (as per clean plan)
- Same info is added to HoodieCleanMetadata
- Metadata table when applying clean metadata, will check for partitions to be deleted and will update the "all_partitions" record for the deleted partitions.
Co-authored-by: sivabalan <n.siva.b@gmail.com>
* Added `DataSkippingFailureMode` to control how DS handles failures in the flow (either "strict", when exception would be thrown, or "fallback" when it will just fallback to the full-scan)
* Make sure tests execute in `DataSkippingFailureMode.Strict`
* Fixed Column Stats Index record merging sequence missing `columnName`
- Provided option to trigger clean every nth commit with default number of commits as 1 so that existing users are not affected.
Co-authored-by: sivabalan <n.siva.b@gmail.com>
* Fixed metadata conversion util to extract schema from `HoodieCommitMetadata`
* Fixed failure to fetch columns to index in empty table
* Abort indexing seq in case there are no columns to index
* Fallback to index at least primary key columns, in case no writer schema could be obtained to index all columns
* Fixed `getRecordFields` incorrectly ignoring default value
* Make sure Hudi metadata fields are also indexed
Refactoring Spark DataSource Relations to avoid code duplication.
Following Relations were in scope:
- BaseFileOnlyViewRelation
- MergeOnReadSnapshotRelaation
- MergeOnReadIncrementalRelation
- Adopt HoodieData in Spark action commit executors
- Make Spark independent DeleteHelper, WriteHelper, MergeHelper in hudi-client-common
- Make HoodieTable in WriteClient APIs have raw type to decouple with Client's generic types
Create new TypedProperties while performing clustering
Add OrderedProperties and minor refactoring
Add javadoc and remove getters from OrderedProperties
NOTE: This change is first part of the series to clean up Hudi's Spark DataSource related implementations, making sure there's minimal code duplication among them, implementations are consistent and performant
This PR is making sure that BaseFileOnlyViewRelation only reads projected columns as well as avoiding unnecessary serde from Row to InternalRow
Brief change log
- Introduced HoodieBaseRDD as a base for all custom RDD impls
- Extracted common fields/methods to HoodieBaseRelation
- Cleaned up and streamlined HoodieBaseFileViewOnlyRelation
- Fixed all of the Relations to avoid superfluous Row <> InternalRow conversions
Desc: Add a hive sync config(hoodie.datasource.hive_sync.sync_comment). This config defaults to false.
While syncing data source to hudi, add column comments to source avro schema, and the sync_comment is true, syncing column comments to the hive table.
Rework of #4761
This diff introduces following changes:
- Write stats are converted to metadata index records during the commit. Making them use the HoodieData type so that the record generation scales up with needs.
- Metadata index init support for bloom filter and column stats partitions.
- When building the BloomFilter from the index records, using the type param stored in the payload instead of hardcoded type.
- Delta writes can change column ranges and the column stats index need to be properly updated with new ranges to be consistent with the table dataset. This fix add column stats index update support for the delta writes.
Co-authored-by: Manoj Govindassamy <manoj.govindassamy@gmail.com>