Re: [DISCUSS] Minimum constraint for segment.ms

2024-03-13 Thread Sergio Daniel Troiano
hey guys,

Regarding to num.recovery.threads.per.data.dir: I agree, in our company we
use the number of vCPUs to do so as this is not competing with ready
cluster traffic.


On Wed, 13 Mar 2024 at 09:29, Luke Chen  wrote:

> Hi Divij,
>
> Thanks for raising this.
> The valid minimum value 1 for `segment.ms` is completely unreasonable.
> Similarly for `segment.bytes`, `metadata.log.segment.ms`,
> `metadata.log.segment.bytes`.
>
> In addition to that, there are also some config default values we'd like to
> propose to change in v4.0.
> We can collect more comments from the community, and come out with a KIP
> for them.
>
> 1. num.recovery.threads.per.data.dir:
> The current default value is 1. But the log recovery is happening before
> brokers are in ready state, which means, we should use all the available
> resource to speed up the log recovery to bring the broker to ready state
> soon. Default value should be... maybe 4 (to be decided)?
>
> 2. Other configs might be able to consider to change the default, but open
> for comments:
>2.1. num.replica.fetchers: default is 1, but that's not enough when
> there are multiple partitions in the cluster
>2.2. `socket.send.buffer.bytes`/`socket.receive.buffer.bytes`:
> Currently, we set 100kb as default value, but that's not enough for
> high-speed network.
>
> Thank you.
> Luke
>
>
> On Tue, Mar 12, 2024 at 1:32 AM Divij Vaidya 
> wrote:
>
> > Hey folks
> >
> > Before I file a KIP to change this in 4.0, I wanted to understand the
> > historical context for the value of the following setting.
> >
> > Currently, segment.ms minimum threshold is set to 1ms [1].
> >
> > Segments are expensive. Every segment uses multiple file descriptors and
> > it's easy to run out of OS limits when creating a large number of
> segments.
> > Large number of segments also delays log loading on startup because of
> > expensive operations such as iterating through all directories &
> > conditionally loading all producer state.
> >
> > I am currently not aware of a reason as to why someone might want to work
> > with a segment.ms of less than ~10s (number chosen arbitrary that looks
> > sane)
> >
> > What was the historical context of setting the minimum threshold to 1ms
> for
> > this setting?
> >
> > [1] https://kafka.apache.org/documentation.html#topicconfigs_segment.ms
> >
> > --
> > Divij Vaidya
> >
>


Re: [DISCUSS] KIP-964: Have visibility when produce requests become "async".

2023-08-08 Thread Sergio Daniel Troiano
Hi everyone!

I have a doubt regarding where the metric should go, I need to know where
the local append happens in the active segment log.

I saw some files which "append" on the segment log but I am a bit confusing
which one is the one:
core/src/main/scala/kafka/log/LocalLog.scala
core/src/main/scala/kafka/log/LogSegment.scala
clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java

At the end I need to identify exactly where the data is appended on the
active segment, this should be only the local append as we are interested
on measuring the local latency of a write in the OS (sync vs async)

Thanks in advance!!!


On Mon, 7 Aug 2023 at 14:43, Sergio Daniel Troiano <
sergio.troi...@adevinta.com> wrote:

> Hi everyone!
>
> I would like to start a discuss thread for this KIP
> <https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=263426647>
>
>
> Thanks
>


[DISCUSS] KIP-964: Have visibility when produce requests become "async".

2023-08-07 Thread Sergio Daniel Troiano
Hi everyone!

I would like to start a discuss thread for this KIP



Thanks


new KIP doubt

2023-08-07 Thread Sergio Daniel Troiano
hey everyone,

Sorry for bothering you, I created a KIP long time ago and now I am
creating a new one here

The problem is I lost my apache credentials for starting a discussion
thread, could you please tell me how I can recover them?

Thanks in advance.
Sergio Troiano


Bug wile create users

2023-07-24 Thread Sergio Daniel Troiano
Hi everyone!

We think we found a bug, we dug deep and found the solution.
Please take a look at the bug reported, I am happy to deliver the patch
https://issues.apache.org/jira/browse/KAFKA-15243


Thanks!


log-cleaner priority while accessing block device (sharing idea)

2023-05-14 Thread Sergio Daniel Troiano
*Hi everyone!*


*In our company we use Kafka in hundreds of clusters in PRO. When the log
cleaner triggers  it “could” create some resource issues (disk I/O).*

*For example when we are running a deployment it is possible one or several
of the brokers run  a log cleaner. As we know there is a throttle config
(50 MiB/s) which is cool.  As we know in a deploy *

*We have consumers that maybe are reading from disk, the streaming data to
be sent to the restarted broker and if we are not lucky a heavy log-cleaner
running.*


*As far as I understand a log-cleaner that runs in 3 seconds or 30 is
exactly the same, I mean as long as we don’t fall “behind” and start
accumulating logs to clean we are fine.*


*So I was thinking it would be nice to avoid the resource competition
between user pain (produce/consume) and the log cleaner, it is
evident producers and consumers should have higher priority to get the disk
access.*



*For that reason I want to propose 2 possible KIPs*



   1. *Can we somehow give less priority on log-cleaner reads when the read
   gets to the I/O scheduler? (I have to investigate) if that is possible this
   would be a clean solution.*



   1. *Monitor the “pending” logs that are candidates for compaction, with
   this we could set a very low throttle in log-cleaner (for example 10 MiB/s)
   and increase it when the metric starts reporting the log cleaner candidates
   are increasing. So we could adjust the best throttle with the minimal
   impact in the cluster leaving the disk access to the producers/consumers*


*What do you think? If you think this could be a solution I could start
working on one of them .*



*Thanks a lot!*


Message conversion and TLS

2023-02-23 Thread Sergio Daniel Troiano
Hey lads,

I was thinking about message conversions and zero copy, as we know TLS will
break the zero copy as the encrypt/decrypt happens in userland, correct?

For example  ProduceMessageConversionsPerSec  will be triggered when a
message must be converted (for example compressing at topic level).

Should we consider the TLS layer a conversion as well? My concern is we are
"hiding" this metric, so let's suppose I have my cluster using TLS (it
means zero-copy is already "broken") then I activate the compression at
topic level.
Now I will see the values in ProduceMessageConversionsPerSec  as the
conversion is happening (tracking the compression at topic level) but the
reality is it was already happening before because of the TLS, so in terms
of performance it's a kind of illusion as it seems the conversion is not
costing any extra CPU cycles.

If you think it is a good idea maybe we could include the count on the
ProduceMessageConversionsPerSec when the TLS is enabled, what do you think?

Thanks in advance.
Sergio Troiano


[Doubt] about breaking zero copy

2022-10-15 Thread Sergio Daniel Troiano
My doubt is regarding the produced batches conversions. (Breaking the zero
copy)

I am producing using Kafka streams 3.0 (same version as the Kafka cluster)

Messages are compressed on the producer.


I am seeing several ProduceConversion  per sec rate in a topic, as we know
the recompression of the batches breaks the “zero copy” which means more
resources to be used by the brokers.



Checking the source code I saw different ways of breaking “in place”
batches.


1 sourceCompression != DestinationCompression (I.e: producer using GZIP and
topic using LZ4)

2 Magic number < 2 or magic number mismatch

3 Offsets


I think this is the code snippet which triggers the method to break the
zero copy

** Code at the bottom of the email



What I suspect is for some reason the offsets are not contiguous on the
produced batches which leads me to the main doubt, what could be a scenario
when this could happen?


I tried to see this with the dump-logs sh tool but of course this is not
possible as Kafka already converted the batches.


Also I thought about transactions  could be the reason of the conversions  as
they use the IsControl batch but as I saw the IsControl batch will always
contain one record (the control), so I assume control batches will never
have other “client generated” records.



So I would appreciate if you tell me can example of the offsets not
contiguos  in a batch, in parallel I will continue my investigation as I am
intrigued about this conversions.



After this I want to write a public document about performance based on
batch conversions.


Thanks in advance


Best regards.


Sergio Troiano






-

recordsIterator.forEachRemaining { record =>
  val expectedOffset = expectedInnerOffset.getAndIncrement()
  val recordError = validateRecordCompression(batchIndex, record).orElse {
validateRecord(batch, topicPartition, record, batchIndex, now,
  timestampType, timestampDiffMaxMs, compactedTopic,
brokerTopicStats).orElse {
  if (batch.magic > RecordBatch.MAGIC_VALUE_V0 && toMagic >
RecordBatch.MAGIC_VALUE_V0) {
if (record.timestamp > maxTimestamp)
  maxTimestamp = record.timestamp

// Some older clients do not implement the V1 internal offsets
correctly.
// Historically the broker handled this by rewriting the batches rather
// than rejecting the request. We must continue this handling
here to avoid
// breaking these clients.
if (record.offset != expectedOffset)
  inPlaceAssignment = false
  }
  None
}
  }


Re: [DISCUSS] KIP-838 Simulate batching and compression

2022-05-16 Thread Sergio Daniel Troiano
Hi Divij,

First of all thanks for your time and dedication.

About point one:
You are right, the idea is to have "in real time" visibility of the way the
clients are using the service as that is translated into a lot of money
saving.
I agree with the further vision although I think we are still far away from
it :)

About the resource usage my idea is to be zero invasive, so taking a few MB
samples once every few hours will be more than enough to understand the
produced pattern, so in this case the CPU usage is only a cost for the
producer and consumer.
Worth to mention that the additional 3% extra usage while producing is
negligible compared to the gain of batching and compression but maybe that
discussion is not related to this KIP, that is a decision between the
cluster admin and the clients.

About the "auto tuning" that is a great idea, again I think it is very
ambitious for the scope of this KIP but if the core of this is properly
done then this can be used in the future.


About point two:
Below is detailed the benefits of bathing and compression :
- Reduction of network bandwidth while data is produced.
- Reduction of disk usage to store the data, less IO for read and write the
segments (supposing the message format has not to be converted)
- Reduction of network traffic while data is replicated.
- Reduction of network traffic while the data is consumer.

The script I propose will output the percentage of network traffic
reduction + the disk space saved.
- Batching will be recommended based on the parameters
$batching-window-time (ms) and $min-records-for-batching the idea is to
check the CreationTime of each batch, lets suppose we use:

batching-window-time = 300
min-records-for-batching = 30

* This means we want to check if at least we can batch together 30 records
in 300 ms, this could be in 2 batches or in 30 (one record per batch)
* If the batching is achievable then we jump the next check to simulate the
compression even if the compression is already applied as batching more
data will improve the compression ratio.
* Finally the payload ( a few MB are brought to memory in order to get its
current size, then it is
compressed and the difference is calculated.


As a side note I think if the classes are properly created this can be
reused in the future for a more "automagic" way of usage. Again I really
like the idea of allowing the cluster to configure the producers (maybe the
producer could have a parameter to allow this)

I did not enter into details about the code as I would like to know if the
idea worth it, I use this "solution" in the company I work and it saved us
a lot of money, for now we have just get the output from the dump-logs.sh
script in order to see the CreateTime and the compression type, this is
a first step but we can't yet simulate the compression.
So for now we reach our clients saying "there is a potential benefit of
cost reduction if you apply these changes in the producer"


I hope this help, please feel free to add more feedback

Best regards.
Sergio Troiano










On Mon, 16 May 2022 at 10:35, Divij Vaidya  wrote:

> Thank you for the KIP Sergio.
>
> High level thoughts:
> 1\ I understand that the idea here is to provide better visibility to the
> admins about potential improvements using compression and modifying batch
> size. I would take it a step further and say that we should be providing
> this visibility in a programmatic push based manner and make this system
> generic enough so that adding new "optimization rules" in the future is
> seamless. Perhaps, have a "diagnostic" mode in the cluster, which can be
> dynamically enabled. In such a mode, the cluster would run a set of
> "optimization" rules (at the cost of additional CPU cycles). One of such
> rules would be the compression rule you mentioned in your KIP. At the end
> of the diagnostic run, the generated report would contain a set of
> recommendations. To begin with, we can introduce this "diagnostic" as a
> one-time run by admin and later, enhance it further to be triggered
> periodically in the cluster automatically (with results being published via
> existing metric libraries). Even further down the line, this could lead to
> "auto-tuning" producer libraries based on recommendations from the server.
>
> KIP implementation specific comments/questions:
> 2\ Can you please add the algorithm that would be used to determine whether
> compression is recommended or not? I am assuming that the algorithm would
> take into account the factors impacting compression optimization such as
> CPU utilization, network bandwidth, decompression cost by the consumers
> etc.
> 3\ Can you please add the algorithm that would be used to determine whether
> batching is recommended?
>
>
> Divij Vaidya
>
>
>
> On Mon, May 1

[DISCUSS] KIP-838 Simulate batching and compression

2022-05-15 Thread Sergio Daniel Troiano
Hey guys!

I would like to start an early discussion on this:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-838+Simulate+batching+and+compression


Thanks!


Re: [VOTE] KIP-824 Allowing dumping segmentlogs limiting the batches in the output

2022-03-29 Thread Sergio Daniel Troiano
Thanks Mickael,

the steps were in the wiki page, sorry for that!!

I have just updated the wiki page,  so the voting was finished and this KIP
is adopted


Best regards.


On Tue, 29 Mar 2022 at 12:32, Mickael Maison 
wrote:

> Hi Sergio,
>
> You've obtained 3 binding votes and over 72 hours have passed so you
> can close the vote, it has passed!
>
> Edit your KIP and mark it as "Accepted" and move it into the Adopted
> KIPs table in
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
>
> Thanks,
> Mickael
>
>
>
> On Mon, Mar 28, 2022 at 10:30 AM Sergio Daniel Troiano
>  wrote:
> >
> > hi guys again,
> >
> > How does it work now? After voting, should I update/modify the KIP? So
> > basically is there any action I should do from my side?
> >
> > Thanks!
> >
> > On Sat, 26 Mar 2022 at 14:58, John Roesler  wrote:
> >
> > > Thanks for the KIP, Sergio!
> > >
> > > I’m +1 (binding)
> > >
> > > Thanks,
> > > John
> > >
> > > On Sat, Mar 26, 2022, at 03:32, David Jacot wrote:
> > > > +1 (binding). Thanks for the KIP.
> > > >
> > > > Best,
> > > > David
> > > >
> > > > Le ven. 25 mars 2022 à 07:11, Luke Chen  a écrit
> :
> > > >
> > > >> Hi Sergio,
> > > >>
> > > >> Thanks for the KIP!
> > > >> +1(binding) from me.
> > > >>
> > > >> Thank you.
> > > >> Luke
> > > >>
> > > >> On Fri, Mar 25, 2022 at 1:40 PM Sergio Daniel Troiano
> > > >>  wrote:
> > > >>
> > > >> > Hi lads,
> > > >> >
> > > >> > I would like to start the vote on this:
> > > >> >
> > > >> >
> > > >>
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-824%3A+Allowing+dumping+segmentlogs+limiting+the+batches+in+the+output
> > > >> >
> > > >> > As an extra information we are already using the patch in our
> > > company, so
> > > >> > thanks to this patch and other extra script we did (maybe I will
> start
> > > >> > another KIP later) we started saving plenty of money.
> > > >> >
> > > >> > Best regards.
> > > >> > Sergio Troiano
> > > >> >
> > > >>
> > >
>


Re: [VOTE] KIP-824 Allowing dumping segmentlogs limiting the batches in the output

2022-03-28 Thread Sergio Daniel Troiano
hi guys again,

How does it work now? After voting, should I update/modify the KIP? So
basically is there any action I should do from my side?

Thanks!

On Sat, 26 Mar 2022 at 14:58, John Roesler  wrote:

> Thanks for the KIP, Sergio!
>
> I’m +1 (binding)
>
> Thanks,
> John
>
> On Sat, Mar 26, 2022, at 03:32, David Jacot wrote:
> > +1 (binding). Thanks for the KIP.
> >
> > Best,
> > David
> >
> > Le ven. 25 mars 2022 à 07:11, Luke Chen  a écrit :
> >
> >> Hi Sergio,
> >>
> >> Thanks for the KIP!
> >> +1(binding) from me.
> >>
> >> Thank you.
> >> Luke
> >>
> >> On Fri, Mar 25, 2022 at 1:40 PM Sergio Daniel Troiano
> >>  wrote:
> >>
> >> > Hi lads,
> >> >
> >> > I would like to start the vote on this:
> >> >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-824%3A+Allowing+dumping+segmentlogs+limiting+the+batches+in+the+output
> >> >
> >> > As an extra information we are already using the patch in our
> company, so
> >> > thanks to this patch and other extra script we did (maybe I will start
> >> > another KIP later) we started saving plenty of money.
> >> >
> >> > Best regards.
> >> > Sergio Troiano
> >> >
> >>
>


Re: [DISCUSS] KIP-824 Allowing dumping segmentlogs limiting the batches in the output

2022-03-25 Thread Sergio Daniel Troiano
Hi David,

I apologize. I missed your suggestion.
By the way I like it and I have applied your suggestion.

About the rejected alternatives I have updated the KIP as well

Best regards
Sergio Troiano

On Fri, 25 Mar 2022 at 06:50, David Jacot  wrote:

> Hi Sergio,
>
> I made a suggestion a few weeks ago about the name about the parameter but
> haven’t got a response for it. Did you consider it?
>
> Do we need to update the rejected alternatives section to mention the
> alternative options discussed in this thread?
>
> Thanks,
> David
>
> Le ven. 25 mars 2022 à 03:44, Luke Chen  a écrit :
>
> > Hi Sergio,
> >
> > Thanks for asking.
> > Since it's been discussed for weeks, you can start the vote anytime.
> >
> > Thank you.
> > Luke
> >
> > On Fri, Mar 25, 2022 at 10:40 AM Sergio Troiano
> >  wrote:
> >
> > > Hey guys,
> > >
> > >
> > > What is the next step? Who decides when it is time for voting?
> > >
> > >
> > > Thanks!
> > >
> > > Sent from my iPhone
> > >
> > > > On 8 Mar 2022, at 19:57, Sergio Daniel Troiano <
> > > sergio.troi...@adevinta.com> wrote:
> > > >
> > > > 
> > > > Hi Michael,
> > > >
> > > > Yes, it's a good idea and I considered it, the main problem is the
> > > FileRecords class does not accept number of batches as a parameter, it
> > > accepts bytes instead, so if we want to do so either we redesign a core
> > > class or we create a new one.
> > > > One of the pretty things (I consider) about this change which will
> > bring
> > > a huge benefit is the change in the code is pretty small and it's on
> the
> > > final script, it does not require any deep change in a core library.
> > > >
> > > > An alternative which requires a big change as well without touching
> the
> > > FileRecords class would be accept number of batches as parameter, then
> > call
> > > the FileRecords.slice() with a small value (bytes) count the batches,
> see
> > > if we can satisfy the number of batches, if not then we call it again
> and
> > > again until we reach the amount of batches. This will require a lot of
> > code
> > > as well
> > > >
> > > > So long story short, the proposal change is quite small, it uses the
> > > current classes and has a big benefit.
> > > >
> > > > Maybe in the future we could consider the FileRecords class to
> support
> > > getting the amount of batches as parameters and we encapsulate this
> logic
> > > in the proper class (FileRecords)
> > > > What do you think?
> > > >
> > > > Thanks
> > > > Sergio
> > > > Thanks
> > > > Sergio
> > > >
> > > >> On Tue, 8 Mar 2022 at 18:32, Mickael Maison <
> mickael.mai...@gmail.com
> > >
> > > wrote:
> > > >> Hi Sergio,
> > > >>
> > > >> Thanks for the KIP. Instead of specifying the size in bytes, have
> you
> > > >> considered specifying it in terms of number of batches? I think
> it's a
> > > >> bit more user friendly than a size in raw bytes.
> > > >> For example: --num-batches: The number of batches to read from the
> log
> > > segment.
> > > >>
> > > >> Thanks,
> > > >> Mickael
> > > >>
> > > >>
> > > >> On Tue, Mar 8, 2022 at 5:27 AM Sergio Daniel Troiano
> > > >>  wrote:
> > > >> >
> > > >> > Hi Luke,
> > > >> >
> > > >> > Make sense, done!
> > > >> >
> > > >> > Thank you.
> > > >> > Sergio Troiano
> > > >> >
> > > >> > On Tue, 8 Mar 2022 at 03:02, Luke Chen  wrote:
> > > >> >
> > > >> > > Hi Sergio,
> > > >> > >
> > > >> > > > I don't want this to minimize the main feature I want to
> deploy
> > > as I
> > > >> > > think the
> > > >> > > message size limit is not as important as the limiting the
> amount
> > of
> > > >> > > batches.
> > > >> > >
> > > >> > > Agree! Let's focus on the feature of limiting the batch amounts.
> > > >> > >
> > > >> > > One more commen

[VOTE] KIP-824 Allowing dumping segmentlogs limiting the batches in the output

2022-03-24 Thread Sergio Daniel Troiano
Hi lads,

I would like to start the vote on this:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-824%3A+Allowing+dumping+segmentlogs+limiting+the+batches+in+the+output

As an extra information we are already using the patch in our company, so
thanks to this patch and other extra script we did (maybe I will start
another KIP later) we started saving plenty of money.

Best regards.
Sergio Troiano


Re: [DISCUSS] KIP-824 Allowing dumping segmentlogs limiting the batches in the output

2022-03-08 Thread Sergio Daniel Troiano
Hi Michael,

Yes, it's a good idea and I considered it, the main problem is the
*FileRecords* class does not accept number of batches as a parameter, it
accepts bytes instead, so if we want to do so either we redesign a core
class or we create a new one.
One of the pretty things (I consider) about this change which will bring a
huge benefit is the change in the code is pretty small and it's on the
final script, it does not require any deep change in a core library.

An alternative which requires a big change as well without touching the
FileRecords class would be accept number of batches as parameter, then call
the FileRecords.slice() with a small value (bytes) count the batches, see
if we can satisfy the number of batches, if not then we call it again and
again until we reach the amount of batches. This will require a lot of code
as well

So long story short, the proposal change is quite small, it uses the
current classes and has a big benefit.

Maybe in the future we could consider the FileRecords class to support
getting the amount of batches as parameters and we encapsulate this logic
in the proper class (FileRecords)
What do you think?

Thanks
Sergio
Thanks
Sergio

On Tue, 8 Mar 2022 at 18:32, Mickael Maison 
wrote:

> Hi Sergio,
>
> Thanks for the KIP. Instead of specifying the size in bytes, have you
> considered specifying it in terms of number of batches? I think it's a
> bit more user friendly than a size in raw bytes.
> For example: --num-batches: The number of batches to read from the log
> segment.
>
> Thanks,
> Mickael
>
>
> On Tue, Mar 8, 2022 at 5:27 AM Sergio Daniel Troiano
>  wrote:
> >
> > Hi Luke,
> >
> > Make sense, done!
> >
> > Thank you.
> > Sergio Troiano
> >
> > On Tue, 8 Mar 2022 at 03:02, Luke Chen  wrote:
> >
> > > Hi Sergio,
> > >
> > > > I don't want this to minimize the main feature I want to deploy as I
> > > think the
> > > message size limit is not as important as the limiting the amount of
> > > batches.
> > >
> > > Agree! Let's focus on the feature of limiting the batch amounts.
> > >
> > > One more comment to the KIP:
> > > 1. Could you put the new parameter description into the KIP proposed
> change
> > > section? That would make it much clear.
> > >
> > >
> > > Thank you.
> > > Luke
> > >
> > > On Mon, Mar 7, 2022 at 8:44 PM Sergio Daniel Troiano
> > >  wrote:
> > >
> > > > hey Luke,
> > > >
> > > > I am interested in expanding the KIP scope but I am a bit concerned
> this
> > > > could create a lot of noise and confusion as they look like very
> similar
> > > > parameters, I agree this is a small change, so I think if I do it
> > > properly
> > > > it should not be a problem at all, I just will need a couple more of
> days
> > > > as I want to create the proper tests as well.
> > > >
> > > > I have a doubt about editing the KIP, I mean should I add this as a
> new
> > > > feature as well?, should I describe this as a side effect finding? I
> > > don't
> > > > want this to minimize the main feature I want to deploy as I think
> the
> > > > message size limit is not as important as the limiting the amount of
> > > > batches.
> > > >
> > > > It is up to you, if you guys consider we must add this in this KIP
> then I
> > > > will be happy to do it. 
> > > >
> > > > Best regards.
> > > > Sergio Troiano
> > > >
> > > > On Mon, 7 Mar 2022 at 02:01, Luke Chen  wrote:
> > > >
> > > > > Hi Sergio,
> > > > >
> > > > > Thanks for your explanation.
> > > > > Make sense to me.
> > > > >
> > > > > > Only interesting thing that I have just found is
> *max-message-size
> > > *is
> > > > > not
> > > > > used while dump logs are requested, instead it is used by dumpIndex
> > > > >
> > > > > Are you interested in expanding the scope of this KIP to include
> the
> > > > > *max-message-size* in dumping logs?
> > > > > I think it's good because it will also be a small change, and no
> need
> > > to
> > > > go
> > > > > through another KIP discussing/voting process. But I'm fine if you
> want
> > > > to
> > > > > keep this KIP as is, and create another JIRA ticket for future
> work.
> > > > >
> > > > > Thank you.
&g

Re: [DISCUSS] KIP-824 Allowing dumping segmentlogs limiting the batches in the output

2022-03-07 Thread Sergio Daniel Troiano
Hi Luke,

Make sense, done!

Thank you.
Sergio Troiano

On Tue, 8 Mar 2022 at 03:02, Luke Chen  wrote:

> Hi Sergio,
>
> > I don't want this to minimize the main feature I want to deploy as I
> think the
> message size limit is not as important as the limiting the amount of
> batches.
>
> Agree! Let's focus on the feature of limiting the batch amounts.
>
> One more comment to the KIP:
> 1. Could you put the new parameter description into the KIP proposed change
> section? That would make it much clear.
>
>
> Thank you.
> Luke
>
> On Mon, Mar 7, 2022 at 8:44 PM Sergio Daniel Troiano
>  wrote:
>
> > hey Luke,
> >
> > I am interested in expanding the KIP scope but I am a bit concerned this
> > could create a lot of noise and confusion as they look like very similar
> > parameters, I agree this is a small change, so I think if I do it
> properly
> > it should not be a problem at all, I just will need a couple more of days
> > as I want to create the proper tests as well.
> >
> > I have a doubt about editing the KIP, I mean should I add this as a new
> > feature as well?, should I describe this as a side effect finding? I
> don't
> > want this to minimize the main feature I want to deploy as I think the
> > message size limit is not as important as the limiting the amount of
> > batches.
> >
> > It is up to you, if you guys consider we must add this in this KIP then I
> > will be happy to do it. 
> >
> > Best regards.
> > Sergio Troiano
> >
> > On Mon, 7 Mar 2022 at 02:01, Luke Chen  wrote:
> >
> > > Hi Sergio,
> > >
> > > Thanks for your explanation.
> > > Make sense to me.
> > >
> > > > Only interesting thing that I have just found is *max-message-size
> *is
> > > not
> > > used while dump logs are requested, instead it is used by dumpIndex
> > >
> > > Are you interested in expanding the scope of this KIP to include the
> > > *max-message-size* in dumping logs?
> > > I think it's good because it will also be a small change, and no need
> to
> > go
> > > through another KIP discussing/voting process. But I'm fine if you want
> > to
> > > keep this KIP as is, and create another JIRA ticket for future work.
> > >
> > > Thank you.
> > > Luke
> > >
> > > On Mon, Mar 7, 2022 at 6:02 AM Sergio Daniel Troiano
> > >  wrote:
> > >
> > > > hey Luke,
> > > >
> > > > Let me answer them:
> > > > 1. If the *max-batches-size* is too small that results in no records
> > > > output, will we output any information to the user?
> > > >
> > > > If the  *max-batches-size*is even smaller than the first batch then
> > there
> > > > won't be any output, this is handled by FileRecords class, I think
> this
> > > is
> > > > correct as this is the expected behaviour.
> > > >
> > > > 2. After your explanation, I guess the use of *max-batches-size*
> won't
> > > > conflict with *max-message-size*, right?
> > > >
> > > > Only interesting thing that I have just found is *max-message-size
> *is
> > > not
> > > > used while dump logs are requested, instead it is used by dumpIndex
> > > > <
> > > >
> > >
> >
> https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/tools/DumpLogSegments.scala#L150
> > > > >
> > > > so,
> > > > this feature is not working for dumping logs, even though I checked
> if
> > > > there is a unit test for this and there is not any. Maybe we could
> > > create a
> > > > ticket for this?
> > > >
> > > > Regards.
> > > >
> > > >
> > > > On Sat, 5 Mar 2022 at 10:36, Luke Chen  wrote:
> > > >
> > > > > Hi Sergio,
> > > > >
> > > > > Thanks for the explanation! Very clear!
> > > > > I think we should put this example and explanation into KIP.
> > > > >
> > > > > Other comments:
> > > > > 1. If the *max-batches-size* is too small that results in no
> records
> > > > > output, will we output any information to the user?
> > > > > 2. After your explanation, I guess the use of *max-batches-size*
> > won't
> > > > > conflict with *max-message-size*, right?
> > > > > That is, user can set the 2 arguments at the same time. Is that
> 

Re: [DISCUSS] KIP-824 Allowing dumping segmentlogs limiting the batches in the output

2022-03-07 Thread Sergio Daniel Troiano
hey Luke,

I am interested in expanding the KIP scope but I am a bit concerned this
could create a lot of noise and confusion as they look like very similar
parameters, I agree this is a small change, so I think if I do it properly
it should not be a problem at all, I just will need a couple more of days
as I want to create the proper tests as well.

I have a doubt about editing the KIP, I mean should I add this as a new
feature as well?, should I describe this as a side effect finding? I don't
want this to minimize the main feature I want to deploy as I think the
message size limit is not as important as the limiting the amount of
batches.

It is up to you, if you guys consider we must add this in this KIP then I
will be happy to do it. 

Best regards.
Sergio Troiano

On Mon, 7 Mar 2022 at 02:01, Luke Chen  wrote:

> Hi Sergio,
>
> Thanks for your explanation.
> Make sense to me.
>
> > Only interesting thing that I have just found is *max-message-size *is
> not
> used while dump logs are requested, instead it is used by dumpIndex
>
> Are you interested in expanding the scope of this KIP to include the
> *max-message-size* in dumping logs?
> I think it's good because it will also be a small change, and no need to go
> through another KIP discussing/voting process. But I'm fine if you want to
> keep this KIP as is, and create another JIRA ticket for future work.
>
> Thank you.
> Luke
>
> On Mon, Mar 7, 2022 at 6:02 AM Sergio Daniel Troiano
>  wrote:
>
> > hey Luke,
> >
> > Let me answer them:
> > 1. If the *max-batches-size* is too small that results in no records
> > output, will we output any information to the user?
> >
> > If the  *max-batches-size*is even smaller than the first batch then there
> > won't be any output, this is handled by FileRecords class, I think this
> is
> > correct as this is the expected behaviour.
> >
> > 2. After your explanation, I guess the use of *max-batches-size* won't
> > conflict with *max-message-size*, right?
> >
> > Only interesting thing that I have just found is *max-message-size *is
> not
> > used while dump logs are requested, instead it is used by dumpIndex
> > <
> >
> https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/tools/DumpLogSegments.scala#L150
> > >
> > so,
> > this feature is not working for dumping logs, even though I checked if
> > there is a unit test for this and there is not any. Maybe we could
> create a
> > ticket for this?
> >
> > Regards.
> >
> >
> > On Sat, 5 Mar 2022 at 10:36, Luke Chen  wrote:
> >
> > > Hi Sergio,
> > >
> > > Thanks for the explanation! Very clear!
> > > I think we should put this example and explanation into KIP.
> > >
> > > Other comments:
> > > 1. If the *max-batches-size* is too small that results in no records
> > > output, will we output any information to the user?
> > > 2. After your explanation, I guess the use of *max-batches-size* won't
> > > conflict with *max-message-size*, right?
> > > That is, user can set the 2 arguments at the same time. Is that
> correct?
> > >
> > > Thank you.
> > > Luke
> > >
> > > Thank you.
> > > Luke
> > >
> > > On Sat, Mar 5, 2022 at 4:47 PM Sergio Daniel Troiano
> > >  wrote:
> > >
> > > > hey Luke,
> > > >
> > > > thanks for the interest, it is a good question, please let me explain
> > > you:
> > > >
> > > > *max-message-size *a filter for the size of each batch, so for
> example
> > if
> > > > Iset --max-message-size 1000 bytes and my segment log has 300
> batches,
> > > 150
> > > > of them has a size of 500 bytes  and the other 150 has a size of 2000
> > > bytes
> > > > then the script will skip the las 150 ones as each batch is heavier
> > than
> > > > the limit.
> > > >
> > > > In the other hand following the same example above with
> > *max-batches-size
> > > > *set
> > > > to 1000 bytes it will only print out the first 2 batches (500 bytes
> > each)
> > > > and stop, This will avoid reading the whole file
> > > >
> > > >
> > > > Also if all of them are smaller than 1000 bytes it will end up
> printing
> > > out
> > > > all the batches.
> > > > The idea of my change is to limit the *amount* of batches no matter
> > their
> > > > size.
> > > >
> > > > I hope this reply helps.
> > &

Re: [DISCUSS] KIP-824 Allowing dumping segmentlogs limiting the batches in the output

2022-03-06 Thread Sergio Daniel Troiano
hey Luke,

Let me answer them:
1. If the *max-batches-size* is too small that results in no records
output, will we output any information to the user?

If the  *max-batches-size*is even smaller than the first batch then there
won't be any output, this is handled by FileRecords class, I think this is
correct as this is the expected behaviour.

2. After your explanation, I guess the use of *max-batches-size* won't
conflict with *max-message-size*, right?

Only interesting thing that I have just found is *max-message-size *is not
used while dump logs are requested, instead it is used by dumpIndex
<https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/tools/DumpLogSegments.scala#L150>
so,
this feature is not working for dumping logs, even though I checked if
there is a unit test for this and there is not any. Maybe we could create a
ticket for this?

Regards.


On Sat, 5 Mar 2022 at 10:36, Luke Chen  wrote:

> Hi Sergio,
>
> Thanks for the explanation! Very clear!
> I think we should put this example and explanation into KIP.
>
> Other comments:
> 1. If the *max-batches-size* is too small that results in no records
> output, will we output any information to the user?
> 2. After your explanation, I guess the use of *max-batches-size* won't
> conflict with *max-message-size*, right?
> That is, user can set the 2 arguments at the same time. Is that correct?
>
> Thank you.
> Luke
>
> Thank you.
> Luke
>
> On Sat, Mar 5, 2022 at 4:47 PM Sergio Daniel Troiano
>  wrote:
>
> > hey Luke,
> >
> > thanks for the interest, it is a good question, please let me explain
> you:
> >
> > *max-message-size *a filter for the size of each batch, so for example if
> > Iset --max-message-size 1000 bytes and my segment log has 300 batches,
> 150
> > of them has a size of 500 bytes  and the other 150 has a size of 2000
> bytes
> > then the script will skip the las 150 ones as each batch is heavier than
> > the limit.
> >
> > In the other hand following the same example above with *max-batches-size
> > *set
> > to 1000 bytes it will only print out the first 2 batches (500 bytes each)
> > and stop, This will avoid reading the whole file
> >
> >
> > Also if all of them are smaller than 1000 bytes it will end up printing
> out
> > all the batches.
> > The idea of my change is to limit the *amount* of batches no matter their
> > size.
> >
> > I hope this reply helps.
> > Best regards.
> >
> > On Sat, 5 Mar 2022 at 08:00, Luke Chen  wrote:
> >
> > > Hi Sergio,
> > >
> > > Thanks for the KIP!
> > >
> > > One question:
> > > I saw there's a `max-message-size` argument that seems to do the same
> > thing
> > > as you want.
> > > Could you help explain what's the difference between `max-message-size`
> > and
> > > `max-batches-size`?
> > >
> > > Thank you.
> > > Luke
> > >
> > > On Sat, Mar 5, 2022 at 3:21 AM Kirk True 
> wrote:
> > >
> > > > Hi Sergio,
> > > >
> > > > Thanks for the KIP. I don't know anything about the log segment
> > > internals,
> > > > but the logic and implementation seem sound.
> > > >
> > > > Three questions:
> > > >  1. Since the --max-batches-size unit is bytes, does it matter if
> that
> > > > size doesn't align to a record boundary?
> > > >  2. Can you add a check to make sure that --max-batches-size doesn't
> > > allow
> > > > the user to pass in a negative number?
> > > >  3. Can you add/update any unit tests related to the DumpLogSegments
> > > > arguments?
> > > > Thanks,
> > > > Kirk
> > > >
> > > > On Thu, Mar 3, 2022, at 1:32 PM, Sergio Daniel Troiano wrote:
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-824%3A+Allowing+dumping+segmentlogs+limiting+the+batches+in+the+output
> > > > >
> > > >
> > >
> >
>


Re: [DISCUSS] KIP-824 Allowing dumping segmentlogs limiting the batches in the output

2022-03-06 Thread Sergio Daniel Troiano
hey Kirk,

Thanks for the questions, please let me answer them:

1. This is handled by the *FileRecords class, *now the open uses the slice
<https://github.com/apache/kafka/pull/11842/files#diff-27b1f2e66462b1f8ef9b21da31ae9bdc76576b590458cd92d3b7d8b4042a2e10R253>
 which takes care of the end
<https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/tools/DumpLogSegments.scala#L309>
bytes . There will be some trailing bytes which is totally fine, you can
see here it was expected even here
<https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/tools/DumpLogSegments.scala#L309>
before my proposal, I added an extra conditional to avoid printing the
warning in the main script here
<https://github.com/apache/kafka/pull/11842/files#diff-27b1f2e66462b1f8ef9b21da31ae9bdc76576b590458cd92d3b7d8b4042a2e10R310>
.

2. *FileRecords *class already checks the value
<https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java#L167>,
we do the same, similar case when we set the maxMessageSizeOpt
<https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/tools/DumpLogSegments.scala#L428>


3. I have just added the unit test 




On Fri, 4 Mar 2022 at 20:21, Kirk True  wrote:

> Hi Sergio,
>
> Thanks for the KIP. I don't know anything about the log segment internals,
> but the logic and implementation seem sound.
>
> Three questions:
>  1. Since the --max-batches-size unit is bytes, does it matter if that
> size doesn't align to a record boundary?
>  2. Can you add a check to make sure that --max-batches-size doesn't allow
> the user to pass in a negative number?
>  3. Can you add/update any unit tests related to the DumpLogSegments
> arguments?
> Thanks,
> Kirk
>
> On Thu, Mar 3, 2022, at 1:32 PM, Sergio Daniel Troiano wrote:
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-824%3A+Allowing+dumping+segmentlogs+limiting+the+batches+in+the+output
> >
>


Re: [DISCUSS] KIP-824 Allowing dumping segmentlogs limiting the batches in the output

2022-03-05 Thread Sergio Daniel Troiano
hey Luke,

thanks for the interest, it is a good question, please let me explain you:

*max-message-size *a filter for the size of each batch, so for example if
Iset --max-message-size 1000 bytes and my segment log has 300 batches, 150
of them has a size of 500 bytes  and the other 150 has a size of 2000 bytes
then the script will skip the las 150 ones as each batch is heavier than
the limit.

In the other hand following the same example above with *max-batches-size  *set
to 1000 bytes it will only print out the first 2 batches (500 bytes each)
and stop, This will avoid reading the whole file


Also if all of them are smaller than 1000 bytes it will end up printing out
all the batches.
The idea of my change is to limit the *amount* of batches no matter their
size.

I hope this reply helps.
Best regards.

On Sat, 5 Mar 2022 at 08:00, Luke Chen  wrote:

> Hi Sergio,
>
> Thanks for the KIP!
>
> One question:
> I saw there's a `max-message-size` argument that seems to do the same thing
> as you want.
> Could you help explain what's the difference between `max-message-size` and
> `max-batches-size`?
>
> Thank you.
> Luke
>
> On Sat, Mar 5, 2022 at 3:21 AM Kirk True  wrote:
>
> > Hi Sergio,
> >
> > Thanks for the KIP. I don't know anything about the log segment
> internals,
> > but the logic and implementation seem sound.
> >
> > Three questions:
> >  1. Since the --max-batches-size unit is bytes, does it matter if that
> > size doesn't align to a record boundary?
> >  2. Can you add a check to make sure that --max-batches-size doesn't
> allow
> > the user to pass in a negative number?
> >  3. Can you add/update any unit tests related to the DumpLogSegments
> > arguments?
> > Thanks,
> > Kirk
> >
> > On Thu, Mar 3, 2022, at 1:32 PM, Sergio Daniel Troiano wrote:
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-824%3A+Allowing+dumping+segmentlogs+limiting+the+batches+in+the+output
> > >
> >
>


[DISCUSS] KIP-824 Allowing dumping segmentlogs limiting the batches in the output

2022-03-03 Thread Sergio Daniel Troiano
https://cwiki.apache.org/confluence/display/KAFKA/KIP-824%3A+Allowing+dumping+segmentlogs+limiting+the+batches+in+the+output


[DISCUSS] KIP-824 Allowing dumping segmentlogs limiting the batches in the output

2022-03-03 Thread Sergio Daniel Troiano
https://cwiki.apache.org/confluence/display/KAFKA/KIP-824%3A+Allowing+dumping+segmentlogs+limiting+the+batches+in+the+output