Hi all, I'm working on a client library for kafka, and I can't seem to find
the difference between truncation of a RecordBatch due to log compaction
and truncation of a RecordBatch due to the MaxBytes requested by the
reader, is there a flag set in the message that I can refere
Hi Guozhang,
Thanks very much, this was really useful.
Jamie
-Original Message-
From: Guozhang Wang
To: users
Sent: Tue, 23 Jul 2019 2:40
Subject: Re: Truncation
Hi Jamie,
The most relevant materials I can think of would be in KIP-101:
https://cwiki.apache.org/confluence/display
Hi Jamie,
The most relevant materials I can think of would be in KIP-101:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-101+-+Alter+Replication+Protocol+to+use+Leader+Epoch+rather+than+High+Watermark+for+Truncation
Although it is a bit out-dated it still contains most significant design
Hi All,
Could someone confirm what truncation happens when a partition changes from a
follower to a leader and why?
Any help would be greatly appreciated.
Many Thanks,
Jamie
Hi there!
We are running kafka 0.11.0 with 0.10.0 message format configured for a
topic
The topic has 1 partition + 3 replicas, unclean.leader.election.enable is
set to false.
We have reasons to believe that an old partition leader did not truncate
its dirty log tail
before syncing with new leade
replication configuration (3 replicas,
min.insync.replicas=2, enable.unclean.leader.election=false). We saw the
'Halting because log truncation is not allowed’ fatal error happened right
after a ISR change happened(ISR shrunk first and expanded immediately). My
suspicious is that the fol
rg/jira/browse/KAFKA-1211
> <https://issues.apache.org/jira/browse/KAFKA-1211>> which is being addressed
> in KIP-101
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-101+-+Alter+Replication+Protocol+to+use+Leader+Generation+rather+than+High+Watermark+for+Truncation
>
&
ache.org/jira/browse/KAFKA-1211> which is being addressed
in KIP-101
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-101+-+Alter+Replication+Protocol+to+use+Leader+Generation+rather+than+High+Watermark+for+Truncation>.
If you can reproduce in a more generally scenario we would be very
me.
>
> [2016-12-16 16:12:47,085] FATAL [ReplicaFetcherThread-0-3], Halting because
> log truncation is not allowed for topic __consumer_offsets, Current leader
> 3's latest offset 5910081 is less than replica 1's latest offset 5910082
> (kafka.server.ReplicaFe
Hi,
We saw the following FATAL error in 2 of our brokers (3 in total, the active
controller doesn’t have this) and they crashed in the same time.
[2016-12-16 16:12:47,085] FATAL [ReplicaFetcherThread-0-3], Halting because log
truncation is not allowed for topic __consumer_offsets, Current
data loss required!)
> >
> > Anyone else have suggestions? **Brokers halting due to a simple network
> hiccup is, shall we say, not good!**
> >
> > -Peter
> >
> >
> >> On Jun 25, 2016, at 12:28 PM, Morellato, Wanny <
> wanny.morell...@concur.com> wr
t; -Peter
>
>
>> On Jun 25, 2016, at 12:28 PM, Morellato, Wanny
>> wrote:
>>
>> Hi all,
>>
>> My kafka brokers (0.9.0.1) are refusing to restart and they return the
>> following error
>>
>> Halting because log truncation is n
tart and they return the
> following error
>
> Halting because log truncation is not allowed for topic __consumer_offsets,
> Current leader 11's latest offset 79445540 is less than replica 13's latest
> offset 79445565 (kafka.server.ReplicaFetcherThread)
>
> Deleting
Hi all,
My kafka brokers (0.9.0.1) are refusing to restart and they return the
following error
Halting because log truncation is not allowed for topic __consumer_offsets,
Current leader 11's latest offset 79445540 is less than replica 13's latest
offse
ge:
>> [2016-02-25 00:29:39,236] FATAL [ReplicaFetcherThread-0-1], Halting
>> because log truncation is not allowed for topic test, Current leader 1's
>> latest offset 0 is less than replica 2's latest offset 151
>> (kafka.server.ReplicaFetcherThread)
>>
>> I
ot; wrote:
>Hi,
>
>I ran into a scenario where one of my brokers would continually shutdown, with
>the error message:
>[2016-02-25 00:29:39,236] FATAL [ReplicaFetcherThread-0-1], Halting because
>log truncation is not allowed for topic test, Current leader 1's latest offset
&
ually shutdown,
> with the error message:
> [2016-02-25 00:29:39,236] FATAL [ReplicaFetcherThread-0-1], Halting
> because log truncation is not allowed for topic test, Current leader 1's
> latest offset 0 is less than replica 2's latest offset 151
> (kafka.server.ReplicaFetch
Hi,
I ran into a scenario where one of my brokers would continually shutdown, with
the error message:
[2016-02-25 00:29:39,236] FATAL [ReplicaFetcherThread-0-1], Halting because log
truncation is not allowed for topic test, Current leader 1's latest offset 0 is
less than replica 2
18 matches
Mail list logo