kafka-users mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Karolis Pocius <karolis.poc...@sentiance.com.INVALID>
Subject Replica movement between log directories
Date Tue, 02 Jul 2019 14:43:54 GMT
Not having much luck with replica movement between directories, so I'd
appreciate if someone validated the steps that I'm taking:

1. Create topics to move json file (with a single topic)
2. Generate a candidate partition reassignment
3. Take the above and replace all instances of "any" with
"/path-to-log-dir" (I want certain partitions moved to a specific log dir
that is the same on each of the five brokers in the cluster)
4. Create reassignment json with the data from step #3
5. Execute reassignment with an increased timeout, just to be safe

What happens next is that some partitions reassign just fine, while others
throw a warning and get stuck forever. Here's the full log for one of the
attempted reassignments:

[2019-07-02 13:58:40,330] INFO [Log partition=topic.0-7, dir=/kafka-data-2]
Loading producer state till offset 0 with message format version 2
(kafka.log.Log)
[2019-07-02 13:58:40,330] INFO [Log partition=topic.0-7, dir=/kafka-data-2]
Completed load of log with 1 segments, log start offset 0 and log end
offset 0 in 1 ms (kafka.log.Log)
[2019-07-02 13:58:40,330] INFO Created log for partition topic.0-7 in
/kafka-data-2 with properties {compression.type -> producer,
message.downconversion.enable -> true, min.insync.replicas -> 2,
segment.jitter.ms -> 0, cleanup.policy -> delete, flush.ms ->
9223372036854775807, segment.bytes -> 1073741824, retention.ms ->
604800000, flush.messages -> 9223372036854775807, message.format.version ->
2.2-IV1, file.delete.delay.ms -> 60000, max.compaction.lag.ms ->
9223372036854775807, max.message.bytes -> 52428800, min.compaction.lag.ms
-> 0, message.timestamp.type -> LogAppendTime, preallocate -> false,
min.cleanable.dirty.ratio -> 0.5, index.interval.bytes -> 4096,
unclean.leader.election.enable -> false, retention.bytes -> -1,
delete.retention.ms -> 86400000, segment.ms -> 604800000,
message.timestamp.difference.max.ms -> 9223372036854775807,
segment.index.bytes -> 10485760}. (kafka.log.LogManager)
[2019-07-02 13:58:40,331] INFO [Partition topic.0-7 broker=2] No
checkpointed highwatermark is found for partition topic.0-7
(kafka.cluster.Partition)
[2019-07-02 13:58:40,331] INFO Replica loaded for partition topic.0-7 with
initial high watermark 0 (kafka.cluster.Replica)
[2019-07-02 13:58:40,331] INFO [ReplicaAlterLogDirsManager on broker 2]
Added fetcher to broker BrokerEndPoint(id=2, host=localhost:-1) for
partitions Map(topic.0-7 -> (offset=0, leaderEpoch=84))
(kafka.server.ReplicaAlterLogDirsManager)
[2019-07-02 13:58:40,389] INFO [ReplicaAlterLogDirsThread-0]: Truncating
partition topic.0-7 to local high watermark 0
(kafka.server.ReplicaAlterLogDirsThread)
[2019-07-02 13:58:40,389] INFO [Log partition=topic.0-7, dir=/kafka-data-2]
Truncating to 0 has no effect as the largest offset in the log is -1
(kafka.log.Log)
[2019-07-02 13:58:41,043] INFO [ReplicaFetcherManager on broker 2] Removed
fetcher for partitions Set(topic.0-7) (kafka.server.ReplicaFetcherManager)
[2019-07-02 13:58:41,043] INFO [ReplicaFetcherManager on broker 2] Added
fetcher to broker BrokerEndPoint(id=0,
host=<ip-redacted>.ec2.internal:9092) for partitions Map(topic.0-7 ->
(offset=59338, leaderEpoch=85)) (kafka.server.ReplicaFetcherManager)
[2019-07-02 13:58:41,203] INFO [Log partition=topic.0-7, dir=/kafka-data-1]
Truncating to 59338 has no effect as the largest offset in the log is 59337
(kafka.log.Log)
[2019-07-02 13:58:41,227] INFO [ReplicaAlterLogDirsThread-0]: Truncating
partition topic.0-7 to local high watermark 0
(kafka.server.ReplicaAlterLogDirsThread)
[2019-07-02 13:58:41,227] INFO [Log partition=topic.0-7, dir=/kafka-data-2]
Truncating to 0 has no effect as the largest offset in the log is -1
(kafka.log.Log)
[2019-07-02 13:58:41,229] INFO [ReplicaAlterLogDirsThread-0]:
Beginning/resuming copy of partition topic.0-7 from offset 0. Including
this partition, there are 5 remaining partitions to copy by this thread.
(kafka.server.ReplicaAlterLogDirsThread)
[2019-07-02 13:58:41,229] INFO [ReplicaAlterLogDirsThread-0]: Partition
topic.0-7 has an older epoch (84) than the current leader. Will await the
new LeaderAndIsr state before resuming fetching.
(kafka.server.ReplicaAlterLogDirsThread)
[2019-07-02 13:58:41,229] WARN [ReplicaAlterLogDirsThread-0]: Partition
topic.0-7 marked as failed (kafka.server.ReplicaAlterLogDirsThread)
[2019-07-02 13:58:41,667] INFO [ReplicaAlterLogDirsThread-0]: Shutting down
(kafka.server.ReplicaAlterLogDirsThread)
[2019-07-02 13:58:41,667] INFO [ReplicaAlterLogDirsThread-0]: Shutdown
completed (kafka.server.ReplicaAlterLogDirsThread)
[2019-07-02 13:58:41,667] INFO [ReplicaAlterLogDirsThread-0]: Stopped
(kafka.server.ReplicaAlterLogDirsThread)

I have upgraded from 2.2.1 to 2.3.0 (haven't changed inter.broker.protocol
yet) hoping that KAFKA-8346 would somehow improve the situation, but it
seems that it just keeps the thread from dying.

Any pointers to what might be going wrong here would be appreciated.

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message