You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by "danny0405 (via GitHub)" <gi...@apache.org> on 2023/02/09 09:18:17 UTC

[GitHub] [hudi] danny0405 opened a new pull request, #7907: [RFC-61] Lockless multi writer support

danny0405 opened a new pull request, #7907:
URL: https://github.com/apache/hudi/pull/7907

   ### Change Logs
   
   RFC-61: lockless multi writer support
   
   ### Impact
   
   no
   
   ### Risk level (write none, low medium or high below)
   
   none
   
   ### Documentation Update
   
   _Describe any necessary documentation update if there is any new feature, config, or user-facing change_
   
   - _The config description must be updated if new configs are added or the default value of the configs are changed_
   - _Any new feature or user-facing change requires updating the Hudi website. Please create a Jira ticket, attach the
     ticket number here and follow the [instruction](https://hudi.apache.org/contribute/developer-setup#website) to make
     changes to the website._
   
   ### Contributor's checklist
   
   - [ ] Read through [contributor's guide](https://hudi.apache.org/contribute/how-to-contribute)
   - [ ] Change Logs and Impact were stated clearly
   - [ ] Adequate tests were added if applicable
   - [ ] CI passed
   


-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] vinothchandar commented on a diff in pull request #7907: [HUDI-5672][RFC-61] Lockless multi writer support

Posted by "vinothchandar (via GitHub)" <gi...@apache.org>.
vinothchandar commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1122538301


##########
rfc/rfc-61/rfc-61.md:
##########
@@ -0,0 +1,98 @@
+# RFC-61: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.

Review Comment:
   compaction or merge on read for queries.



##########
rfc/rfc-61/rfc-61.md:
##########
@@ -0,0 +1,98 @@
+# RFC-61: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Determistic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,

Review Comment:
   Can't the write_token be the write commit_time? we can make collisions highly improbably using some suffix/hash if needed. 



##########
rfc/rfc-61/rfc-61.md:
##########
@@ -0,0 +1,98 @@
+# RFC-61: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,

Review Comment:
   whatever we do, would be nice to generalize across indexes.



##########
rfc/rfc-61/rfc-61.md:
##########
@@ -0,0 +1,98 @@
+# RFC-61: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Determistic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+#### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).

Review Comment:
   +1 . I wonder if we need arrival time based combining as well i.e instant time.



##########
rfc/rfc-61/rfc-61.md:
##########
@@ -0,0 +1,98 @@
+# RFC-61: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Determistic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+#### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+### The Unique Ids for Different Writers
+
+In order to avoid file name conflicts for different writers, here we introduce a new write config option: `hoodie.write.client.id`,

Review Comment:
   can this be auto generated. Love to avoid another config for this. 



##########
rfc/rfc-61/rfc-61.md:
##########
@@ -0,0 +1,98 @@
+# RFC-61: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Determistic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+#### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+### The Unique Ids for Different Writers
+
+In order to avoid file name conflicts for different writers, here we introduce a new write config option: `hoodie.write.client.id`,
+the option value is then part of the task write token, so that different task from the different writers always diff in file name.
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${instant}.log.${version}_${task_token}-${client_id}

Review Comment:
   again, won't the deltacommit time work - as `task_token`



##########
rfc/rfc-61/rfc-61.md:
##########
@@ -0,0 +1,98 @@
+# RFC-61: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Determistic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.

Review Comment:
   the insert problem can be handled with a more expensive conflict resolution strategy, that incrementally checks for PK violations. let me think about this more and sketch a solution if I can.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1325822626


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,215 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing

Review Comment:
   Yeah, I will add the example to the doc.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking multi writer support

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1309670651


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,119 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,

Review Comment:
   Sure, It is called out in the log file naming part.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [HUDI-6495][RFC-66] Non-blocking Concurrency Control [hudi]

Posted by "beyond1920 (via GitHub)" <gi...@apache.org>.
beyond1920 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1360569304


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.

Review Comment:
   Is Non-blocking concurrent control only work for insert and update, and not for insert overwrite? 
   The following points are different for `insert overwrite` with `insert` or `upsert`:
   1. Generating a fixed file group based on the bucket number is not applicable for `insert overwrite`. Using the same file group before and after insert overwrite will leads to incorrect results.
   
   2. If there are multiple writers, the one that fires first may finish later. Is it the one that fired first overwrites the data generated by the one that fired later, or is it the one that finished first overwrites the data generated by the one that finished later?
   
   3. Assuming we have two commit: t1 -> commit1, t2 -> commit2. When generating `partitionToReplaceIds` metadata for insert ovewriter, a lock might be needed here.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] LinMingQiang commented on a diff in pull request #7907: [HUDI-5672][RFC-61] Lockless multi writer support

Posted by "LinMingQiang (via GitHub)" <gi...@apache.org>.
LinMingQiang commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1102294460


##########
rfc/rfc-61/rfc-61.md:
##########
@@ -0,0 +1,98 @@
+# RFC-61: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Determistic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.

Review Comment:
   > Can you explain why the same file is written? In my understanding, different writers will write to different log files, and they will be merged into the same `File Group` during the Compaction phase. Because it's `INSERT` operation, it doesn't need to resolve the same primary key causing conflicts.
   > 
   > Maybe my understanding is wrong, I'd be grateful if you could help me point out.
   
   Sorry, I made a mistake. The new design dose not write the same file. The reason why only bucket index layout can be supported is because, When writing new keys, different writers may assign different filegroups. 
   
   
   



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1329428267


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,246 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+```python
+# new file slicing
+file_slice_barriers = []
+# sort the base file by start time reversely and get the barriers
+for base_file in fg.getBaseFilesSortedReverselyByStartTime():
+  file_slice_barriers.add(instant_time(base_file))
+
+file_slices = []
+for log_file in fg.getLogFilesSortedByCompletionTime():
+  completion_time = completion_time(log_file)
+  file_slice = find_file_slice(completion_time, file_slice_barriers)
+  file_slices.add(file_slice)
+
+def find_file_slice(completion_time, file_slice_barriers):
+  # Note: needs to handle special case where file_slice_barriers is empty (no base file)

Review Comment:
   You are right, thanks for the clarification ~



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] bvaradar commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "bvaradar (via GitHub)" <gi...@apache.org>.
bvaradar commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1339451946


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}

Review Comment:
   Is task_token same as write_token mentioned below ?



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [HUDI-6495][RFC-66] Non-blocking Concurrency Control [hudi]

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1349831280


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}

Review Comment:
   Yeah, that's why hasing index is a prerequisite for it.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] codope commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "codope (via GitHub)" <gi...@apache.org>.
codope commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1325792572


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,215 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing

Review Comment:
   Just to make it concrete for the readers, we can add an example.
   Let's take this example of one filegroup and consider the following sequence of commits:
   
   1. `fg_t10` -> very first commit with start time, end time as `[t10, t20]`.
   2. `l1` -> log file version 1 start time, end time as `[t21, t40]`.
   3. `l2` -> concurrent log file version 2 `[t30, t50]`.
   4. `fg_t60` -> base file due to compaction `[t60, t80]`.
   5. `l3` -> concurrent log file version 3 `[t35, t90]`.
   
   In this case, `file_slice_barriers` list is [t60, t10]. For a query at `t100`, `getAllFileSlices` should return the following list:
   [
       {t60, fg_t60.parquet, {l3}},
       {t10, fg_t10.parquet, {l1, l2}}
   ]
   This assumes that file slicing is done based on completion time.



##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,215 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing

Review Comment:
   Just to make it concrete for the readers, we can add an example.
   Let's take this example of one filegroup and consider the following sequence of commits:
   
   1. `fg_t10` -> very first commit with start time, end time as `[t10, t20]`.
   2. `l1` -> log file version 1 start time, end time as `[t21, t40]`.
   3. `l2` -> concurrent log file version 2 `[t30, t50]`.
   4. `fg_t60` -> base file due to compaction `[t60, t80]`.
   5. `l3` -> concurrent log file version 3 `[t35, t90]`.
   
   In this case, `file_slice_barriers` list is [t60, t10]. For a query at `t100`, `getAllFileSlices` should return the following list:
   [
       {t60, fg_t60.parquet, {l3}},
       {t10, fg_t10.parquet, {l1, l2}}
   ]
   This assumes that file slicing is done based on completion time.



##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,215 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+```python
+# new file slicing
+file_slice_barriers = []
+# sort the base file by start time reversely and get the barriers
+for base_file in fg.getBaseFilesSortedReverselyByStartTime():
+  file_slice_barriers.add(instant_time(base_file))
+
+file_slices = []
+for log_file in fg.getLogFilesSortedByCompletionTime():
+  completion_time = completion_time(log_file)
+  file_slice = find_file_slice(completion_time, file_slice_barriers)
+  file_slices.add(file_slice)
+
+def find_file_slice(completion_time, file_slice_barriers):
+  # Note: needs to handle special case where file_slice_barriers is empty (no base file)
+  for barrier in file_slice_barriers:
+    if (barrier < completion_time):
+      # each file slice is attached to a barrier, returns the existing file slice or a fresh new one based on the barrier.
+      return get_or_create_file_slice(barrier)
+```
+
+### Pseudocode for Time Travel Queries
+
+```python
+# This is based on the new file slicing.
+# find the write 'w' with completionTime(w) < t, among all the writes that completed before t 
+# (Note t may not match an existing completion time directly) 
+max_completion_time = findMaxCompletedTimeBefore(t) 
+
+fileSlicesToRead = [] 
+for fg in fileGroups:
+  fileSlice = latestFileSliceThatIsOverlappingWithCompletionTime(fg, max_completion_time)
+  fileSlicesToRead.append(fileSlice)
+return fileSlicesToRead
+
+def latestFileSliceThatIsOverlappingWithCompletionTime(fg, max_completion_time):

Review Comment:
   Apart from an example, let's also write why this is correct. We can simply reason based on contradiction or show that the algo doesn't miss any valid file slice (or add any uncommitted slice) walking down the temporal graph of flice slice based on completion time.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] bvaradar commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "bvaradar (via GitHub)" <gi...@apache.org>.
bvaradar commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1339453088


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token

Review Comment:
   During read time, Can we use the completion time of the delta-commit to order as this is what we are using for file slice ? 
   
   So completion_time and version_number will give total ordering of the log files.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1329531650


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,246 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+```python
+# new file slicing

Review Comment:
   Thanks Sagar~



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [HUDI-6495][RFC-66] Non-blocking Concurrency Control [hudi]

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on PR #7907:
URL: https://github.com/apache/hudi/pull/7907#issuecomment-1786320842

   > > > for this feature, how do we handle the failure writing commits,will it be rollback by other writing tasks?
   > > 
   > > 
   > > There is no failure thrown actively because the conflicts would be reslved automically.
   > 
   > not the conflict failure but some other failure such as input data not correct. now we use partition lock for hudi but in some case there`s some failure commits and it won`t been rollback or archived.
   
   The failures are either taken care of by the writer if the failed writes rollback strategy is configured as `EAGER` or by the cleaner if there are multiple writers with the rollback strategy configured as `LAZY`.


-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking multi writer support

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1309669080


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,119 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${instant}.log.${version}_${task_token}

Review Comment:
   > delta committime
   
   you mean the instant time (start time) ?, we will do that.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] BruceKellan commented on a diff in pull request #7907: [HUDI-5672][RFC-61] Lockless multi writer support

Posted by "BruceKellan (via GitHub)" <gi...@apache.org>.
BruceKellan commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1101268040


##########
rfc/rfc-61/rfc-61.md:
##########
@@ -0,0 +1,98 @@
+# RFC-61: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Determistic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.

Review Comment:
   If we are using a MOR table but not a bucket index layout, can we support lockless multi writer for INSERT? Will there be any problem?



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-5672][RFC-61] Lockless multi writer support

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1126125345


##########
rfc/rfc-61/rfc-61.md:
##########
@@ -0,0 +1,98 @@
+# RFC-61: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Determistic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+#### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+### The Unique Ids for Different Writers
+
+In order to avoid file name conflicts for different writers, here we introduce a new write config option: `hoodie.write.client.id`,

Review Comment:
   Yeah, instant time actually works well here.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] Zouxxyy commented on pull request #7907: [HUDI-5672][RFC-66] Lockless multi writer support

Posted by "Zouxxyy (via GitHub)" <gi...@apache.org>.
Zouxxyy commented on PR #7907:
URL: https://github.com/apache/hudi/pull/7907#issuecomment-1596516873

   @danny0405 In the current single-writer mode, the compaction generation and the writer are actually serialized. If we want to support multiple writers, then we have to resolve the write conflicts between compaction generation and writers. Has this RFC considered this issue?


-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [HUDI-6495][RFC-66] Non-blocking Concurrency Control [hudi]

Posted by "vinothchandar (via GitHub)" <gi...@apache.org>.
vinothchandar merged PR #7907:
URL: https://github.com/apache/hudi/pull/7907


-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [HUDI-6495][RFC-66] Non-blocking Concurrency Control [hudi]

Posted by "vinothchandar (via GitHub)" <gi...@apache.org>.
vinothchandar commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1361412028


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+Assume we have the following inputs to build the file slices:
+1. `base_files_by_file_id`: a map of filegroup id to list of base files.
+2. `log_files_by_file_id`: a map of feilgroup id to list of log files.
+3. `timeline`: Hudi write timeline .
+
+The pseudocode below introduces a notion of **file slice barriers**, which contain a list of instant (start) times of 
+the base files in descending order. Barriers will help in demarcating file slices. The below pseudocode builds the file 
+slices per file group. Building file groups from file slices is not shown but can be easily done.
+
+```python
+# new file slicing
+def build_file_slices(base_files_by_file_id, log_files_by_file_id, timeline):
+    # get set of all filegroup ids
+    file_id_set = base_files_by_file_id.keys
+    file_id_set.add_all(log_files_by_file_id.keys)
+    
+    for file_id in file_id_set:
+        # sort the base files by descending order of instant (start) time, i.e. last written base file first
+        base_files_in_file_id = base_files_by_file_id[file_id].sort(BASE_FILE_REVERSE_COMPARATOR)
+        # filter out log files that have been log-compacted
+        log_files_in_file_id = handle_log_compaction(log_files_by_file_id[file_id], timeline)
+        # sort the log files by ascending order of completion time
+        log_files_in_file_id.sort(LOG_FILE_COMPARATOR)
+        # get list of file slice barriers for this fielgroup id
+        file_slice_barriers = get_file_slice_barriers(base_files_in_file_id, log_files_in_file_id)    
+        # build file slices
+        file_slices = []
+        for log_file in log_files_in_file_id:
+            file_slice = find_file_slice(log_file, file_slice_barriers)
+            file_slices.add(file_slice)
+            
+            
+# Given all log files for a file id, filter out such log files that have been log-compacted.
+def handle_log_compaction(log_files_in_file_id, timeline):
+    log_compaction_instants = timeline.get_completed_log_compaction()
+    for log_compaction_instant in log_compaction_instant:
+        log_files_compacted = get_log_files_compacted(log_compaction_instant)
+        log_files_in_file_id.remove(log_files_compacted)
+        
+    return log_files_in_file_id
+
+
+# Given base files and log files for a filegroup id, return a list containing file slice barriers. 
+def get_file_slice_barriers(base_files_in_file_id, log_files_in_file_id):
+    file_slice_barriers = []
+    if base_files_in_file_id.size > 0:
+        for base_file in base_files_in_file_id:
+            file_slice_barriers.add(instant_time(base_file))

Review Comment:
   this is so that we can associate any log file with completion time < instant_time of `base_file` with the prev file slice.



##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+Assume we have the following inputs to build the file slices:
+1. `base_files_by_file_id`: a map of filegroup id to list of base files.
+2. `log_files_by_file_id`: a map of feilgroup id to list of log files.
+3. `timeline`: Hudi write timeline .
+
+The pseudocode below introduces a notion of **file slice barriers**, which contain a list of instant (start) times of 
+the base files in descending order. Barriers will help in demarcating file slices. The below pseudocode builds the file 
+slices per file group. Building file groups from file slices is not shown but can be easily done.
+
+```python
+# new file slicing
+def build_file_slices(base_files_by_file_id, log_files_by_file_id, timeline):
+    # get set of all filegroup ids
+    file_id_set = base_files_by_file_id.keys
+    file_id_set.add_all(log_files_by_file_id.keys)
+    
+    for file_id in file_id_set:
+        # sort the base files by descending order of instant (start) time, i.e. last written base file first
+        base_files_in_file_id = base_files_by_file_id[file_id].sort(BASE_FILE_REVERSE_COMPARATOR)
+        # filter out log files that have been log-compacted
+        log_files_in_file_id = handle_log_compaction(log_files_by_file_id[file_id], timeline)
+        # sort the log files by ascending order of completion time
+        log_files_in_file_id.sort(LOG_FILE_COMPARATOR)
+        # get list of file slice barriers for this fielgroup id
+        file_slice_barriers = get_file_slice_barriers(base_files_in_file_id, log_files_in_file_id)    
+        # build file slices
+        file_slices = []
+        for log_file in log_files_in_file_id:
+            file_slice = find_file_slice(log_file, file_slice_barriers)
+            file_slices.add(file_slice)
+            
+            
+# Given all log files for a file id, filter out such log files that have been log-compacted.
+def handle_log_compaction(log_files_in_file_id, timeline):
+    log_compaction_instants = timeline.get_completed_log_compaction()
+    for log_compaction_instant in log_compaction_instant:
+        log_files_compacted = get_log_files_compacted(log_compaction_instant)
+        log_files_in_file_id.remove(log_files_compacted)
+        
+    return log_files_in_file_id
+
+
+# Given base files and log files for a filegroup id, return a list containing file slice barriers. 
+def get_file_slice_barriers(base_files_in_file_id, log_files_in_file_id):
+    file_slice_barriers = []
+    if base_files_in_file_id.size > 0:
+        for base_file in base_files_in_file_id:
+            file_slice_barriers.add(instant_time(base_file))
+    elif log_files_in_file_id.size > 0:
+        # for a file group with no base file, the instant time of the earliest log file is the barrier
+        file_slice_barriers.add(instant_time(log_files_in_file_id[0]))
+    
+    return file_slice_barriers
+
+
+def find_file_slice(log_file, file_slice_barriers):
+  completion_time = completion_time(log_file)
+  for barrier in file_slice_barriers:
+    if (barrier < completion_time):
+      # each file slice is attached to a barrier, returns the existing file slice or a fresh new one based on the barrier.
+      # note that since file_slice_barriers is reverse sorted, we would return the file slice 
+      # corresponding to the max barrier just less than the completion_time

Review Comment:
   i.e max barrier that is less than log's completion time.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [HUDI-6495][RFC-66] Non-blocking Concurrency Control [hudi]

Posted by "beyond1920 (via GitHub)" <gi...@apache.org>.
beyond1920 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1363440409


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.

Review Comment:
   Besides, could Non-blocking concurrent control work for 'bulk insert'?
   Bulk insert writes data into base files, if there are multiple bulk insert job, there might exists multiple base files in the same bucket. 
   Maybe one `bulk insert` job and multiple `insert`/`upsert` could run concurrently.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1339602364


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}

Review Comment:
   No, the algorithm is compatible, old naming is a special case for the new one, because while we read the old table, the log file completion time is actually the same with it's base file start time, and that makes the file slicing same as before.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] bvaradar commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "bvaradar (via GitHub)" <gi...@apache.org>.
bvaradar commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1339458846


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+Assume we have the following inputs to build the file slices:
+1. `base_files_by_file_id`: a map of filegroup id to list of base files.
+2. `log_files_by_file_id`: a map of feilgroup id to list of log files.
+3. `timeline`: Hudi write timeline .
+
+The pseudocode below introduces a notion of **file slice barriers**, which contain a list of instant (start) times of 
+the base files in descending order. Barriers will help in demarcating file slices. The below pseudocode builds the file 
+slices per file group. Building file groups from file slices is not shown but can be easily done.
+
+```python
+# new file slicing
+def build_file_slices(base_files_by_file_id, log_files_by_file_id, timeline):
+    # get set of all filegroup ids
+    file_id_set = base_files_by_file_id.keys
+    file_id_set.add_all(log_files_by_file_id.keys)
+    
+    for file_id in file_id_set:
+        # sort the base files by descending order of instant (start) time, i.e. last written base file first
+        base_files_in_file_id = base_files_by_file_id[file_id].sort(BASE_FILE_REVERSE_COMPARATOR)
+        # filter out log files that have been log-compacted
+        log_files_in_file_id = handle_log_compaction(log_files_by_file_id[file_id], timeline)
+        # sort the log files by ascending order of completion time
+        log_files_in_file_id.sort(LOG_FILE_COMPARATOR)
+        # get list of file slice barriers for this fielgroup id
+        file_slice_barriers = get_file_slice_barriers(base_files_in_file_id, log_files_in_file_id)    
+        # build file slices
+        file_slices = []
+        for log_file in log_files_in_file_id:
+            file_slice = find_file_slice(log_file, file_slice_barriers)
+            file_slices.add(file_slice)
+            
+            
+# Given all log files for a file id, filter out such log files that have been log-compacted.
+def handle_log_compaction(log_files_in_file_id, timeline):

Review Comment:
   I would like to explicitly call out that we still need to read compaction plans when determining file-slice as not all file slices will need to be scheduled for compaction. 



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] Zouxxyy commented on pull request #7907: [HUDI-5672][RFC-66] Lockless multi writer support

Posted by "Zouxxyy (via GitHub)" <gi...@apache.org>.
Zouxxyy commented on PR #7907:
URL: https://github.com/apache/hudi/pull/7907#issuecomment-1598064670

   > We can add a expliciet lock while scheduling these plans.
   
   @danny0405 The current asynchronous compaction schedule has the risk of losing data, maybe we can solve this first


-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] Zouxxyy commented on a diff in pull request #7907: [HUDI-5672][RFC-66] Lockless multi writer support

Posted by "Zouxxyy (via GitHub)" <gi...@apache.org>.
Zouxxyy commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1233533548


##########
rfc/rfc-61/rfc-61.md:
##########
@@ -0,0 +1,98 @@
+# RFC-61: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Determistic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+#### The Compaction Procedure

Review Comment:
   @danny0405  In the current single-writer mode, the compaction generation and the writer are actually serialized. If we want to support multiple writers, then we have to resolve the write conflicts between compaction generation and writers. Has this RFC considered this issue?



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] fengjian428 commented on a diff in pull request #7907: [HUDI-5672][RFC-61] Lockless multi writer support

Posted by "fengjian428 (via GitHub)" <gi...@apache.org>.
fengjian428 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1155047834


##########
rfc/rfc-61/rfc-61.md:
##########
@@ -0,0 +1,98 @@
+# RFC-61: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Determistic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+#### The Compaction Procedure

Review Comment:
   as I know, for now Hudi does not support generate compaction plan during data writing, need to solve this problem



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking multi writer support

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1322329718


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,165 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, because we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${instant}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by base_commit_time, version_number, write_token

Review Comment:
   Then there must be some retries/attempts. The commit finalizing and rollback should handle that, the fs view should shadow these invalid files.



##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,165 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, because we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${instant}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by base_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+```python
+# new file slicing
+file_slice_barriers = []
+# sort the base file by start time reversely and get the barriers

Review Comment:
   exactly.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] vinothchandar commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking multi writer support

Posted by "vinothchandar (via GitHub)" <gi...@apache.org>.
vinothchandar commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1322274752


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,165 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, because we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${instant}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by base_commit_time, version_number, write_token

Review Comment:
   can it be such that for the same base_commit_time, same version_number, we get a write token x written after write token y, such that x < y. Would that cause any issues?



##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,119 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${instant}.log.${version}_${task_token}

Review Comment:
   can we please update the log file name
   



##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,165 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, because we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${instant}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by base_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+```python
+# new file slicing
+file_slice_barriers = []
+# sort the base file by start time reversely and get the barriers

Review Comment:
   whats start time? same as instant time?



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] codope commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "codope (via GitHub)" <gi...@apache.org>.
codope commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1328749606


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,246 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+```python
+# new file slicing

Review Comment:
   This whole logic needs to loop per file group id. Let's add that in the pseudocode.



##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,246 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+```python
+# new file slicing
+file_slice_barriers = []
+# sort the base file by start time reversely and get the barriers
+for base_file in fg.getBaseFilesSortedReverselyByStartTime():
+  file_slice_barriers.add(instant_time(base_file))
+
+file_slices = []
+for log_file in fg.getLogFilesSortedByCompletionTime():
+  completion_time = completion_time(log_file)
+  file_slice = find_file_slice(completion_time, file_slice_barriers)
+  file_slices.add(file_slice)
+
+def find_file_slice(completion_time, file_slice_barriers):
+  # Note: needs to handle special case where file_slice_barriers is empty (no base file)
+  for barrier in file_slice_barriers:
+    if (barrier < completion_time):
+      # each file slice is attached to a barrier, returns the existing file slice or a fresh new one based on the barrier.
+      return get_or_create_file_slice(barrier)
+
+all_file_slices = {}
+def get_or_create_file_slice(barrier):
+  if barrier in all_file_slices:
+    return all_file_slices[barrier]
+  else:
+    file_slice = FileSlice(barrier)

Review Comment:
   Can you please add how this constructor will work with the `barrier`? I am assuming it gets all the log files (for a particular file id) whose completion time is less than the `barrier`.



##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,246 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+```python
+# new file slicing
+file_slice_barriers = []
+# sort the base file by start time reversely and get the barriers
+for base_file in fg.getBaseFilesSortedReverselyByStartTime():
+  file_slice_barriers.add(instant_time(base_file))
+
+file_slices = []
+for log_file in fg.getLogFilesSortedByCompletionTime():
+  completion_time = completion_time(log_file)
+  file_slice = find_file_slice(completion_time, file_slice_barriers)
+  file_slices.add(file_slice)
+
+def find_file_slice(completion_time, file_slice_barriers):

Review Comment:
   doesn't it need to take `log_file` as well? How can file slice be constructed without log files? The expectation is that it returns a file slice that contains a set of log files and fielgroup id.



##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,246 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+```python
+# new file slicing
+file_slice_barriers = []
+# sort the base file by start time reversely and get the barriers
+for base_file in fg.getBaseFilesSortedReverselyByStartTime():
+  file_slice_barriers.add(instant_time(base_file))
+
+file_slices = []
+for log_file in fg.getLogFilesSortedByCompletionTime():
+  completion_time = completion_time(log_file)
+  file_slice = find_file_slice(completion_time, file_slice_barriers)
+  file_slices.add(file_slice)
+
+def find_file_slice(completion_time, file_slice_barriers):
+  # Note: needs to handle special case where file_slice_barriers is empty (no base file)

Review Comment:
   Let's complete this part. Barrier and filegroup id need to have one-one mapping. If there are no base files, the instant time of the earliest log file can be the barrier, and so all log files get added to the same slice. 



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] bvaradar commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "bvaradar (via GitHub)" <gi...@apache.org>.
bvaradar commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1339456272


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}

Review Comment:
   Do we need any special flag in the log file naming to distinguish between pre 1.x hudi log file and 1.0 log files ?



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-5672][RFC-61] Lockless multi writer support

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1105598730


##########
rfc/rfc-61/rfc-61.md:
##########
@@ -0,0 +1,98 @@
+# RFC-61: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Determistic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.

Review Comment:
   Yeah, you guys got it, just to ensure that all the changes for one record key happens in one FileGroup.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on pull request #7907: [HUDI-5672][RFC-66] Lockless multi writer support

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on PR #7907:
URL: https://github.com/apache/hudi/pull/7907#issuecomment-1598060094

   > then we have to resolve the write conflicts between compaction generation and writers.
   
   We can add a expliciet lock while scheduling these plans.


-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] vinothchandar commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking multi writer support

Posted by "vinothchandar (via GitHub)" <gi...@apache.org>.
vinothchandar commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1308905831


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,119 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.

Review Comment:
   Can we update rfc with exact assumptions on locks



##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,119 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,

Review Comment:
   We can add instant time to log file names? Can you explicitly call out all format changes needed here?



##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,119 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.

Review Comment:
   Essentially the compaction start time is assumed to encompass all logs with completion time before that?



##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,119 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${instant}.log.${version}_${task_token}

Review Comment:
   Otherwise hou do we map from completiontime toinstant time '



##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,119 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,

Review Comment:
   Can the record level index change anything here?



##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,119 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${instant}.log.${version}_${task_token}

Review Comment:
   Can the log file name contain the delta committime as well?



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking multi writer support

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1309670169


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,119 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.

Review Comment:
   exactly



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking multi writer support

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1309671455


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,119 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,

Review Comment:
   Not much, the RLI need to maintain the index in very heavy style and it needs OCC to keep index in consistency, both of these restrictions are not very suitable for streaming write.



##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,119 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.

Review Comment:
   Sure.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on pull request #7907: [HUDI-6495][RFC-66] Non-blocking multi writer support

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on PR #7907:
URL: https://github.com/apache/hudi/pull/7907#issuecomment-1698506743

   > I feel we need to add lot more details here; including how new file slice generation works for all different types of queries. Make assumptions and format changes clear
   
   Yeah, we can supplement the demo code once we make that clear enough internally.


-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1328193248


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,239 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+```python
+# new file slicing
+file_slice_barriers = []
+# sort the base file by start time reversely and get the barriers
+for base_file in fg.getBaseFilesSortedReverselyByStartTime():
+  file_slice_barriers.add(instant_time(base_file))
+
+file_slices = []
+for log_file in fg.getLogFilesSortedByCompletionTime():
+  completion_time = completion_time(log_file)
+  file_slice = find_file_slice(completion_time, file_slice_barriers)
+  file_slices.add(file_slice)
+
+def find_file_slice(completion_time, file_slice_barriers):
+  # Note: needs to handle special case where file_slice_barriers is empty (no base file)
+  for barrier in file_slice_barriers:
+    if (barrier < completion_time):
+      # each file slice is attached to a barrier, returns the existing file slice or a fresh new one based on the barrier.
+      return get_or_create_file_slice(barrier)
+```
+
+#### A Demo Conducive to Comprehension
+
+Just to make it concrete for the readers, we can add an example.
+Let's take this example of one filegroup and consider the following sequence of commits:
+
+```xml
+fg_t10 -> very first commit with start time, end time as [t10, t20].
+l1 -> log file version 1 start time, end time as [t21, t40].
+l2 -> concurrent log file version 2 [t30, t50].
+fg_t60 -> base file due to compaction [t60, t80].
+l3 -> concurrent log file version 3 [t35, t90].
+```
+
+In this case, file_slice_barriers list is [t60, t10]. For a query at t100, getAllFileSlices should return the following list:
+
+```xml
+[
+{t60, fg_t60.parquet, {l3}},
+{t10, fg_t10.parquet, {l1, l2}}
+]
+```
+
+This assumes that file slicing is done based on completion time.

Review Comment:
   Exactly.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [HUDI-6495][RFC-66] Non-blocking Concurrency Control [hudi]

Posted by "vinothchandar (via GitHub)" <gi...@apache.org>.
vinothchandar commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1361415450


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}

Review Comment:
   we can solve this orthogonally, at the cost of a background clustering. but insert-insert conflict resolution needs some hash bucketing (since row level locks are a terrible choice for data lake workloads) 



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [HUDI-6495][RFC-66] Non-blocking Concurrency Control [hudi]

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1364782280


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.

Review Comment:
   Yeah, I don't think bulk insert needs a NB-CC, just throw exception under NB-CC should be okay.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] yihua commented on pull request #7907: [HUDI-5672][RFC-66] Lockless multi writer support

Posted by "yihua (via GitHub)" <gi...@apache.org>.
yihua commented on PR #7907:
URL: https://github.com/apache/hudi/pull/7907#issuecomment-1555107431

   @danny0405 I updated the RFC number.


-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [HUDI-6495][RFC-66] Non-blocking Concurrency Control [hudi]

Posted by "vinothchandar (via GitHub)" <gi...@apache.org>.
vinothchandar commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1361414800


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.

Review Comment:
   @beyond1920 I think for two concurrent `insert overwrite` operations, we would stick with OCC mode. Good call, to add as limitation.
   
   > s it the one that finished later overwrites the data generated by the one that finished first?
   This . or it can resolve using an event time (ordering field) as today. 
   
   



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [HUDI-6495][RFC-66] Non-blocking Concurrency Control [hudi]

Posted by "xicm (via GitHub)" <gi...@apache.org>.
xicm commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1349815428


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}

Review Comment:
   Just a reminder, we should consider that the records with same key should be written to the same file group when multi writers do the first writing.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1329428053


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,246 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+```python
+# new file slicing
+file_slice_barriers = []
+# sort the base file by start time reversely and get the barriers
+for base_file in fg.getBaseFilesSortedReverselyByStartTime():
+  file_slice_barriers.add(instant_time(base_file))
+
+file_slices = []
+for log_file in fg.getLogFilesSortedByCompletionTime():
+  completion_time = completion_time(log_file)
+  file_slice = find_file_slice(completion_time, file_slice_barriers)
+  file_slices.add(file_slice)
+
+def find_file_slice(completion_time, file_slice_barriers):
+  # Note: needs to handle special case where file_slice_barriers is empty (no base file)
+  for barrier in file_slice_barriers:
+    if (barrier < completion_time):
+      # each file slice is attached to a barrier, returns the existing file slice or a fresh new one based on the barrier.
+      return get_or_create_file_slice(barrier)
+
+all_file_slices = {}
+def get_or_create_file_slice(barrier):
+  if barrier in all_file_slices:
+    return all_file_slices[barrier]
+  else:
+    file_slice = FileSlice(barrier)

Review Comment:
   Just pass in the file group id and instant time(barrier) like what we do today.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1340868197


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+Assume we have the following inputs to build the file slices:
+1. `base_files_by_file_id`: a map of filegroup id to list of base files.
+2. `log_files_by_file_id`: a map of feilgroup id to list of log files.
+3. `timeline`: Hudi write timeline .
+
+The pseudocode below introduces a notion of **file slice barriers**, which contain a list of instant (start) times of 
+the base files in descending order. Barriers will help in demarcating file slices. The below pseudocode builds the file 
+slices per file group. Building file groups from file slices is not shown but can be easily done.
+
+```python
+# new file slicing
+def build_file_slices(base_files_by_file_id, log_files_by_file_id, timeline):
+    # get set of all filegroup ids
+    file_id_set = base_files_by_file_id.keys
+    file_id_set.add_all(log_files_by_file_id.keys)
+    
+    for file_id in file_id_set:
+        # sort the base files by descending order of instant (start) time, i.e. last written base file first
+        base_files_in_file_id = base_files_by_file_id[file_id].sort(BASE_FILE_REVERSE_COMPARATOR)
+        # filter out log files that have been log-compacted
+        log_files_in_file_id = handle_log_compaction(log_files_by_file_id[file_id], timeline)
+        # sort the log files by ascending order of completion time
+        log_files_in_file_id.sort(LOG_FILE_COMPARATOR)
+        # get list of file slice barriers for this fielgroup id
+        file_slice_barriers = get_file_slice_barriers(base_files_in_file_id, log_files_in_file_id)    
+        # build file slices
+        file_slices = []
+        for log_file in log_files_in_file_id:
+            file_slice = find_file_slice(log_file, file_slice_barriers)
+            file_slices.add(file_slice)
+            
+            
+# Given all log files for a file id, filter out such log files that have been log-compacted.
+def handle_log_compaction(log_files_in_file_id, timeline):
+    log_compaction_instants = timeline.get_completed_log_compaction()
+    for log_compaction_instant in log_compaction_instant:

Review Comment:
   We put the logic here but actually the exising mechanism still works, the log compaction file replacing behaves pretty much like the replace commit, here during the file slicing, there is no need to care about whether this log file is replaced or not, just put all the log files in the correct file slice and then the existing replacement would take over the other logic.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] vinothchandar commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "vinothchandar (via GitHub)" <gi...@apache.org>.
vinothchandar commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1340820011


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+Assume we have the following inputs to build the file slices:
+1. `base_files_by_file_id`: a map of filegroup id to list of base files.
+2. `log_files_by_file_id`: a map of feilgroup id to list of log files.
+3. `timeline`: Hudi write timeline .
+
+The pseudocode below introduces a notion of **file slice barriers**, which contain a list of instant (start) times of 
+the base files in descending order. Barriers will help in demarcating file slices. The below pseudocode builds the file 
+slices per file group. Building file groups from file slices is not shown but can be easily done.
+
+```python
+# new file slicing
+def build_file_slices(base_files_by_file_id, log_files_by_file_id, timeline):
+    # get set of all filegroup ids
+    file_id_set = base_files_by_file_id.keys
+    file_id_set.add_all(log_files_by_file_id.keys)
+    
+    for file_id in file_id_set:
+        # sort the base files by descending order of instant (start) time, i.e. last written base file first
+        base_files_in_file_id = base_files_by_file_id[file_id].sort(BASE_FILE_REVERSE_COMPARATOR)
+        # filter out log files that have been log-compacted
+        log_files_in_file_id = handle_log_compaction(log_files_by_file_id[file_id], timeline)
+        # sort the log files by ascending order of completion time
+        log_files_in_file_id.sort(LOG_FILE_COMPARATOR)
+        # get list of file slice barriers for this fielgroup id
+        file_slice_barriers = get_file_slice_barriers(base_files_in_file_id, log_files_in_file_id)    
+        # build file slices
+        file_slices = []
+        for log_file in log_files_in_file_id:
+            file_slice = find_file_slice(log_file, file_slice_barriers)
+            file_slices.add(file_slice)

Review Comment:
   nit: We won't be adding a file slice object per log file right? I see that `file_slices` is a list and not say a sortedset. Do you intend something like that?



##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+Assume we have the following inputs to build the file slices:
+1. `base_files_by_file_id`: a map of filegroup id to list of base files.
+2. `log_files_by_file_id`: a map of feilgroup id to list of log files.
+3. `timeline`: Hudi write timeline .
+
+The pseudocode below introduces a notion of **file slice barriers**, which contain a list of instant (start) times of 
+the base files in descending order. Barriers will help in demarcating file slices. The below pseudocode builds the file 
+slices per file group. Building file groups from file slices is not shown but can be easily done.
+
+```python
+# new file slicing
+def build_file_slices(base_files_by_file_id, log_files_by_file_id, timeline):
+    # get set of all filegroup ids
+    file_id_set = base_files_by_file_id.keys
+    file_id_set.add_all(log_files_by_file_id.keys)
+    
+    for file_id in file_id_set:
+        # sort the base files by descending order of instant (start) time, i.e. last written base file first
+        base_files_in_file_id = base_files_by_file_id[file_id].sort(BASE_FILE_REVERSE_COMPARATOR)
+        # filter out log files that have been log-compacted
+        log_files_in_file_id = handle_log_compaction(log_files_by_file_id[file_id], timeline)
+        # sort the log files by ascending order of completion time
+        log_files_in_file_id.sort(LOG_FILE_COMPARATOR)
+        # get list of file slice barriers for this fielgroup id
+        file_slice_barriers = get_file_slice_barriers(base_files_in_file_id, log_files_in_file_id)    
+        # build file slices
+        file_slices = []
+        for log_file in log_files_in_file_id:
+            file_slice = find_file_slice(log_file, file_slice_barriers)
+            file_slices.add(file_slice)
+            
+            
+# Given all log files for a file id, filter out such log files that have been log-compacted.
+def handle_log_compaction(log_files_in_file_id, timeline):
+    log_compaction_instants = timeline.get_completed_log_compaction()
+    for log_compaction_instant in log_compaction_instant:

Review Comment:
   this is going to make it expensive to do the slicing right? we have to read the commit metadata for all log compactions in the timeline ? Can we include these and let the workers/executors skip them during actual read?



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] LinMingQiang commented on a diff in pull request #7907: [HUDI-5672][RFC-61] Lockless multi writer support

Posted by "LinMingQiang (via GitHub)" <gi...@apache.org>.
LinMingQiang commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1156022791


##########
rfc/rfc-61/rfc-61.md:
##########
@@ -0,0 +1,98 @@
+# RFC-61: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Determistic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+#### The Compaction Procedure

Review Comment:
   The mark just tells writers to  write into a new instant file. If you have time, we can discuss this topic online.
   
   



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1325831839


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,215 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+```python
+# new file slicing
+file_slice_barriers = []
+# sort the base file by start time reversely and get the barriers
+for base_file in fg.getBaseFilesSortedReverselyByStartTime():
+  file_slice_barriers.add(instant_time(base_file))
+
+file_slices = []
+for log_file in fg.getLogFilesSortedByCompletionTime():
+  completion_time = completion_time(log_file)
+  file_slice = find_file_slice(completion_time, file_slice_barriers)
+  file_slices.add(file_slice)
+
+def find_file_slice(completion_time, file_slice_barriers):
+  # Note: needs to handle special case where file_slice_barriers is empty (no base file)
+  for barrier in file_slice_barriers:
+    if (barrier < completion_time):
+      # each file slice is attached to a barrier, returns the existing file slice or a fresh new one based on the barrier.
+      return get_or_create_file_slice(barrier)
+```
+
+### Pseudocode for Time Travel Queries
+
+```python
+# This is based on the new file slicing.
+# find the write 'w' with completionTime(w) < t, among all the writes that completed before t 
+# (Note t may not match an existing completion time directly) 
+max_completion_time = findMaxCompletedTimeBefore(t) 
+
+fileSlicesToRead = [] 
+for fg in fileGroups:
+  fileSlice = latestFileSliceThatIsOverlappingWithCompletionTime(fg, max_completion_time)
+  fileSlicesToRead.append(fileSlice)
+return fileSlicesToRead
+
+def latestFileSliceThatIsOverlappingWithCompletionTime(fg, max_completion_time):

Review Comment:
   Not very clear how to do that, maybe @yihua  can help here.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking multi writer support

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1322328542


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,165 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, because we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${instant}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by base_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |

Review Comment:
   insert-insert should not have conflicts in any use cases. We do not cover clustering-upsert conflict here.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-5672][RFC-61] Lockless multi writer support

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1155502850


##########
rfc/rfc-61/rfc-61.md:
##########
@@ -0,0 +1,98 @@
+# RFC-61: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Determistic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+#### The Compaction Procedure

Review Comment:
   > does not support generate compaction plan during data writing
   
   Yes, we can only schedule a fresh new compaction plan after a commit/delta_commit had succeed, for `during data writing` did you indicate the procedure of on-going data writing?



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] fengjian428 commented on a diff in pull request #7907: [HUDI-5672][RFC-61] Lockless multi writer support

Posted by "fengjian428 (via GitHub)" <gi...@apache.org>.
fengjian428 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1156003396


##########
rfc/rfc-61/rfc-61.md:
##########
@@ -0,0 +1,98 @@
+# RFC-61: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Determistic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+#### The Compaction Procedure

Review Comment:
   > generate
   
   @danny0405 yes, data should writing into new instant, hence the Instant generated by compaction plan in this situation, data will loss as long as some data writing to old instant after Compaction plan was generated.
   
   @LinMingQiang  do you mean that mark will stop all the writers before a compaction plan ?
   



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] LinMingQiang commented on a diff in pull request #7907: [HUDI-5672][RFC-61] Lockless multi writer support

Posted by "LinMingQiang (via GitHub)" <gi...@apache.org>.
LinMingQiang commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1155513800


##########
rfc/rfc-61/rfc-61.md:
##########
@@ -0,0 +1,98 @@
+# RFC-61: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Determistic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+#### The Compaction Procedure

Review Comment:
   May be we can generate an alignment mark before  generate compaction plan, and the writer can scroll through the new file after discovering the mark. And then generate a schedule plan.
   
   



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] beyond1920 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking multi writer support

Posted by "beyond1920 (via GitHub)" <gi...@apache.org>.
beyond1920 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1312463342


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,124 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, because we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 

Review Comment:
   And I guess we would only create new log files, and will not append to existed log files.



##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,119 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${instant}.log.${version}_${task_token}

Review Comment:
   In the picture of [Non-Serial Compaction Plan Schedule], there are two log files named `t2_t5_v2.w2` and `t1_t3_v1.w1`, the log files names contain instant time and completion time. 
   But the log name rule here only contain instant time, does not contain completion time.
   I suppose the rules here should be followed, and the log file names in the picture are for easier understanding, I guess?



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1339600606


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token

Review Comment:
   Actually it is not necessary, we have the ordering field to keep the correctness, a log file that completes later may be flushed with a smaller start instant time, that means if a file is written first it may be sorted to the latest with a largest completion time.
   
   From my understanding, either we sort by start or end time is okay.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [HUDI-6495][RFC-66] Non-blocking Concurrency Control [hudi]

Posted by "beyond1920 (via GitHub)" <gi...@apache.org>.
beyond1920 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1363440409


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.

Review Comment:
   Besides, could Non-blocking concurrent control work for `bulk insert`?
   Bulk insert writes data into base files, if there are multiple bulk insert job, there might exists multiple base files in the same bucket. 
   Maybe one `bulk insert` job and multiple `insert`/`upsert` could be allowed run concurrently.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [HUDI-6495][RFC-66] Non-blocking Concurrency Control [hudi]

Posted by "beyond1920 (via GitHub)" <gi...@apache.org>.
beyond1920 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1360569304


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.

Review Comment:
   Is Non-blocking concurrent control only work for insert and update, and not for insert overwrite? 
   The following points are different for `insert overwrite` with `insert` or `upsert`:
   1. Generating a fixed file group based on the bucket number is not applicable for `insert overwrite`. Using the same file group before and after insert overwrite will leads to incorrect results.
   
   2. If there are multiple writers, the one that fires first may finish later. Is it the one that fired later overwrites the data generated by the one that fired first, or is it the one that finished later overwrites the data generated by the one that finished first?
   
   3. Assuming we have two commit: t1 -> commit1, t2 -> commit2. When generating `partitionToReplaceIds` metadata for insert overwrite job, a lock might be needed here.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1339594892


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}

Review Comment:
   yes



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking multi writer support

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1322324393


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,119 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${instant}.log.${version}_${task_token}

Review Comment:
   Updated, but I'm -1 for introduing the `base Transaction timestamp` in the file name, no one knows what is that.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [HUDI-6495][RFC-66] Non-blocking Concurrency Control [hudi]

Posted by "psendyk (via GitHub)" <gi...@apache.org>.
psendyk commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1349133996


##########
rfc/rfc-61/rfc-61.md:
##########
@@ -0,0 +1,98 @@
+# RFC-61: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Determistic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.

Review Comment:
   @danny0405 First of all, this is an exciting change to the concurrency model. Would the `BUCKET` index type also be required for a single ingestion writer in `INSERT` mode and multiple data modification writers (i.e. no new records) in `UPSERT` mode? From my understanding, the `BUCKET` index type would only be required for multiple ingestion writers, specifically in the case of them writing the same _new_ record key. But if the records already exist, the writers modifying them will add deltas to the same file groups no matter the index type.
   We've been running into some write conflicts between our ingestion and GDPR deletion; even though our table is partitioned on time, we have some late arrivals that are causing the conflicts. When we created the table, the `BUCKET` index wasn't even available. Given that the index type cannot be changed, it'd be annoying to have to rewrite our entire table (>1PB size). I was hoping we could use the non-blocking concurrency control with the `SIMPLE` index type under the assumption that we have only one ingestion writer that adds new records, and all other writers only modify the existing records. Curious your thoughts on this, and whether it's even something that you'd want to allow in the config since it might be unsafe for most use cases.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] LinMingQiang commented on a diff in pull request #7907: [HUDI-5672][RFC-61] Lockless multi writer support

Posted by "LinMingQiang (via GitHub)" <gi...@apache.org>.
LinMingQiang commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1101638612


##########
rfc/rfc-61/rfc-61.md:
##########
@@ -0,0 +1,98 @@
+# RFC-61: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Determistic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.

Review Comment:
   If we are using a MOR table but not a bucket index layout, can we support lockless multi writer for INSERT? Will there be any problem?
   It will cause multiple writers to write to the same file and cause conflicts.
   



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] LinMingQiang commented on a diff in pull request #7907: [HUDI-5672][RFC-61] Lockless multi writer support

Posted by "LinMingQiang (via GitHub)" <gi...@apache.org>.
LinMingQiang commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1101638612


##########
rfc/rfc-61/rfc-61.md:
##########
@@ -0,0 +1,98 @@
+# RFC-61: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Determistic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.

Review Comment:
   > If we are using a MOR table but not a bucket index layout, can we support lockless multi writer for INSERT? Will there be any problem?
   It will cause multiple writers to write to the same file and cause conflicts.
   



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] LinMingQiang commented on a diff in pull request #7907: [HUDI-5672][RFC-61] Lockless multi writer support

Posted by "LinMingQiang (via GitHub)" <gi...@apache.org>.
LinMingQiang commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1101637043


##########
rfc/rfc-61/rfc-61.md:
##########
@@ -0,0 +1,98 @@
+# RFC-61: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Determistic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.

Review Comment:
   It will cause multiple writers to write to the same file and cause conflicts.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1328193199


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,239 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+```python
+# new file slicing
+file_slice_barriers = []
+# sort the base file by start time reversely and get the barriers
+for base_file in fg.getBaseFilesSortedReverselyByStartTime():
+  file_slice_barriers.add(instant_time(base_file))
+
+file_slices = []
+for log_file in fg.getLogFilesSortedByCompletionTime():
+  completion_time = completion_time(log_file)
+  file_slice = find_file_slice(completion_time, file_slice_barriers)
+  file_slices.add(file_slice)
+
+def find_file_slice(completion_time, file_slice_barriers):
+  # Note: needs to handle special case where file_slice_barriers is empty (no base file)
+  for barrier in file_slice_barriers:
+    if (barrier < completion_time):
+      # each file slice is attached to a barrier, returns the existing file slice or a fresh new one based on the barrier.
+      return get_or_create_file_slice(barrier)

Review Comment:
   More details added.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] codope commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking multi writer support

Posted by "codope (via GitHub)" <gi...@apache.org>.
codope commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1321525253


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,165 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,

Review Comment:
   Another use case is clustering and upsert conflict. Can that be handled by non-blocking concurrency control?



##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,165 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.

Review Comment:
   Why can't we support COW? Even for MOR what if user has not configured inline/async compaction? Suppose user is doing offline compaction.



##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,165 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, because we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${instant}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by base_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |

Review Comment:
   i would say let's keep the base instant time too unless there is some inefficiency. 
   Otherwise we will have to add the base instatnt time to log file header (or repeated in log block header)



##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,119 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${instant}.log.${version}_${task_token}

Review Comment:
   Yes i think log file naming needs to be udpated. It can contain both.
   `[File Id]_[Base Transaction timestamp]_[Deltacommit instant time].[Log File Extension].[Log File Version]_[File Write Token]`



##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,165 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, because we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${instant}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by base_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |

Review Comment:
   What will happen on insert-insert conflict or clustering-upsert conflict? Will one of the writers fail? If so, do we retry?



##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,165 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, because we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${instant}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by base_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+```python

Review Comment:
   thanks for adding the new file slicing.
   let's also add how we handle snapshot, time travel, incremental and cdc queries with the new slicing.



##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,124 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, because we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 

Review Comment:
   yes it will simplify rollbacks as well. Check
   https://github.com/apache/hudi/pull/9594



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking multi writer support

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1322328068


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,165 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, because we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${instant}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by base_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |

Review Comment:
   > let's keep the base instant time too unless there is some inefficiency.
   
   There is inefficiency because the base instant time needs additional look up to the fs view, which is burdensome for the driver especially for large table with updates to multiple data partitions. This is good chance to remove that, if position deletes needs base instant time, do the very specific fs view look-up and put it in the log blocks.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [HUDI-6495][RFC-66] Non-blocking Concurrency Control [hudi]

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1339600606


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token

Review Comment:
   Actually it is not necessary, we have the ordering field to keep the correctness, a log file that completes later may be flushed with a smaller start instant time, that means if a file is written first it may be sorted to the latest with the largest completion time.
   
   From my comprehension, sorting either by start or end time is okay.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [HUDI-6495][RFC-66] Non-blocking Concurrency Control [hudi]

Posted by "peanut-chenzhong (via GitHub)" <gi...@apache.org>.
peanut-chenzhong commented on PR #7907:
URL: https://github.com/apache/hudi/pull/7907#issuecomment-1785080428

   > > for this feature, how do we handle the failure writing commits,will it be rollback by other writing tasks?
   > 
   > There is no failure thrown actively because the conflicts would be reslved automically.
   
   not the conflict failure but some other failure such as input data not correct. 
   now we use partition lock for hudi but in some case there`s some failure commits and it won`t been rollback or archived.


-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] bvaradar commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "bvaradar (via GitHub)" <gi...@apache.org>.
bvaradar commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1339455600


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,165 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, because we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${instant}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by base_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |

Review Comment:
   If we include base instant time in the log files, it will become confusing because the logical file-slice will span across base instant-times



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1339603396


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+Assume we have the following inputs to build the file slices:
+1. `base_files_by_file_id`: a map of filegroup id to list of base files.
+2. `log_files_by_file_id`: a map of feilgroup id to list of log files.
+3. `timeline`: Hudi write timeline .
+
+The pseudocode below introduces a notion of **file slice barriers**, which contain a list of instant (start) times of 
+the base files in descending order. Barriers will help in demarcating file slices. The below pseudocode builds the file 
+slices per file group. Building file groups from file slices is not shown but can be easily done.
+
+```python
+# new file slicing
+def build_file_slices(base_files_by_file_id, log_files_by_file_id, timeline):
+    # get set of all filegroup ids
+    file_id_set = base_files_by_file_id.keys
+    file_id_set.add_all(log_files_by_file_id.keys)
+    
+    for file_id in file_id_set:
+        # sort the base files by descending order of instant (start) time, i.e. last written base file first
+        base_files_in_file_id = base_files_by_file_id[file_id].sort(BASE_FILE_REVERSE_COMPARATOR)
+        # filter out log files that have been log-compacted
+        log_files_in_file_id = handle_log_compaction(log_files_by_file_id[file_id], timeline)
+        # sort the log files by ascending order of completion time
+        log_files_in_file_id.sort(LOG_FILE_COMPARATOR)
+        # get list of file slice barriers for this fielgroup id
+        file_slice_barriers = get_file_slice_barriers(base_files_in_file_id, log_files_in_file_id)    
+        # build file slices
+        file_slices = []
+        for log_file in log_files_in_file_id:
+            file_slice = find_file_slice(log_file, file_slice_barriers)
+            file_slices.add(file_slice)
+            
+            
+# Given all log files for a file id, filter out such log files that have been log-compacted.
+def handle_log_compaction(log_files_in_file_id, timeline):

Review Comment:
   Yes, the pending compaction base filres are exlcuded always.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] BruceKellan commented on a diff in pull request #7907: [HUDI-5672][RFC-61] Lockless multi writer support

Posted by "BruceKellan (via GitHub)" <gi...@apache.org>.
BruceKellan commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1102190591


##########
rfc/rfc-61/rfc-61.md:
##########
@@ -0,0 +1,98 @@
+# RFC-61: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Determistic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.

Review Comment:
   Can you explain why the same file is written? In my understanding, different writers will write to different log files, and they will be merged into the same `File Group` during the Compaction phase. Because it's `INSERT` operation, it doesn't need to resolve the same primary key causing conflicts.
   
   Maybe my understanding is wrong, I'd be grateful if you could help me point out.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-5672][RFC-61] Lockless multi writer support

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1126124036


##########
rfc/rfc-61/rfc-61.md:
##########
@@ -0,0 +1,98 @@
+# RFC-61: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Determistic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,

Review Comment:
   It can be and that would eliminate the explicit config options.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] vinothchandar commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "vinothchandar (via GitHub)" <gi...@apache.org>.
vinothchandar commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1328040583


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,239 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+```python
+# new file slicing
+file_slice_barriers = []
+# sort the base file by start time reversely and get the barriers
+for base_file in fg.getBaseFilesSortedReverselyByStartTime():
+  file_slice_barriers.add(instant_time(base_file))
+
+file_slices = []
+for log_file in fg.getLogFilesSortedByCompletionTime():
+  completion_time = completion_time(log_file)
+  file_slice = find_file_slice(completion_time, file_slice_barriers)
+  file_slices.add(file_slice)
+
+def find_file_slice(completion_time, file_slice_barriers):
+  # Note: needs to handle special case where file_slice_barriers is empty (no base file)
+  for barrier in file_slice_barriers:
+    if (barrier < completion_time):
+      # each file slice is attached to a barrier, returns the existing file slice or a fresh new one based on the barrier.
+      return get_or_create_file_slice(barrier)

Review Comment:
   where is this defined? Its kind of hard to understand fully without this? 



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] vinothchandar commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "vinothchandar (via GitHub)" <gi...@apache.org>.
vinothchandar commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1328135812


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,239 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+```python
+# new file slicing
+file_slice_barriers = []
+# sort the base file by start time reversely and get the barriers
+for base_file in fg.getBaseFilesSortedReverselyByStartTime():
+  file_slice_barriers.add(instant_time(base_file))
+
+file_slices = []
+for log_file in fg.getLogFilesSortedByCompletionTime():
+  completion_time = completion_time(log_file)
+  file_slice = find_file_slice(completion_time, file_slice_barriers)
+  file_slices.add(file_slice)
+
+def find_file_slice(completion_time, file_slice_barriers):
+  # Note: needs to handle special case where file_slice_barriers is empty (no base file)
+  for barrier in file_slice_barriers:
+    if (barrier < completion_time):
+      # each file slice is attached to a barrier, returns the existing file slice or a fresh new one based on the barrier.
+      return get_or_create_file_slice(barrier)
+```
+
+#### A Demo Conducive to Comprehension
+
+Just to make it concrete for the readers, we can add an example.
+Let's take this example of one filegroup and consider the following sequence of commits:
+
+```xml
+fg_t10 -> very first commit with start time, end time as [t10, t20].
+l1 -> log file version 1 start time, end time as [t21, t40].
+l2 -> concurrent log file version 2 [t30, t50].
+fg_t60 -> base file due to compaction [t60, t80].
+l3 -> concurrent log file version 3 [t35, t90].
+```
+
+In this case, file_slice_barriers list is [t60, t10]. For a query at t100, getAllFileSlices should return the following list:
+
+```xml
+[
+{t60, fg_t60.parquet, {l3}},
+{t10, fg_t10.parquet, {l1, l2}}
+]
+```
+
+This assumes that file slicing is done based on completion time.
+
+### Pseudocode for Time Travel Queries
+
+```python
+# This is based on the new file slicing.
+# find the write 'w' with completionTime(w) < t, among all the writes that completed before t 
+# (Note t may not match an existing completion time directly) 
+max_completion_time = findMaxCompletedTimeBefore(t) 
+
+fileSlicesToRead = [] 
+for fg in fileGroups:
+  fileSlice = latestFileSliceThatIsOverlappingWithCompletionTime(fg, max_completion_time)
+  fileSlicesToRead.append(fileSlice)
+return fileSlicesToRead
+
+def latestFileSliceThatIsOverlappingWithCompletionTime(fg, max_completion_time):
+  # getAllFileSlices() returns the file slices as it does today ordered reversely by instant time.
+  for f_slice in fg.getAllFileSlices():

Review Comment:
   Are these guaranteed to be non-overlapping ie a given log file or base file occurs in only one slice?
   



##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,239 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+```python
+# new file slicing
+file_slice_barriers = []
+# sort the base file by start time reversely and get the barriers
+for base_file in fg.getBaseFilesSortedReverselyByStartTime():
+  file_slice_barriers.add(instant_time(base_file))
+
+file_slices = []
+for log_file in fg.getLogFilesSortedByCompletionTime():
+  completion_time = completion_time(log_file)
+  file_slice = find_file_slice(completion_time, file_slice_barriers)
+  file_slices.add(file_slice)
+
+def find_file_slice(completion_time, file_slice_barriers):
+  # Note: needs to handle special case where file_slice_barriers is empty (no base file)
+  for barrier in file_slice_barriers:
+    if (barrier < completion_time):
+      # each file slice is attached to a barrier, returns the existing file slice or a fresh new one based on the barrier.
+      return get_or_create_file_slice(barrier)
+```
+
+#### A Demo Conducive to Comprehension
+
+Just to make it concrete for the readers, we can add an example.
+Let's take this example of one filegroup and consider the following sequence of commits:
+
+```xml
+fg_t10 -> very first commit with start time, end time as [t10, t20].
+l1 -> log file version 1 start time, end time as [t21, t40].
+l2 -> concurrent log file version 2 [t30, t50].
+fg_t60 -> base file due to compaction [t60, t80].
+l3 -> concurrent log file version 3 [t35, t90].
+```
+
+In this case, file_slice_barriers list is [t60, t10]. For a query at t100, getAllFileSlices should return the following list:
+
+```xml
+[
+{t60, fg_t60.parquet, {l3}},
+{t10, fg_t10.parquet, {l1, l2}}
+]
+```
+
+This assumes that file slicing is done based on completion time.

Review Comment:
   t10 and t60 are both start times right?



##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,239 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+```python
+# new file slicing
+file_slice_barriers = []
+# sort the base file by start time reversely and get the barriers
+for base_file in fg.getBaseFilesSortedReverselyByStartTime():
+  file_slice_barriers.add(instant_time(base_file))
+
+file_slices = []
+for log_file in fg.getLogFilesSortedByCompletionTime():
+  completion_time = completion_time(log_file)
+  file_slice = find_file_slice(completion_time, file_slice_barriers)
+  file_slices.add(file_slice)
+
+def find_file_slice(completion_time, file_slice_barriers):
+  # Note: needs to handle special case where file_slice_barriers is empty (no base file)
+  for barrier in file_slice_barriers:
+    if (barrier < completion_time):
+      # each file slice is attached to a barrier, returns the existing file slice or a fresh new one based on the barrier.
+      return get_or_create_file_slice(barrier)
+```
+
+#### A Demo Conducive to Comprehension
+
+Just to make it concrete for the readers, we can add an example.
+Let's take this example of one filegroup and consider the following sequence of commits:
+
+```xml
+fg_t10 -> very first commit with start time, end time as [t10, t20].
+l1 -> log file version 1 start time, end time as [t21, t40].
+l2 -> concurrent log file version 2 [t30, t50].
+fg_t60 -> base file due to compaction [t60, t80].
+l3 -> concurrent log file version 3 [t35, t90].
+```
+
+In this case, file_slice_barriers list is [t60, t10]. For a query at t100, getAllFileSlices should return the following list:
+
+```xml
+[
+{t60, fg_t60.parquet, {l3}},
+{t10, fg_t10.parquet, {l1, l2}}
+]
+```
+
+This assumes that file slicing is done based on completion time.
+
+### Pseudocode for Time Travel Queries
+
+```python
+# This is based on the new file slicing.
+# find the write 'w' with completionTime(w) < t, among all the writes that completed before t 
+# (Note t may not match an existing completion time directly) 
+max_completion_time = findMaxCompletedTimeBefore(t) 
+
+fileSlicesToRead = [] 
+for fg in fileGroups:
+  fileSlice = latestFileSliceThatIsOverlappingWithCompletionTime(fg, max_completion_time)
+  fileSlicesToRead.append(fileSlice)
+return fileSlicesToRead
+
+def latestFileSliceThatIsOverlappingWithCompletionTime(fg, max_completion_time):
+  # getAllFileSlices() returns the file slices as it does today ordered reversely by instant time.
+  for f_slice in fg.getAllFileSlices():
+    # the min_completion time may takes time if there is a base file.
+    fs_min_completion_time = getMinCompletionTime(f_slice)
+    fs_max_completion_time = getMaxCompletionTime(f_slice)
+
+    if (fs_min_completion_time <= max_completion_time <= fs_max_completion_time):
+      # Needs to remove the logs files that completed later than the maxCompletionTime
+      return fix_file_slice_by_max_completion_time(f_slice, maxCompletionTime)
+  return empty
+```
+
+### Pseudocode for Incremental/CDC Queries
+
+```python
+#1. Fetch from metadata table all base and log files with completion or instant time between t1 (exclusive) and t2 (inclusive), 

Review Comment:
   Fix comment to remove dependency on mt?the completion time may come from just timeline



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1328187835


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,239 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+```python
+# new file slicing
+file_slice_barriers = []
+# sort the base file by start time reversely and get the barriers
+for base_file in fg.getBaseFilesSortedReverselyByStartTime():
+  file_slice_barriers.add(instant_time(base_file))
+
+file_slices = []
+for log_file in fg.getLogFilesSortedByCompletionTime():
+  completion_time = completion_time(log_file)
+  file_slice = find_file_slice(completion_time, file_slice_barriers)
+  file_slices.add(file_slice)
+
+def find_file_slice(completion_time, file_slice_barriers):
+  # Note: needs to handle special case where file_slice_barriers is empty (no base file)
+  for barrier in file_slice_barriers:
+    if (barrier < completion_time):
+      # each file slice is attached to a barrier, returns the existing file slice or a fresh new one based on the barrier.
+      return get_or_create_file_slice(barrier)
+```
+
+#### A Demo Conducive to Comprehension
+
+Just to make it concrete for the readers, we can add an example.
+Let's take this example of one filegroup and consider the following sequence of commits:
+
+```xml
+fg_t10 -> very first commit with start time, end time as [t10, t20].
+l1 -> log file version 1 start time, end time as [t21, t40].
+l2 -> concurrent log file version 2 [t30, t50].
+fg_t60 -> base file due to compaction [t60, t80].
+l3 -> concurrent log file version 3 [t35, t90].
+```
+
+In this case, file_slice_barriers list is [t60, t10]. For a query at t100, getAllFileSlices should return the following list:
+
+```xml
+[
+{t60, fg_t60.parquet, {l3}},
+{t10, fg_t10.parquet, {l1, l2}}
+]
+```
+
+This assumes that file slicing is done based on completion time.
+
+### Pseudocode for Time Travel Queries
+
+```python
+# This is based on the new file slicing.
+# find the write 'w' with completionTime(w) < t, among all the writes that completed before t 
+# (Note t may not match an existing completion time directly) 
+max_completion_time = findMaxCompletedTimeBefore(t) 
+
+fileSlicesToRead = [] 
+for fg in fileGroups:
+  fileSlice = latestFileSliceThatIsOverlappingWithCompletionTime(fg, max_completion_time)
+  fileSlicesToRead.append(fileSlice)
+return fileSlicesToRead
+
+def latestFileSliceThatIsOverlappingWithCompletionTime(fg, max_completion_time):
+  # getAllFileSlices() returns the file slices as it does today ordered reversely by instant time.
+  for f_slice in fg.getAllFileSlices():

Review Comment:
   Exactly.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1340868393


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token
+```
+
+For a single writer, its log files natual order is preserved by the auto increasing version_number,
+for multiple writers, we try the best to preserve the natual order still with the version number,
+but the write_token has deterministic priority in sorting, which breaks the file generation order.
+
+![log file sequence](log_file_sequence.png)
+
+### Format changes
+
+| Type                       | Changes                                                                                                                            |
+|----------------------------|------------------------------------------------------------------------------------------------------------------------------------|
+| Commit/compaction metadata | No changes                                                                                                                         |
+| Commit file name           | we are only going to add completion time in completed metadata filename for all actions                                            |
+| Log file format            | Adding deltacommit instant time to log file name. The file name does not contain base instant time; No changes to log block format |
+
+### Commit Protocol
+
+| Type                            | Content                                                                                                                                       |
+|---------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------|
+| Writer expectations             | Writer does not need to look up the latest file slice to fetch the base commit time anymore, it always uses the current instant time directly |
+| Conflict Resolution             | No explicit conflict resolution, it is handled by the compactor                                                                               |
+| Reader expectations             | Readers still query based on file slices, see the pseudocode for new file slicing                                                             |
+| Synchronization and correctness | Still need some mutex on commit log (timeline). Getting the completion time and adding to the timeline needs to happen atomically             |
+
+**Note:** Clustering and updates conflict resolution is not in the scope of non-blocking concurrency control, you still need the OCC for course-grained failures and retries. 
+
+## Appendix
+
+### Pseudocode for New File Slicing
+
+Assume we have the following inputs to build the file slices:
+1. `base_files_by_file_id`: a map of filegroup id to list of base files.
+2. `log_files_by_file_id`: a map of feilgroup id to list of log files.
+3. `timeline`: Hudi write timeline .
+
+The pseudocode below introduces a notion of **file slice barriers**, which contain a list of instant (start) times of 
+the base files in descending order. Barriers will help in demarcating file slices. The below pseudocode builds the file 
+slices per file group. Building file groups from file slices is not shown but can be easily done.
+
+```python
+# new file slicing
+def build_file_slices(base_files_by_file_id, log_files_by_file_id, timeline):
+    # get set of all filegroup ids
+    file_id_set = base_files_by_file_id.keys
+    file_id_set.add_all(log_files_by_file_id.keys)
+    
+    for file_id in file_id_set:
+        # sort the base files by descending order of instant (start) time, i.e. last written base file first
+        base_files_in_file_id = base_files_by_file_id[file_id].sort(BASE_FILE_REVERSE_COMPARATOR)
+        # filter out log files that have been log-compacted
+        log_files_in_file_id = handle_log_compaction(log_files_by_file_id[file_id], timeline)
+        # sort the log files by ascending order of completion time
+        log_files_in_file_id.sort(LOG_FILE_COMPARATOR)
+        # get list of file slice barriers for this fielgroup id
+        file_slice_barriers = get_file_slice_barriers(base_files_in_file_id, log_files_in_file_id)    
+        # build file slices
+        file_slices = []
+        for log_file in log_files_in_file_id:
+            file_slice = find_file_slice(log_file, file_slice_barriers)
+            file_slices.add(file_slice)

Review Comment:
   No, it should be a set or map.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1339600606


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.
+
+### Basic Work Flow
+
+#### Writing Log Files Separately In Sequence
+
+Basically, each writer flushes the log files in sequence, the log file rolls over for different versioning number,
+a pivotal thing needs to note here is that we need to make the write_token unique for the same version log files with the same base instant time,
+so that the file name does not conflict for the writers.
+
+The log files generated by a single writer can still preserve the sequence by versioning number, which is important if the natual order is needed for single writer events.
+
+![multi-writer](multi_writer.png)
+
+### The Compaction Procedure
+
+The compaction service is the duty role that actually resoves the conflicts. Within a file group, it sorts the files then merge all the record payloads for a record key.
+The event time sequence is respected by combining the payloads with even time field provided by the payload (known as the `preCombine` field in Hudi).
+
+![compaction procedure](compaction.png)
+
+#### Non-Serial Compaction Plan Schedule
+Currently, the compaction plan scheduling must be in serial order with the writers, that means, while scheduling the compaction plan, no ongoing writers should be writing to
+the table. This restriction makes the compaction almost impossible for multi streaming writers because there is always an instant writing to the table for streaming ingestion.
+
+In order to unblock the compaction plan scheduling and keep the completeness of the readers, we introduce the completion time for file slice generation:
+
+- Support quick look-up from instant time ➝ completion time, the [HUDI-6539](https://issues.apache.org/jira/browse/HUDI-6539) supports fast completion time queries on archived timeline, based on this, we are able
+to support flexible completion time queries on both active and archived timeline, see [HUDI-6725](https://issues.apache.org/jira/browse/HUDI-6725);
+- New compaction plan scheduling to comply with the completion time, that is: only log files that have smaller completion time(than the compaction start instant time) should be considered
+- New file slice generation strategy, a log file with smaller instant time (than the compaction instant time) but greater completion time should be assigned to a new file slice
+- By combining #2 and #3, in general, we are slicing by comparing the compaction start time with the log files completion time.
+
+<img src="non_serial_compaction.png" alt="drawing" width="400"/>
+
+Assumes we have two log files, with instant time & completion time as [t1, t3] and [t2, t5], at t4, a compaction plan was scheduled,
+the plan does not include file t2_t5_v1.w2, in the reader view, the log file should be assigned to a different file slice than the t4 instant time.
+
+#### Global Monotonically Increasing Timestamp
+
+In order to make the time deterministic among cloud storages, we use a logical time generated by a special **TimeGenerator**, see [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) for details.
+
+#### Assumption On Locks
+In order to get global monotonically increasing time generation, we may introduce lock providers on the instant time generation and the creation of completed metadata file.
+That means, for each instant time/completion time generation request, there is a try-lock action. As a special case, we store the completion time as part of the completed
+metadata file name, the time generation and file creation should have atomicity altogether, they should be guarded under the same lock of regular time generation request.
+
+### The Log File Naming Convention
+
+We use the current instant time instead of the base commit time in the file name,
+so that tasks from different writers can not conflict in file name. We can also parse the file name to fetch the instant time quickly. 
+Finally, the log name is with the following pattern:
+
+```shell
+${uuid}_${delta_commit instant time}.log.${version}_${task_token}
+```
+
+### The Sorting Rules for Log Files from Different Writers
+
+The sorting rules is important because it somehow decides the natual order processing,
+especially when the event time field are the same, and we don't know which payload to choose when combining.
+Here we can keep using the log file name compactor of current codebase, that is:
+
+```sql
+order by delta_commit_time, version_number, write_token

Review Comment:
   Actually it is not necessary, we have the ordering field to keep the correctness, actually a log file that completes later may be flushed with a smaller start instant time, that means if a file is written first it may be sorted to the latest if it's completion time is the largest.
   
   From my understanding, either we sort by start or end time is okay.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] bvaradar commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking Concurrency Control

Posted by "bvaradar (via GitHub)" <gi...@apache.org>.
bvaradar commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1339423476


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.
+
+#### Lazy Cleaning Strategy
+
+Config the cleaning strategy as lazy so that the pending instants are not rolled back by the other active writers.

Review Comment:
   cleaning -> rollback



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [HUDI-6495][RFC-66] Non-blocking Concurrency Control [hudi]

Posted by "peanut-chenzhong (via GitHub)" <gi...@apache.org>.
peanut-chenzhong commented on PR #7907:
URL: https://github.com/apache/hudi/pull/7907#issuecomment-1783720492

   for this feature, how do we handle the failure writing commits,will it be rollback by other writing tasks?


-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking multi writer support

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1322325872


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,165 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.

Review Comment:
   Because the conflict resolution for COW is limited to OCC, that is decided by the COW behaviors. Offline compaction should also work well.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking multi writer support

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1322326466


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,165 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,

Review Comment:
   No, we have a plan for that but it's only a draft, need more discusstions to find out the solution.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on a diff in pull request #7907: [HUDI-6495][RFC-66] Non-blocking multi writer support

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1322326466


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,165 @@
+# RFC-66: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,

Review Comment:
   No, we have a plan for that but it's only a draft, need more discusstions to pin out the solution.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [HUDI-6495][RFC-66] Non-blocking Concurrency Control [hudi]

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1349439025


##########
rfc/rfc-61/rfc-61.md:
##########
@@ -0,0 +1,98 @@
+# RFC-61: Lockless Multi Writer
+
+## Proposers
+- @danny0405
+- @ForwardXu
+- @SteNicholas
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, explicit lock is not very capable of putting into production, in this RFC, we propse a lockless solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Determistic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.

Review Comment:
   > I was hoping we could use the non-blocking concurrency control with the SIMPLE index type under the assumption that we have only one ingestion writer that adds new records, and all other writers only modify the existing records.
   
   We can definitely extend the conflict resolution strategy if we have some assumptions like this, so a desired state is any concurrent modifications to the same file group can work smothly with semantics of NB-CC, no matter what the index type is.
   
   In general, thanks for the feedback @psendyk , I think it's a very valuable response with real use cases.
   



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [HUDI-6495][RFC-66] Non-blocking Concurrency Control [hudi]

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on PR #7907:
URL: https://github.com/apache/hudi/pull/7907#issuecomment-1783975790

   > for this feature, how do we handle the failure writing commits,will it be rollback by other writing tasks?
   
   There is no failure thrown actively because the conflicts would be reslved automically.


-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [HUDI-6495][RFC-66] Non-blocking Concurrency Control [hudi]

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1361456912


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.

Review Comment:
   Yes, we definitely need a fix for insert overwrite, let's fire a GH issue to tackle this.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [HUDI-6495][RFC-66] Non-blocking Concurrency Control [hudi]

Posted by "beyond1920 (via GitHub)" <gi...@apache.org>.
beyond1920 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1360569304


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.

Review Comment:
   Is Non-blocking concurrent control only work for insert and update, and not for insert overwrite? 
   The following points are different for `insert overwrite` with `insert` or `upsert`:
   1. Generating a fixed file group based on the bucket number is not applicable for `insert overwrite`. Using the same file group before and after insert overwrite will leads to incorrect results.
   
   2. If there are multiple writers, the one that fires first may finish later. Is it the one that fired first overwrites the data generated by the one that fired later, or is it the one that finished first overwrites the data generated by the one that finished later?
   
   3. Assuming we have two commit: t1 -> commit1, t2 -> commit2. Commit2 is fired later, it may complete earlier. When generating `partitionToReplaceIds` metadata for this job, a lock might be needed here.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [HUDI-6495][RFC-66] Non-blocking Concurrency Control [hudi]

Posted by "beyond1920 (via GitHub)" <gi...@apache.org>.
beyond1920 commented on code in PR #7907:
URL: https://github.com/apache/hudi/pull/7907#discussion_r1363440409


##########
rfc/rfc-66/rfc-66.md:
##########
@@ -0,0 +1,318 @@
+# RFC-66: Non-blocking Concurrency Control
+
+## Proposers
+- @danny0405
+- @ForwardXu
+
+## Approvers
+-
+
+## Status
+
+JIRA: [Lockless multi writer support](https://issues.apache.org/jira/browse/HUDI-5672)
+
+## Abstract
+As you know, Hudi already supports basic OCC with abundant lock providers.
+But for multi streaming ingestion writers, the OCC does not work well because the conflicts happen in very high frequency.
+Expand it a little bit, with hashing index, all the writers have deterministic hashing algorithm for distributing the records by primary keys,
+all the keys are evenly distributed in all the data buckets, for a single data flushing in one writer, almost all the data buckets are appended with new inputs,
+so the conflict would very possibility happen for mul-writer because almost all the data buckets are being written by multiple writers at the same time;
+For bloom filter index, things are different, but remember that we have a small file load rebalance strategy to writer into the **small** bucket in higher priority,
+that means, multiple writers prune to write into the same **small** buckets at the same time, that's how conflicts happen.
+
+In general, for multiple streaming writers ingestion, OCC is not very feasible in production, in this RFC, we propose a non-blocking solution for streaming ingestion.
+
+## Background
+
+Streaming jobs are naturally suitable for data ingestion, it has no complexity of pipeline orchestration and has a smother write workload.
+Most of the raw data set we are handling today are generating all the time in streaming way.
+
+Based on that, many requests for multiple writers' ingestion are derived. With multi-writer ingestion, several streaming events with the same schema can be drained into one Hudi table,
+the Hudi table kind of becomes a UNION table view for all the input data set. This is a very common use case because in reality, the data sets are usually scattered all over the data sources.
+
+Another very useful use case we wanna unlock is the real-time data set join. One of the biggest pain point in streaming computation is the dataset join,
+the engine like Flink has basic supports for all kind of SQL JOINs, but it stores the input records within its inner state-backend which is a huge cost for pure data join with no additional computations.
+In [HUDI-3304](https://issues.apache.org/jira/browse/HUDI-3304), we introduced a `PartialUpdateAvroPayload`, in combination with the lockless multi-writer,
+we can implement N-ways data sources join in real-time! Hudi would take care of the payload join during compaction service procedure.
+
+## Design
+
+### The Precondition
+
+#### MOR Table Type Is Required
+
+The table type must be `MERGE_ON_READ`, so that we can defer the conflict resolution to the compaction phase. The compaction service would resolve the conflicts of the same keys by respecting the event time sequence of the events.
+
+#### Deterministic Bucketing Strategy
+
+Deterministic bucketing strategy is required, because the same records keys from different writers are desired to be distributed into the same bucket, not only for UPSERTs, but also for all the new INSERTs.

Review Comment:
   Besides, could Non-blocking concurrent control work for `bulk insert`?
   Bulk insert writes data into base files, if there are multiple bulk insert job, there might exists multiple base files in the same bucket. 
   Maybe one `bulk insert` job and multiple `insert`/`upsert` could run concurrently.



-- 
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-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org