Skip to content
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

Add API of PullConsumer #342

Open
wants to merge 2 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,134 @@
/*
* 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.rocketmq.client.apis.consumer;

import java.io.Closeable;
import java.io.IOException;
import java.time.Duration;
import java.util.Collection;
import java.util.List;
import java.util.Optional;
import org.apache.rocketmq.client.apis.ClientException;
import org.apache.rocketmq.client.apis.message.MessageQueue;
import org.apache.rocketmq.client.apis.message.MessageView;

public interface PullConsumer extends Closeable {
/**
* Get the consumer group of the consumer.
*/
String getConsumerGroup();

/**
* @param topic the topic that needs to be monitored.
* @param listener the callback to detect the message queue changes.
*/
void registerMessageQueueChangeListenerByTopic(String topic, TopicMessageQueueChangeListener listener);

/**
* Fetch message queues of the topic.
*/
Collection<MessageQueue> fetchMessageQueues(String topic) throws ClientException;

/**
* Manually assign a list of message queues to this consumer.
*
* <p>This interface does not allow for incremental assignment and will replace the previous assignment (if
* previous assignment existed).
*
* @param messageQueues the list of message queues that are to be assigned to this consumer.
*/
void assign(Collection<MessageQueue> messageQueues);

/**
* Fetch messages from assigned message queues specified by {@link #assign(Collection)}.
*
* @param timeout the maximum time to block.
* @return list of fetched messages.
zhouxinyu marked this conversation as resolved.
Show resolved Hide resolved
*/
List<MessageView> poll(Duration timeout);
Copy link
Member Author

@aaron-ai aaron-ai Mar 6, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we also provide an asynchronous interface for PullConsumer#poll? If we only offer a synchronous interface, it means that each call will occupy one user thread. Typically, the poll interface is called for multiple queues, so occupying one thread may not have a significant impact. However, an asynchronous interface can provide greater flexibility, such as forming a future chain with the user's own methods, reducing unnecessary threads when multiple clients exist in one process, and potentially offering other benefits.

We welcome everyone to join the discussion.


/**
* Overrides the fetch offsets that the consumer will use on the next poll. If this method is invoked for the same
* message queue more than once, the latest offset will be used on the next {@link #poll(Duration)}.
*
* @param messageQueue the message queue to override the fetch offset.
* @param offset message offset.
*/
void seek(MessageQueue messageQueue, long offset);

/**
* Suspending message pulling from the message queues.
*
* @param messageQueues message queues that need to be suspended.
*/
void pause(Collection<MessageQueue> messageQueues);

/**
* Resuming message pulling from the message queues.
*
* @param messageQueues message queues that need to be resumed.
*/
void resume(Collection<MessageQueue> messageQueues);

/**
* Look up the offsets for the given message queue by timestamp. The returned offset for each message queue is the
* earliest offset whose timestamp is greater than or equal to the given timestamp in the corresponding message
* queue.
*
* @param messageQueue message queue that needs to be looked up.
* @param timestamp the timestamp for which to search.
* @return the offset of the message queue, or {@link Optional#empty()} if there is no message.
*/
Optional<Long> offsetForTimestamp(MessageQueue messageQueue, Long timestamp);

/**
* Get the latest committed offset for the given message queue.
*
* @return the latest committed offset, or {@link Optional#empty()} if there was no prior commit.
*/
Optional<Long> committed(MessageQueue messageQueue);

/**
* Commit offset manually.
*/
void commit() throws ClientException;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The semantics of commit are unclear, for example, for a single queue client, the pull progress is 200, and the poll function returns 100 at this time, we don't know the consumption progress. If we commit 200 will lose the message, commit 100 is not correct either. For users, it is not the case that the previous batch of messages needs to be consumed before taking the next batch. So the meaning here is auto commit. For stream frameworks, it is usually expected that commit and ckpt are atomic, and I think commit(mq, offset) should also be added as a manual interface.


/**
* Overrides the fetch offsets with the beginning offset that the consumer will use on the next poll. If this
* method is invoked for the same message queue more than once, the latest offset will be used on the next
* {@link #poll(Duration)}.
*
* @param messageQueue the message queue to seek.
*/
void seekToBegin(MessageQueue messageQueue) throws ClientException;

/**
* Overrides the fetch offsets with the end offset that the consumer will use on the next poll. If this method is
* invoked for the same message queue more than once, the latest offset will be used on the next
* {@link #poll(Duration)}.
*
* @param messageQueue the message queue to seek.
*/
void seekToEnd(MessageQueue messageQueue) throws ClientException;

/**
* Close the pull consumer and release all related resources.
*/
@Override
void close() throws IOException;
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,82 @@
/*
* 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.rocketmq.client.apis.consumer;

import java.time.Duration;
import org.apache.rocketmq.client.apis.ClientConfiguration;
import org.apache.rocketmq.client.apis.ClientException;

public interface PullConsumerBuilder {
/**
* Set the client configuration for the consumer.
*
* @param clientConfiguration client's configuration.
* @return the consumer builder instance.
*/
PullConsumerBuilder setClientConfiguration(ClientConfiguration clientConfiguration);

/**
* Set the load balancing group for the consumer.
*
* @param consumerGroup consumer load balancing group.
* @return the consumer builder instance.
*/
PullConsumerBuilder setConsumerGroup(String consumerGroup);

/**
* Automate the consumer's offset commit.
*
* @return the consumer builder instance.
*/
PullConsumerBuilder enableAutoCommit(boolean enable);

/**
* Set the consumer's offset commit interval if auto commit is enabled.
*
* @param duration offset commit interval
* @return the consumer builder instance.
*/
PullConsumerBuilder setAutoCommitInterval(Duration duration);

/**
* Set the maximum number of messages cached locally.
*
* @param count message count.
* @return the consumer builder instance.
*/
PullConsumerBuilder setMaxCacheMessageCountEachQueue(int count);

/**
* Set the maximum bytes of messages cached locally.
*
* @param bytes message size.
* @return the consumer builder instance.
*/
PullConsumerBuilder setMaxCacheMessageSizeInBytesEachQueue(int bytes);

/**
* Finalize the build of {@link PullConsumer} and start.
*
* <p>This method will block until the pull consumer starts successfully.
*
* <p>Especially, if this method is invoked more than once, different pull consumer will be created and started.
*
* @return the pull consumer instance.
*/
PullConsumer build() throws ClientException;
}
Original file line number Diff line number Diff line change
Expand Up @@ -93,7 +93,7 @@ public interface PushConsumer extends Closeable {
*
* <p>Nothing occurs if the specified topic does not exist in subscription expressions of the push consumer.
*
* @param topic the topic to remove the subscription.
* @param topic the topic to remove from the subscription.
* @return push consumer instance.
*/
PushConsumer unsubscribe(String topic) throws ClientException;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ public interface PushConsumerBuilder {
*
* <p>This method will block until the push consumer starts successfully.
*
* <p>Especially, if this method is invoked more than once, different push consumers will be created and started.
* <p>Especially, if this method is invoked more than once, different push consumer will be created and started.
*
* @return the push consumer instance.
*/
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
/*
* 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.rocketmq.client.apis.consumer;

import java.util.Set;
import org.apache.rocketmq.client.apis.message.MessageQueue;

public interface TopicMessageQueueChangeListener {
/**
* This method will be invoked in the condition of queue numbers changed, These scenarios occur when the topic is
* expanded or shrunk.
*
* @param topic the topic to listen.
* @param messageQueues latest message queues of the topic.
*/
void onChanged(String topic, Set<MessageQueue> messageQueues);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
/*
* 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.rocketmq.client.apis.message;

public interface MessageQueue {
/**
* Topic of the current message queue.
*/
String getTopic();

/**
* Get the identifier of the current message queue.
*/
String getId();
zhouxinyu marked this conversation as resolved.
Show resolved Hide resolved
}