[ 
https://issues.apache.org/jira/browse/KAFKA-8270?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jiangtao Liu updated KAFKA-8270:
--------------------------------
    Description: 
What's the issue?
{quote}There were log segments, which can not be deleted over configured 
retention hours.
{quote}
What are impacts? 
{quote} # Log space keep in increasing and finally cause space shortage.
 # There are lots of log segment rolled with a small size. e.g log segment may 
be only 50mb, not the expected 1gb.
 # Kafka stream or client may experience missing data.{quote}
How to reproduce it?
 # 
{quote}Configure message.timestamp.type as CreateTime.
{quote}
 # 
{quote}Reset Kafka client's system clock time a future time, e.g 03/04/*2025*, 
3:25:52 PM [GMT-08:00|https://www.epochconverter.com/timezones?q=1741130752]
{quote}

What's Next?
 # 
{quote}Go to check the timestamp in log time index and record (e.g 
00000000035957300794.log|timeindex). You should see all the value of the 
timestamp is messed up with future time after `03/04/*2025*, 3:25:52 PM 
[GMT-08:00|https://www.epochconverter.com/timezones?q=1741130752]`.  
{quote}
 # 
{quote}You may also see the new rolled log segment's size smaller than the 
configured log segment size).
{quote}
 # 
{quote}Log segment (00000000035957300794.log|timeindex) and any of rolled log 
segments will not be deleted.
{quote}

 

What's the particular logic to cause this issue?
 # 
{quote}private def deletableSegments(predicate: (LogSegment, 
Option[LogSegment]) => 
Boolean)|[https://github.com/apache/kafka/blob/1.1/core/src/main/scala/kafka/log/Log.scala#L1227]]
 will always return empty deletable log segments.
{quote}
 

  was:
What's the issue?
{quote}There were log segments, which can not be deleted over configured 
retention hours.
{quote}
What are impacts? 
{quote} # Log space keep in increasing and finally cause space shortage.
 # There are lots of log segment rolled with a small size. e.g log segment may 
be only 50mb, not the expected 1gb.
 # Kafka stream or client may experience missing data.{quote}
How to reproduce it?
 # 
{quote}Configure message.timestamp.type as CreateTime.
{quote}
 # 
{quote}Reset Kafka client's system clock time a future time, e.g 03/04/*2025*, 
3:25:52 PM [GMT-08:00|https://www.epochconverter.com/timezones?q=1741130752]
{quote}

What's Next?
 # 
{quote}Go to check the timestamp in log time index and record (e.g 
00000000035957300794.log|timeindex). You should see all the value of the 
timestamp is messed up with future time after `03/04/*2025*, 3:25:52 PM 
[GMT-08:00|https://www.epochconverter.com/timezones?q=1741130752]`.  
{quote}
 # 
{quote}You may also see the new rolled log segment's size smaller than the 
configured log segment size).
{quote}
 # 
{quote}Log segment (00000000035957300794.log|timeindex) and any of rolled log 
segments will not be deleted.
{quote}

 

What's the particular logic to cause this issue?
 # 
{quote}private def deletableSegments(predicate: (LogSegment, 
Option[LogSegment]) => 
Boolean)|[https://github.com/apache/kafka/blob/1.1/core/src/main/scala/kafka/log/Log.scala#L1227]]
 will always return empty deletable log segments.
{quote}


> Kafka timestamp-based retention policy is not working when Kafka client's 
> time is not reliable.
> -----------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-8270
>                 URL: https://issues.apache.org/jira/browse/KAFKA-8270
>             Project: Kafka
>          Issue Type: Bug
>          Components: log, log cleaner, logging
>    Affects Versions: 1.1.1
>            Reporter: Jiangtao Liu
>            Priority: Major
>              Labels: storage
>         Attachments: Screen Shot 2019-04-20 at 10.57.59 PM.png
>
>
> What's the issue?
> {quote}There were log segments, which can not be deleted over configured 
> retention hours.
> {quote}
> What are impacts? 
> {quote} # Log space keep in increasing and finally cause space shortage.
>  # There are lots of log segment rolled with a small size. e.g log segment 
> may be only 50mb, not the expected 1gb.
>  # Kafka stream or client may experience missing data.{quote}
> How to reproduce it?
>  # 
> {quote}Configure message.timestamp.type as CreateTime.
> {quote}
>  # 
> {quote}Reset Kafka client's system clock time a future time, e.g 
> 03/04/*2025*, 3:25:52 PM 
> [GMT-08:00|https://www.epochconverter.com/timezones?q=1741130752]
> {quote}
> What's Next?
>  # 
> {quote}Go to check the timestamp in log time index and record (e.g 
> 00000000035957300794.log|timeindex). You should see all the value of the 
> timestamp is messed up with future time after `03/04/*2025*, 3:25:52 PM 
> [GMT-08:00|https://www.epochconverter.com/timezones?q=1741130752]`.  
> {quote}
>  # 
> {quote}You may also see the new rolled log segment's size smaller than the 
> configured log segment size).
> {quote}
>  # 
> {quote}Log segment (00000000035957300794.log|timeindex) and any of rolled log 
> segments will not be deleted.
> {quote}
>  
> What's the particular logic to cause this issue?
>  # 
> {quote}private def deletableSegments(predicate: (LogSegment, 
> Option[LogSegment]) => 
> Boolean)|[https://github.com/apache/kafka/blob/1.1/core/src/main/scala/kafka/log/Log.scala#L1227]]
>  will always return empty deletable log segments.
> {quote}
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to