This is an automated email from the ASF dual-hosted git repository.

eolivelli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new c6f27f0240 LedgerHandle: eliminate unnecessasary synchronization on 
LedgerHandle.getLength() (#4516)
c6f27f0240 is described below

commit c6f27f0240b8561501f2d2203473062a962b158d
Author: Enrico Olivelli <[email protected]>
AuthorDate: Sat Nov 16 16:45:17 2024 +0100

    LedgerHandle: eliminate unnecessasary synchronization on 
LedgerHandle.getLength() (#4516)
---
 .../org/apache/bookkeeper/client/LedgerHandle.java | 24 +++++++++++-----------
 .../apache/bookkeeper/client/LedgerRecoveryOp.java |  4 ++--
 .../bookkeeper/client/ReadOnlyLedgerHandle.java    |  2 +-
 .../apache/bookkeeper/client/MockLedgerHandle.java |  2 +-
 4 files changed, 16 insertions(+), 16 deletions(-)

diff --git 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
index 6a15cb42f7..6c0c5cc2ab 100644
--- 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
+++ 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
@@ -52,6 +52,7 @@ import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
 import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
 import org.apache.bookkeeper.client.AsyncCallback.AddCallbackWithLatency;
 import org.apache.bookkeeper.client.AsyncCallback.CloseCallback;
@@ -137,7 +138,7 @@ public class LedgerHandle implements WriteHandle {
      */
     private int stickyBookieIndex;
 
-    long length;
+    final AtomicLong length;
     final DigestManager macManager;
     final DistributionSchedule distributionSchedule;
     final RateLimiter throttler;
@@ -188,10 +189,10 @@ public class LedgerHandle implements WriteHandle {
         LedgerMetadata metadata = versionedMetadata.getValue();
         if (metadata.isClosed()) {
             lastAddConfirmed = lastAddPushed = metadata.getLastEntryId();
-            length = metadata.getLength();
+            length = new AtomicLong(metadata.getLength());
         } else {
             lastAddConfirmed = lastAddPushed = INVALID_ENTRY_ID;
-            length = 0;
+            length = new AtomicLong();
         }
 
         this.pendingAddsSequenceHead = lastAddConfirmed;
@@ -365,7 +366,7 @@ public class LedgerHandle implements WriteHandle {
                 LedgerMetadata metadata = versionedMetadata.getValue();
                 if (metadata.isClosed()) {
                     lastAddConfirmed = lastAddPushed = 
metadata.getLastEntryId();
-                    length = metadata.getLength();
+                    length.set(metadata.getLength());
                 }
                 return true;
             } else {
@@ -422,9 +423,8 @@ public class LedgerHandle implements WriteHandle {
      * @param delta
      * @return the length of the ledger after the addition
      */
-    synchronized long addToLength(long delta) {
-        this.length += delta;
-        return this.length;
+    long addToLength(long delta) {
+        return length.addAndGet(delta);
     }
 
     /**
@@ -433,8 +433,8 @@ public class LedgerHandle implements WriteHandle {
      * @return the length of the ledger in bytes
      */
     @Override
-    public synchronized long getLength() {
-        return this.length;
+    public long getLength() {
+        return this.length.get();
     }
 
     /**
@@ -559,7 +559,7 @@ public class LedgerHandle implements WriteHandle {
 
                         // taking the length must occur after draining, as 
draining changes the length
                         lastEntry = lastAddPushed = 
LedgerHandle.this.lastAddConfirmed;
-                        finalLength = LedgerHandle.this.length;
+                        finalLength = LedgerHandle.this.length.get();
                         handleState = HandleState.CLOSED;
                     }
 
@@ -1649,7 +1649,7 @@ public class LedgerHandle implements WriteHandle {
             lacUpdateMissesCounter.inc();
         }
         lastAddPushed = Math.max(lastAddPushed, lac);
-        length = Math.max(length, len);
+        length.accumulateAndGet(len, (current, value) -> Math.max(current, 
value));
     }
 
     /**
@@ -1985,7 +1985,7 @@ public class LedgerHandle implements WriteHandle {
             isClosed = metadata.isClosed();
             if (isClosed) {
                 lastAddConfirmed = metadata.getLastEntryId();
-                length = metadata.getLength();
+                length.set(metadata.getLength());
             }
         }
         if (isClosed) {
diff --git 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java
 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java
index fe697ef4e0..16a95cbc89 100644
--- 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java
+++ 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java
@@ -121,7 +121,7 @@ class LedgerRecoveryOp implements ReadEntryListener, 
AddCallback {
                                 lh.lastAddPushed = lh.lastAddConfirmed = 
Math.max(data.getLastAddConfirmed(),
                                         (lastEnsembleEntryId - 1));
 
-                                lh.length = data.getLength();
+                                lh.length.set(data.getLength());
                                 lh.pendingAddsSequenceHead = 
lh.lastAddConfirmed;
                                 startEntryToRead = endEntryToRead = 
lh.lastAddConfirmed;
                             }
@@ -192,7 +192,7 @@ class LedgerRecoveryOp implements ReadEntryListener, 
AddCallback {
              * be added again when processing the call to add it.
              */
             synchronized (lh) {
-                lh.length = entry.getLength() - (long) data.length;
+                lh.length.set(entry.getLength() - (long) data.length);
                 // check whether entry id is expected, so we won't overwritten 
any entries by mistake
                 if (entry.getEntryId() != lh.lastAddPushed + 1) {
                     LOG.error("Unexpected to recovery add entry {} as entry {} 
for ledger {}.",
diff --git 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java
 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java
index 9e883a8246..95e8666660 100644
--- 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java
+++ 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java
@@ -300,7 +300,7 @@ class ReadOnlyLedgerHandle extends LedgerHandle implements 
LedgerMetadataListene
         long lac, len;
         synchronized (this) {
             lac = lastAddConfirmed;
-            len = length;
+            len = length.get();
         }
         LOG.info("Closing recovered ledger {} at entry {}", getId(), lac);
         CompletableFuture<Versioned<LedgerMetadata>> f = new 
MetadataUpdateLoop(
diff --git 
a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockLedgerHandle.java
 
b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockLedgerHandle.java
index 660bf0f80c..b8d1c3a135 100644
--- 
a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockLedgerHandle.java
+++ 
b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockLedgerHandle.java
@@ -84,7 +84,7 @@ public class MockLedgerHandle extends LedgerHandle {
         metadata = LedgerMetadataBuilder.from(metadata)
                 .withClosedState()
                 .withLastEntryId(lastEntry)
-                .withLength(length)
+                .withLength(length.get())
                 .build();
         setLedgerMetadata(getVersionedLedgerMetadata(), new 
Versioned<>(metadata, new LongVersion(1L)));
 

Reply via email to