Skip to content

Conversation

@wzx140
Copy link
Contributor

@wzx140 wzx140 commented Nov 30, 2022

Change Logs

  1. Modify composeSchemaEvolutionTransformer to get the evolved schema
  2. Support schema evolution for parquet
  3. Clear cache in HoodieInMemoryHashIndex after each test in sql test
  4. Add prefixFieldName to multi record keys in RowKeyGenerator
  5. Sometimes, we need to extract the recordKey from the partition-dropped data. Change RowKeyGenerator's behaviors to be consistent with avro key generator.
  6. Fix kyro register. "AbstractMethodError". I found that relocation shade of kryo changes the method signature. We should not let Spark call HoodieSparkKryoRegistrar's method related to kryo.

RFC46 Phase 1
To achieve stated goals of avoiding unnecessary conversions into intermediate representation (Avro), we promote HoodieRecord to become a standardized API of interacting with a single record holding internal engine-specific representation of the payload and replace all accesses from HoodieRecordPayload. We extract Record Combining (Merge) API from HoodieRecordPayload into a standalone, stateless component (engine). More information can be found in RFC46.

HoodieAvroRecordMerger and HoodiePayload are used together to support all existing features. Now the spark part has been realized. The following is the caveats with HoodieSparkRecordMerger

  • Only Parquet log is now supported
  • For sql merge api, it will automatically backoff to HoodieAvroRecordMerger
  • Not support hoodie.metadata.index.column.stats
  • SparkMerger should use "HoodieSparkValidateDuplicateKeyRecordMerger" with "hoodie.sql.insert.mode=strict"

Impact

Rebase release-feature-rfc46 on master

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

none

Documentation Update

Describe any necessary documentation update if there is any new feature, config, or user-facing change

  • The config description must be updated if new configs are added or the default value of the configs are changed
  • Any new feature or user-facing change requires updating the Hudi website. Please create a Jira ticket, attach the
    ticket number here and follow the instruction to make
    changes to the website.

Contributor's checklist

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

@wzx140 wzx140 changed the title Rfc46 rebase RFC46 rebase Nov 30, 2022
@wzx140 wzx140 changed the title RFC46 rebase [RFC46] rebase Nov 30, 2022
@wzx140 wzx140 changed the title [RFC46] rebase [HUDI-3378][HUDI-3379][HUDI-3381] RFC46 rebase Nov 30, 2022
@wzx140 wzx140 changed the title [HUDI-3378][HUDI-3379][HUDI-3381] RFC46 rebase [HUDI-3378] RFC46 rebase Nov 30, 2022
@wzx140 wzx140 force-pushed the rfc46-rebase branch 5 times, most recently from 00cb133 to 7a65409 Compare December 3, 2022 04:14
@wzx140
Copy link
Contributor Author

wzx140 commented Dec 3, 2022

@hudi-bot run azure

@wzx140
Copy link
Contributor Author

wzx140 commented Dec 3, 2022

@alexeykudinkin alexeykudinkin self-requested a review December 13, 2022 06:31
@wzx140 wzx140 force-pushed the rfc46-rebase branch 2 times, most recently from 2cfcca5 to 1930cfe Compare December 13, 2022 06:52
@wzx140
Copy link
Contributor Author

wzx140 commented Dec 13, 2022

@hudi-bot run azure

Copy link
Contributor

@alexeykudinkin alexeykudinkin left a comment

Choose a reason for hiding this comment

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

Great job @wzx140!

@alexeykudinkin alexeykudinkin force-pushed the rfc46-rebase branch 3 times, most recently from 1930cfe to 060d8e2 Compare December 14, 2022 02:04
wzx140 and others added 9 commits December 14, 2022 10:49
[minor] add more test for rfc46 (apache#7003)

## Change Logs

 - Add HoodieSparkValidateDuplicateKeyRecordMerger behaving the same as ValidateDuplicateKeyPayload. We should
 use it with config "hoodie.sql.insert.mode=strict".
 - Fix nest field exist in HoodieCatalystExpressionUtils
 - Fix rewrite in HoodieInternalRowUtiles to support type promoted as avro
 - Fallback to avro when use "merge into" sql
 - Fix some schema handling issue
 - Support delta streamer
 - Convert parquet schema to spark schema and then avro schema(in
 org.apache.hudi.io.storage.HoodieSparkParquetReader#getSchema). Some types in avro are not compatible with
 parquet. For ex, decimal as int32/int64 in parquet will convert to int/long in avro. Because avro do not has decimal as
 int/long . We will lose the logic type info if we directly convert it to avro schema.
 - Support schema evolution in parquet block

[Minor] fix multi deser avro payload (apache#7021)

In HoodieAvroRecord, we will call isDelete, shouldIgnore before we write it to the file. Each method will deserialize HoodiePayload. So we add deserialization method in HoodieRecord and call this method once before calling isDelete or shouldIgnore.

Co-authored-by: wangzixuan.wzxuan <[email protected]>
Co-authored-by: Alexey Kudinkin <[email protected]>
Co-authored-by: Alexey Kudinkin <[email protected]>

[MINOR] Properly registering target classes w/ Kryo (apache#7026)

* Added `HoodieKryoRegistrar` registering necessary Hudi's classes w/ Kryo to make their serialization more efficient (by serializing just the class id, in-liue the fully qualified class-name)

* Redirected Kryo registration to `HoodieKryoRegistrar`

* Registered additional classes likely to be serialized by Kryo

* Updated tests

* Fixed serialization of Avro's `Utf8` to serialize just the bytes

* Added tests

* Added custom `AvroUtf8Serializer`;
Tidying up

* Extracted `HoodieCommonKryoRegistrar` to leverage in `SerializationUtils`

* `HoodieKryoRegistrar` > `HoodieSparkKryoRegistrar`;
Rebased `HoodieSparkKryoRegistrar` onto `HoodieCommonKryoRegistrar`

* `lint`

* Fixing compilation for Spark 2.x

* Disabling flaky test

[MINOR] Make sure all `HoodieRecord`s are appropriately serializable by Kryo (apache#6977)

* Make sure `HoodieRecord`, `HoodieKey`, `HoodieRecordLocation` are all `KryoSerializable`

* Revisited `HoodieRecord` serialization hooks to make sure they a) could not be overridden, b) provide for hooks to properly
serialize record's payload;
Implemented serialization hooks for `HoodieAvroIndexedRecord`;
Implemented serialization hooks for `HoodieEmptyRecord`;

* Revisited `HoodieRecord` serialization hooks to make sure they a) could not be overridden, b) provide for hooks to properly
serialize record's payload;
Implemented serialization hooks for `HoodieAvroIndexedRecord`;
Implemented serialization hooks for `HoodieEmptyRecord`;
Implemented serialization hooks for `HoodieAvroRecord`;

* Revisited `HoodieSparkRecord` to transiently hold on to the schema so that it could project row

* Implemented serialization hooks for `HoodieSparkRecord`

* Added `TestHoodieSparkRecord`

* Added tests for Avro-based records

* Added test for `HoodieEmptyRecord`

* Fixed sealing/unsealing for `HoodieRecord` in `HoodieBackedTableMetadataWriter`

* Properly handle deflated records

* Fixing `Row`s encoding

* Fixed `HoodieRecord` to be properly sealed/unsealed

* Fixed serialization of the `HoodieRecordGlobalLocation`

[MINOR] Additional fixes for apache#6745 (apache#6947)

* Tidying up

* Tidying up more

* Cleaning up duplication

* Tidying up

* Revisited legacy operating mode configuration

* Tidying up

* Cleaned up `projectUnsafe` API

* Fixing compilation

* Cleaning up `HoodieSparkRecord` ctors;
Revisited mandatory unsafe-projection

* Fixing compilation

* Cleaned up `ParquetReader` initialization

* Revisited `HoodieSparkRecord` to accept either `UnsafeRow` or `HoodieInternalRow`, and avoid unnecessary copying after unsafe-projection

* Cleaning up redundant exception spec

* Make sure `updateMetadataFields` properly wraps `InternalRow` into `HoodieInternalRow` if necessary;
Cleaned up `MetadataValues`

* Fixed meta-fields extraction and `HoodieInternalRow` composition w/in `HoodieSparkRecord`

* De-duplicate `HoodieSparkRecord` ctors;
Make sure either only `UnsafeRow` or `HoodieInternalRow` are permitted inside `HoodieSparkRecord`

* Removed unnecessary copying

* Cleaned up projection for `HoodieSparkRecord` (dropping partition columns);
Removed unnecessary copying

* Fixing compilation

* Fixing compilation (for Flink)

* Cleaned up File Raders' interfaces:
  - Extracted `HoodieSeekingFileReader` interface (for key-ranged reads)
  - Pushed down concrete implementation methods into `HoodieAvroFileReaderBase` from the interfaces

* Cleaned up File Readers impls (inline with then new interfaces)

* Rebsaed `HoodieBackedTableMetadata` onto new `HoodieSeekingFileReader`

* Tidying up

* Missing licenses

* Re-instate custom override for `HoodieAvroParquetReader`;
Tidying up

* Fixed missing cloning w/in `HoodieLazyInsertIterable`

* Fixed missing cloning in deduplication flow

* Allow `HoodieSparkRecord` to hold `ColumnarBatchRow`

* Missing licenses

* Fixing compilation

* Missing changes

* Fixed Spark 2.x validation whether the row was read as a batch

Fix comment in RFC46 (apache#6745)

* rename

* add MetadataValues in updateMetadataValues

* remove singleton in fileFactory

* add truncateRecordKey

* remove hoodieRecord#setData

* rename HoodieAvroRecord

* fix code style

* fix HoodieSparkRecordSerializer

* fix benchmark

* fix SparkRecordUtils

* instantiate HoodieWriteConfig on the fly

* add test

* fix HoodieSparkRecordSerializer. Replace Java's object serialization with kryo

* add broadcast

* fix comment

* remove unnecessary broadcast

* add unsafe check in spark record

* fix getRecordColumnValues

* remove spark.sql.parquet.writeLegacyFormat

* fix unsafe projection

* fix

* pass external schema

* update doc

* rename back to HoodieAvroRecord

* fix

* remove comparable wrapper

* fix comment

* fix comment

* fix comment

* fix comment

* simplify row copy

* fix ParquetReaderIterator

Co-authored-by: Shawy Geng <[email protected]>
Co-authored-by: wangzixuan.wzxuan <[email protected]>

[RFC-46][HUDI-4414] Update the RFC-46 doc to fix comments feedback (apache#6132)

* Update the RFC-46 doc to fix comments feedback

* fix

Co-authored-by: wangzixuan.wzxuan <[email protected]>

[HUDI-4301] [HUDI-3384][HUDI-3385] Spark specific file reader/writer.(apache#5629)

* [HUDI-4301] [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

* add schema finger print

* add benchmark

* a new way to config the merger

* fix

Co-authored-by: wangzixuan.wzxuan <[email protected]>
Co-authored-by: gengxiaoyu <[email protected]>

[HUDI-3350][HUDI-3351] Support HoodieMerge API and Spark engine-specific  HoodieRecord (apache#5627)

Co-authored-by: wangzixuan.wzxuan <[email protected]>

[HUDI-4344] fix usage of HoodieDataBlock#getRecordIterator (apache#6005)

Co-authored-by: wangzixuan.wzxuan <[email protected]>

[HUDI-4292][RFC-46] Update doc to align with the Record Merge API changes (apache#5927)

[MINOR] Fix type casting in TestHoodieHFileReaderWriter

[HUDI-3378][HUDI-3379][HUDI-3381] Migrate usage of HoodieRecordPayload and raw Avro payload to HoodieRecord (apache#5522)

Co-authored-by: Alexey Kudinkin <[email protected]>
Co-authored-by: wangzixuan.wzxuan <[email protected]>
wangzixuan.wzxuan added 2 commits December 14, 2022 10:49
@wzx140
Copy link
Contributor Author

wzx140 commented Dec 14, 2022

@hudi-bot run azure

1 similar comment
@wzx140
Copy link
Contributor Author

wzx140 commented Dec 14, 2022

@hudi-bot run azure

@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

@wzx140
Copy link
Contributor Author

wzx140 commented Dec 14, 2022

@xushiyan
Copy link
Member

Screen Shot 2022-12-14 at 10 21 20 PM

@xushiyan
Copy link
Member

great job! kudos to @wzx140 @alexeykudinkin @minihippo !

@xushiyan xushiyan merged commit a5bda3a into apache:master Dec 14, 2022
danny0405 added a commit to danny0405/hudi that referenced this pull request Mar 15, 2023
danny0405 added a commit that referenced this pull request Mar 16, 2023
danny0405 added a commit to danny0405/hudi that referenced this pull request Mar 23, 2023
nsivabalan pushed a commit to nsivabalan/hudi that referenced this pull request Mar 23, 2023
fengjian428 pushed a commit to fengjian428/hudi that referenced this pull request Apr 5, 2023
## Change Logs

 - Add HoodieSparkValidateDuplicateKeyRecordMerger behaving the same as ValidateDuplicateKeyPayload. We should use it with config "hoodie.sql.insert.mode=strict".
 - Fix nest field exist in HoodieCatalystExpressionUtils
 - Fix rewrite in HoodieInternalRowUtiles to support type promoted as avro
 - Fallback to avro when use "merge into" sql
 - Fix some schema handling issue
 - Support delta streamer
 - Convert parquet schema to spark schema and then avro schema(in
 org.apache.hudi.io.storage.HoodieSparkParquetReader#getSchema). Some types in avro are not compatible with
 parquet. For ex, decimal as int32/int64 in parquet will convert to int/long in avro. Because avro do not has decimal as
 int/long . We will lose the logic type info if we directly convert it to avro schema.
 - Support schema evolution in parquet block

[Minor] fix multi deser avro payload (apache#7021)

In HoodieAvroRecord, we will call isDelete, shouldIgnore before we write it to the file. Each method will deserialize HoodiePayload. So we add deserialization method in HoodieRecord and call this method once before calling isDelete or shouldIgnore.

Co-authored-by: wangzixuan.wzxuan <[email protected]>
Co-authored-by: Alexey Kudinkin <[email protected]>
Co-authored-by: Alexey Kudinkin <[email protected]>

[MINOR] Properly registering target classes w/ Kryo (apache#7026)

* Added `HoodieKryoRegistrar` registering necessary Hudi's classes w/ Kryo to make their serialization more efficient (by serializing just the class id, in-liue the fully qualified class-name)

[MINOR] Make sure all `HoodieRecord`s are appropriately serializable by Kryo (apache#6977)

* Make sure `HoodieRecord`, `HoodieKey`, `HoodieRecordLocation` are all `KryoSerializable`

* Revisited `HoodieRecord` serialization hooks to make sure they a) could not be overridden, b) provide for hooks to properly
serialize record's payload;
Implemented serialization hooks for `HoodieAvroIndexedRecord`;
Implemented serialization hooks for `HoodieEmptyRecord`;
Implemented serialization hooks for `HoodieAvroRecord`;

* Revisited `HoodieSparkRecord` to transiently hold on to the schema so that it could project row

[MINOR] Additional fixes for apache#6745 (apache#6947)

Co-authored-by: Shawy Geng <[email protected]>
Co-authored-by: wangzixuan.wzxuan <[email protected]>

[RFC-46][HUDI-4414] Update the RFC-46 doc to fix comments feedback (apache#6132)

Co-authored-by: wangzixuan.wzxuan <[email protected]>

[HUDI-4301] [HUDI-3384][HUDI-3385] Spark specific file reader/writer.(apache#5629)

Co-authored-by: wangzixuan.wzxuan <[email protected]>
Co-authored-by: gengxiaoyu <[email protected]>

[HUDI-3350][HUDI-3351] Support HoodieMerge API and Spark engine-specific  HoodieRecord (apache#5627)

Co-authored-by: wangzixuan.wzxuan <[email protected]>

[HUDI-4344] fix usage of HoodieDataBlock#getRecordIterator (apache#6005)

Co-authored-by: wangzixuan.wzxuan <[email protected]>

[HUDI-4292][RFC-46] Update doc to align with the Record Merge API changes (apache#5927)

[MINOR] Fix type casting in TestHoodieHFileReaderWriter

[HUDI-3378][HUDI-3379][HUDI-3381] Migrate usage of HoodieRecordPayload and raw Avro payload to HoodieRecord (apache#5522)

Co-authored-by: Alexey Kudinkin <[email protected]>
Co-authored-by: wangzixuan.wzxuan <[email protected]>
Co-authored-by: gengxiaoyu <[email protected]>
fengjian428 pushed a commit to fengjian428/hudi that referenced this pull request Apr 5, 2023
stayrascal pushed a commit to stayrascal/hudi that referenced this pull request Apr 20, 2023
h1ap pushed a commit to h1ap/hudi that referenced this pull request May 15, 2023
@danny0405
Copy link
Contributor

This PR introduces so many regressions, don't think it is a valuable PR from the gains. Let's be more conservertive to accept huge PRs and more rigorous for code reviewing without sufficient tests.

@yihua yihua changed the title [HUDI-3378] RFC46 rebase [HUDI-3378] RFC-46 rebase Sep 22, 2023
KnightChess pushed a commit to KnightChess/hudi that referenced this pull request Jan 2, 2024
@SuppressWarnings("unchecked")
@Override
protected final IndexedRecord readRecordPayload(Kryo kryo, Input input) {
// NOTE: We're leveraging Spark's default [[GenericAvroSerializer]] to serialize Avro
Copy link
Contributor

Choose a reason for hiding this comment

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

@wzx140 GenericRecord.class is interface , this will cause InstantiationError. yes or no ?

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

Labels

None yet

Projects

Status: ✅ Done

Development

Successfully merging this pull request may close these issues.

6 participants