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/07/06 13:10:40 UTC

[GitHub] [pulsar] lordcheng10 opened a new pull request, #16420: [fix][broker] fix No such ledger exception

lordcheng10 opened a new pull request, #16420:
URL: https://github.com/apache/pulsar/pull/16420

   ### Motivation
   In the log we found No such ledger exception:
   07:31:55.118 [broker-topic-workers-OrderedScheduler-0-0] ERROR org.apache.pulsar.broker.service.persistent.PersistentDispatcherSingleActiveConsumer - [persistent://tenant_g_cdg_cft_tdbank__cft/b_cdg_cft_dz_lct/lcttengan_item_dc_db_xx_t_item_dc_netvalue_x_tengan_hh-partition-2 / sort_lj_cft_pulsar_to_thive_gz_cft_1_v2_9_lcttengan_item_dc_db_xx_t_item_dc_netvalue_x_tengan_hh_consumer_group-Consumer{subscription=PersistentSubscription{topic=persistent://tenant_g_cdg_cft_tdbank__cft/b_cdg_cft_dz_lct/lcttengan_item_dc_db_xx_t_item_dc_netvalue_x_tengan_hh-partition-2, name=sort_lj_cft_pulsar_to_thive_gz_cft_1_v2_9_lcttengan_item_dc_db_xx_t_item_dc_netvalue_x_tengan_hh_consumer_group}, consumerId=62, consumerName=55bd8, address=/11.168.42.86:35383}] Error reading entries at 16092024:0 : No such ledger exists on Metadata Server - Retrying to read in 58.032 seconds
   
   The reasons are as follows:
   1. Frequent Full GC occurs on the broker, causing zk to time out:
   22:29:06.119 [main-EventThread] ERROR org.apache.bookkeeper.zookeeper.ZooKeeperWatcherBase - ZooKeeper client connection to the ZooKeeper server has expired!
   2.When the create leadger is completed, the ledger will be put into ledgers and update currentLedger,the updateLedgersListAfterRollover method is executed, and the metadata to zookeeper fails to write:
   <img width="1449" alt="image" src="https://user-images.githubusercontent.com/19296967/177555986-96b40448-c831-4622-94d9-1385a4605d3e.png">
   https://github.com/apache/pulsar/blob/4c958a9a1a0468656e37851293cf1930041d2c93/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java#L1484-L1489
   
   3.In the failure callback method operationFailed, the corresponding ledger will be removed from the ledgers, but the currentLedger still points to the ledger that failed to create, but the ledger will be deleted here through bookKeeper.asyncDeleteLedger. When reading data, it will be read through the currentLedger. Since the ledger has been deleted, the final error is reported: No such ledger 
   https://github.com/apache/pulsar/blob/4c958a9a1a0468656e37851293cf1930041d2c93/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java#L1509-L1522
   
   
   
   
   
   
   
   
   
   
   ### Modifications
   
   
   
   ### Documentation
   
   Check the box below or label this PR directly.
   
   Need to update docs? 
   
   - [ ] `doc-required` 
   (Your PR needs to update docs and you will update later)
     
   - [ ] `doc-not-needed` 
   (Please explain why)
     
   - [ ] `doc` 
   (Your PR contains doc changes)
   
   - [ ] `doc-complete`
   (Docs have been already added)


-- 
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] lordcheng10 commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r915434308


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1482,18 +1482,17 @@ public synchronized void createComplete(int rc, final LedgerHandle lh, Object ct
             lastLedgerCreationFailureTimestamp = clock.millis();
         } else {
             log.info("[{}] Created new ledger {}", name, lh.getId());
-            ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());
-            currentLedger = lh;
-            currentLedgerEntries = 0;
-            currentLedgerSize = 0;
-
             final MetaStoreCallback<Void> cb = new MetaStoreCallback<Void>() {
                 @Override
                 public void operationComplete(Void v, Stat stat) {
                     if (log.isDebugEnabled()) {
                         log.debug("[{}] Updating of ledgers list after create complete. version={}", name, stat);
                     }
                     ledgersStat = stat;
+                    ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());

Review Comment:
   Your concern is right, I will try to update the zookeeper metadata with a temporary ledgersTmp, and update the ledgers after the zookeeper is successfully written. @mattisonchao @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] lordcheng10 commented on pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#issuecomment-1177908819

   /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] mattisonchao commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r915598218


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1545,21 +1546,21 @@ public void operationFailed(MetaStoreException e) {
                 }
             };
 
-            updateLedgersListAfterRollover(cb);
+            updateLedgersListAfterRollover(cb, getManagedLedgerInfo(ledgersTmp));
         }
     }
-
-    private void updateLedgersListAfterRollover(MetaStoreCallback<Void> callback) {
+    private void updateLedgersListAfterRollover(MetaStoreCallback<Void> callback, ManagedLedgerInfo mlInfo) {
         if (!metadataMutex.tryLock()) {
             // Defer update for later
-            scheduledExecutor.schedule(() -> updateLedgersListAfterRollover(callback), 100, TimeUnit.MILLISECONDS);
+            scheduledExecutor.schedule(() -> updateLedgersListAfterRollover(callback, mlInfo),

Review Comment:
   @lordcheng10  
   Yes, I also don't think it's a big problem, but maybe we can leave this comment here. Perhaps someone can find some things that will be affected by 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] lordcheng10 commented on pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#issuecomment-1177497148

   /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] lordcheng10 commented on pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#issuecomment-1177571470

   /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] mattisonchao commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r914893010


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1483,17 +1483,16 @@ public synchronized void createComplete(int rc, final LedgerHandle lh, Object ct
         } else {
             log.info("[{}] Created new ledger {}", name, lh.getId());
             ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());

Review Comment:
   Maybe we can also move this line to metadata persistent success? and then we don't need to remove it when store metadata failed.



-- 
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] lordcheng10 commented on pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#issuecomment-1176460054

   /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] mattisonchao commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r915382317


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1482,18 +1482,17 @@ public synchronized void createComplete(int rc, final LedgerHandle lh, Object ct
             lastLedgerCreationFailureTimestamp = clock.millis();
         } else {
             log.info("[{}] Created new ledger {}", name, lh.getId());
-            ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());
-            currentLedger = lh;
-            currentLedgerEntries = 0;
-            currentLedgerSize = 0;
-
             final MetaStoreCallback<Void> cb = new MetaStoreCallback<Void>() {
                 @Override
                 public void operationComplete(Void v, Stat stat) {
                     if (log.isDebugEnabled()) {
                         log.debug("[{}] Updating of ledgers list after create complete. version={}", name, stat);
                     }
                     ledgersStat = stat;
+                    ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());

Review Comment:
   I still have concerns because in this method, if the ledger is not persisted to the metadata, it means the ledger cannot be used. But we put it in the ledger map; it is not persistent and may be used by other operations under multi-threaded conditions.
   
   However, I'm not sure what it will affect. Just a minor concern. Please feel free to go on.



-- 
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] mattisonchao commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r915598218


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1545,21 +1546,21 @@ public void operationFailed(MetaStoreException e) {
                 }
             };
 
-            updateLedgersListAfterRollover(cb);
+            updateLedgersListAfterRollover(cb, getManagedLedgerInfo(ledgersTmp));
         }
     }
-
-    private void updateLedgersListAfterRollover(MetaStoreCallback<Void> callback) {
+    private void updateLedgersListAfterRollover(MetaStoreCallback<Void> callback, ManagedLedgerInfo mlInfo) {
         if (!metadataMutex.tryLock()) {
             // Defer update for later
-            scheduledExecutor.schedule(() -> updateLedgersListAfterRollover(callback), 100, TimeUnit.MILLISECONDS);
+            scheduledExecutor.schedule(() -> updateLedgersListAfterRollover(callback, mlInfo),

Review Comment:
   @lordcheng10  
   Yes, I also don't think it's a big problem, but maybe we can leave this comment here. Perhaps someone can find some things that will be affected by 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] lordcheng10 commented on pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#issuecomment-1177267241

   /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 merged pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
Jason918 merged PR #16420:
URL: https://github.com/apache/pulsar/pull/16420


-- 
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] lordcheng10 commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r915576289


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1545,21 +1546,21 @@ public void operationFailed(MetaStoreException e) {
                 }
             };
 
-            updateLedgersListAfterRollover(cb);
+            updateLedgersListAfterRollover(cb, getManagedLedgerInfo(ledgersTmp));
         }
     }
-
-    private void updateLedgersListAfterRollover(MetaStoreCallback<Void> callback) {
+    private void updateLedgersListAfterRollover(MetaStoreCallback<Void> callback, ManagedLedgerInfo mlInfo) {
         if (!metadataMutex.tryLock()) {
             // Defer update for later
-            scheduledExecutor.schedule(() -> updateLedgersListAfterRollover(callback), 100, TimeUnit.MILLISECONDS);
+            scheduledExecutor.schedule(() -> updateLedgersListAfterRollover(callback, mlInfo),

Review Comment:
   
   
   
   
   > Why do we need to create a new method?
   In order to avoid affecting other logic that uses getManagedLedgerInfo, a getManagedLedgerInfo method is added here.
   The new method has made some changes again, please review again, thanks!



##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1545,21 +1546,21 @@ public void operationFailed(MetaStoreException e) {
                 }
             };
 
-            updateLedgersListAfterRollover(cb);
+            updateLedgersListAfterRollover(cb, getManagedLedgerInfo(ledgersTmp));
         }
     }
-
-    private void updateLedgersListAfterRollover(MetaStoreCallback<Void> callback) {
+    private void updateLedgersListAfterRollover(MetaStoreCallback<Void> callback, ManagedLedgerInfo mlInfo) {
         if (!metadataMutex.tryLock()) {
             // Defer update for later
-            scheduledExecutor.schedule(() -> updateLedgersListAfterRollover(callback), 100, TimeUnit.MILLISECONDS);
+            scheduledExecutor.schedule(() -> updateLedgersListAfterRollover(callback, mlInfo),

Review Comment:
   
   
   
   > Why do we need to create a new method?
   
   
   In order to avoid affecting other logic that uses getManagedLedgerInfo, a getManagedLedgerInfo method is added here.
   The new method has made some changes again, please review again, thanks!



-- 
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] lordcheng10 commented on pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#issuecomment-1177319166

   > I think we are not handling this condition well:
   If the process runs at this line, we continue to retry later. But at the same time, the ledgers is changed. However, we still use old data to update. I'm not sure if it is a big problem. But I think we have to clarify it.
   I think I have to give a change request. @lordcheng10 Could you help to confirm it?
   
   This is a problem, I made a modification, please review it again. @mattisonchao 


-- 
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] lordcheng10 commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r915567000


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -3569,6 +3570,10 @@ private ManagedLedgerInfo getManagedLedgerInfo() {
         return buildManagedLedgerInfo(ledgers);
     }
 
+    private ManagedLedgerInfo getManagedLedgerInfo(NavigableMap<Long, LedgerInfo> ledgersTmp) {
+        return buildManagedLedgerInfo(ledgersTmp);

Review Comment:
   OK



-- 
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] lordcheng10 commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r914859782


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1494,6 +1493,7 @@ public void operationComplete(Void v, Stat stat) {
                         log.debug("[{}] Updating of ledgers list after create complete. version={}", name, stat);
                     }
                     ledgersStat = stat;
+                    currentLedger = lh;

Review Comment:
   OK



-- 
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] github-actions[bot] commented on pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#issuecomment-1176205097

   @lordcheng10 Please provide a correct documentation label for your PR.
   Instructions see [Pulsar Documentation Label Guide](https://docs.google.com/document/d/1Qw7LHQdXWBW9t2-r-A7QdFDBwmZh6ytB4guwMoXHqc0).


-- 
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] lordcheng10 commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r915434308


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1482,18 +1482,17 @@ public synchronized void createComplete(int rc, final LedgerHandle lh, Object ct
             lastLedgerCreationFailureTimestamp = clock.millis();
         } else {
             log.info("[{}] Created new ledger {}", name, lh.getId());
-            ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());
-            currentLedger = lh;
-            currentLedgerEntries = 0;
-            currentLedgerSize = 0;
-
             final MetaStoreCallback<Void> cb = new MetaStoreCallback<Void>() {
                 @Override
                 public void operationComplete(Void v, Stat stat) {
                     if (log.isDebugEnabled()) {
                         log.debug("[{}] Updating of ledgers list after create complete. version={}", name, stat);
                     }
                     ledgersStat = stat;
+                    ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());

Review Comment:
   Your concern is right, I will try to update the zookeeper metadata with a temporary ledgersTmp, and update the ledgers after the update is successful. @mattisonchao @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 diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
Jason918 commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r915516072


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1482,18 +1482,18 @@ public synchronized void createComplete(int rc, final LedgerHandle lh, Object ct
             lastLedgerCreationFailureTimestamp = clock.millis();
         } else {
             log.info("[{}] Created new ledger {}", name, lh.getId());
-            ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());
-            currentLedger = lh;
-            currentLedgerEntries = 0;
-            currentLedgerSize = 0;
-
+            NavigableMap<Long, LedgerInfo> ledgersTmp = new ConcurrentSkipListMap<>(ledgers);

Review Comment:
   If I understand this right, we should add `lh` in `ledgersTmp`?



-- 
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] lordcheng10 commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r915473555


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1482,18 +1482,17 @@ public synchronized void createComplete(int rc, final LedgerHandle lh, Object ct
             lastLedgerCreationFailureTimestamp = clock.millis();
         } else {
             log.info("[{}] Created new ledger {}", name, lh.getId());
-            ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());
-            currentLedger = lh;
-            currentLedgerEntries = 0;
-            currentLedgerSize = 0;
-
             final MetaStoreCallback<Void> cb = new MetaStoreCallback<Void>() {
                 @Override
                 public void operationComplete(Void v, Stat stat) {
                     if (log.isDebugEnabled()) {
                         log.debug("[{}] Updating of ledgers list after create complete. version={}", name, stat);
                     }
                     ledgersStat = stat;
+                    ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());

Review Comment:
   fixed. @merlimat @hangc0276 @aloyszhang @mattisonchao PTAL,thanks!



-- 
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 diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
Jason918 commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r915527164


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -3569,6 +3570,10 @@ private ManagedLedgerInfo getManagedLedgerInfo() {
         return buildManagedLedgerInfo(ledgers);
     }
 
+    private ManagedLedgerInfo getManagedLedgerInfo(NavigableMap<Long, LedgerInfo> ledgersTmp) {
+        return buildManagedLedgerInfo(ledgersTmp);

Review Comment:
   The `ledgersTmp` can be avoided by extending `buildManagedLedgerInfo`. Just add the new LedgerInfo after line 3578.



-- 
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] hangc0276 commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
hangc0276 commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r914859005


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1494,6 +1493,7 @@ public void operationComplete(Void v, Stat stat) {
                         log.debug("[{}] Updating of ledgers list after create complete. version={}", name, stat);
                     }
                     ledgersStat = stat;
+                    currentLedger = lh;

Review Comment:
   Move 
   ```
   currentLedgerEntries = 0;
   currentLedgerSize = 0;
   ```
   here?



-- 
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] lordcheng10 commented on pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#issuecomment-1176459359

   /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] mattisonchao commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r915608301


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1545,21 +1546,21 @@ public void operationFailed(MetaStoreException e) {
                 }
             };
 
-            updateLedgersListAfterRollover(cb);
+            updateLedgersListAfterRollover(cb, getManagedLedgerInfo(ledgersTmp));
         }
     }
-
-    private void updateLedgersListAfterRollover(MetaStoreCallback<Void> callback) {
+    private void updateLedgersListAfterRollover(MetaStoreCallback<Void> callback, ManagedLedgerInfo mlInfo) {
         if (!metadataMutex.tryLock()) {
             // Defer update for later
-            scheduledExecutor.schedule(() -> updateLedgersListAfterRollover(callback), 100, TimeUnit.MILLISECONDS);
+            scheduledExecutor.schedule(() -> updateLedgersListAfterRollover(callback, mlInfo),

Review Comment:
   @lordcheng10 
   I'm not very sure about that, maybe we need more eyes on it.
   



-- 
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] lordcheng10 commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r915520892


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1482,18 +1482,18 @@ public synchronized void createComplete(int rc, final LedgerHandle lh, Object ct
             lastLedgerCreationFailureTimestamp = clock.millis();
         } else {
             log.info("[{}] Created new ledger {}", name, lh.getId());
-            ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());
-            currentLedger = lh;
-            currentLedgerEntries = 0;
-            currentLedgerSize = 0;
-
+            NavigableMap<Long, LedgerInfo> ledgersTmp = new ConcurrentSkipListMap<>(ledgers);

Review Comment:
   Fixed,PTAL,thanks! @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] lordcheng10 commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r915576289


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1545,21 +1546,21 @@ public void operationFailed(MetaStoreException e) {
                 }
             };
 
-            updateLedgersListAfterRollover(cb);
+            updateLedgersListAfterRollover(cb, getManagedLedgerInfo(ledgersTmp));
         }
     }
-
-    private void updateLedgersListAfterRollover(MetaStoreCallback<Void> callback) {
+    private void updateLedgersListAfterRollover(MetaStoreCallback<Void> callback, ManagedLedgerInfo mlInfo) {
         if (!metadataMutex.tryLock()) {
             // Defer update for later
-            scheduledExecutor.schedule(() -> updateLedgersListAfterRollover(callback), 100, TimeUnit.MILLISECONDS);
+            scheduledExecutor.schedule(() -> updateLedgersListAfterRollover(callback, mlInfo),

Review Comment:
   
   
   
   > Why do we need to create a new method?
   
   
   In order to avoid affecting other logic that uses getManagedLedgerInfo, a getManagedLedgerInfo method is added here.
   The new method has made some changes again, please review again, thanks! @mattisonchao 



-- 
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] lordcheng10 commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r915573306


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1482,18 +1482,19 @@ public synchronized void createComplete(int rc, final LedgerHandle lh, Object ct
             lastLedgerCreationFailureTimestamp = clock.millis();
         } else {
             log.info("[{}] Created new ledger {}", name, lh.getId());
-            ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());
-            currentLedger = lh;
-            currentLedgerEntries = 0;
-            currentLedgerSize = 0;
-
+            NavigableMap<Long, LedgerInfo> ledgersTmp = new ConcurrentSkipListMap<>(ledgers);
+            ledgersTmp.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());

Review Comment:
   fixed.



-- 
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] lordcheng10 commented on pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#issuecomment-1178447977

   /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] lordcheng10 commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r915521392


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1562,6 +1562,20 @@ private void updateLedgersListAfterRollover(MetaStoreCallback<Void> callback) {
         store.asyncUpdateLedgerIds(name, getManagedLedgerInfo(), ledgersStat, callback);
     }
 
+    private void updateLedgersListAfterRollover(MetaStoreCallback<Void> callback, ManagedLedgerInfo mlInfo) {

Review Comment:
   fixed.



-- 
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] mattisonchao commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r915538687


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -3569,6 +3570,10 @@ private ManagedLedgerInfo getManagedLedgerInfo() {
         return buildManagedLedgerInfo(ledgers);
     }
 
+    private ManagedLedgerInfo getManagedLedgerInfo(NavigableMap<Long, LedgerInfo> ledgersTmp) {
+        return buildManagedLedgerInfo(ledgersTmp);

Review Comment:
   Why do we need to create a new method?



-- 
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] mattisonchao commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r915530980


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1482,18 +1482,19 @@ public synchronized void createComplete(int rc, final LedgerHandle lh, Object ct
             lastLedgerCreationFailureTimestamp = clock.millis();
         } else {
             log.info("[{}] Created new ledger {}", name, lh.getId());
-            ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());
-            currentLedger = lh;
-            currentLedgerEntries = 0;
-            currentLedgerSize = 0;
-
+            NavigableMap<Long, LedgerInfo> ledgersTmp = new ConcurrentSkipListMap<>(ledgers);

Review Comment:
   Does the local variable need to use `Concurrent`? Maybe some immutable container is better.



##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1482,18 +1482,19 @@ public synchronized void createComplete(int rc, final LedgerHandle lh, Object ct
             lastLedgerCreationFailureTimestamp = clock.millis();
         } else {
             log.info("[{}] Created new ledger {}", name, lh.getId());
-            ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());
-            currentLedger = lh;
-            currentLedgerEntries = 0;
-            currentLedgerSize = 0;
-
+            NavigableMap<Long, LedgerInfo> ledgersTmp = new ConcurrentSkipListMap<>(ledgers);
+            ledgersTmp.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());

Review Comment:
   Cleanup: Maybe we can avoid building `schemainfo` twice?  another is line 1494.



##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1545,21 +1546,21 @@ public void operationFailed(MetaStoreException e) {
                 }
             };
 
-            updateLedgersListAfterRollover(cb);
+            updateLedgersListAfterRollover(cb, getManagedLedgerInfo(ledgersTmp));
         }
     }
-
-    private void updateLedgersListAfterRollover(MetaStoreCallback<Void> callback) {
+    private void updateLedgersListAfterRollover(MetaStoreCallback<Void> callback, ManagedLedgerInfo mlInfo) {
         if (!metadataMutex.tryLock()) {
             // Defer update for later
-            scheduledExecutor.schedule(() -> updateLedgersListAfterRollover(callback), 100, TimeUnit.MILLISECONDS);
+            scheduledExecutor.schedule(() -> updateLedgersListAfterRollover(callback, mlInfo),

Review Comment:
   I think we are not handling this condition well:
   
   If the process runs at this line, we continue to retry later. But at the same time, the `ledgers` is changed. However, we still use old data to update. I'm not sure if it is a big problem. But I think we have to clarify it.
   I think I have to give a change request.  @lordcheng10  Could you help to confirm it?



##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -3569,6 +3570,10 @@ private ManagedLedgerInfo getManagedLedgerInfo() {
         return buildManagedLedgerInfo(ledgers);
     }
 
+    private ManagedLedgerInfo getManagedLedgerInfo(NavigableMap<Long, LedgerInfo> ledgersTmp) {
+        return buildManagedLedgerInfo(ledgersTmp);

Review Comment:
   Why do we need to create a new method?



-- 
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] mattisonchao commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r915376024


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1483,17 +1483,16 @@ public synchronized void createComplete(int rc, final LedgerHandle lh, Object ct
         } else {
             log.info("[{}] Created new ledger {}", name, lh.getId());
             ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());

Review Comment:
   I got @merlimat comment, sorry.



-- 
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] lordcheng10 commented on pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#issuecomment-1176981500

   /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] lordcheng10 commented on pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#issuecomment-1176365281

   /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] lordcheng10 commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r915578315


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1545,21 +1546,21 @@ public void operationFailed(MetaStoreException e) {
                 }
             };
 
-            updateLedgersListAfterRollover(cb);
+            updateLedgersListAfterRollover(cb, getManagedLedgerInfo(ledgersTmp));
         }
     }
-
-    private void updateLedgersListAfterRollover(MetaStoreCallback<Void> callback) {
+    private void updateLedgersListAfterRollover(MetaStoreCallback<Void> callback, ManagedLedgerInfo mlInfo) {
         if (!metadataMutex.tryLock()) {
             // Defer update for later
-            scheduledExecutor.schedule(() -> updateLedgersListAfterRollover(callback), 100, TimeUnit.MILLISECONDS);
+            scheduledExecutor.schedule(() -> updateLedgersListAfterRollover(callback, mlInfo),

Review Comment:
   > I think we are not handling this condition well:
   If the process runs at this line, we continue to retry later. But at the same time, the ledgers is changed. However, we still use old data to update. I'm not sure if it is a big problem. But I think we have to clarify it.
   I think I have to give a change request. @lordcheng10 Could you help to confirm it?
   
   
   
   This is eventual consistency, I don't think it's a problem.



-- 
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] mattisonchao commented on pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#issuecomment-1177252881

   @hangc0276 
   
   This pull request made a lot of change after your review. Could you have time to review it again?


-- 
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] mattisonchao commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r914884809


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1483,17 +1483,16 @@ public synchronized void createComplete(int rc, final LedgerHandle lh, Object ct
         } else {
             log.info("[{}] Created new ledger {}", name, lh.getId());
             ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());

Review Comment:
   What will this ledger do?



-- 
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] lordcheng10 commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r915570830


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -3569,6 +3570,10 @@ private ManagedLedgerInfo getManagedLedgerInfo() {
         return buildManagedLedgerInfo(ledgers);
     }
 
+    private ManagedLedgerInfo getManagedLedgerInfo(NavigableMap<Long, LedgerInfo> ledgersTmp) {
+        return buildManagedLedgerInfo(ledgersTmp);

Review Comment:
   Fixed.PTAL,thanks! @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] lordcheng10 commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r914926203


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1483,17 +1483,16 @@ public synchronized void createComplete(int rc, final LedgerHandle lh, Object ct
         } else {
             log.info("[{}] Created new ledger {}", name, lh.getId());
             ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());

Review Comment:
   Fixed. @mattisonchao 



##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1483,17 +1483,16 @@ public synchronized void createComplete(int rc, final LedgerHandle lh, Object ct
         } else {
             log.info("[{}] Created new ledger {}", name, lh.getId());
             ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());

Review Comment:
   updated. @mattisonchao 



-- 
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] lordcheng10 commented on pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#issuecomment-1178495136

   /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] lordcheng10 commented on pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#issuecomment-1176213768

   @codelipenghui @eolivelli PTAL,thanks!


-- 
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] lordcheng10 commented on pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#issuecomment-1176213462

   @hangc0276 PTAL,thanks!


-- 
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] merlimat commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
merlimat commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r915252434


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1482,18 +1482,17 @@ public synchronized void createComplete(int rc, final LedgerHandle lh, Object ct
             lastLedgerCreationFailureTimestamp = clock.millis();
         } else {
             log.info("[{}] Created new ledger {}", name, lh.getId());
-            ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());
-            currentLedger = lh;
-            currentLedgerEntries = 0;
-            currentLedgerSize = 0;
-
             final MetaStoreCallback<Void> cb = new MetaStoreCallback<Void>() {
                 @Override
                 public void operationComplete(Void v, Stat stat) {
                     if (log.isDebugEnabled()) {
                         log.debug("[{}] Updating of ledgers list after create complete. version={}", name, stat);
                     }
                     ledgersStat = stat;
+                    ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());

Review Comment:
   @lordcheng10 Adding to the ledgers map cannot be done here, because that map is used when writing to the z-node (eg: `buildManagedLedgerInfo(Map<Long, LedgerInfo> ledgers)`. 



-- 
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] aloyszhang commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r915464235


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1482,18 +1482,17 @@ public synchronized void createComplete(int rc, final LedgerHandle lh, Object ct
             lastLedgerCreationFailureTimestamp = clock.millis();
         } else {
             log.info("[{}] Created new ledger {}", name, lh.getId());
-            ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());
-            currentLedger = lh;
-            currentLedgerEntries = 0;
-            currentLedgerSize = 0;
-
             final MetaStoreCallback<Void> cb = new MetaStoreCallback<Void>() {
                 @Override
                 public void operationComplete(Void v, Stat stat) {
                     if (log.isDebugEnabled()) {
                         log.debug("[{}] Updating of ledgers list after create complete. version={}", name, stat);
                     }
                     ledgersStat = stat;
+                    ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());

Review Comment:
   Agree with @mattisonchao. This may lead to data loss.
   
   Putting the newly created ledger into the `NavigableMap<Long, LedgerInfo> ledgers`, the new ledger can be seen immediately which means can write data into this ledger, and then if the meta-store operation failed, the ledger will be removed from both the `ledgers` and bookie, all data in this ledger are 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] lordcheng10 commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r914896614


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1483,17 +1483,16 @@ public synchronized void createComplete(int rc, final LedgerHandle lh, Object ct
         } else {
             log.info("[{}] Created new ledger {}", name, lh.getId());
             ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());

Review Comment:
   agree



-- 
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] lordcheng10 commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r915360149


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1482,18 +1482,17 @@ public synchronized void createComplete(int rc, final LedgerHandle lh, Object ct
             lastLedgerCreationFailureTimestamp = clock.millis();
         } else {
             log.info("[{}] Created new ledger {}", name, lh.getId());
-            ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());
-            currentLedger = lh;
-            currentLedgerEntries = 0;
-            currentLedgerSize = 0;
-
             final MetaStoreCallback<Void> cb = new MetaStoreCallback<Void>() {
                 @Override
                 public void operationComplete(Void v, Stat stat) {
                     if (log.isDebugEnabled()) {
                         log.debug("[{}] Updating of ledgers list after create complete. version={}", name, stat);
                     }
                     ledgersStat = stat;
+                    ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());

Review Comment:
   fixed. PTAL,thanks! @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 diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
Jason918 commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r915512032


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1562,6 +1562,20 @@ private void updateLedgersListAfterRollover(MetaStoreCallback<Void> callback) {
         store.asyncUpdateLedgerIds(name, getManagedLedgerInfo(), ledgersStat, callback);
     }
 
+    private void updateLedgersListAfterRollover(MetaStoreCallback<Void> callback, ManagedLedgerInfo mlInfo) {

Review Comment:
   It seems we can just modify in previous method `updateLedgersListAfterRollover`, not creating new one.



-- 
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] lordcheng10 commented on pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#issuecomment-1178582958

   @codelipenghui PTAL,thanks!


-- 
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] lordcheng10 commented on a diff in pull request #16420: [fix][broker] fix No such ledger exception

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #16420:
URL: https://github.com/apache/pulsar/pull/16420#discussion_r915573037


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -1482,18 +1482,19 @@ public synchronized void createComplete(int rc, final LedgerHandle lh, Object ct
             lastLedgerCreationFailureTimestamp = clock.millis();
         } else {
             log.info("[{}] Created new ledger {}", name, lh.getId());
-            ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());
-            currentLedger = lh;
-            currentLedgerEntries = 0;
-            currentLedgerSize = 0;
-
+            NavigableMap<Long, LedgerInfo> ledgersTmp = new ConcurrentSkipListMap<>(ledgers);

Review Comment:
   fixed.  
   remove ledgersTmp. @mattisonchao PTAL,thanks!



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