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 2022/12/04 10:18:12 UTC

[GitHub] [pulsar] labuladong opened a new issue, #18731: [Bug] shadow topic replicator may fail due to retention policy

labuladong opened a new issue, #18731:
URL: https://github.com/apache/pulsar/issues/18731

   ### Search before asking
   
   - [X] I searched in the [issues](https://github.com/apache/pulsar/issues) and found nothing similar.
   
   
   ### Version
   
   Master branch
   
   ### Minimal reproduce step
   
   Based on the flaky test https://github.com/apache/pulsar/issues/18705, I modified it to reproduce the bug with a high probability:
   
   ```java
   @Test
       public void testShadowTopicConsuming() throws Exception {
           String sourceTopic = newShadowSourceTopicName();
           String shadowTopic = sourceTopic + "-shadow";
           admin.topics().createNonPartitionedTopic(sourceTopic);
           admin.topics().createShadowTopic(shadowTopic, sourceTopic);
           admin.topics().setShadowTopics(sourceTopic, Lists.newArrayList(shadowTopic));
           // wait init finish
           Thread.sleep(1000 * 3);
   
           PersistentTopic brokerSourceTopic =
                   (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(sourceTopic).get().get();
           ManagedLedgerConfig managedLedgerConfig = brokerSourceTopic.getManagedLedger().getConfig();
           // rollover ledger frequently
           managedLedgerConfig.setMaxEntriesPerLedger(2);
           managedLedgerConfig.setMinimumRolloverTime(1, TimeUnit.MILLISECONDS);
           managedLedgerConfig.setMaximumRolloverTime(1, TimeUnit.SECONDS);
   
   
           @Cleanup Producer<byte[]> producer = pulsarClient.newProducer()
                   .sendTimeout(1, TimeUnit.DAYS)
                   .topic(sourceTopic).create();
           @Cleanup Consumer<byte[]> consumer =
                   pulsarClient.newConsumer().topic(shadowTopic).subscriptionName("sub").subscribe();
           ArrayList<MessageId> idList = new ArrayList<>();
           for (int i = 0; i < 10; i++) {
               byte[] content = ("Hello Shadow Topic" + i).getBytes(StandardCharsets.UTF_8);
               MessageId id = producer.send(content);
               log.info("msg send to source topic, id={}", id);
               idList.add(id);
           }
           for (int i = 0; i < 10; i++) {
               byte[] content = ("Hello Shadow Topic" + i).getBytes(StandardCharsets.UTF_8);
               MessageId id = idList.get(i);
               // this msg would be null with a high probability
               Message<byte[]> msg = consumer.receive(5, TimeUnit.SECONDS);
               
               System.out.println("receive: " + msg.getMessageId());
               Assert.assertEquals(msg.getMessageId(), id);
               Assert.assertEquals(msg.getValue(), content);
           }
       }
   ```
   
   The consumers on the shadow topic cannot receive new messages due to `internalTrimLedgers` deleting the metadata of old ledgers.
   
   Log output:
   
   ```shell
   
   2022-12-04T17:51:40,886 - INFO  - [metadata-store-12-1:Slf4jRequestLogWriter@62] - 127.0.0.1 - - [04/12月/2022:17:51:40 +0800] "GET /admin/v2/schemas/prop/ns-abc/topic-c97789885d35e455-shadow/schema HTTP/1.1" 404 29 "-" "Pulsar-Java-v2.11.0-SNAPSHOT" 9
   2022-12-04T17:51:40,887 - WARN  - [pulsar-io-6-12:HttpClient@235] - [http://localhost:63449/admin/v2/schemas/prop/ns-abc/topic-c97789885d35e455-shadow/schema] HTTP get request failed: Schema not found
   2022-12-04T17:51:40,890 - INFO  - [pulsar-4-2:Slf4jRequestLogWriter@62] - 127.0.0.1 - - [04/12月/2022:17:51:40 +0800] "GET /admin/v2/persistent/prop/ns-abc/topic-c97789885d35e455-shadow/partitions?checkAllowAutoCreation=true HTTP/1.1" 200 32 "-" "Pulsar-Java-v2.11.0-SNAPSHOT" 3
   2022-12-04T17:51:40,897 - INFO  - [pulsar-web-32-8:Slf4jRequestLogWriter@62] - 127.0.0.1 - - [04/12月/2022:17:51:40 +0800] "GET /lookup/v2/topic/persistent/prop/ns-abc/topic-c97789885d35e455-shadow HTTP/1.1" 200 217 "-" "Pulsar-Java-v2.11.0-SNAPSHOT" 5
   2022-12-04T17:51:40,898 - INFO  - [pulsar-io-6-13:ConnectionPool@248] - [[id: 0x8ba4f938, L:/127.0.0.1:63483 - R:localhost/127.0.0.1:63424]] Connected to server
   2022-12-04T17:51:40,899 - INFO  - [pulsar-io-6-14:ServerCnx@304] - New connection from /127.0.0.1:63483
   2022-12-04T17:51:40,900 - INFO  - [pulsar-io-6-13:ProducerImpl@1650] - [persistent://prop/ns-abc/topic-c97789885d35e455-shadow] [pulsar.repl-persistent://prop/ns-abc/topic-c97789885d35e455-->persistent://prop/ns-abc/topic-c97789885d35e455-shadow] Creating producer on cnx [id: 0x8ba4f938, L:/127.0.0.1:63483 - R:localhost/127.0.0.1:63424]
   2022-12-04T17:51:40,907 - INFO  - [metadata-store-12-1:ServerCnx@1484] - [/127.0.0.1:63483] Created new producer: Producer{topic=PersistentTopic{topic=persistent://prop/ns-abc/topic-c97789885d35e455-shadow}, client=/127.0.0.1:63483, producerName=pulsar.repl-persistent://prop/ns-abc/topic-c97789885d35e455-->persistent://prop/ns-abc/topic-c97789885d35e455-shadow, producerId=0}
   2022-12-04T17:51:40,907 - INFO  - [pulsar-io-6-13:ProducerImpl@1705] - [persistent://prop/ns-abc/topic-c97789885d35e455-shadow] [pulsar.repl-persistent://prop/ns-abc/topic-c97789885d35e455-->persistent://prop/ns-abc/topic-c97789885d35e455-shadow] Created producer on cnx [id: 0x8ba4f938, L:/127.0.0.1:63483 - R:localhost/127.0.0.1:63424]
   2022-12-04T17:51:40,908 - INFO  - [pulsar-io-6-13:ManagedCursorImpl@2426] - [prop/ns-abc/persistent/topic-c97789885d35e455-pulsar.repl-persistent%3A%2F%2Fprop%2Fns-abc%2Ftopic-c97789885d35e455-shadow] Rewind from 3:0 to 3:0
   2022-12-04T17:51:40,908 - INFO  - [pulsar-io-6-13:PersistentReplicator@145] - [persistent://prop/ns-abc/topic-c97789885d35e455-->persistent://prop/ns-abc/topic-c97789885d35e455-shadow | test] Created replicator producer
   2022-12-04T17:51:40,919 - INFO  - [pulsar-4-4:Slf4jRequestLogWriter@62] - 127.0.0.1 - - [04/12月/2022:17:51:40 +0800] "GET /admin/v2/persistent/prop/ns-abc/topic-c97789885d35e455/partitions?checkAllowAutoCreation=true HTTP/1.1" 200 32 "-" "Pulsar-Java-v2.11.0-SNAPSHOT" 2
   2022-12-04T17:51:40,923 - INFO  - [pulsar-web-32-15:Slf4jRequestLogWriter@62] - 127.0.0.1 - - [04/12月/2022:17:51:40 +0800] "GET /lookup/v2/topic/persistent/prop/ns-abc/topic-c97789885d35e455 HTTP/1.1" 200 217 "-" "Pulsar-Java-v2.11.0-SNAPSHOT" 2
   2022-12-04T17:51:40,924 - INFO  - [pulsar-client-io-37-1:ConnectionPool@248] - [[id: 0x9d49d00b, L:/127.0.0.1:63485 - R:localhost/127.0.0.1:63424]] Connected to server
   2022-12-04T17:51:40,925 - INFO  - [pulsar-io-6-16:ServerCnx@304] - New connection from /127.0.0.1:63485
   2022-12-04T17:51:40,927 - INFO  - [pulsar-client-io-37-1:ProducerImpl@1650] - [persistent://prop/ns-abc/topic-c97789885d35e455] [null] Creating producer on cnx [id: 0x9d49d00b, L:/127.0.0.1:63485 - R:localhost/127.0.0.1:63424]
   2022-12-04T17:51:40,928 - INFO  - [pulsar-io-6-16:ServerCnx@1484] - [/127.0.0.1:63485] Created new producer: Producer{topic=PersistentTopic{topic=persistent://prop/ns-abc/topic-c97789885d35e455}, client=/127.0.0.1:63485, producerName=test-0-1, producerId=0}
   2022-12-04T17:51:40,928 - INFO  - [pulsar-client-io-37-1:ProducerImpl@1705] - [persistent://prop/ns-abc/topic-c97789885d35e455] [test-0-1] Created producer on cnx [id: 0x9d49d00b, L:/127.0.0.1:63485 - R:localhost/127.0.0.1:63424]
   2022-12-04T17:51:40,934 - INFO  - [pulsar-4-5:Slf4jRequestLogWriter@62] - 127.0.0.1 - - [04/12月/2022:17:51:40 +0800] "GET /admin/v2/persistent/prop/ns-abc/topic-c97789885d35e455-shadow/partitions?checkAllowAutoCreation=true HTTP/1.1" 200 32 "-" "Pulsar-Java-v2.11.0-SNAPSHOT" 2
   2022-12-04T17:51:40,936 - INFO  - [pulsar-web-32-16:Slf4jRequestLogWriter@62] - 127.0.0.1 - - [04/12月/2022:17:51:40 +0800] "GET /lookup/v2/topic/persistent/prop/ns-abc/topic-c97789885d35e455-shadow HTTP/1.1" 200 217 "-" "Pulsar-Java-v2.11.0-SNAPSHOT" 2
   2022-12-04T17:51:40,936 - INFO  - [pulsar-client-io-37-1:ConsumerImpl@785] - [persistent://prop/ns-abc/topic-c97789885d35e455-shadow][sub] Subscribing to topic on cnx [id: 0x9d49d00b, L:/127.0.0.1:63485 - R:localhost/127.0.0.1:63424], consumerId 0
   2022-12-04T17:51:40,937 - INFO  - [pulsar-io-6-16:ServerCnx@1052] - [/127.0.0.1:63485] Subscribing on topic persistent://prop/ns-abc/topic-c97789885d35e455-shadow / sub
   2022-12-04T17:51:40,938 - INFO  - [pulsar-io-6-16:ManagedCursorImpl@661] - [prop/ns-abc/persistent/topic-c97789885d35e455-shadow] Cursor sub recovered to position 3:-1
   2022-12-04T17:51:40,941 - INFO  - [bookkeeper-ml-scheduler-OrderedScheduler-4-0:ManagedLedgerImpl$7@994] - [prop/ns-abc/persistent/topic-c97789885d35e455-shadow] Opened new cursor: ManagedCursorImpl{ledger=prop/ns-abc/persistent/topic-c97789885d35e455-shadow, name=sub, ackPos=3:-1, readPos=3:0}
   2022-12-04T17:51:40,942 - INFO  - [bookkeeper-ml-scheduler-OrderedScheduler-4-0:ManagedCursorImpl@2426] - [prop/ns-abc/persistent/topic-c97789885d35e455-shadow-sub] Rewind from 3:0 to 3:0
   2022-12-04T17:51:40,942 - INFO  - [bookkeeper-ml-scheduler-OrderedScheduler-4-0:ServerCnx@1144] - [/127.0.0.1:63485] Created subscription on topic persistent://prop/ns-abc/topic-c97789885d35e455-shadow / sub
   2022-12-04T17:51:40,942 - INFO  - [pulsar-client-io-37-1:ConsumerImpl@919] - [persistent://prop/ns-abc/topic-c97789885d35e455-shadow][sub] Subscribed to topic on localhost/127.0.0.1:63424 -- consumer: 0
   2022-12-04T17:51:40,946 - INFO  - [main:ShadowTopicTest@165] - msg send to source topic, id=3:0:-1
   2022-12-04T17:51:40,948 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:OpAddEntry@241] - [prop/ns-abc/persistent/topic-c97789885d35e455] Closing ledger 3 for being full
   2022-12-04T17:51:40,949 - INFO  - [main:ShadowTopicTest@165] - msg send to source topic, id=3:1:-1
   2022-12-04T17:51:40,950 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ManagedLedgerImpl@835] - [prop/ns-abc/persistent/topic-c97789885d35e455] Creating a new ledger
   2022-12-04T17:51:40,950 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:PulsarMockBookKeeper@122] - Creating ledger 6
   2022-12-04T17:51:40,951 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ManagedLedgerImpl@1551] - [prop/ns-abc/persistent/topic-c97789885d35e455] Created new ledger 6
   2022-12-04T17:51:40,958 - INFO  - [main:ShadowTopicTest@165] - msg send to source topic, id=6:0:-1
   2022-12-04T17:51:40,960 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:OpAddEntry@241] - [prop/ns-abc/persistent/topic-c97789885d35e455] Closing ledger 6 for being full
   2022-12-04T17:51:40,960 - INFO  - [main:ShadowTopicTest@165] - msg send to source topic, id=6:1:-1
   2022-12-04T17:51:40,966 - INFO  - [bookkeeper-ml-scheduler-OrderedScheduler-2-0:ManagedLedgerImpl$18@2697] - [prop/ns-abc/persistent/topic-c97789885d35e455] End TrimConsumedLedgers. ledgers=1 totalSize=100
   2022-12-04T17:51:40,966 - INFO  - [bookkeeper-ml-scheduler-OrderedScheduler-2-0:ManagedLedgerImpl$18@2704] - [prop/ns-abc/persistent/topic-c97789885d35e455] Removing ledger 3 - size: 100
   2022-12-04T17:51:40,962 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ShadowManagedLedgerImpl@272] - [prop/ns-abc/persistent/topic-c97789885d35e455-shadow][prop/ns-abc/persistent/topic-c97789885d35e455] new SourceManagedLedgerInfo:ledgerInfo {
     ledgerId: 3
     entries: 2
     size: 100
     timestamp: 1670147500948
   }
   ledgerInfo {
     ledgerId: 6
     timestamp: 1670147500961
   }
   , prevStat=Stat(path=/managed-ledgers/prop/ns-abc/persistent/topic-c97789885d35e455, version=1, creationTimestamp=0, modificationTimestamp=0, ephemeral=true, createdBySelf=false),stat=Stat(path=/managed-ledgers/prop/ns-abc/persistent/topic-c97789885d35e455, version=2, creationTimestamp=0, modificationTimestamp=0, ephemeral=true, createdBySelf=false)
   2022-12-04T17:51:40,993 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ShadowManagedLedgerImpl@304] - [prop/ns-abc/persistent/topic-c97789885d35e455-shadow] Old ledger info updated in source,ledgerId=3
   2022-12-04T17:51:40,993 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ManagedLedgerImpl@835] - [prop/ns-abc/persistent/topic-c97789885d35e455] Creating a new ledger
   2022-12-04T17:51:40,994 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ShadowManagedLedgerImpl@272] - [prop/ns-abc/persistent/topic-c97789885d35e455-shadow][prop/ns-abc/persistent/topic-c97789885d35e455] new SourceManagedLedgerInfo:ledgerInfo {
     ledgerId: 6
     entries: 2
     size: 100
     timestamp: 1670147500960
   }
   , prevStat=Stat(path=/managed-ledgers/prop/ns-abc/persistent/topic-c97789885d35e455, version=2, creationTimestamp=0, modificationTimestamp=0, ephemeral=true, createdBySelf=false),stat=Stat(path=/managed-ledgers/prop/ns-abc/persistent/topic-c97789885d35e455, version=3, creationTimestamp=0, modificationTimestamp=0, ephemeral=true, createdBySelf=false)
   2022-12-04T17:51:40,994 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ShadowManagedLedgerImpl@304] - [prop/ns-abc/persistent/topic-c97789885d35e455-shadow] Old ledger info updated in source,ledgerId=6
   2022-12-04T17:51:40,994 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ShadowManagedLedgerImpl@356] - [prop/ns-abc/persistent/topic-c97789885d35e455-shadow]ledgers deleted in source, size=1
   2022-12-04T17:51:40,994 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ManagedLedgerImpl@2734] - [prop/ns-abc/persistent/topic-c97789885d35e455-shadow] Ledger 3 contains the current last confirmed entry 3:1, and it is going to be deleted
   2022-12-04T17:51:40,994 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:PulsarMockBookKeeper@122] - Creating ledger 7
   2022-12-04T17:51:40,995 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ManagedLedgerImpl@1551] - [prop/ns-abc/persistent/topic-c97789885d35e455] Created new ledger 7
   2022-12-04T17:51:40,995 - WARN  - [mock-pulsar-bk-OrderedExecutor-0-0:ManagedLedgerImpl@2378] - Cursor: ManagedCursorImpl{ledger=prop/ns-abc/persistent/topic-c97789885d35e455-shadow, name=sub, ackPos=3:-1, readPos=3:2} does not exist in the managed-ledger.
   2022-12-04T17:51:40,996 - WARN  - [mock-pulsar-bk-OrderedExecutor-0-0:ManagedLedgerImpl@2378] - Cursor: ManagedCursorImpl{ledger=prop/ns-abc/persistent/topic-c97789885d35e455-shadow, name=sub, ackPos=3:-1, readPos=3:2} does not exist in the managed-ledger.
   2022-12-04T17:51:40,996 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:Producer@696] - Disconnecting producer: Producer{topic=PersistentTopic{topic=persistent://prop/ns-abc/topic-c97789885d35e455-shadow}, client=/127.0.0.1:63483, producerName=pulsar.repl-persistent://prop/ns-abc/topic-c97789885d35e455-->persistent://prop/ns-abc/topic-c97789885d35e455-shadow, producerId=0}
   2022-12-04T17:51:40,996 - WARN  - [mock-pulsar-bk-OrderedExecutor-0-0:PersistentTopic@590] - [persistent://prop/ns-abc/topic-c97789885d35e455-shadow] Failed to persist msg in store: Unexpected add entry op when complete the add entry op.
   2022-12-04T17:51:40,996 - INFO  - [pulsar-io-6-13:ClientCnx@778] - [localhost/127.0.0.1:63424] Broker notification of Closed producer: 0
   2022-12-04T17:51:40,996 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:PersistentTopic@523] - [persistent://prop/ns-abc/topic-c97789885d35e455-shadow] Un-fencing topic...
   2022-12-04T17:51:40,996 - WARN  - [mock-pulsar-bk-OrderedExecutor-0-0:PersistentTopic@590] - [persistent://prop/ns-abc/topic-c97789885d35e455-shadow] Failed to persist msg in store: Unexpected add entry op when complete the add entry op.
   2022-12-04T17:51:40,996 - INFO  - [pulsar-io-6-13:ConnectionHandler@144] - [persistent://prop/ns-abc/topic-c97789885d35e455-shadow] [pulsar.repl-persistent://prop/ns-abc/topic-c97789885d35e455-->persistent://prop/ns-abc/topic-c97789885d35e455-shadow] Closed connection [id: 0x8ba4f938, L:/127.0.0.1:63483 - R:localhost/127.0.0.1:63424] -- Will try again in 0.1 s
   2022-12-04T17:51:40,997 - WARN  - [pulsar-io-6-13:ClientCnx@722] - [id: 0x8ba4f938, L:/127.0.0.1:63483 - R:localhost/127.0.0.1:63424] Received send error from server: PersistenceError : org.apache.bookkeeper.mledger.ManagedLedgerException: Unexpected add entry op when complete the add entry op.
   2022-12-04T17:51:40,998 - INFO  - [main:ShadowTopicTest@165] - msg send to source topic, id=7:0:-1
   2022-12-04T17:51:40,998 - INFO  - [pulsar-io-6-14:ServerCnx@316] - Closed connection from /127.0.0.1:63483
   2022-12-04T17:51:40,998 - WARN  - [pulsar-io-6-13:ClientCnx@722] - [id: 0x8ba4f938, L:/127.0.0.1:63483 ! R:localhost/127.0.0.1:63424] Received send error from server: PersistenceError : org.apache.bookkeeper.mledger.ManagedLedgerException: Unexpected add entry op when complete the add entry op.
   2022-12-04T17:51:40,998 - INFO  - [pulsar-io-6-13:ClientCnx@294] - [id: 0x8ba4f938, L:/127.0.0.1:63483 ! R:localhost/127.0.0.1:63424] Disconnected
   2022-12-04T17:51:41,000 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:OpAddEntry@241] - [prop/ns-abc/persistent/topic-c97789885d35e455] Closing ledger 7 for being full
   2022-12-04T17:51:41,000 - INFO  - [main:ShadowTopicTest@165] - msg send to source topic, id=7:1:-1
   2022-12-04T17:51:41,001 - INFO  - [bookkeeper-ml-scheduler-OrderedScheduler-2-0:ManagedLedgerImpl$18@2697] - [prop/ns-abc/persistent/topic-c97789885d35e455] End TrimConsumedLedgers. ledgers=1 totalSize=100
   2022-12-04T17:51:41,001 - INFO  - [bookkeeper-ml-scheduler-OrderedScheduler-2-0:ManagedLedgerImpl$18@2704] - [prop/ns-abc/persistent/topic-c97789885d35e455] Removing ledger 6 - size: 100
   2022-12-04T17:51:41,001 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ShadowManagedLedgerImpl@272] - [prop/ns-abc/persistent/topic-c97789885d35e455-shadow][prop/ns-abc/persistent/topic-c97789885d35e455] new SourceManagedLedgerInfo:ledgerInfo {
     ledgerId: 6
     entries: 2
     size: 100
     timestamp: 1670147500960
   }
   ledgerInfo {
     ledgerId: 7
     timestamp: 1670147501001
   }
   , prevStat=Stat(path=/managed-ledgers/prop/ns-abc/persistent/topic-c97789885d35e455, version=3, creationTimestamp=0, modificationTimestamp=0, ephemeral=true, createdBySelf=false),stat=Stat(path=/managed-ledgers/prop/ns-abc/persistent/topic-c97789885d35e455, version=4, creationTimestamp=0, modificationTimestamp=0, ephemeral=true, createdBySelf=false)
   2022-12-04T17:51:41,001 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ShadowManagedLedgerImpl@304] - [prop/ns-abc/persistent/topic-c97789885d35e455-shadow] Old ledger info updated in source,ledgerId=6
   2022-12-04T17:51:41,001 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ManagedLedgerImpl@835] - [prop/ns-abc/persistent/topic-c97789885d35e455] Creating a new ledger
   2022-12-04T17:51:41,001 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:PulsarMockBookKeeper@122] - Creating ledger 8
   2022-12-04T17:51:41,002 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ManagedLedgerImpl@1551] - [prop/ns-abc/persistent/topic-c97789885d35e455] Created new ledger 8
   2022-12-04T17:51:41,002 - WARN  - [mock-pulsar-bk-OrderedExecutor-0-0:ManagedLedgerImpl@2378] - Cursor: ManagedCursorImpl{ledger=prop/ns-abc/persistent/topic-c97789885d35e455-shadow, name=sub, ackPos=3:-1, readPos=3:2} does not exist in the managed-ledger.
   2022-12-04T17:51:41,006 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ShadowManagedLedgerImpl@272] - [prop/ns-abc/persistent/topic-c97789885d35e455-shadow][prop/ns-abc/persistent/topic-c97789885d35e455] new SourceManagedLedgerInfo:ledgerInfo {
     ledgerId: 7
     entries: 2
     size: 100
     timestamp: 1670147501000
   }
   , prevStat=Stat(path=/managed-ledgers/prop/ns-abc/persistent/topic-c97789885d35e455, version=4, creationTimestamp=0, modificationTimestamp=0, ephemeral=true, createdBySelf=false),stat=Stat(path=/managed-ledgers/prop/ns-abc/persistent/topic-c97789885d35e455, version=5, creationTimestamp=0, modificationTimestamp=0, ephemeral=true, createdBySelf=false)
   2022-12-04T17:51:41,006 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ShadowManagedLedgerImpl@304] - [prop/ns-abc/persistent/topic-c97789885d35e455-shadow] Old ledger info updated in source,ledgerId=7
   2022-12-04T17:51:41,006 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ShadowManagedLedgerImpl@356] - [prop/ns-abc/persistent/topic-c97789885d35e455-shadow]ledgers deleted in source, size=1
   2022-12-04T17:51:41,010 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:PulsarMockBookKeeper@122] - Creating ledger 9
   2022-12-04T17:51:41,010 - WARN  - [mock-pulsar-bk-OrderedExecutor-0-0:ManagedLedgerImpl@1957] - [prop/ns-abc/persistent/topic-c97789885d35e455-shadow] Failed to close a Ledger ReadHandle:
   java.util.concurrent.CompletionException: org.apache.bookkeeper.mledger.ManagedLedgerException$LedgerNotExistException: No such ledger exists on Bookies
   	at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:332) ~[?:?]
   	at java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:347) ~[?:?]
   	at java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:708) ~[?:?]
   	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:510) ~[?:?]
   	at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:2162) ~[?:?]
   	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.lambda$getLedgerHandle$20(ManagedLedgerImpl.java:1940) ~[classes/:?]
   	at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:863) ~[?:?]
   	at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:841) ~[?:?]
   	at java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:482) ~[?:?]
   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136) ~[?:?]
   	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635) ~[?:?]
   	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) ~[netty-common-4.1.77.Final.jar:4.1.77.Final]
   	at java.lang.Thread.run(Thread.java:833) ~[?:?]
   Caused by: org.apache.bookkeeper.mledger.ManagedLedgerException$LedgerNotExistException: No such ledger exists on Bookies
   2022-12-04T17:51:41,010 - ERROR - [mock-pulsar-bk-OrderedExecutor-0-0:ManagedLedgerImpl@1865] - [prop/ns-abc/persistent/topic-c97789885d35e455-shadow] Error opening ledger for reading at position 6:0 - org.apache.bookkeeper.mledger.ManagedLedgerException$LedgerNotExistException: No such ledger exists on Bookies
   2022-12-04T17:51:41,010 - WARN  - [mock-pulsar-bk-OrderedExecutor-0-0:OpReadEntry@123] - [prop/ns-abc/persistent/topic-c97789885d35e455-shadow][sub] read failed from ledger at position:6:0
   org.apache.bookkeeper.mledger.ManagedLedgerException$LedgerNotExistException: No such ledger exists on Bookies
   2022-12-04T17:51:41,011 - ERROR - [broker-topic-workers-OrderedExecutor-6-0:PersistentDispatcherSingleActiveConsumer@502] - [persistent://prop/ns-abc/topic-c97789885d35e455-shadow / sub-Consumer{subscription=PersistentSubscription{topic=persistent://prop/ns-abc/topic-c97789885d35e455-shadow, name=sub}, consumerId=0, consumerName=6cad1, address=/127.0.0.1:63485}] Error reading entries at 7:0 : No such ledger exists on Bookies - Retrying to read in 15.0 seconds
   2022-12-04T17:51:41,011 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ShadowManagedLedgerImpl@272] - [prop/ns-abc/persistent/topic-c97789885d35e455-shadow][prop/ns-abc/persistent/topic-c97789885d35e455] new SourceManagedLedgerInfo:ledgerInfo {
     ledgerId: 7
     entries: 2
     size: 100
     timestamp: 1670147501000
   }
   ledgerInfo {
     ledgerId: 8
     timestamp: 1670147501011
   }
   , prevStat=Stat(path=/managed-ledgers/prop/ns-abc/persistent/topic-c97789885d35e455, version=5, creationTimestamp=0, modificationTimestamp=0, ephemeral=true, createdBySelf=false),stat=Stat(path=/managed-ledgers/prop/ns-abc/persistent/topic-c97789885d35e455, version=6, creationTimestamp=0, modificationTimestamp=0, ephemeral=true, createdBySelf=false)
   2022-12-04T17:51:41,012 - INFO  - [main:ShadowTopicTest@165] - msg send to source topic, id=8:0:-1
   2022-12-04T17:51:41,018 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:OpAddEntry@241] - [prop/ns-abc/persistent/topic-c97789885d35e455] Closing ledger 8 for being full
   2022-12-04T17:51:41,019 - INFO  - [main:ShadowTopicTest@165] - msg send to source topic, id=8:1:-1
   2022-12-04T17:51:41,019 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ManagedLedgerImpl@835] - [prop/ns-abc/persistent/topic-c97789885d35e455] Creating a new ledger
   2022-12-04T17:51:41,020 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:PulsarMockBookKeeper@122] - Creating ledger 10
   2022-12-04T17:51:41,020 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ManagedLedgerImpl@1551] - [prop/ns-abc/persistent/topic-c97789885d35e455] Created new ledger 10
   2022-12-04T17:51:41,021 - INFO  - [bookkeeper-ml-scheduler-OrderedScheduler-2-0:ManagedLedgerImpl$18@2697] - [prop/ns-abc/persistent/topic-c97789885d35e455] End TrimConsumedLedgers. ledgers=1 totalSize=100
   2022-12-04T17:51:41,021 - INFO  - [bookkeeper-ml-scheduler-OrderedScheduler-4-0:ManagedCursorImpl$30@3007] - [prop/ns-abc/persistent/topic-c97789885d35e455-shadow] Updated cursor sub with ledger id 9 md-position=7:-1 rd-position=7:0
   2022-12-04T17:51:41,021 - INFO  - [bookkeeper-ml-scheduler-OrderedScheduler-2-0:ManagedLedgerImpl$18@2704] - [prop/ns-abc/persistent/topic-c97789885d35e455] Removing ledger 7 - size: 100
   2022-12-04T17:51:41,026 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ShadowManagedLedgerImpl@272] - [prop/ns-abc/persistent/topic-c97789885d35e455-shadow][prop/ns-abc/persistent/topic-c97789885d35e455] new SourceManagedLedgerInfo:ledgerInfo {
     ledgerId: 8
     entries: 2
     size: 100
     timestamp: 1670147501018
   }
   , prevStat=Stat(path=/managed-ledgers/prop/ns-abc/persistent/topic-c97789885d35e455, version=6, creationTimestamp=0, modificationTimestamp=0, ephemeral=true, createdBySelf=false),stat=Stat(path=/managed-ledgers/prop/ns-abc/persistent/topic-c97789885d35e455, version=7, creationTimestamp=0, modificationTimestamp=0, ephemeral=true, createdBySelf=false)
   2022-12-04T17:51:41,026 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ShadowManagedLedgerImpl@356] - [prop/ns-abc/persistent/topic-c97789885d35e455-shadow]ledgers deleted in source, size=1
   2022-12-04T17:51:41,098 - INFO  - [broker-client-shared-timer-11-1:ConnectionHandler@148] - [persistent://prop/ns-abc/topic-c97789885d35e455-shadow] [pulsar.repl-persistent://prop/ns-abc/topic-c97789885d35e455-->persistent://prop/ns-abc/topic-c97789885d35e455-shadow] Reconnecting after timeout
   2022-12-04T17:51:41,102 - INFO  - [pulsar-web-32-1:Slf4jRequestLogWriter@62] - 127.0.0.1 - - [04/12月/2022:17:51:41 +0800] "GET /lookup/v2/topic/persistent/prop/ns-abc/topic-c97789885d35e455-shadow HTTP/1.1" 200 217 "-" "Pulsar-Java-v2.11.0-SNAPSHOT" 3
   2022-12-04T17:51:41,103 - INFO  - [pulsar-io-6-2:ConnectionPool@248] - [[id: 0x00cf3873, L:/127.0.0.1:63489 - R:localhost/127.0.0.1:63424]] Connected to server
   2022-12-04T17:51:41,104 - INFO  - [pulsar-io-6-3:ServerCnx@304] - New connection from /127.0.0.1:63489
   2022-12-04T17:51:41,105 - INFO  - [pulsar-io-6-2:ProducerImpl@1650] - [persistent://prop/ns-abc/topic-c97789885d35e455-shadow] [pulsar.repl-persistent://prop/ns-abc/topic-c97789885d35e455-->persistent://prop/ns-abc/topic-c97789885d35e455-shadow] Creating producer on cnx [id: 0x00cf3873, L:/127.0.0.1:63489 - R:localhost/127.0.0.1:63424]
   2022-12-04T17:51:41,106 - INFO  - [pulsar-io-6-3:ServerCnx@1484] - [/127.0.0.1:63489] Created new producer: Producer{topic=PersistentTopic{topic=persistent://prop/ns-abc/topic-c97789885d35e455-shadow}, client=/127.0.0.1:63489, producerName=pulsar.repl-persistent://prop/ns-abc/topic-c97789885d35e455-->persistent://prop/ns-abc/topic-c97789885d35e455-shadow, producerId=0}
   2022-12-04T17:51:41,106 - INFO  - [pulsar-io-6-2:ProducerImpl@1705] - [persistent://prop/ns-abc/topic-c97789885d35e455-shadow] [pulsar.repl-persistent://prop/ns-abc/topic-c97789885d35e455-->persistent://prop/ns-abc/topic-c97789885d35e455-shadow] Created producer on cnx [id: 0x00cf3873, L:/127.0.0.1:63489 - R:localhost/127.0.0.1:63424]
   2022-12-04T17:51:41,106 - INFO  - [pulsar-io-6-2:ProducerImpl@1886] - [persistent://prop/ns-abc/topic-c97789885d35e455-shadow] [pulsar.repl-persistent://prop/ns-abc/topic-c97789885d35e455-->persistent://prop/ns-abc/topic-c97789885d35e455-shadow] Re-Sending 4 messages to server
   2022-12-04T17:51:41,126 - INFO  - [bookkeeper-ml-scheduler-OrderedScheduler-2-0:ManagedLedgerImpl@2383] - Reset cursor:NonDurableCursorImpl{ledger=prop/ns-abc/persistent/topic-c97789885d35e455, ackPos=8:1, readPos=8:2} to 10:-1 since ledger consumed completely
   2022-12-04T17:51:41,128 - INFO  - [main:ShadowTopicTest@165] - msg send to source topic, id=10:0:-1
   2022-12-04T17:51:41,130 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:OpAddEntry@241] - [prop/ns-abc/persistent/topic-c97789885d35e455] Closing ledger 10 for being full
   2022-12-04T17:51:41,130 - INFO  - [main:ShadowTopicTest@165] - msg send to source topic, id=10:1:-1
   2022-12-04T17:51:41,130 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ShadowManagedLedgerImpl@272] - [prop/ns-abc/persistent/topic-c97789885d35e455-shadow][prop/ns-abc/persistent/topic-c97789885d35e455] new SourceManagedLedgerInfo:ledgerInfo {
     ledgerId: 8
     entries: 2
     size: 100
     timestamp: 1670147501018
   }
   ledgerInfo {
     ledgerId: 10
     timestamp: 1670147501130
   }
   , prevStat=Stat(path=/managed-ledgers/prop/ns-abc/persistent/topic-c97789885d35e455, version=7, creationTimestamp=0, modificationTimestamp=0, ephemeral=true, createdBySelf=false),stat=Stat(path=/managed-ledgers/prop/ns-abc/persistent/topic-c97789885d35e455, version=8, creationTimestamp=0, modificationTimestamp=0, ephemeral=true, createdBySelf=false)
   receive: 3:0:-1
   receive: 3:1:-1
   2022-12-04T17:51:41,136 - INFO  - [bookkeeper-ml-scheduler-OrderedScheduler-2-0:ManagedLedgerImpl$18@2697] - [prop/ns-abc/persistent/topic-c97789885d35e455] End TrimConsumedLedgers. ledgers=1 totalSize=100
   2022-12-04T17:51:41,136 - INFO  - [bookkeeper-ml-scheduler-OrderedScheduler-2-0:ManagedLedgerImpl$18@2704] - [prop/ns-abc/persistent/topic-c97789885d35e455] Removing ledger 8 - size: 100
   2022-12-04T17:51:41,141 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ShadowManagedLedgerImpl@272] - [prop/ns-abc/persistent/topic-c97789885d35e455-shadow][prop/ns-abc/persistent/topic-c97789885d35e455] new SourceManagedLedgerInfo:ledgerInfo {
     ledgerId: 10
     entries: 2
     size: 100
     timestamp: 1670147501130
   }
   , prevStat=Stat(path=/managed-ledgers/prop/ns-abc/persistent/topic-c97789885d35e455, version=8, creationTimestamp=0, modificationTimestamp=0, ephemeral=true, createdBySelf=false),stat=Stat(path=/managed-ledgers/prop/ns-abc/persistent/topic-c97789885d35e455, version=9, creationTimestamp=0, modificationTimestamp=0, ephemeral=true, createdBySelf=false)
   2022-12-04T17:51:41,141 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ShadowManagedLedgerImpl@304] - [prop/ns-abc/persistent/topic-c97789885d35e455-shadow] Old ledger info updated in source,ledgerId=10
   2022-12-04T17:51:41,141 - INFO  - [mock-pulsar-bk-OrderedExecutor-0-0:ShadowManagedLedgerImpl@356] - [prop/ns-abc/persistent/topic-c97789885d35e455-shadow]ledgers deleted in source, size=1
   2022-12-04T17:51:46,145 - INFO  - [pulsar-io-6-16:ServerCnx@1843] - [/127.0.0.1:63485] Closing consumer: consumerId=0
   2022-12-04T17:51:46,145 - INFO  - [pulsar-io-6-16:AbstractDispatcherSingleActiveConsumer@187] - Removing consumer Consumer{subscription=PersistentSubscription{topic=persistent://prop/ns-abc/topic-c97789885d35e455-shadow, name=sub}, consumerId=0, consumerName=6cad1, address=/127.0.0.1:63485}
   2022-12-04T17:51:46,147 - INFO  - [pulsar-io-6-16:ServerCnx@1879] - [/127.0.0.1:63485] Closed consumer, consumerId=0
   2022-12-04T17:51:46,148 - INFO  - [pulsar-client-io-37-1:ConsumerImpl@1061] - [persistent://prop/ns-abc/topic-c97789885d35e455-shadow] [sub] Closed consumer
   2022-12-04T17:51:46,149 - INFO  - [pulsar-io-6-16:ServerCnx@1827] - [PersistentTopic{topic=persistent://prop/ns-abc/topic-c97789885d35e455}][test-0-1] Closing producer on cnx /127.0.0.1:63485. producerId=0
   2022-12-04T17:51:46,149 - INFO  - [pulsar-io-6-16:ServerCnx@1831] - [PersistentTopic{topic=persistent://prop/ns-abc/topic-c97789885d35e455}][test-0-1] Closed producer on cnx /127.0.0.1:63485. producerId=0
   2022-12-04T17:51:46,150 - INFO  - [pulsar-client-io-37-1:ProducerImpl@1058] - [persistent://prop/ns-abc/topic-c97789885d35e455] [test-0-1] Closed Producer
   
   java.lang.NullPointerException: Cannot invoke "org.apache.pulsar.client.api.Message.getMessageId()" because "msg" is null
   
   	at org.apache.pulsar.broker.service.persistent.ShadowTopicTest.testShadowTopicConsuming(ShadowTopicTest.java:172)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)
   	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.base/java.lang.reflect.Method.invoke(Method.java:568)
   	at org.testng.internal.invokers.MethodInvocationHelper.invokeMethod(MethodInvocationHelper.java:139)
   	at org.testng.internal.invokers.TestInvoker.invokeMethod(TestInvoker.java:677)
   	at org.testng.internal.invokers.TestInvoker.invokeTestMethod(TestInvoker.java:221)
   	at org.testng.internal.invokers.MethodRunner.runInSequence(MethodRunner.java:50)
   	at org.testng.internal.invokers.TestInvoker$MethodInvocationAgent.invoke(TestInvoker.java:962)
   	at org.testng.internal.invokers.TestInvoker.invokeTestMethods(TestInvoker.java:194)
   	at org.testng.internal.invokers.TestMethodWorker.invokeTestMethods(TestMethodWorker.java:148)
   	at org.testng.internal.invokers.TestMethodWorker.run(TestMethodWorker.java:128)
   	at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
   	at org.testng.TestRunner.privateRun(TestRunner.java:806)
   	at org.testng.TestRunner.run(TestRunner.java:601)
   	at org.testng.SuiteRunner.runTest(SuiteRunner.java:433)
   	at org.testng.SuiteRunner.runSequentially(SuiteRunner.java:427)
   	at org.testng.SuiteRunner.privateRun(SuiteRunner.java:387)
   	at org.testng.SuiteRunner.run(SuiteRunner.java:330)
   	at org.testng.SuiteRunnerWorker.runSuite(SuiteRunnerWorker.java:52)
   	at org.testng.SuiteRunnerWorker.run(SuiteRunnerWorker.java:95)
   	at org.testng.TestNG.runSuitesSequentially(TestNG.java:1256)
   	at org.testng.TestNG.runSuitesLocally(TestNG.java:1176)
   	at org.testng.TestNG.runSuites(TestNG.java:1099)
   	at org.testng.TestNG.run(TestNG.java:1067)
   	at com.intellij.rt.testng.IDEARemoteTestNG.run(IDEARemoteTestNG.java:66)
   	at com.intellij.rt.testng.RemoteTestNGStarter.main(RemoteTestNGStarter.java:109)
   
   
   ```
   
   ### What did you expect to see?
   
   Consumer receive all 10 messages.
   
   ### What did you see instead?
   
   Consumer will receive a null message and cause an assertion error.
   
   ### Anything else?
   
   _No response_
   
   ### Are you willing to submit a PR?
   
   - [ ] I'm willing to submit a PR!


-- 
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.apache.org

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


[GitHub] [pulsar] Jason918 commented on issue #18731: [Bug] shadow topic replicator may fail due to retention policy

Posted by GitBox <gi...@apache.org>.
Jason918 commented on issue #18731:
URL: https://github.com/apache/pulsar/issues/18731#issuecomment-1338901017

   > > i think we can add an `durable` shadow cursor on the source topic side which tracks the slowest subscription position on the shadow topic side. after we get this the data is protected for shadow topic without being delete. look forward to others idea.
   > 
   > @Jason918 hi,how about this idea?I would like to discuss in the email thread if you think this is acceptable ^_^
   
   As @labuladong mentioned above, changing it to "durable cursor" won't solve the problem. 
   In the current design, there is no guarantee that shadow topic subscriptions will consume all the messages. Messages deleted from the source topic work as the message is expired.
   This behavior is not perfect, I am open to other ideas.


-- 
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] labuladong commented on issue #18731: [Bug] shadow topic replicator may fail due to retention policy

Posted by GitBox <gi...@apache.org>.
labuladong commented on issue #18731:
URL: https://github.com/apache/pulsar/issues/18731#issuecomment-1336599502

   I try to change `ledger.newNonDurableCursor` to `ledger.openCursor`, but this unit test still may fail... I guess it's due to the async update of metadata. I notice there are locks in `ManagedLedger` to protect async operation:
   
   https://github.com/apache/pulsar/blob/3df506de099f27d4c3b7fbf2064db881ffda0402/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java#L208-L215
   
   But `ShadowManagedLedger` didn't use any locks. I'm not sure if it is the root cause 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] lifepuzzlefun commented on issue #18731: [Bug] shadow topic replicator may fail due to retention policy

Posted by GitBox <gi...@apache.org>.
lifepuzzlefun commented on issue #18731:
URL: https://github.com/apache/pulsar/issues/18731#issuecomment-1336437279

   Just to share what i find.
   the UT above will trigger ledger rollover frequently. 
   when ledgerClose the method `trimConsumedLedgersInBackground` called.
   which will trigger delete all the available ledger without waiting for the shadow replicator.
   ![image](https://user-images.githubusercontent.com/13600283/205498734-7dc71908-8a43-4380-ac11-9aa57986cfb4.png)
   
   the current shadow replicator will create a non-durable cursor. 
   
   but after i change the code use an durable cursor for shadow replicator. the message is still be skipped without being send to shadow topic consumers. then i find the cursor in sourceTopic side is only trick the progress for the shadow replicator. the process of the shadow topic consumer is not tracked by the sourceTopic. so the message "lost"


-- 
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 issue #18731: [Bug] shadow topic replicator may fail due to retention policy

Posted by GitBox <gi...@apache.org>.
Jason918 commented on issue #18731:
URL: https://github.com/apache/pulsar/issues/18731#issuecomment-1338864227

   > Could you explain why we use a `non-durable` cursor in a shadow topic?
   
   Because we only need to replicate the latest messages to the shadow topic(to populate the entry cache and update LAC). No need to use the durable cursor.


-- 
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] lifepuzzlefun commented on issue #18731: [Bug] shadow topic replicator may fail due to retention policy

Posted by GitBox <gi...@apache.org>.
lifepuzzlefun commented on issue #18731:
URL: https://github.com/apache/pulsar/issues/18731#issuecomment-1336438544

   i think we can add an `durable` shadow cursor on the source topic side which tracks the slowest subscription position on the shadow topic side. after we get this the data is protected for shadow topic without being delete. look forward to others idea. 


-- 
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 issue #18731: [Bug] shadow topic replicator may fail due to retention policy

Posted by GitBox <gi...@apache.org>.
Jason918 commented on issue #18731:
URL: https://github.com/apache/pulsar/issues/18731#issuecomment-1338903936

   As for the flaky unit test, I think we can create a consumer for the source topic, to prevent the ledger to be deleted.
   
   But the messages at-least-once delivery guarantee is another issue that we can optimize for shadow topics.


-- 
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] labuladong commented on issue #18731: [Bug] shadow topic replicator may fail due to retention policy

Posted by GitBox <gi...@apache.org>.
labuladong commented on issue #18731:
URL: https://github.com/apache/pulsar/issues/18731#issuecomment-1336369866

   cc @Jason918 


-- 
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] labuladong closed issue #18731: [Bug] shadow topic replicator may fail due to retention policy

Posted by GitBox <gi...@apache.org>.
labuladong closed issue #18731: [Bug] shadow topic replicator may fail due to retention policy
URL: https://github.com/apache/pulsar/issues/18731


-- 
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] labuladong commented on issue #18731: [Bug] shadow topic replicator may fail due to retention policy

Posted by GitBox <gi...@apache.org>.
labuladong commented on issue #18731:
URL: https://github.com/apache/pulsar/issues/18731#issuecomment-1359060702

   > there is no guarantee that shadow topic subscriptions will consume all the messages. 
   > But the messages at-least-once delivery guarantee is another issue that we can optimize for shadow topics.
   
   So the message missing due to retention is the expected behavior, this issue can be closed.


-- 
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] lifepuzzlefun commented on issue #18731: [Bug] shadow topic replicator may fail due to retention policy

Posted by GitBox <gi...@apache.org>.
lifepuzzlefun commented on issue #18731:
URL: https://github.com/apache/pulsar/issues/18731#issuecomment-1338893654

   > i think we can add an `durable` shadow cursor on the source topic side which tracks the slowest subscription position on the shadow topic side. after we get this the data is protected for shadow topic without being delete. look forward to others idea. 
   
   @Jason918 hi,how about this idea?I would like to discuss in the email thread if you think this is acceptable ^_^


-- 
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] labuladong commented on issue #18731: [Bug] shadow topic replicator may fail due to retention policy

Posted by GitBox <gi...@apache.org>.
labuladong commented on issue #18731:
URL: https://github.com/apache/pulsar/issues/18731#issuecomment-1336588984

   @lifepuzzlefun I think you are right, the `non-durable` cursor of the shadow topic is the root cause of this issue:
   
   https://github.com/apache/pulsar/blob/3df506de099f27d4c3b7fbf2064db881ffda0402/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java#L1682-L1689
   
    Could you explain why we use a `non-durable` cursor in a shadow topic? @Jason918 


-- 
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 issue #18731: [Bug] shadow topic replicator may fail due to retention policy

Posted by GitBox <gi...@apache.org>.
Jason918 commented on issue #18731:
URL: https://github.com/apache/pulsar/issues/18731#issuecomment-1338887648

   > I try to change `ledger.newNonDurableCursor` to `ledger.openCursor`, but this unit test still may fail... I guess it's due to the async update of metadata. I notice there are locks in `ManagedLedger` to protect async operation:
   
   Yes, This is an async process. Shadow topic reads the `ledgerIds` from the same zk node from the source topic. And shadow topic can't stop source topic from deleting ledgers (either because of retention policy nor admin ops.)


-- 
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