-
Notifications
You must be signed in to change notification settings - Fork 3.6k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
PIP-13-1/3: Provide TopicsConsumer
to consume from several topics under same namespace
#1103
Changes from all commits
2ae7189
6fde7a9
a63e6c9
37ed39b
29da59c
a43cb0f
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Large diffs are not rendered by default.
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -468,17 +468,20 @@ public void redeliverUnacknowledgedMessages() { | |
} | ||
|
||
@Override | ||
public void redeliverUnacknowledgedMessages(Set<MessageIdImpl> messageIds) { | ||
public void redeliverUnacknowledgedMessages(Set<MessageId> messageIds) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why are you changing this to MessageId? You're casting to MessageIdImpl anyhow. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks, As above reply, we also need this redeliverUnacknowledgedMessages method in Consumer.java handling TopicMessageIdImpl. would like to change this in Consumer.java and make the case in each child class. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. In Consumer.java it doesn't know about the MessageIds. The call takes no parameters. TopicsMessageIdImpl should be a specialization of MessageIdImpl (I assumed it was when I originally commented, until i saw Matteo's comment to the same effect). If it is a specialization of TopicsMessageIdImpl, then this signature doesn't need to change. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Oh, it is in ConsumerBase.java, sorry for the wrong reference of Consumer.java. |
||
checkArgument(messageIds.stream().findFirst().get() instanceof MessageIdImpl); | ||
if (conf.getSubscriptionType() != SubscriptionType.Shared) { | ||
// We cannot redeliver single messages if subscription type is not Shared | ||
redeliverUnacknowledgedMessages(); | ||
return; | ||
} | ||
removeExpiredMessagesFromQueue(messageIds); | ||
messageIds.stream() | ||
.collect(Collectors.groupingBy(MessageIdImpl::getPartitionIndex, Collectors.toSet())) | ||
.forEach((partitionIndex, messageIds1) -> | ||
consumers.get(partitionIndex).redeliverUnacknowledgedMessages(messageIds1)); | ||
.map(messageId -> (MessageIdImpl)messageId) | ||
.collect(Collectors.groupingBy(MessageIdImpl::getPartitionIndex, Collectors.toSet())) | ||
.forEach((partitionIndex, messageIds1) -> | ||
consumers.get(partitionIndex).redeliverUnacknowledgedMessages( | ||
messageIds1.stream().map(mid -> (MessageId)mid).collect(Collectors.toSet()))); | ||
resumeReceivingFromPausedConsumersIfNeeded(); | ||
} | ||
|
||
|
@@ -546,10 +549,10 @@ public UnAckedMessageTracker getUnAckedMessageTracker() { | |
return unAckedMessageTracker; | ||
} | ||
|
||
private void removeExpiredMessagesFromQueue(Set<MessageIdImpl> messageIds) { | ||
private void removeExpiredMessagesFromQueue(Set<MessageId> messageIds) { | ||
Message peek = incomingMessages.peek(); | ||
if (peek != null) { | ||
if (!messageIds.contains((MessageIdImpl) peek.getMessageId())) { | ||
if (!messageIds.contains(peek.getMessageId())) { | ||
// first message is not expired, then no message is expired in queue. | ||
return; | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,49 @@ | ||
/** | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, | ||
* software distributed under the License is distributed on an | ||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
* KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations | ||
* under the License. | ||
*/ | ||
package org.apache.pulsar.client.impl; | ||
|
||
import org.apache.pulsar.client.api.MessageId; | ||
|
||
public class TopicMessageIdImpl implements MessageId { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This could be extending MessageIdImpl instead of composing There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. +1. This would reduce the amount of change needed elsewhere also. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks. Here TopicMessageIdImpl is mainly a wrapper for MessageIdImpl, We need keep a reference of MessageIdImpl, because it will be used for one internal-sub-consumer. |
||
private final String topicName; | ||
private final MessageId messageId; | ||
|
||
TopicMessageIdImpl(String topicName, MessageId messageId) { | ||
this.topicName = topicName; | ||
this.messageId = messageId; | ||
} | ||
|
||
public String getTopicName() { | ||
return topicName; | ||
} | ||
|
||
public MessageId getInnerMessageId() { | ||
return messageId; | ||
} | ||
|
||
@Override | ||
public byte[] toByteArray() { | ||
return messageId.toByteArray(); | ||
} | ||
|
||
@Override | ||
public int compareTo(MessageId o) { | ||
return messageId.compareTo(o); | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
These methods should be converted into builder API from #1089