You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2021/09/14 05:11:09 UTC

[GitHub] [pulsar] Jason918 opened a new pull request #12032: [consumer] Add seek by index for consumer

Jason918 opened a new pull request #12032:
URL: https://github.com/apache/pulsar/pull/12032


   ### Motivation
   
   Currently we can reset the read position of a cursor by message id or timestamp. Since we formerly introduced index in broker metadata since 2.9.0, reset cursor by index is very helpful in other protocol handler (KoP or RoP). 
   
   ### Modifications
   
   Added seekByIndex in consumer api.
   The implementation is similar to seeking by timestamp.
   
   ### Verifying this change
   
   - [ ] Make sure that the change passes the CI checks.
   
   This change added tests and can be verified as follows:
     - PersistentMessageFinderByIndexTest.java is unit test for the core finder implementation in broker.
     - ConsumerSeekTest.java is the integration test for client implementation.
   
   ### Does this pull request potentially affect one of the following parts:
   
   *If `yes` was chosen, please highlight the changes*
   
     - Dependencies (does it add or upgrade a dependency): ( no)
     - The public API: (yes?)  added optional field 'index' in CommandSeek
     - The schema: ( no ) 
     - The default values of configurations: (no)
     - The wire protocol: (no)
     - The rest endpoints: (no)
     - The admin cli options: (no)
     - Anything that affects deployment: ( no)
   
   ### Documentation
   
   Check the box below and label this PR (if you have committer privilege).
   
   Need to update docs? 
   
   - [ ] doc-required 
     
     (If you need help on updating docs, create a doc issue)
     
   - [ ] no-need-doc 
     
     (Please explain why)
     
   - [x] doc 
     
     (If this PR contains doc changes) 
   User can refer to the java doc in file pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Consumer.java
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] Jason918 commented on pull request #12032: [consumer] Add seek by index for consumer

Posted by GitBox <gi...@apache.org>.
Jason918 commented on pull request #12032:
URL: https://github.com/apache/pulsar/pull/12032#issuecomment-925466982


   > it looks like we try to bake some sort of support for other message protocol into the broker itself?
   > will this just open up the potential to add more patches to make supporting other message protocol easier, then how far are we willing to go?
   
   Message protocol handler is one of the great features in pulsar.
   IMHO, It's important and useful to make other message protocol implementation easier. It's great for the messaging ecosystem. So we should add more new features supporting other message protocol as long as it does not breaking the design philosophy of pulsar.
   
   @codelipenghui @jiazhai  What do you think of this?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] BewareMyPower commented on a change in pull request #12032: [consumer] Add seek by index for consumer

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on a change in pull request #12032:
URL: https://github.com/apache/pulsar/pull/12032#discussion_r717232489



##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConsumerSeekTest.java
##########
@@ -0,0 +1,108 @@
+/**
+ * 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 java.nio.charset.StandardCharsets;
+import java.util.HashSet;
+import java.util.Set;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.MessageId;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerConsumerBase;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class ConsumerSeekTest extends ProducerConsumerBase {
+
+    @BeforeClass
+    @Override
+    protected void setup() throws Exception {
+        Set<String> interceptorNames = new HashSet<>();
+        interceptorNames.add("org.apache.pulsar.common.intercept.AppendBrokerTimestampMetadataInterceptor");
+        interceptorNames.add("org.apache.pulsar.common.intercept.AppendIndexMetadataInterceptor");
+        conf.setBrokerEntryMetadataInterceptors(interceptorNames);
+        conf.setExposingBrokerEntryMetadataToClientEnabled(true);
+        enableBrokerInterceptor = true;
+        internalSetup();
+        producerBaseSetup();
+    }
+
+    @AfterClass
+    @Override
+    protected void cleanup() throws Exception {
+        internalCleanup();
+    }
+
+    @Test
+    public void testConsumerSeekByIndex() throws Exception {

Review comment:
       It's better to move the test to `SubscriptionSeekTest`. And I don't think validating `markDeletePosition` is a good way, you can take other tests in `SubscriptionSeekTest` as reference. 

##########
File path: pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Consumer.java
##########
@@ -599,6 +599,15 @@
      */
     void seek(long timestamp) throws PulsarClientException;
 
+
+    /**
+     * Reset the subscription associated with this consumer to a specific message index.
+     *
+     * @param index
+     *            the message index where to reposition the subscription
+     */

Review comment:
       We should define the behavior for corner cases. For example, if there're 10 messages in a topic, i.e. the offset range varies from 0 to 9. What will happen if the given index is -1 or 10?
   
   There is another corner case. If the offset range is `[10, 20)`, the oldest messages (`[0, 10)`) have been deleted. What will happen if the given index is less than 10?
   
   BTW, it's a change for public API. I think we need a PIP first. /cc @merlimat 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] Jason918 commented on pull request #12032: [consumer] Add seek by index for consumer

Posted by GitBox <gi...@apache.org>.
Jason918 commented on pull request #12032:
URL: https://github.com/apache/pulsar/pull/12032#issuecomment-929830743


   /pulsarbot run-failure-checks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] Jason918 commented on pull request #12032: [consumer] Add seek by index for consumer

Posted by GitBox <gi...@apache.org>.
Jason918 commented on pull request #12032:
URL: https://github.com/apache/pulsar/pull/12032#issuecomment-929802708






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] BewareMyPower commented on pull request #12032: [consumer] Add seek by index for consumer

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on pull request #12032:
URL: https://github.com/apache/pulsar/pull/12032#issuecomment-928816251


   @MarvinCai It's not a support for other protocol, it's an enhancement. Currently Pulsar support the seek operation based on timestamp or message id. However, it doesn't support seek operation based on message count. For example, users might want to seek to 10000 messages before. Currently they cannot achieve this goal.
   Pulsar 2.8.0 introduced BrokerEntryMetadata (PIP-70) and the message index (similar to Kafka's offset), so it becomes possible to support seeking by index now.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MarvinCai commented on pull request #12032: [consumer] Add seek by index for consumer

Posted by GitBox <gi...@apache.org>.
MarvinCai commented on pull request #12032:
URL: https://github.com/apache/pulsar/pull/12032#issuecomment-929850723


   But not all users will use Index or event BrokerEntryMetadata, while all messages have Timestamp and MessageID, that's a bit different in my opinion. Adding a method that rely on a property might not exist for all users is kind of weird. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MarvinCai commented on pull request #12032: [consumer] Add seek by index for consumer

Posted by GitBox <gi...@apache.org>.
MarvinCai commented on pull request #12032:
URL: https://github.com/apache/pulsar/pull/12032#issuecomment-925050452


   it looks like we try to bake some sort of support for other message protocol into the broker itself?
   will this just open up the potential to add more patches to make supporting other message protocol easier, then how far are we willing to go?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] BewareMyPower commented on pull request #12032: [consumer] Add seek by index for consumer

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on pull request #12032:
URL: https://github.com/apache/pulsar/pull/12032#issuecomment-928816251


   @MarvinCai It's not a support for other protocol, it's an enhancement. Currently Pulsar support the seek operation based on timestamp or message id. However, it doesn't support seek operation based on message count. For example, users might want to seek to 10000 messages before. Currently they cannot achieve this goal.
   Pulsar 2.8.0 introduced BrokerEntryMetadata (PIP-70) and the message index (similar to Kafka's offset), so it becomes possible to support seeking by index now.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] Jason918 commented on a change in pull request #12032: [consumer] Add seek by index for consumer

Posted by GitBox <gi...@apache.org>.
Jason918 commented on a change in pull request #12032:
URL: https://github.com/apache/pulsar/pull/12032#discussion_r717614426



##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConsumerSeekTest.java
##########
@@ -0,0 +1,108 @@
+/**
+ * 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 java.nio.charset.StandardCharsets;
+import java.util.HashSet;
+import java.util.Set;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.MessageId;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerConsumerBase;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class ConsumerSeekTest extends ProducerConsumerBase {
+
+    @BeforeClass
+    @Override
+    protected void setup() throws Exception {
+        Set<String> interceptorNames = new HashSet<>();
+        interceptorNames.add("org.apache.pulsar.common.intercept.AppendBrokerTimestampMetadataInterceptor");
+        interceptorNames.add("org.apache.pulsar.common.intercept.AppendIndexMetadataInterceptor");
+        conf.setBrokerEntryMetadataInterceptors(interceptorNames);
+        conf.setExposingBrokerEntryMetadataToClientEnabled(true);
+        enableBrokerInterceptor = true;
+        internalSetup();
+        producerBaseSetup();
+    }
+
+    @AfterClass
+    @Override
+    protected void cleanup() throws Exception {
+        internalCleanup();
+    }
+
+    @Test
+    public void testConsumerSeekByIndex() throws Exception {

Review comment:
       Updated. PTAL.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MarvinCai edited a comment on pull request #12032: [consumer] Add seek by index for consumer

Posted by GitBox <gi...@apache.org>.
MarvinCai edited a comment on pull request #12032:
URL: https://github.com/apache/pulsar/pull/12032#issuecomment-929850723


   But not all users will use Index or even BrokerEntryMetadata, while all messages have Timestamp and MessageID, that's a bit different in my opinion. Adding a method that rely on a property might not exist for all users is kind of weird. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] Jason918 commented on pull request #12032: [consumer] Add seek by index for consumer

Posted by GitBox <gi...@apache.org>.
Jason918 commented on pull request #12032:
URL: https://github.com/apache/pulsar/pull/12032#issuecomment-919024511


   /pulsarbot run-failure-checks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MarvinCai edited a comment on pull request #12032: [consumer] Add seek by index for consumer

Posted by GitBox <gi...@apache.org>.
MarvinCai edited a comment on pull request #12032:
URL: https://github.com/apache/pulsar/pull/12032#issuecomment-929852505


   But let's discuss it with broader community.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] Jason918 commented on pull request #12032: [consumer] Add seek by index for consumer

Posted by GitBox <gi...@apache.org>.
Jason918 commented on pull request #12032:
URL: https://github.com/apache/pulsar/pull/12032#issuecomment-918915634


   /pulsarbot run-failure-checks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] BewareMyPower commented on a change in pull request #12032: [consumer] Add seek by index for consumer

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on a change in pull request #12032:
URL: https://github.com/apache/pulsar/pull/12032#discussion_r717232489



##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConsumerSeekTest.java
##########
@@ -0,0 +1,108 @@
+/**
+ * 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 java.nio.charset.StandardCharsets;
+import java.util.HashSet;
+import java.util.Set;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.MessageId;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerConsumerBase;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class ConsumerSeekTest extends ProducerConsumerBase {
+
+    @BeforeClass
+    @Override
+    protected void setup() throws Exception {
+        Set<String> interceptorNames = new HashSet<>();
+        interceptorNames.add("org.apache.pulsar.common.intercept.AppendBrokerTimestampMetadataInterceptor");
+        interceptorNames.add("org.apache.pulsar.common.intercept.AppendIndexMetadataInterceptor");
+        conf.setBrokerEntryMetadataInterceptors(interceptorNames);
+        conf.setExposingBrokerEntryMetadataToClientEnabled(true);
+        enableBrokerInterceptor = true;
+        internalSetup();
+        producerBaseSetup();
+    }
+
+    @AfterClass
+    @Override
+    protected void cleanup() throws Exception {
+        internalCleanup();
+    }
+
+    @Test
+    public void testConsumerSeekByIndex() throws Exception {

Review comment:
       It's better to move the test to `SubscriptionSeekTest`. And I don't think validating `markDeletePosition` is a good way, you can take other tests in `SubscriptionSeekTest` as reference. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] Jason918 commented on a change in pull request #12032: [consumer] Add seek by index for consumer

Posted by GitBox <gi...@apache.org>.
Jason918 commented on a change in pull request #12032:
URL: https://github.com/apache/pulsar/pull/12032#discussion_r717614426



##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConsumerSeekTest.java
##########
@@ -0,0 +1,108 @@
+/**
+ * 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 java.nio.charset.StandardCharsets;
+import java.util.HashSet;
+import java.util.Set;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.MessageId;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerConsumerBase;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class ConsumerSeekTest extends ProducerConsumerBase {
+
+    @BeforeClass
+    @Override
+    protected void setup() throws Exception {
+        Set<String> interceptorNames = new HashSet<>();
+        interceptorNames.add("org.apache.pulsar.common.intercept.AppendBrokerTimestampMetadataInterceptor");
+        interceptorNames.add("org.apache.pulsar.common.intercept.AppendIndexMetadataInterceptor");
+        conf.setBrokerEntryMetadataInterceptors(interceptorNames);
+        conf.setExposingBrokerEntryMetadataToClientEnabled(true);
+        enableBrokerInterceptor = true;
+        internalSetup();
+        producerBaseSetup();
+    }
+
+    @AfterClass
+    @Override
+    protected void cleanup() throws Exception {
+        internalCleanup();
+    }
+
+    @Test
+    public void testConsumerSeekByIndex() throws Exception {

Review comment:
       Updated. PTAL.

##########
File path: pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Consumer.java
##########
@@ -599,6 +599,15 @@
      */
     void seek(long timestamp) throws PulsarClientException;
 
+
+    /**
+     * Reset the subscription associated with this consumer to a specific message index.
+     *
+     * @param index
+     *            the message index where to reposition the subscription
+     */

Review comment:
       Updated the doc in Consumer.java. 
   And I have send this PR as PIP 100 to the mail list. PTAL.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] Jason918 closed pull request #12032: [PIP 101] Add seek by index feature for consumer

Posted by GitBox <gi...@apache.org>.
Jason918 closed pull request #12032:
URL: https://github.com/apache/pulsar/pull/12032


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] Jason918 commented on pull request #12032: [consumer] Add seek by index for consumer

Posted by GitBox <gi...@apache.org>.
Jason918 commented on pull request #12032:
URL: https://github.com/apache/pulsar/pull/12032#issuecomment-918852114


   /pulsarbot run-failure-checks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] Jason918 commented on pull request #12032: [consumer] Add seek by index for consumer

Posted by GitBox <gi...@apache.org>.
Jason918 commented on pull request #12032:
URL: https://github.com/apache/pulsar/pull/12032#issuecomment-929802708


   /pulsarbot run-failure-checks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MarvinCai commented on pull request #12032: [consumer] Add seek by index for consumer

Posted by GitBox <gi...@apache.org>.
MarvinCai commented on pull request #12032:
URL: https://github.com/apache/pulsar/pull/12032#issuecomment-929850723






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] Jason918 commented on pull request #12032: [PIP 101] Add seek by index feature for consumer

Posted by GitBox <gi...@apache.org>.
Jason918 commented on pull request #12032:
URL: https://github.com/apache/pulsar/pull/12032#issuecomment-1031117390


   Closing this for now, as there are not enough votes.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] BewareMyPower commented on a change in pull request #12032: [consumer] Add seek by index for consumer

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on a change in pull request #12032:
URL: https://github.com/apache/pulsar/pull/12032#discussion_r717234322



##########
File path: pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Consumer.java
##########
@@ -599,6 +599,15 @@
      */
     void seek(long timestamp) throws PulsarClientException;
 
+
+    /**
+     * Reset the subscription associated with this consumer to a specific message index.
+     *
+     * @param index
+     *            the message index where to reposition the subscription
+     */

Review comment:
       We should define the behavior for corner cases. For example, if there're 10 messages in a topic, i.e. the offset range varies from 0 to 9. What will happen if the given index is -1 or 10?
   
   There is another corner case. If the offset range is `[10, 20)`, the oldest messages (`[0, 10)`) have been deleted. What will happen if the given index is less than 10?
   
   BTW, it's a change for public API. I think we need a PIP first. /cc @merlimat 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] Jason918 commented on a change in pull request #12032: [consumer] Add seek by index for consumer

Posted by GitBox <gi...@apache.org>.
Jason918 commented on a change in pull request #12032:
URL: https://github.com/apache/pulsar/pull/12032#discussion_r717615692



##########
File path: pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Consumer.java
##########
@@ -599,6 +599,15 @@
      */
     void seek(long timestamp) throws PulsarClientException;
 
+
+    /**
+     * Reset the subscription associated with this consumer to a specific message index.
+     *
+     * @param index
+     *            the message index where to reposition the subscription
+     */

Review comment:
       Updated the doc in Consumer.java. 
   And I have send this PR as PIP 100 to the mail list. PTAL.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MarvinCai commented on pull request #12032: [consumer] Add seek by index for consumer

Posted by GitBox <gi...@apache.org>.
MarvinCai commented on pull request #12032:
URL: https://github.com/apache/pulsar/pull/12032#issuecomment-929852505


   But let's discuss it with the community.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] MarvinCai edited a comment on pull request #12032: [consumer] Add seek by index for consumer

Posted by GitBox <gi...@apache.org>.
MarvinCai edited a comment on pull request #12032:
URL: https://github.com/apache/pulsar/pull/12032#issuecomment-929850723






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org