-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[RFC-46][HUDI-4414] Update the RFC-46 doc to fix comments feedback #6132
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
rfc/rfc-46/rfc-46.md
Outdated
|
|
||
| Option<HoodieRecord> combineAndGetUpdateValue(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException; | ||
| interface HoodieRecordMerger { | ||
| // combineAndGetUpdateValue and precombine |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Let's update the comment to elaborate on what's expected semantic of this method (for ex, we should mention that this operation should be associative)
rfc/rfc-46/rfc-46.md
Outdated
| } | ||
| @Override | ||
| Option<HoodieRecord> merge(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException { | ||
| // HoodieSparkRecord precombine and combineAndGetUpdateValue |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Same comment as above
rfc/rfc-46/rfc-46.md
Outdated
| already used in production by Hudi users, we will provide a BWC-bridge in the form of instance of `HoodieRecordMerger` called `HoodieAvroRecordMerger`, that will | ||
| be using user-defined subclass of `HoodieRecordPayload` to combine the records. | ||
|
|
||
| Leveraging such bridge will make provide for seamless BWC migration to the 0.11 release, however will be removing the performance |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Typo: "make provide"
rfc/rfc-46/rfc-46.md
Outdated
|
|
||
| Precombine is used to merge records from logs or incoming records; CombineAndGetUpdateValue is used to merge record from log file and record from base file. | ||
| these two merge logics are not exactly the same for some RecordPayload, such as OverwriteWithLatestAvroPaload. | ||
| We add an Enum in HoodieRecord to mark where it comes from(BASE, LOG or WRITE). `HoodieAvroRecordMerger`'s API will look like following: |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't think we even need this: current implementation of OverwriteWithLatestAvroPayload is actually implemented inconsistently right now:
- When de-duping it will use
orderingValto determine the latest record, but - When merging persisted record w/ incoming it will just assume that incoming somehow is more recent than persisted one (even though it could be the opposite
We should actually unify its semantic to be consistent across preCombine and combineAndGetUpdateValue
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
For downward compatibility, I think it is necessary to maintain this difference from OverwriteWithLatestAvroPayload. But in the future implement based on HoodieRecordMerger, this semantics can be unified
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think we should qualify existing behavior as simply a bug and resolve it with this, instead of going out of the way to maintain BWC.
What's your take folks @vinothchandar @prasannarajaperumal?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I agree. But we need to check all the recordPayloads, and the ut involved. Maybe we can do a bug fix later. Because this function is evolving and we can change it.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This will be another breaking change then. I totally understand and am with you that it's not making our life easier, but i think it'd be better if we take it upfront make all the breaking changes at once and be done with it.
For ex, this new Enum is going to stick with us for a while if we'd introduce it and may produce new implementations using it which we will need to unwind in the future, which is exactly what i think we'd try to avoid.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think you're right. I've removed the mark(BASE, LOG or WRITE) in HoodieRecord and unified logic of HoodieSparkRecord.
rfc/rfc-46/rfc-46.md
Outdated
| The MERGE_CLASS_NAME config is engine-aware. If you are not specified the MERGE_CLASS_NAME, MERGE_CLASS_NAME will be specified default according to your engine type. | ||
|
|
||
| ### Public Api in HoodieRecord | ||
| Because we implement different types of records, we need to transfer some func in AvroUtils into HoodieRecord for different data(avro, InternalRow, RowData). |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We might need to re-phrase this to make message a little more clear: we are saying that we will be implementing functionality similar to AvroUtils in HoodieRecord, right?
rfc/rfc-46/rfc-46.md
Outdated
| /** | ||
| * This method used to extract HoodieKey not through keyGenerator. | ||
| */ | ||
| HoodieRecord expansion( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We should not have this method:
- It accepts
payloadClassthat we're eliminating - It accepts
preCombineFieldthat should only be used insideHoodieRecordMerger - Its purpose is unclear
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This method used in LogScanner. It expanses the raw data(avro, InternalRow) to HoodieRecord which has HoodieKey.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
HoodieAvroRecord need to use preCombineField to extract orderingVal. Is it a better way to put payloadClass and preCombineField into the props?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Please check my comment in the PR where it's introduced (#5629 (comment)), i don't think we should have this method and i believe we should be able to get away without it in any form.
rfc/rfc-46/rfc-46.md
Outdated
| /** | ||
| * This method used to extract HoodieKey through keyGenerator. This method used in ClusteringExecutionStrategy. | ||
| */ | ||
| HoodieRecord transform(Properties props, Option<BaseKeyGenerator> keyGen); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Same comment as above for expansion: what this method is needed for?
rfc/rfc-46/rfc-46.md
Outdated
| */ | ||
| HoodieRecord transform(Properties props, Option<BaseKeyGenerator> keyGen); | ||
|
|
||
| Option<IndexedRecord> toIndexedRecord(Schema schema, Properties props) throws IOException; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What's the purpose of this method? Are we using it to fallback to Avro?
If that's the case then we should:
- Return
HoodieAvroRecordinstead ofIndexedRecord - Make sure we annotate it as the one which usages should be very restricted (mostly for the time we migrate and in a few exceptions)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
fixed
rfc/rfc-46/rfc-46.md
Outdated
| /** | ||
| * Is deleted. | ||
| */ | ||
| boolean isPresent(Schema recordSchema, Properties props) throws IOException; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Let's instead call it isDelete
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
fixed
| /** | ||
| * Is EmptyRecord. Generated by ExpressionPayload. | ||
| */ | ||
| boolean shouldIgnore(Schema recordSchema, Properties props) throws IOException; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Are you referring to EmptyHoodieRecordPayload? It's as well used tombstone record (ie delete).
We should be fine with just one deleting mechanism.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No. This method check whether is EmptyRecord SENTINEL.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We should probably update the java-doc then to avoid ref to any particular implementation
|
@alexeykudinkin Could you review these change if it’s convenient for you. Remove HoodieSource that marks the record from base or log. Rename expasion and transform func. |
af53430 to
ec48685
Compare
| /** | ||
| * Spark-specific implementation | ||
| */ | ||
| class HoodieSparkRecordMerger implements HoodieRecordMerger { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Please check my comment in here:
#5629 (comment)
We should allow user to implement single RecordMerger object supporting every engine type
| /** | ||
| * Get column in record to support RDDCustomColumnsSortPartitioner | ||
| */ | ||
| Object getRecordColumnValues(Schema recordSchema, String[] columns, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This should return an array of objects, right?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is consistent with HoodieAvroUtils#getRecordColumnValues . Column value if a single column, or concatenated String values by comma.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@wzx140 understand where you're coming from.
We should have already deprecated getRecordColumnValues as this method is heavily coupled to where it's used currently and unfortunately isn't generic enough to serve its purpose. In this particular case converting the values and concat-ing them as strings doesn't make sense for a generic utility -- whenever someone requests a list of column values they expect to get a list of values (as they are) as compared to receiving a string (!) of concatenated values.
| /** | ||
| * Support bootstrap. | ||
| */ | ||
| HoodieRecord mergeWith(HoodieRecord other, Schema targetSchema) throws IOException; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
As we've discussed prior we should avoid adding any merging semantic to the Record API itself. What this method is going to be used for?
rfc/rfc-46/rfc-46.md
Outdated
| HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, | ||
| Map<String, String> renameCols) throws IOException; | ||
|
|
||
| HoodieRecord updateValues(Schema recordSchema, Properties props, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'd suggest to
- Name it
updateMetadataValuesto avoid confusion (we shouldn't be allowing to modify the record's payload) - Instead of
Map<Sstring, String>let's create a strongly typed Java class w/ all meta-fields and pass it here
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is also used in HoodieHFileDataBlock#serializeRecord to update recordKey field
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@wzx140 we should split these up:
- Only legitimate use-case for us to update fields is Hudi's metadata
HoodieHFileDataBlockshouldn't be modifying existing payload but should instead be rewriting w/o the field it wants to omit. We will tackle that separately, and for the sake of RFC-46 we can create temporary methodtruncateRecordKeywhich will be overwriting record-key value for now (we will deprecate and remove this method after we address this)
We should not leave a loophole where we allow a record to be modified to make sure that nobody can start building against this API
| /** | ||
| * Is EmptyRecord. Generated by ExpressionPayload. | ||
| */ | ||
| boolean shouldIgnore(Schema recordSchema, Properties props) throws IOException; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We should probably update the java-doc then to avoid ref to any particular implementation
| */ | ||
| HoodieRecord getKeyWithKeyGen(Properties props, Option<BaseKeyGenerator> keyGen); | ||
|
|
||
| Option<HoodieAvroIndexedRecord> toIndexedRecord(Schema schema, Properties props) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why do we need HoodieAvroIndexedRecord?
I think HoodieAvroRecord should be enough
rfc/rfc-46/rfc-46.md
Outdated
| /** | ||
| * This method used to extract HoodieKey through parameters. | ||
| */ | ||
| HoodieRecord getKeyWithParams( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sorry, i think i previously was under wrong impression that this method returns HoodieKey, but now i see that it returns the same HoodieRecord and the only thing that is changed essentially is just the name of the method.
I left my comment in the actual PR elaborating why i think we should have no need for this method, PTAL.
|
@wzx140 seems like this PR now carries all the changes related to RFC-46. |
ec48685 to
bb208e9
Compare
bb208e9 to
d0300c2
Compare
|
@alexeykudinkin Sorry for not watching this pr for a long time. Most of the comments have been discussed in slack, is there anything else that needs to be changed? |
alexeykudinkin
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@wzx140 i think we're mostly good to go! Let's update the minor ones, and we should be able to land this one
rfc/rfc-46/rfc-46.md
Outdated
| HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, | ||
| Map<String, String> renameCols) throws IOException; | ||
|
|
||
| HoodieRecord updateValues(Schema recordSchema, Properties props, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@wzx140 we should split these up:
- Only legitimate use-case for us to update fields is Hudi's metadata
HoodieHFileDataBlockshouldn't be modifying existing payload but should instead be rewriting w/o the field it wants to omit. We will tackle that separately, and for the sake of RFC-46 we can create temporary methodtruncateRecordKeywhich will be overwriting record-key value for now (we will deprecate and remove this method after we address this)
We should not leave a loophole where we allow a record to be modified to make sure that nobody can start building against this API
|
@alexeykudinkin Thank you for your suggestion. This will be fixed soon |
|
@alexeykudinkin I have changed func updateMetadataValues(Schema recordSchema, Properties props, MetadataValues metadataValues). truncateRecordKey will be put in HoodieRecordCompatibilityInterface. |
…pache#6132) * Update the RFC-46 doc to fix comments feedback * fix Co-authored-by: wangzixuan.wzxuan <[email protected]>
…pache#6132) * Update the RFC-46 doc to fix comments feedback * fix Co-authored-by: wangzixuan.wzxuan <[email protected]>
…pache#6132) * Update the RFC-46 doc to fix comments feedback * fix Co-authored-by: wangzixuan.wzxuan <[email protected]>
…pache#6132) * Update the RFC-46 doc to fix comments feedback * fix Co-authored-by: wangzixuan.wzxuan <[email protected]>
…pache#6132) * Update the RFC-46 doc to fix comments feedback * fix Co-authored-by: wangzixuan.wzxuan <[email protected]>
[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]>
[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]>
[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]>
[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]>
[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]>
[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]>
[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]>
[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]>
[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]>
[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]>
[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]>
## 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]>
## 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]>
Tips
What is the purpose of the pull request
(For example: This pull request adds quick-start document.)
Brief change log
(for example:)
Verify this pull request
(Please pick either of the following options)
This pull request is a trivial rework / code cleanup without any test coverage.
(or)
This pull request is already covered by existing tests, such as (please describe tests).
(or)
This change added tests and can be verified as follows:
(example:)
Committer checklist
Has a corresponding JIRA in PR title & commit
Commit message is descriptive of the change
CI is green
Necessary doc changes done or have another open PR
For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.