Thunder, thanks for the detailed info.  I can confirm that our incident had
the same (or similar) sequence of messages, when the first upgraded broker
restarted (after having undergone an unclean shutdown).

I think it makes sense at this point, to file a jira issue to track it.
 (Could mostly just copy paste the details in this thread).  I can then add
my notes to the ticket (or maybe file a separate one).

My main question at this point, is whether the issue is related
specifically to an unclean shutdown during an upgrade to 0.8.2.1 (or if
it's just unclean shutdowns in general with 0.8.1.1+ not recovering
properly).  Any thoughts jiangjie?

Jason

On Mon, Apr 6, 2015 at 7:30 PM, Thunder Stumpges <tstump...@ntent.com>
wrote:

> Hi all,
>
> I have collected our logs and paired them down. Jiangjie, I sent you more
> complete logs directly to your mailto from the list (minus the .INVALID).
> Please let me know if you did not get them. I cannot post them here in
> public.
>
> For others, and to continue the conversation, during the initial shutdown
> we failed a clean shutdown due to one corrupt topic. The message in
> server.log was:
>
> [2015-03-31 10:21:46,250] INFO [Kafka Server 6], Remaining partitions to
> move: [__samza_checkpoint_ver_1_for_usersessions_1,0]
> (kafka.server.KafkaServer)
> [2015-03-31 10:21:46,250] INFO [Kafka Server 6], Error code from
> controller: 0 (kafka.server.KafkaServer)
>
> And related message in state-change.log:
>
> [2015-03-31 10:21:42,622] TRACE Controller 6 epoch 23 started leader
> election for partition [__samza_checkpoint_ver_1_for_usersessions_1,0]
> (state.change.logger)
> [2015-03-31 10:21:42,623] ERROR Controller 6 epoch 23 encountered error
> while electing leader for partition
> [__samza_checkpoint_ver_1_for_usersessions_1,0] due to: LeaderAndIsr
> information doesn't exist for partition
> [__samza_checkpoint_ver_1_for_usersessions_1,0] in OnlinePartition state.
> (state.change.logger)
> [2015-03-31 10:21:42,623] TRACE Controller 6 epoch 23 received response
> correlationId 2360 for a request sent to broker id:8,host:xxxxxxx,port:9092
> (state.change.logger)
> [2015-03-31 10:21:42,623] ERROR Controller 6 epoch 23 initiated state
> change for partition [__samza_checkpoint_ver_1_for_usersessions_1,0] from
> OnlinePartition to OnlinePartition failed (state.change.logger)
> kafka.common.StateChangeFailedException: encountered error while electing
> leader for partition [__samza_checkpoint_ver_1_for_usersessions_1,0] due
> to: LeaderAndIsr information doesn't exist for partition
> [__samza_checkpoint_ver_1_for_usersessions_1,0] in OnlinePartition state.
>         at
> kafka.controller.PartitionStateMachine.electLeaderForPartition(PartitionStateMachine.scala:360)
>         at
> kafka.controller.PartitionStateMachine.kafka$controller$PartitionStateMachine$$handleStateChange(PartitionStateMachine.scala:187)
>         at
> kafka.controller.PartitionStateMachine$$anonfun$handleStateChanges$2.apply(PartitionStateMachine.scala:125)
>         at
> kafka.controller.PartitionStateMachine$$anonfun$handleStateChanges$2.apply(PartitionStateMachine.scala:124)
>         at scala.collection.immutable.Set$Set1.foreach(Set.scala:86)
>         at
> kafka.controller.PartitionStateMachine.handleStateChanges(PartitionStateMachine.scala:124)
>         at
> kafka.controller.KafkaController$$anonfun$shutdownBroker$3$$anonfun$apply$1$$anonfun$apply$mcV$sp$3.apply(KafkaController.scala:257)
>         at
> kafka.controller.KafkaController$$anonfun$shutdownBroker$3$$anonfun$apply$1$$anonfun$apply$mcV$sp$3.apply(KafkaController.scala:253)
>         at scala.Option.foreach(Option.scala:197)
>         at
> kafka.controller.KafkaController$$anonfun$shutdownBroker$3$$anonfun$apply$1.apply$mcV$sp(KafkaController.scala:253)
>         at
> kafka.controller.KafkaController$$anonfun$shutdownBroker$3$$anonfun$apply$1.apply(KafkaController.scala:253)
>         at
> kafka.controller.KafkaController$$anonfun$shutdownBroker$3$$anonfun$apply$1.apply(KafkaController.scala:253)
>         at kafka.utils.Utils$.inLock(Utils.scala:538)
>         at
> kafka.controller.KafkaController$$anonfun$shutdownBroker$3.apply(KafkaController.scala:252)
>         at
> kafka.controller.KafkaController$$anonfun$shutdownBroker$3.apply(KafkaController.scala:249)
>         at
> scala.collection.immutable.HashSet$HashSet1.foreach(HashSet.scala:130)
>         at
> scala.collection.immutable.HashSet$HashTrieSet.foreach(HashSet.scala:275)
>         at
> scala.collection.immutable.HashSet$HashTrieSet.foreach(HashSet.scala:275)
>         at
> kafka.controller.KafkaController.shutdownBroker(KafkaController.scala:249)
>         at
> kafka.server.KafkaApis.handleControlledShutdownRequest(KafkaApis.scala:264)
>         at kafka.server.KafkaApis.handle(KafkaApis.scala:192)
>         at
> kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:42)
>         at java.lang.Thread.run(Thread.java:744)
> Caused by: kafka.common.StateChangeFailedException: LeaderAndIsr
> information doesn't exist for partition
> [__samza_checkpoint_ver_1_for_usersessions_1,0] in OnlinePartition state
>         at
> kafka.controller.PartitionStateMachine.getLeaderIsrAndEpochOrThrowException(PartitionStateMachine.scala:391)
>         at
> kafka.controller.PartitionStateMachine.electLeaderForPartition(PartitionStateMachine.scala:324)
>         ... 22 more
>
>
> Then when starting up, we got many angry messages like these:
>
> [2015-03-31 10:24:10,960] WARN [Replica Manager on Broker 6]: Fetch
> request with correlation id 159883938 from client ReplicaFetcherThread-0-6
> on partition [__samza_checkpoint_ver_1_for_usersessions_1,0] failed due to
> Partition [__samza_checkpoint_ver_1_for_usersessions_1,0] doesn't exist on
> 6 (kafka.server.ReplicaManager) [2015-03-31 10:24:10,989] WARN [Replica
> Manager on Broker 6]: While recording the follower position, the partition
> [__samza_checkpoint_ver_1_for_usersessions_1,0] hasn't been created, skip
> updating leader HW (kafka.server.ReplicaManager)
> [2015-03-31 10:24:10,989] WARN [Replica Manager on Broker 6]: While
> recording the follower position, the partition
> [__samza_checkpoint_ver_1_for_usersessions_1,0] hasn't been created, skip
> updating leader HW (kafka.server.ReplicaManager)
>
> Then they change to these:
>
> [2015-03-31 10:24:12,008] WARN [Replica Manager on Broker 6]: Fetch
> request with correlation id 159884643 from client ReplicaFetcherThread-0-6
> on partition [__samza_checkpoint_ver_1_for_usersessions_1,0] failed due to
> Partition [__samza_checkpoint_ver_1_for_usersessions_1,0] doesn't exist on
> 6 (kafka.server.ReplicaManager)
> [2015-03-31 10:24:12,009] WARN [Replica Manager on Broker 6]: While
> recording the follower position, the partition
> [__samza_checkpoint_ver_1_for_usersessions_1,0] hasn't been created, skip
> updating leader HW (kafka.server.ReplicaManager)
> [2015-03-31 10:24:12,009] WARN [Replica Manager on Broker 6]: Fetch
> request with correlation id 158861852 from client ReplicaFetcherThread-0-6
> on partition [__samza_checkpoint_ver_1_for_usersessions_1,0] failed due to
> Partition [__samza_checkpoint_ver_1_for_usersessions_1,0] doesn't exist on
> 6 (kafka.server.ReplicaManager)
> [2015-03-31 10:24:12,009] WARN [Replica Manager on Broker 6]: While
> recording the follower position, the partition
> [__samza_checkpoint_ver_1_for_usersessions_1,0] hasn't been created, skip
> updating leader HW (kafka.server.ReplicaManager)
>
> And a bunch like this:
>
>
> [2015-03-31 10:24:12,019] ERROR [KafkaApi-6] error when handling request
> Name: FetchRequest; Version: 0; CorrelationId: 158861855; ClientId:
> ReplicaFetcherThread-0-6; ReplicaId: 8; MaxWait: 500 ms; MinBytes: 1 bytes;
> RequestInfo: [__samza_checkpoint_ver_1_for_usersessions_1,0] ->
> PartitionFetchInfo(4461,1048576) (kafka.server.KafkaApis)
> kafka.common.NotAssignedReplicaException: Leader 6 failed to record
> follower 8's position -1 since the replica is not recognized to be one of
> the assigned replicas  for partition
> [__samza_checkpoint_ver_1_for_usersessions_1,0]
>         at
> kafka.server.ReplicaManager.updateReplicaLEOAndPartitionHW(ReplicaManager.scala:574)
>         at
> kafka.server.KafkaApis$$anonfun$recordFollowerLogEndOffsets$2.apply(KafkaApis.scala:388)
>         at
> kafka.server.KafkaApis$$anonfun$recordFollowerLogEndOffsets$2.apply(KafkaApis.scala:386)
>         at
> scala.collection.immutable.MapLike$$anon$2$$anonfun$foreach$3.apply(MapLike.scala:109)
>         at
> scala.collection.immutable.MapLike$$anon$2$$anonfun$foreach$3.apply(MapLike.scala:109)
>         at scala.collection.immutable.Map$Map1.foreach(Map.scala:119)
>         at
> scala.collection.immutable.MapLike$$anon$2.foreach(MapLike.scala:109)
>         at
> kafka.server.KafkaApis.recordFollowerLogEndOffsets(KafkaApis.scala:386)
>         at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:351)
>         at kafka.server.KafkaApis.handle(KafkaApis.scala:60)
>         at
> kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:59)
>         at java.lang.Thread.run(Thread.java:744)
>
>
> And it starts truncating the logs (not to 0, guessing as part of becoming
> follower?) ...
> [2015-03-31 10:24:15,897] INFO [ReplicaFetcherManager on broker 6] Removed
> fetcher for partitions
> [my-store-changelog,1],[stage_vsw.avrodto.addelivery.internalstats.trafficshaperlog,3],+
> a bunch of partitions... (kafka.server.ReplicaFetcherManager)
> [2015-03-31 10:24:16,013] INFO Truncating log my-store-changelog-1 to
> offset 3736. (kafka.log.Log)
> [2015-03-31 10:24:16,014] INFO Truncating log xyz.topic1-3 to offset 2930.
> (kafka.log.Log)
>  [2015-03-31 10:24:16,014] INFO Truncating log xyz.topic2-2 to offset
> 10176. (kafka.log.Log)
>
> This continues for many / all partitions, then... It starts deleting logs!?
>
> [2015-03-31 10:24:38,449] INFO [ReplicaFetcherManager on broker 6] Removed
> fetcher for partitions [my-store-changelog,1],[xyz.topic1,1],...
> (kafka.server.ReplicaFetcherManager)
> [2015-03-31 10:24:40,700] INFO Scheduling log segment 6587 for log
> xyz.topic1-2 for deletion. (kafka.log.Log)
> [2015-03-31 10:24:40,701] INFO Scheduling log segment 6594 for log
> xyz.topic1-2 for deletion. (kafka.log.Log)
> [2015-03-31 10:24:40,701] INFO Scheduling log segment 6595 for log
> xyz.topic1-2 for deletion. (kafka.log.Log)
> [2015-03-31 10:24:40,702] INFO Scheduling log segment 203953 for log
> xyz.topic2-4 for deletion. (kafka.log.Log)
> [2015-03-31 10:24:40,702] INFO Scheduling log segment 210571 for log
> xyz.topic2-4 for deletion. (kafka.log.Log)
> [2015-03-31 10:24:40,702] INFO Scheduling log segment 211471 for log
> xyz.topic2-4 for deletion. (kafka.log.Log)
>
> then it starts actually deleting them... this goes on for a good 20
> minutes...
>  [2015-03-31 10:25:40,704] INFO Deleting segment 6587 from log
> xyz.topic1-2. (kafka.log.Log)
> [2015-03-31 10:25:40,716] INFO Deleting index
> /data4/kafka-data/xyz.topic1-2/00000000000000006587.index.deleted
> (kafka.log.OffsetIndex)
> [2015-03-31 10:25:40,716] INFO Deleting segment 6594 from log
> xyz.topic1-2. (kafka.log.Log)
> [2015-03-31 10:25:40,717] INFO Deleting index
> /data4/kafka-data/xyz.topic1-2/00000000000000006594.index.deleted
> (kafka.log.OffsetIndex)
> [2015-03-31 10:25:40,717] INFO Deleting segment 6595 from log
> xyz.topic1-2. (kafka.log.Log)
> [2015-03-31 10:25:40,717] INFO Deleting index
> /data4/kafka-data/xyz.topic1-2/00000000000000006595.index.deleted
> (kafka.log.OffsetIndex)
> [2015-03-31 10:25:40,717] INFO Deleting segment 203953 from log
> xyz.topic2-4. (kafka.log.Log)
> [2015-03-31 10:25:40,722] INFO Deleting segment 210571 from log
> xyz.topic2-4. (kafka.log.Log)
> [2015-03-31 10:25:40,729] INFO Deleting index
> /data4/kafka-data/xyz.topic2-4/00000000000000203953.index.deleted
> (kafka.log.OffsetIndex)
> [2015-03-31 10:25:40,729] INFO Deleting segment 211471 from log
> xyz.topic2-4. (kafka.log.Log)
>
> I don't know that we knew what was happening exactly at this time, only
> that it was not sync'ing up with the others. I think the sys-engineer
> stopped it after about 20 minutes to see what was wrong... I think by this
> point the damage was done. And actually in these logs, I don't see any of
> the "No checkpointed highwatermark is found for partition" messages in this
> sequence though, not sure what's up.
>
> Any ideas? It makes us more than a little nervous to restart nodes if they
> will just "decide" to delete segments. Under what conditions would this
> happen?
>
> Thanks!
> Thunder
>
>
> -----Original Message-----
> From: Thunder Stumpges [mailto:tstump...@ntent.com]
> Sent: Friday, April 03, 2015 12:10 PM
> To: users@kafka.apache.org
> Subject: RE: Problem with node after restart no partitions?
>
> Likewise, I was not "at the wheel" when this was happening, and there very
> well could have been a similar situation of not waiting for a controlled
> shutdown to complete successfully.
>
> Fortunately we did not end up in exactly your situation where the entire
> cluster went down, but I can say I know we never had more than 1 of the
> three nodes shut down during this situation, and twice in two days we lost
> all data on the node and it had to re-sync ALL of its data over again.
>
> I am in the process of trying to pull the controller and server logs for
> the first portion of our issue right now. Will follow up when they are
> available.
>
> Thanks,
> Thunder
>
>
> -----Original Message-----
> From: Jason Rosenberg [mailto:j...@squareup.com]
> Sent: Friday, April 03, 2015 10:50 AM
> To: users@kafka.apache.org
> Subject: Re: Problem with node after restart no partitions?
>
> I will provide what I can (we don't have separate logs for controller,
> etc., it's all integrated in a single log with log4j for us, we embed Kafka
> in a java container, etc).  Are there specific log classes you'd be
> interested in seeing?  (I can look at the default log4j configs to see
> what's set up normally for the 'controller' log)....
>
> We have most of the logs saved away (but not all of them).
>
> Sorry, I didn't mean to hijack this thread (so will start a new thread
> soonly).....
>
> Jason
>
> On Fri, Apr 3, 2015 at 1:37 PM, Jiangjie Qin <j...@linkedin.com.invalid>
> wrote:
>
> > This sounds a very serious issueĊ  Could you provide the controller log
> > and the log for the first broker on which you tried controlled
> > shutdown and upgrade?
> >
> > On 4/3/15, 8:57 AM, "Jason Rosenberg" <j...@squareup.com> wrote:
> >
> > >I'm preparing a longer post here, but we recently ran into a similar
> > >scenario.  Not sure yet if it's the same thing you saw (but it feels
> > >similar).  We were also doing a rolling upgrade from 0.8.1.1 to
> > >0.8.2.1, and during the controlled shutdown of the first node (of a 4
> > >node cluster), the controlled shutdown was taking longer than normal
> > >(it timed out several times and was retrying controlled shutdown),
> > >and unfortunately, our deployment system decided to kill it hard (so
> > >it was in the middle of it's 4th controlled shutdown retry, etc.).
> > >
> > >Anyway, when the node came back, it naturally decided to 'restore'
> > >most of it's partitions, which took some time (but only like 5
> > >minutes).  What's weird is it didn't decide to resync data from other
> > >replicas, instead it just restored partitions locally.  During this
> > >time, the rest of the cluster failed to elect any new leaders, and so
> > >for 5 minutes, those partitions were unavailable (and we saw a flood
> > >of failed FetcherManager exceptions from the other nodes in the
> > >cluster).  Most of the partitions were empty (e.g. there's no way the
> > >other replicas were behind and not in the ISR normally).  During this
> > >5 minutes, producers were unable to send messages due to
> > >NotLeaderForPartition exceptions.  Apparently the controller was still
> sending them to the unavailable broker.
> > >
> > >Finally, when the first node finally came up, the other nodes were
> > >somewhat happy again (but a few partitions remained under-replicated
> > >indefinitely).
> > >Because of this, we decided to pause the rolling restart, and try to
> > >wait for the under-replicated partitions to get insync.
> > >Unfortunately, about an hour later, the whole cluster went foobar
> > >(e.g. partitions became unavailable, brokers logged a flood of
> > >Fetcher errors, producers couldn't find a valid leader, metadata
> > >requests timed out, etc.).  In a panic, we reverted that first node
> > >back to 0.8.1.1. This did not help, unfortunately, so, deciding we'd
> > >already probably lost data at this point (and producers could not
> > >send data due to (NotLeaderForPartition exceptions)), we decided to
> > >just forcibly do the upgrade to 0.8.2.1.  This was all a bad
> > >situation, of course.
> > >
> > >So, now we have the cluster stable at 0.8.2.1, but like you, we are
> > >very, very nervous about doing any kind of restart to any of our
> > >nodes.  We lost data, primarily in the form of producers failing to
> > >send during the periods of unavailability.
> > >
> > >It looks like the root cause, in our case, was a flood of topics
> > >created (long-since unused and empty).  This appears to have caused
> > >the longer than normal controlled shutdown, which in turn, led to the
> > >followon problems.
> > >However, in the past, we've seen a controlled shutdown failure result
> > >in an unclean shutdown, but usually the cluster recovers (e.g. it
> > >elects new leaders, and when the new node comes back, it recovers
> > >it's partitions that were uncleanly shutdown).  That did not happen
> > >this time (the rest of the cluster got in an apparent infinite loop
> > >where it tried repeatedly (e.g.
> > >500K times a minute) to fetch partitions that were unavailable).
> > >
> > >I'm preparing a longer post with more detail (will take a bit of time).
> > >
> > >Jason
> > >
> > >On Thu, Apr 2, 2015 at 10:19 PM, Gwen Shapira <gshap...@cloudera.com>
> > >wrote:
> > >
> > >> wow, thats scary for sure.
> > >>
> > >> Just to be clear - all you did is restart *one* broker in the cluster?
> > >> everything else was ok before the restart? and that was controlled
> > >> shutdown?
> > >>
> > >> Gwen
> > >>
> > >> On Wed, Apr 1, 2015 at 11:54 AM, Thunder Stumpges
> > >> <tstump...@ntent.com>
> > >> wrote:
> > >>
> > >> > Well it appears we lost all the data on the one node again. It
> > >>appears to
> > >> > be all or part of KAFKA-1647<
> > >> > https://issues.apache.org/jira/browse/KAFKA-1647> as we saw this
> > >> > in
> > >>our
> > >> > logs (for all topics):
> > >> >
> > >> > [2015-04-01 10:46:58,901] WARN Partition [logactivity-redirect,3]
> > >> > on broker 6: No checkpointed highwatermark is found for partition
> > >> > [logactivity-redirect,3] (kafka.cluster.Partition)
> > >> > [2015-04-01 10:46:58,902] WARN Partition [pageimpression,1] on
> > >> > broker
> > >>6:
> > >> > No checkpointed highwatermark is found for partition
> > >>[pageimpression,1]
> > >> > (kafka.cluster.Partition)
> > >> > [2015-04-01 10:46:58,904] WARN Partition
> > >> > [campaignplatformtarget,6] on broker 6: No checkpointed
> > >> > highwatermark is found for partition [campaignplatformtarget,6]
> > >> > (kafka.cluster.Partition)
> > >> > [2015-04-01 10:46:58,905] WARN Partition [trackingtags-c2,1] on
> > >>broker 6:
> > >> > No checkpointed highwatermark is found for partition
> > >>[trackingtags-c2,1]
> > >> > (kafka.cluster.Partition)
> > >> >
> > >> > Followed by:
> > >> >
> > >> > [2015-04-01 10:46:58,911] INFO Truncating log trafficshaperlog-3
> > >> > to
> > >> offset
> > >> > 0. (kafka.log.Log)
> > >> > [2015-04-01 10:46:58,928] INFO Truncating log videorecrequest-0
> > >> > to
> > >>offset
> > >> > 0. (kafka.log.Log)
> > >> > [2015-04-01 10:46:58,928] INFO Truncating log filteredredirect-2
> > >> > to
> > >> offset
> > >> > 0. (kafka.log.Log)
> > >> > [2015-04-01 10:46:58,985] INFO Truncating log precheckrequest-3
> > >> > to
> > >>offset
> > >> > 0. (kafka.log.Log)
> > >> > [2015-04-01 10:46:58,990] INFO Truncating log filteredclicklog-8
> > >> > to
> > >> offset
> > >> > 0. (kafka.log.Log)
> > >> >
> > >> > Followed by:
> > >> > [2015-04-01 10:46:59,107] INFO Scheduling log segment 90276704
> > >> > for log
> > >> > pageview-0 for deletion. (kafka.log.Log)
> > >> > [2015-04-01 10:46:59,107] INFO Scheduling log segment 90593329
> > >> > for log
> > >> > pageview-0 for deletion. (kafka.log.Log)
> > >> > [2015-04-01 10:46:59,107] INFO Scheduling log segment 90906048
> > >> > for log
> > >> > pageview-0 for deletion. (kafka.log.Log)
> > >> > [2015-04-01 10:46:59,107] INFO Scheduling log segment 91226773
> > >> > for log
> > >> > pageview-0 for deletion. (kafka.log.Log)
> > >> >
> > >> >
> > >> > The strange thing however is that I don't believe we ever had the
> > >> scenario
> > >> > mentioned in the bug (all brokers for a topic down) As we have 0
> > >>unclean
> > >> > leader elections, our applications never complained the cluster
> > >> > was
> > >>down,
> > >> > and we never actually stopped more than one node (the node in
> > >>question).
> > >> >
> > >> > Anyway, really unsure if this could have somehow been related to
> > >> > the attempted upgrade to 0.8.2.1 (the occurrence this morning was
> > >> > NOT, it
> > >> was a
> > >> > simple stop, restart broker)
> > >> >
> > >> > We are really nervous to do any kafka service restarts now that
> > >> > this
> > >>has
> > >> > happened twice on this machine.
> > >> > Any suggestions? Should we go back and make another attempt to
> > >>upgrade to
> > >> > 0.8.2.1 ?
> > >> >
> > >> > Thanks,
> > >> > Thunder
> > >> >
> > >> >
> > >> > From: Thunder Stumpges
> > >> > Sent: Wednesday, April 01, 2015 10:30 AM
> > >> > To: kafka-us...@incubator.apache.org
> > >> > Subject: Problem with node after restart no partitions?
> > >> >
> > >> > Hi group,
> > >> >
> > >> > We've been trying to track down a problem this morning for a
> > >> > little
> > >> while,
> > >> > and thought I'd ask here while we keep looking.
> > >> >
> > >> > We have 3 nodes (rep-3) running 8.1.1. We attempted a rolling
> > >> > upgrade yesterday to 8.2.1, and on the first node, after
> > >> > restarting, a single
> > >> topic
> > >> > (a samza intermediate topic) started throwing replica fetcher
> > >> > errors
> > >>over
> > >> > and over ("NotLeaderForPartition"). There may or may not have
> > >> > been
> > >>other
> > >> > things attempted at this time (not by me so I cannot say for sure).
> > >> Anyway
> > >> > we ended up rolling back to 8.1.1 and ALL data had been DELETED
> > >> > from
> > >>that
> > >> > node. It spent most of yesterday re-syncing, and came into sync
> > >> > last
> > >> night,
> > >> > and a rebalance made everything run smoothly (*except for these
> > >> > damn replica fetcher errors for that one partition).
> > >> >
> > >> > Today my colleague attempted the "unsupported" topic delete
> > >> > command
> > >>for
> > >> > the "bad" partition, and bounced that one troublesome node.
> > >> >
> > >> > Upon coming up, I can see in server.log that it is reading in all
> > >> > of
> > >>the
> > >> > segments in, and then starts spitting out a samza topic fetch
> > >> > error,
> > >>and
> > >> > through JMX the "ReplicaManager".LeaderCount is 0. It is not
> > >>attempting
> > >> to
> > >> > fetch or load any topics.
> > >> >
> > >> > The other two brokers are showing under-replicated (obviously).
> > >> > What
> > >>is
> > >> > going wrong? How can we get that samza topic really and truly gone?
> > >>(if
> > >> > that is the cause of the broker not coming up)
> > >> >
> > >> > Thanks,
> > >> > Thunder
> > >> >
> > >> >
> > >>
> >
> >
>

Reply via email to