prasannarajaperumal commented on code in PR #6256: URL: https://github.com/apache/hudi/pull/6256#discussion_r938603827
########## rfc/rfc-51/rfc-51.md: ########## @@ -148,20 +152,27 @@ hudi_cdc_table/ Under a partition directory, the `.log` file with `CDCBlock` above will keep the changing data we have to materialize. -There is an option to control what data is written to `CDCBlock`, that is `hoodie.table.cdc.supplemental.logging`. See the description of this config above. +#### Write-on-indexing vs Write-on-compaction Review Comment: This is an interesting trade-off. I understand Compaction is the one common place today where we know for sure the op type for all supported indexing. I am not sure about marrying 2 different concepts here - CDC stream freshness and Query efficiency (compaction frequency). This could be confusing for customers who dont know the implementation details. I was thinking along the lines of a new table service, ChangeTrackLoggingService, we can then - Asynchronous cdc option in addition - no cdc related ingestion failures and no performance hit. - Optional for other indexes, but mandatory for bucket indexing when cdc is on - Cant be frequently than compaction - lightweight when supplimental.logging is off. determines the CDC stream freshness. This also means a different timeline instant to manage. I will think more on this and wait for suggestions as well. In general I am not a big fan of supplimental.logging=true - I think duplicating data values is something we should avoid completely. I dont even know how to deal with data inconsistencies between cdc blocks and data blocks if we end up in that state. cc @xushiyan @danny0405 @YannByron @vinothchandar ########## rfc/rfc-51/rfc-51.md: ########## @@ -64,69 +65,72 @@ We follow the debezium output format: four columns as shown below Note: the illustration here ignores all the Hudi metadata columns like `_hoodie_commit_time` in `before` and `after` columns. -## Goals +## Design Goals 1. Support row-level CDC records generation and persistence; 2. Support both MOR and COW tables; 3. Support all the write operations; 4. Support Spark DataFrame/SQL/Streaming Query; -## Implementation +## Configurations -### CDC Architecture +| key | default | description | +|-----------------------------------------------------|----------|--------------------------------------------------------------------------------------------------------------------------------------------------| +| hoodie.table.cdc.enabled | `false` | The master switch of the CDC features. If `true`, writers and readers will respect CDC configurations and behave accordingly. | +| hoodie.table.cdc.supplemental.logging | `false` | If `true`, persist the required information about the changed data, including `before`. If `false`, only `op` and record keys will be persisted. | +| hoodie.table.cdc.supplemental.logging.include_after | `false` | If `true`, persist `after` as well. | -![](arch.jpg) +To perform CDC queries, users need to set `hoodie.table.cdc.enable=true` and `hoodie.datasource.query.type=incremental`. -Note: Table operations like `Compact`, `Clean`, `Index` do not write/change any data. So we don't need to consider them in CDC scenario. - -### Modifiying code paths +| key | default | description | +|----------------------------------------|------------|--------------------------------------| +| hoodie.table.cdc.enabled | `false` | set to `true` for CDC queries | +| hoodie.datasource.query.type | `snapshot` | set to `incremental` for CDC queries | Review Comment: +1 to @YannByron - yes enabling CDC does not automatically mean incremental queries will always produce CDC format. We need a incremental.output.format to distingush between (latest_state - default, cdc) ``` hoodie.datasource.query.type=incremental hoodie.datasource.incremental.output.format=cdc ``` ########## rfc/rfc-51/rfc-51.md: ########## @@ -64,69 +65,72 @@ We follow the debezium output format: four columns as shown below Note: the illustration here ignores all the Hudi metadata columns like `_hoodie_commit_time` in `before` and `after` columns. -## Goals +## Design Goals 1. Support row-level CDC records generation and persistence; 2. Support both MOR and COW tables; 3. Support all the write operations; 4. Support Spark DataFrame/SQL/Streaming Query; -## Implementation +## Configurations -### CDC Architecture +| key | default | description | +|-----------------------------------------------------|----------|--------------------------------------------------------------------------------------------------------------------------------------------------| +| hoodie.table.cdc.enabled | `false` | The master switch of the CDC features. If `true`, writers and readers will respect CDC configurations and behave accordingly. | +| hoodie.table.cdc.supplemental.logging | `false` | If `true`, persist the required information about the changed data, including `before`. If `false`, only `op` and record keys will be persisted. | +| hoodie.table.cdc.supplemental.logging.include_after | `false` | If `true`, persist `after` as well. | -![](arch.jpg) +To perform CDC queries, users need to set `hoodie.table.cdc.enable=true` and `hoodie.datasource.query.type=incremental`. -Note: Table operations like `Compact`, `Clean`, `Index` do not write/change any data. So we don't need to consider them in CDC scenario. - -### Modifiying code paths +| key | default | description | +|----------------------------------------|------------|--------------------------------------| +| hoodie.table.cdc.enabled | `false` | set to `true` for CDC queries | +| hoodie.datasource.query.type | `snapshot` | set to `incremental` for CDC queries | +| hoodie.datasource.read.start.timestamp | - | requried. | +| hoodie.datasource.read.end.timestamp | - | optional. | -![](points.jpg) +### Logical File Types -### Config Definitions +We define 4 logical file types for the CDC scenario. -Define a new config: +- CDC_LOG_File: a file consists of CDC Blocks with the changing data related to one commit. + - For COW tables, this file type refers to newly written log files alongside base files. The log files in this case only contain CDC info. + - For MOR tables, this file type refers to the typical log files in MOR tables. CDC info will be persisted as log blocks in the log files. +- ADD_BASE_File: a normal base file for a specified instant and a specified file group. All the data in this file are new-incoming. For example, we first write data to a new file group. So we can load this file, treat each record in this as the value of `after`, and the value of `op` of each record is `i`. +- REMOVE_BASE_FILE: a normal base file for a specified instant and a specified file group, but this file is empty. A file like this will be generated when we delete all the data in a file group. So we need to find the previous version of the file group, load it, treat each record in this as the value of `before`, and the value of `op` of each record is `d`. +- REPLACED_FILE_GROUP: a file group that be replaced totally, like `DELETE_PARTITION` and `INSERT_OVERWRITE` operations. We load this file group, treat all the records as the value of `before`, and the value of `op` of each record is `d`. -| key | default | description | -| --- | --- | --- | -| hoodie.table.cdc.enabled | false | `true` represents the table to be used for CDC queries and will write cdc data if needed. | -| hoodie.table.cdc.supplemental.logging | true | If true, persist all the required information about the change data, including 'before' and 'after'. Otherwise, just persist the 'op' and the record key. | +Note: -Other existing config that can be reused in cdc mode is as following: -Define another query mode named `cdc`, which is similar to `snapshpt`, `read_optimized` and `incremental`. -When read in cdc mode, set `hoodie.datasource.query.type` to `cdc`. +**`CDC_LOG_File` is a new file type and written out for CDC**. `ADD_BASE_File`, `REMOVE_BASE_FILE`, and `REPLACED_FILE_GROUP` represent the existing data files in the CDC scenario. -| key | default | description | -| --- |---| --- | -| hoodie.datasource.query.type | snapshot | set to cdc, enable the cdc quey mode | -| hoodie.datasource.read.start.timestamp | - | requried. | -| hoodie.datasource.read.end.timestamp | - | optional. | +For examples: +- `INSERT` operation will maybe create a list of new data files. These files will be treated as ADD_BASE_FILE; +- `DELETE_PARTITION` operation will replace a list of file slice. For each of these, we get the cdc data in the `REPLACED_FILE_GROUP` way. +## When `supplemental.logging=false` -### CDC File Types +In this mode, we minimized the additional storage for CDC information. -Here we define 5 cdc file types in CDC scenario. +- When write, the logging process is similar to the one described in section "When `supplemental.logging=true`", just that only change type `op` and record key are persisted. +- When read, changed info will be inferred on-the-fly, which costs more computation power. -- CDC_LOG_File: a file consists of CDC Blocks with the changing data related to one commit. - - when `hoodie.table.cdc.supplemental.logging` is true, it keeps all the fields about the change data, including `op`, `ts_ms`, `before` and `after`. When query hudi table in cdc query mode, load this file and return directly. - - when `hoodie.table.cdc.supplemental.logging` is false, it just keeps the `op` and the key of the changing record. When query hudi table in cdc query mode, we need to load the previous version and the current one of the touched file slice to extract the other info like `before` and `after` on the fly. -- ADD_BASE_File: a normal base file for a specified instant and a specified file group. All the data in this file are new-incoming. For example, we first write data to a new file group. So we can load this file, treat each record in this as the value of `after`, and the value of `op` of each record is `i`. -- REMOVE_BASE_FILE: a normal base file for a specified instant and a specified file group, but this file is empty. A file like this will be generated when we delete all the data in a file group. So we need to find the previous version of the file group, load it, treat each record in this as the value of `before`, and the value of `op` of each record is `d`. -- MOR_LOG_FILE: a normal log file. For this type, we need to load the previous version of file slice, and merge each record in the log file with this data loaded separately to determine how the record has changed, and get the value of `before` and `after`. -- REPLACED_FILE_GROUP: a file group that be replaced totally, like `DELETE_PARTITION` and `INSERT_OVERWRITE` operations. We load this file group, treat all the records as the value of `before`, and the value of `op` of each record is `d`. +Detailed inference algorithms are illustrated in this [design document](https://docs.google.com/document/d/1vb6EwTqGE0XBpZxWH6grUP2erCjQYb1dS4K24Dk3vL8/). Review Comment: Overall I think there is a difference in the design proposed in HUDI-3478 and in HUDI-1771 HUDI-1771 is a simple elegant design but as mentioned in [comment](https://github.com/apache/hudi/pull/5436#issuecomment-1206165913) - I am more inclined to the RFC-51 design that tracks the op and row_keys in the CDC log block and the reader doing a hash based join with this and the required file slices to compute the CDC stream. This will be a batched row_keys lookup and we should be efficient in doing that in all engines and the complexity is close to O(updates) and that is very reasonable rather than O(data accessed) like @YannByron mentions here. ########## rfc/rfc-51/rfc-51.md: ########## @@ -64,69 +65,72 @@ We follow the debezium output format: four columns as shown below Note: the illustration here ignores all the Hudi metadata columns like `_hoodie_commit_time` in `before` and `after` columns. -## Goals +## Design Goals 1. Support row-level CDC records generation and persistence; 2. Support both MOR and COW tables; 3. Support all the write operations; 4. Support Spark DataFrame/SQL/Streaming Query; Review Comment: Looks like the scope is going beyond Spark - lets mention that in the goals. ########## rfc/rfc-51/rfc-51.md: ########## @@ -64,69 +65,72 @@ We follow the debezium output format: four columns as shown below Note: the illustration here ignores all the Hudi metadata columns like `_hoodie_commit_time` in `before` and `after` columns. -## Goals +## Design Goals 1. Support row-level CDC records generation and persistence; 2. Support both MOR and COW tables; 3. Support all the write operations; 4. Support Spark DataFrame/SQL/Streaming Query; -## Implementation +## Configurations -### CDC Architecture +| key | default | description | +|-----------------------------------------------------|----------|--------------------------------------------------------------------------------------------------------------------------------------------------| +| hoodie.table.cdc.enabled | `false` | The master switch of the CDC features. If `true`, writers and readers will respect CDC configurations and behave accordingly. | +| hoodie.table.cdc.supplemental.logging | `false` | If `true`, persist the required information about the changed data, including `before`. If `false`, only `op` and record keys will be persisted. | +| hoodie.table.cdc.supplemental.logging.include_after | `false` | If `true`, persist `after` as well. | Review Comment: Combine Instead of hoodie.table.cdc.supplemental.logging, hoodie.table.cdc.supplemental.logging.include_after Make this a generic extensible config on what to log - hoodie.table.cdc.supplemental.logging.mode? possible values could be - min_cdc_metadata (op and key) - cdc_data_before - cdc_data_before_after (something to that effect ... ) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@hudi.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org