Skip to content

Conversation

@alexeykudinkin
Copy link
Contributor

Change Logs

This PR adjusts the way new and existing HoodieRecords implementations are being serialized by Kryo t/h explicit implementation of KryoSerializable interfaces for

  • HodieRecord (and inheritors)
  • HoodieKey
  • HoodieRecordLocation

Additionally, HoodieSparkRecord serialization is streamlined to always project the Row into UnsafeRow before serialization to avoid penalty of serializing whole object tree.

Impact

No impact

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

Risk: medium

Documentation Update

N/A

Contributor's checklist

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

Alexey Kudinkin added 11 commits October 17, 2022 19:57
…ld not be overridden, b) provide for hooks to properly

serialize record's payload;
Implemented serialization hooks for `HoodieAvroIndexedRecord`;
Implemented serialization hooks for `HoodieEmptyRecord`;
…ld 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`;
@alexeykudinkin alexeykudinkin force-pushed the ak/rfc46-kryo-serde-fix branch from 30a32fe to 416dda2 Compare October 18, 2022 04:28

kryo.writeObjectOrNull(output, key, HoodieKey.class);
kryo.writeObjectOrNull(output, currentLocation, recLocSerializer);
kryo.writeObjectOrNull(output, newLocation, recLocSerializer);
Copy link
Contributor

Choose a reason for hiding this comment

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

How about the performance gains for records SE/DE after this change ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It's less about performance gains in the HoodieRecord per se and more about fixing ser/de for HoodieSparkRecord

  • Previous PR actually removed mandatory conversion to UnsafeRow w/in the HoodieSparkRecord (for performance reasons)
  • Conversion to UnsafeRow is still required though upon ser/de (for shuffling). This change is to handle that.

* (by Kryo)
*/
public HoodieSparkRecord(InternalRow data) {
private final transient StructType schema;
Copy link
Contributor

Choose a reason for hiding this comment

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

I think this is unnecessary. The schema is only used in construction.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

We actually now don't do projection in ctor, instead we only do it when serializing HoodieSparkRecord

@Override
protected final void writeRecordPayload(InternalRow payload, Kryo kryo, Output output) {
// NOTE: [[payload]] could be null if record has already been deflated
UnsafeRow unsafeRow = convertToUnsafeRow(payload, schema);
Copy link
Contributor

Choose a reason for hiding this comment

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

HoodieInternalRow must hold UnsafeRow. Does it need to unsafe projection? We use HoodieInternalRow because we want to modify metaField.
I don't really understand why we should make sure that SparkRecord only hold UnsafeRow. HoodieInternalRow hold UnsafeRow too. They basically make no difference.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is done to simplify ser/deserialization sequnce:

  • We always project to UnsafeRow before serialization, therefore
  • We need to decode just the UnsafeRow when we deserialize
  • Ser/de of UnsafeRow is as simple as just writing out the bytes

* - partitionPath : the partition path of a record.
*/
public class HoodieKey implements Serializable {
public class HoodieKey implements Serializable, KryoSerializable {
Copy link
Contributor

Choose a reason for hiding this comment

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

We have used kryo as the serializer of shuffle. I wonder that Why we explicitly specify KryoSerializable? Same to HoodieRecord.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Correct. We want to enforce that

  • HoodieRecord are serializable by Kryo
  • That we control serialization sequence

@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

@alexeykudinkin
Copy link
Contributor Author

CI is green:

Screen Shot 2022-10-19 at 9 15 48 PM

https://dev.azure.com/apache-hudi-ci-org/apache-hudi-ci/_build/results?buildId=12318&view=results

@alexeykudinkin alexeykudinkin merged commit 9a1aa0a into apache:release-feature-rfc46 Oct 20, 2022
wzx140 added a commit to wzx140/hudi that referenced this pull request Nov 30, 2022
[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]>
wzx140 added a commit to wzx140/hudi that referenced this pull request Dec 1, 2022
[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]>
wzx140 added a commit to wzx140/hudi that referenced this pull request Dec 2, 2022
[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]>
wzx140 added a commit to wzx140/hudi that referenced this pull request Dec 3, 2022
[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]>
wzx140 added a commit to wzx140/hudi that referenced this pull request Dec 9, 2022
[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]>
wzx140 added a commit to wzx140/hudi that referenced this pull request Dec 13, 2022
[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]>
alexeykudinkin pushed a commit to wzx140/hudi that referenced this pull request Dec 13, 2022
[minor] add more test for rfc46 (apache#7003)

 - 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]>
wzx140 added a commit to wzx140/hudi that referenced this pull request Dec 13, 2022
[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]>
alexeykudinkin pushed a commit to wzx140/hudi that referenced this pull request Dec 13, 2022
[minor] add more test for rfc46 (apache#7003)

 - 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]>
alexeykudinkin pushed a commit to onehouseinc/hudi that referenced this pull request Dec 14, 2022
[minor] add more test for rfc46 (apache#7003)

 - 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]>
wzx140 added a commit to wzx140/hudi that referenced this pull request Dec 14, 2022
[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]>
xushiyan pushed a commit that referenced this pull request Dec 14, 2022
## 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 (#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 (#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 (#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 #6745 (#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 (#6132)

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

[HUDI-4301] [HUDI-3384][HUDI-3385] Spark specific file reader/writer.(#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 (#5627)

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

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

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

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

[MINOR] Fix type casting in TestHoodieHFileReaderWriter

[HUDI-3378][HUDI-3379][HUDI-3381] Migrate usage of HoodieRecordPayload and raw Avro payload to HoodieRecord (#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
## 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]>
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