:
On Thu, 29 May 2025 at 10:02, Henry Haiying Cai
wrote:
> Jorge,
> Thanks for your detailed questions, please see my answer inline below.
> On Tuesday, May 27, 2025 at 12:48:01 AM PDT, Jorge Esteban Quilcate
> Otoya wrote:
>
> Hi Henry,
>
> Thanks for the propos
system based solution (such as AWS FSx) or
their equivalents in other cloud providers. Between S3 and S3E1Z, I think some
companies might still prefer S3 since the replication within S3 is across AZs
but S3 is slower. There is tradeoff they need to make.
Thanks,
Jorge.
On Wed, 14 May 2025
gt; > > Thank you for your detailed reply. The answer makes sense to me, and
> you're
> > > right, KIP-1176 has a clear and specific scope and is expected to have
> a
> > > quick path to implement it.
> > >
> > > I also want to discuss the me
able. And make
the configuration doc clear to users about the potential fall back
behavior, I think it will be fine. About the metadata topic in KIP-405,
like I described above, we should not change the retention config for it to
avoid data loss.
Does that make sense?
Thank you.
Luke
On
increased, not decreased, then when the broker
restarted, the wal metadata consumer will need more and more time to read
all the wal metadata before it can serve the produce request. That will be
a problem sometime in the future.
Thank you.
Luke
On Fri, May 9, 2025 at 5:52 AM Henry Haiying Cai
w
n AutoMQ, WAL is solely
> for recovery and is expected to be uploaded to standard S3 as soon as
> possible, without metadata management. I think KIP-1176 might not need it
> either; during recovery, we can simply scan the WAL to restore the
> metadata.
>
> Regards,
> Xinyu
>
AL log segments. Is an
>> internal topic necessary for managing metadata? In AutoMQ, WAL is solely
>> for recovery and is expected to be uploaded to standard S3 as soon as
>> possible, without metadata management. I think KIP-1176 might not need it
>> either; during recovery, we
mply scan the WAL to restore the metadata.
Regards,
Xinyu
On Thu, May 8, 2025 at 2:00 AM Henry Haiying Cai
wrote:
> Xinyu,
> Thanks for your time reading the KIP and detailed comments. We are
> honored to have technical leaders from AutoMQ to look at our work.
> Please see my
cloud market.
Overall, KIP-1176 is a great idea for using S3EOZ to eliminate cross-AZ
replication traffic. Well done!
Disclaimer: I work for AutoMQ, but I am wearing the community hat to join
this discussion thread.
Regards,
Xinyu
On Wed, May 7, 2025 at 9:13 AM Henry Haiying Cai
wrote:
>
since our current
implementation is to copy all the way to LOG_END between leader and follower
broker (through object storage), it seems it would naturally support
replicating transactional messages as well.
On Tuesday, May 6, 2025 at 12:20:43 PM PDT, Henry Haiying Cai
wrote:
Ch
community early to get some
feedbacks.
On Tuesday, May 6, 2025 at 01:59:37 AM PDT, Chia-Ping Tsai
wrote:
hi Henry Haiying Cai
Thanks for sharing the cool idea. Out of curiosity, do you already have an
implementation running in your cluster?
Best,
Chia-Ping
On 2025/05/06 05:56:28 Henry
luded in object storage, but again we are
basically replacing the information exchange in the current
FetchRequest/Response.
Once again, this is quite exciting, so thanks for the contribution!
Best,
Christo
On Thu, 1 May 2025 at 19:01, Henry Haiying Cai
wrote:
> Luke,
> Thanks for you
If it's not too late into the party, we (Slack/Salesforce) also have submitted
KIP-1176: Tiered Storage for Active Log Segments . Our proposal is not a
diskless offering but an incremental evolution on top of KIP-405: Tiered
Storage. We propose using background tasks to upload a section of ac
code base. KIP-1150 is a much ambitious effort
with a complete revamp of Kafka storage and memory management system.
Thank you.
Luke
On Thu, May 1, 2025 at 1:45 PM Henry Haiying Cai
wrote:
> Link to the KIP:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1176%3A+Tiered+Storage+fo
again after the log segment is rolled?
> I'm thinking if it's possible we learn(borrow) some ideas from KIP-1150?
> This way, we can achieve the similar goal since we accumulate (combine)
> data in multiple partitions and upload to S3 to save the cost.
>
> Thank you.
> L
Link to the KIP:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1176%3A+Tiered+Storage+for+Active+Log+Segment
Motivation
In KIP-405, the community has proposed and implemented the tiered storage for
old Kafka log segment files, when the log segments is older than
local.retention.ms, it be
16 matches
Mail list logo