Am I missing something or is the handling of fetch.wait.max.ms racy? In 0.8
code I don't see any locking (might just be missing it) around the initial
check to see if we can satisfy the event and enqueuing a delayed request in the
purgatory, so if data arrives that satisfies the request it might
Thanks Jun, that answers my question. I'll see if we can find some time to work
on these.
/Sam
On Apr 29, 2014, at 5:24 PM, Jun Rao (JIRA) wrote:
> Jun Rao created KAFKA-1430:
> --
>
> Summary: Purgatory redesign
> Key: KAFKA-1430
>
Hey,
I now have roughly a day a week I can dedicate to working on Kafka, so I am
looking for issues in the 0.8.1 batch that you think might be good starting
points. Input would be much appreciated.
Speaking of issues, I think it would be good to either fix
https://issues.apache.org/jira/browse
continue work on
https://issues.apache.org/jira/browse/KAFKA-330) and would potentially want to
change the build so it creates separate server and client artifacts.
/Sam
> Folks--any other ideas?
>
> -Jay
>
>
>
> On Sun, Jun 23, 2013 at 11:01 PM, Sam Meder
> wrote:
I think it makes sense to kill the hadoop consumer/producer code in Kafka,
given, as you said, Camus and the simplicity of the Hadoop producer.
/Sam
On Jul 2, 2013, at 5:01 PM, Jay Kreps wrote:
> We currently have a contrib package for consuming and producing messages
> from mapreduce (
> http
We're expecting to occasionally have to deal with pretty large messages being
sent to Kafka. We will of course set the fetch size appropriately high, but are
concerned about the behavior when the message exceeds the fetch size. As far as
I can tell the current behavior when a message that is too
I also noticed that there are two properties related to messages size on the
server: replica.fetch.max.bytes and message.max.bytes. What happens when
replica.fetch.max.bytes is lower than message.max.bytes? Should there even be
two properties?
/Sam
On Jul 31, 2013, at 5:25 PM, Sam Meder
row MessageSizeTooLargeException
> to the caller if the fetch size is too small.
>
> Thanks,
>
> Jun
>
>
> On Wed, Jul 31, 2013 at 8:25 AM, Sam Meder wrote:
>
>> We're expecting to occasionally have to deal with pretty large messages
>> being sent to Kafka. We
:
> server: replica.fetch.max.bytes should be >= message.max.bytes. Otherwise,
> the follower will get stuck when replicating data from the leader.
>
> Thanks,
>
> Jun
>
>
> On Wed, Jul 31, 2013 at 10:10 AM, Sam Meder wrote:
>
>> I also noticed that there are two
We've recently come across a scenario where we see consumers resetting their
offsets to earliest and which as far as I can tell may also lead to data loss
(we're running with ack = -1 to avoid loss). This seems to happen when we time
out on doing a regular shutdown and instead kill -9 the kafka
y
is
1. controlled shutdown of brokers 1 & 2 in parallel (kill -9 if they don't shut
down)
2. start brokers 1 & 2
3. controlled shutdown of brokers 3 & 4 in parallel (kill -9 if they don't shut
down)
4. start brokers 3 & 4
/Sam
>
> -Jay
>
>
> On Thu, Au
his could
> happen.
Hmm, but it shouldn't really be made leader if it isn't even in the isr, should
it?
/Sam
> With ack = -1, we guarantee that all the replicas in the in sync
> set have received the message before exposing the message to the consumer.
>
> On 8/29/13 8:32 A
replicas online at
some point and the replica that came back first was a little short on data...
How do you automate waiting for the broker to come up? Just keep monitoring the
process and keep trying to connect to the port?
/Sam
On Aug 29, 2013, at 6:40 PM, Sam Meder wrote:
>
> On Aug 29
Thinking about it some more I guess you are really talking about monitoring
UnderReplicatedPartitionCount during a restart?
/Sam
On Sep 6, 2013, at 5:46 PM, Sam Meder wrote:
> On Aug 29, 2013, at 11:12 PM, Neha Narkhede wrote:
>
>>>> How do you automate waiting for the
ugh a kill -9, OOM, HW failure -
what ever floats your boat). For this scenario it would be great if there was a
way to wait for the recovery process to finish. For now we can just wait for
the server port to become available, but something more explicit would be great.
/Sam
>
>
> Th
The latest consumer changes to read data from Zookeeper during rebalance have
made the consumer rebalance code incompatible with older versions (making
rolling upgrades without downtime hard). The problem relates to how partitions
are ordered. The old code seems to have returned the partitions s
Feel free to upload a patch as well.
>
> Thanks,
> Neha
>
>
> On Thu, Sep 19, 2013 at 8:17 AM, Sam Meder wrote:
>
>> The latest consumer changes to read data from Zookeeper during rebalance
>> have made the consumer rebalance code incompatible with older ve
For what it is worth I am currently looking into a problem that sounds
suspiciously related. We're seeing no node exceptions for the consumer node
during rebalance:
Caused by: org.apache.zookeeper.KeeperException$NoNodeException:
KeeperErrorCode = NoNode for
/consumers/es_consumer/ids/es_consu
I'm not quite sure how we get into this state, but we've seen this a few times
now. Basically, one of our brokers (broker 1 in this case) gets into a state
where ISR updates fail forever:
[2013-10-16 06:19:12,448] ERROR Conditional update of path
/brokers/topics/search-gateway-wal/partitions/5/
[
https://issues.apache.org/jira/browse/KAFKA-1029?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13946076#comment-13946076
]
Sam Meder commented on KAFKA-1029:
--
We've seen this as well (once or twice in
[
https://issues.apache.org/jira/browse/KAFKA-493?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13970427#comment-13970427
]
Sam Meder commented on KAFKA-493:
-
Do you know if VisualVM uses sampling to arrive at
Sam Meder created KAFKA-1417:
Summary: Very slow initial high-level consumer startup in low
traffic/blocking fetch scenario
Key: KAFKA-1417
URL: https://issues.apache.org/jira/browse/KAFKA-1417
Project
[
https://issues.apache.org/jira/browse/KAFKA-1417?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13979211#comment-13979211
]
Sam Meder commented on KAFKA-1417:
--
The simple, although not the most efficient solu
[
https://issues.apache.org/jira/browse/KAFKA-1417?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13979267#comment-13979267
]
Sam Meder commented on KAFKA-1417:
--
I think the timeout is somewhat arbitrary, but s
[
https://issues.apache.org/jira/browse/KAFKA-1417?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13980651#comment-13980651
]
Sam Meder commented on KAFKA-1417:
--
It should, let me take a look at the 0.8.1
[
https://issues.apache.org/jira/browse/KAFKA-1417?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13981825#comment-13981825
]
Sam Meder commented on KAFKA-1417:
--
Looks fine in 0.8.1
> Very slow initial hig
[
https://issues.apache.org/jira/browse/KAFKA-1417?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder resolved KAFKA-1417.
--
Resolution: Fixed
Fix Version/s: 0.8.1
> Very slow initial high-level consumer startup in
Sam Meder created KAFKA-1447:
Summary: Controlled shutdown deadlock when trying to send state
updates
Key: KAFKA-1447
URL: https://issues.apache.org/jira/browse/KAFKA-1447
Project: Kafka
Issue
[
https://issues.apache.org/jira/browse/KAFKA-1447?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13996532#comment-13996532
]
Sam Meder commented on KAFKA-1447:
--
0.8.0
> Controlled shutdown deadlock when tr
[
https://issues.apache.org/jira/browse/KAFKA-1447?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13997524#comment-13997524
]
Sam Meder commented on KAFKA-1447:
--
We'll be rolling out 0.8.1.1 soon, but
Sam Meder created KAFKA-1461:
Summary: Replica fetcher threads do not implement any back-off
behavior
Key: KAFKA-1461
URL: https://issues.apache.org/jira/browse/KAFKA-1461
Project: Kafka
Issue
[
https://issues.apache.org/jira/browse/KAFKA-1461?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-1461:
-
Summary: Replica fetcher thread does not implement any back-off behavior
(was: Replica fetcher threads
[
https://issues.apache.org/jira/browse/KAFKA-1447?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14002177#comment-14002177
]
Sam Meder commented on KAFKA-1447:
--
I don't have those logs anymore, should hav
[
https://issues.apache.org/jira/browse/KAFKA-997?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-997:
Resolution: Won't Fix
Status: Resolved (was: Patch Available)
> Provide a strict verificat
[
https://issues.apache.org/jira/browse/KAFKA-997?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14010310#comment-14010310
]
Sam Meder commented on KAFKA-997:
-
Looked at this a bit more a while ago and am just g
[
https://issues.apache.org/jira/browse/KAFKA-493?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14202228#comment-14202228
]
Sam Meder commented on KAFKA-493:
-
Sorry, don't have that anymore - it's
Sam Meder created KAFKA-938:
---
Summary: High CPU usage when more or less idle
Key: KAFKA-938
URL: https://issues.apache.org/jira/browse/KAFKA-938
Project: Kafka
Issue Type: Bug
Components
[
https://issues.apache.org/jira/browse/KAFKA-938?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-938:
Status: Patch Available (was: Open)
> High CPU usage when more or less i
[
https://issues.apache.org/jira/browse/KAFKA-938?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-938:
Attachment: timeunit.patch
> High CPU usage when more or less i
Sam Meder created KAFKA-943:
---
Summary: Move all configuration key string to constants
Key: KAFKA-943
URL: https://issues.apache.org/jira/browse/KAFKA-943
Project: Kafka
Issue Type: Improvement
[
https://issues.apache.org/jira/browse/KAFKA-943?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-943:
Status: Patch Available (was: Open)
> Move all configuration key string to consta
[
https://issues.apache.org/jira/browse/KAFKA-943?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-943:
Attachment: configConstants.patch
> Move all configuration key string to consta
[
https://issues.apache.org/jira/browse/KAFKA-945?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13686490#comment-13686490
]
Sam Meder commented on KAFKA-945:
-
We had a similar issue and it was due to a chang
Sam Meder created KAFKA-946:
---
Summary: Kafka Hadoop Consumer fails when verifying message
checksum
Key: KAFKA-946
URL: https://issues.apache.org/jira/browse/KAFKA-946
Project: Kafka
Issue Type
[
https://issues.apache.org/jira/browse/KAFKA-946?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-946:
Attachment: hadoop_consumer.patch
> Kafka Hadoop Consumer fails when verifying message check
[
https://issues.apache.org/jira/browse/KAFKA-946?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-946:
Status: Patch Available (was: Open)
Attached a patch that simply passes the full message buffer instead of
Sam Meder created KAFKA-956:
---
Summary: High-level consumer fails to check topic metadata
response for errors
Key: KAFKA-956
URL: https://issues.apache.org/jira/browse/KAFKA-956
Project: Kafka
[
https://issues.apache.org/jira/browse/KAFKA-956?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-956:
Status: Patch Available (was: Open)
> High-level consumer fails to check topic metadata response
[
https://issues.apache.org/jira/browse/KAFKA-956?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-956:
Attachment: consumer_metadata_fetch.patch
Patch to throw an exception if no partition info is available
[
https://issues.apache.org/jira/browse/KAFKA-956?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13693683#comment-13693683
]
Sam Meder commented on KAFKA-956:
-
Regarding 1. - how does the consumer recover
[
https://issues.apache.org/jira/browse/KAFKA-956?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13694034#comment-13694034
]
Sam Meder commented on KAFKA-956:
-
Right, but the topic already exists at the tim
[
https://issues.apache.org/jira/browse/KAFKA-956?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13694150#comment-13694150
]
Sam Meder commented on KAFKA-956:
-
Basically the scenario is (roughly):
1. Start cons
[
https://issues.apache.org/jira/browse/KAFKA-956?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13694516#comment-13694516
]
Sam Meder commented on KAFKA-956:
-
Seems like we need to allow for rebalancing on
[
https://issues.apache.org/jira/browse/KAFKA-956?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13696789#comment-13696789
]
Sam Meder commented on KAFKA-956:
-
I guess we can carry my patch locally for now (we d
[
https://issues.apache.org/jira/browse/KAFKA-956?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13696943#comment-13696943
]
Sam Meder commented on KAFKA-956:
-
I don't think it has anything to do with topic
[
https://issues.apache.org/jira/browse/KAFKA-943?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13699027#comment-13699027
]
Sam Meder commented on KAFKA-943:
-
I would actually argue that it is easier on exte
[
https://issues.apache.org/jira/browse/KAFKA-943?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13699128#comment-13699128
]
Sam Meder commented on KAFKA-943:
-
So we also have a config management system externa
[
https://issues.apache.org/jira/browse/KAFKA-943?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13701803#comment-13701803
]
Sam Meder commented on KAFKA-943:
-
How about an option that turns on strict valida
[
https://issues.apache.org/jira/browse/KAFKA-969?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13705971#comment-13705971
]
Sam Meder commented on KAFKA-969:
-
I feel that Kafka servers and clients should really
Sam Meder created KAFKA-994:
---
Summary: High level consumer doesn't throw an exception when the
message it is trying to fetch exceeds the configured fetch size
Key: KAFKA-994
URL: https://issues.apache.org/jira/b
[
https://issues.apache.org/jira/browse/KAFKA-994?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-994:
Description:
The high level consumer code is supposed to throw an exception when it
encounters a message
[
https://issues.apache.org/jira/browse/KAFKA-994?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-994:
Fix Version/s: 0.8
Status: Patch Available (was: Open)
> High level consumer doesn'
[
https://issues.apache.org/jira/browse/KAFKA-994?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-994:
Attachment: messageSize.patch
> High level consumer doesn't throw an exception when the messa
Sam Meder created KAFKA-995:
---
Summary: Enforce that the value for replica.fetch.max.bytes is
always >= the value for message.max.bytes
Key: KAFKA-995
URL: https://issues.apache.org/jira/browse/KAFKA-
[
https://issues.apache.org/jira/browse/KAFKA-995?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-995:
Status: Patch Available (was: Open)
> Enforce that the value for replica.fetch.max.bytes is alw
[
https://issues.apache.org/jira/browse/KAFKA-995?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-995:
Attachment: replica_fetch_size_config.patch
> Enforce that the value for replica.fetch.max.bytes
[
https://issues.apache.org/jira/browse/KAFKA-994?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13726535#comment-13726535
]
Sam Meder commented on KAFKA-994:
-
Yea, I realized that after the initial write-up,
[
https://issues.apache.org/jira/browse/KAFKA-881?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-881:
Attachment: kafka-roll-0.8.patch
> Kafka broker not respecting log.roll.ho
[
https://issues.apache.org/jira/browse/KAFKA-881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13726789#comment-13726789
]
Sam Meder commented on KAFKA-881:
-
Attached a version of this patch that applies aga
[
https://issues.apache.org/jira/browse/KAFKA-943?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-943:
Resolution: Won't Fix
Status: Resolved (was: Patch Available)
Resolving this issue since the
Sam Meder created KAFKA-997:
---
Summary: Provide a strict verification mode when reading
configuration properties
Key: KAFKA-997
URL: https://issues.apache.org/jira/browse/KAFKA-997
Project: Kafka
[
https://issues.apache.org/jira/browse/KAFKA-997?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-997:
Status: Patch Available (was: Open)
> Provide a strict verification mode when reading configurat
[
https://issues.apache.org/jira/browse/KAFKA-997?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-997:
Attachment: strict-verification.patch
Add strict verification property and use it in tests and similar
[
https://issues.apache.org/jira/browse/KAFKA-997?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-997:
Attachment: strict-verification-2.patch
> Provide a strict verification mode when reading configurat
[
https://issues.apache.org/jira/browse/KAFKA-997?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-997:
Attachment: (was: strict-verification.patch)
> Provide a strict verification mode when read
Sam Meder created KAFKA-1010:
Summary: Concurrency issue in getCluster() causes rebalance
failure and dead consumer
Key: KAFKA-1010
URL: https://issues.apache.org/jira/browse/KAFKA-1010
Project: Kafka
[
https://issues.apache.org/jira/browse/KAFKA-1010?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-1010:
-
Attachment: get_cluster_0_8.patch
Simply move getCluster() call intro retry loop and eliminate second
[
https://issues.apache.org/jira/browse/KAFKA-1010?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-1010:
-
Status: Patch Available (was: Open)
> Concurrency issue in getCluster() causes rebalance fail
[
https://issues.apache.org/jira/browse/KAFKA-1010?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13742354#comment-13742354
]
Sam Meder commented on KAFKA-1010:
--
weird, patch -p1 -i get_cluster_0_8.patch wo
[
https://issues.apache.org/jira/browse/KAFKA-1010?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-1010:
-
Attachment: get_cluster_0_8_git.patch
> Concurrency issue in getCluster() causes rebalance fail
[
https://issues.apache.org/jira/browse/KAFKA-1010?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-1010:
-
Attachment: (was: get_cluster_0_8.patch)
> Concurrency issue in getCluster() causes rebala
[
https://issues.apache.org/jira/browse/KAFKA-1010?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13742365#comment-13742365
]
Sam Meder commented on KAFKA-1010:
--
Git formatted patch is now atta
Sam Meder created KAFKA-1029:
Summary: Zookeeper leader election stuck in ephemeral node retry
loop
Key: KAFKA-1029
URL: https://issues.apache.org/jira/browse/KAFKA-1029
Project: Kafka
Issue
[
https://issues.apache.org/jira/browse/KAFKA-1029?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-1029:
-
Attachment: 0001-KAFKA-1029-Use-brokerId-instead-of-leaderId-when-tri.patch
> Zookeeper lea
[
https://issues.apache.org/jira/browse/KAFKA-1029?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-1029:
-
Status: Patch Available (was: Open)
> Zookeeper leader election stuck in ephemeral node retry l
[
https://issues.apache.org/jira/browse/KAFKA-1029?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-1029:
-
Attachment: (was:
0001-KAFKA-1029-Use-brokerId-instead-of-leaderId-when-tri.patch)
> Zookee
[
https://issues.apache.org/jira/browse/KAFKA-1029?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-1029:
-
Attachment: 0002-KAFKA-1029-Use-brokerId-instead-of-leaderId-when-tri.patch
> Zookeeper lea
[
https://issues.apache.org/jira/browse/KAFKA-1029?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13751431#comment-13751431
]
Sam Meder commented on KAFKA-1029:
--
Updated patch to also include moving the list
[
https://issues.apache.org/jira/browse/KAFKA-956?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-956:
Resolution: Fixed
Status: Resolved (was: Patch Available)
I can confirm that this issue was fixed
[
https://issues.apache.org/jira/browse/KAFKA-956?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder closed KAFKA-956.
---
> High-level consumer fails to check topic metadata response for err
Sam Meder created KAFKA-1062:
Summary: Reading topic metadata from zookeeper leads to
incompatible ordering of partition list
Key: KAFKA-1062
URL: https://issues.apache.org/jira/browse/KAFKA-1062
Project
[
https://issues.apache.org/jira/browse/KAFKA-1062?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-1062:
-
Attachment: sorted.patch
> Reading topic metadata from zookeeper leads to incompatible ordering
[
https://issues.apache.org/jira/browse/KAFKA-1062?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-1062:
-
Status: Patch Available (was: Open)
> Reading topic metadata from zookeeper leads to incompati
[
https://issues.apache.org/jira/browse/KAFKA-946?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13775988#comment-13775988
]
Sam Meder commented on KAFKA-946:
-
Rebased patch attached.
> Kafka
[
https://issues.apache.org/jira/browse/KAFKA-946?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-946:
Attachment: (was: hadoop_consumer.patch)
> Kafka Hadoop Consumer fails when verifying mess
[
https://issues.apache.org/jira/browse/KAFKA-946?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Meder updated KAFKA-946:
Attachment: hadoop_consumer_1.patch
> Kafka Hadoop Consumer fails when verifying message check
[
https://issues.apache.org/jira/browse/KAFKA-330?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13530293#comment-13530293
]
Sam Meder commented on KAFKA-330:
-
I've been doing a lot of manual resetting o
[
https://issues.apache.org/jira/browse/KAFKA-330?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13584725#comment-13584725
]
Sam Meder commented on KAFKA-330:
-
Any news on this?
> Add delet
98 matches
Mail list logo