You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by iv...@apache.org on 2018/12/03 12:37:08 UTC

[bookkeeper] branch master updated: Users of LedgerMetadata should use the api interface

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

ivank 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 8e0e9fb  Users of LedgerMetadata should use the api interface
8e0e9fb is described below

commit 8e0e9fb0fd2792bcce5ce9466ab5d2e1423f756f
Author: Ivan Kelly <iv...@apache.org>
AuthorDate: Mon Dec 3 13:37:03 2018 +0100

    Users of LedgerMetadata should use the api interface
    
    All users to LedgerMetadata should use api.LedgerMetadata rather than
    client.LedgerMetadata.
    
    Some methods have been promoted to the interface to allow this.
    
    Other methods have been moved out to a utility class that acts purely
    on api.LedgerMetadata.
    
    client.LedgerMetadata has been renamed to LedgerMetadataImpl.
    
    Master issue: #723
    
    
    Reviewers: Enrico Olivelli <eo...@gmail.com>, Sijie Guo <si...@apache.org>
    
    This closes #1852 from ivankelly/kill-store-ctime
---
 .../org/apache/bookkeeper/bookie/BookieShell.java  |  2 +-
 .../org/apache/bookkeeper/client/BookKeeper.java   |  3 +-
 .../apache/bookkeeper/client/BookKeeperAdmin.java  |  6 +-
 .../apache/bookkeeper/client/EnsembleUtils.java    |  1 +
 .../apache/bookkeeper/client/LedgerCreateOp.java   |  3 +-
 .../apache/bookkeeper/client/LedgerFragment.java   |  2 +-
 .../org/apache/bookkeeper/client/LedgerHandle.java | 14 ++--
 .../apache/bookkeeper/client/LedgerHandleAdv.java  |  1 +
 .../bookkeeper/client/LedgerMetadataBuilder.java   | 15 ++--
 ...LedgerMetadata.java => LedgerMetadataImpl.java} | 95 ++++++----------------
 .../bookkeeper/client/LedgerMetadataUtils.java     | 79 ++++++++++++++++++
 .../org/apache/bookkeeper/client/LedgerOpenOp.java |  1 +
 .../apache/bookkeeper/client/LedgerRecoveryOp.java |  1 +
 .../bookkeeper/client/MetadataUpdateLoop.java      |  1 +
 .../apache/bookkeeper/client/PendingReadOp.java    |  5 +-
 .../client/ReadLastConfirmedAndEntryOp.java        |  1 +
 .../bookkeeper/client/ReadOnlyLedgerHandle.java    |  9 +-
 .../apache/bookkeeper/client/UpdateLedgerOp.java   |  1 +
 .../bookkeeper/client/api/LedgerMetadata.java      | 31 +++++++
 .../bookkeeper/conf/AbstractConfiguration.java     |  1 +
 .../bookkeeper/conf/ClientConfiguration.java       |  3 -
 .../bookkeeper/meta/AbstractZkLedgerManager.java   |  2 +-
 .../bookkeeper/meta/CleanupLedgerManager.java      |  2 +-
 .../org/apache/bookkeeper/meta/LedgerManager.java  |  2 +-
 .../bookkeeper/meta/LedgerMetadataSerDe.java       |  6 +-
 .../bookkeeper/meta/MSLedgerManagerFactory.java    |  2 +-
 .../proto/BookkeeperInternalCallbacks.java         |  2 +-
 .../bookkeeper/replication/ReplicationWorker.java  |  2 +-
 .../server/http/service/GetLedgerMetaService.java  |  2 +-
 .../server/http/service/ListLedgerService.java     |  2 +-
 .../apache/bookkeeper/bookie/BookieShellTest.java  |  2 +-
 .../apache/bookkeeper/bookie/CompactionTest.java   |  2 +-
 .../bookie/TestGcOverreplicatedLedger.java         |  2 +-
 ...KeeperDiskSpaceWeightedLedgerPlacementTest.java | 18 ++--
 .../bookkeeper/client/BookieRecoveryTest.java      |  5 +-
 .../org/apache/bookkeeper/client/ClientUtil.java   |  1 +
 .../client/GenericEnsemblePlacementPolicyTest.java |  2 +-
 .../bookkeeper/client/HandleFailuresTest.java      |  1 +
 .../apache/bookkeeper/client/LedgerClose2Test.java |  1 +
 .../bookkeeper/client/LedgerHandleAdapter.java     |  7 --
 .../bookkeeper/client/LedgerMetadataTest.java      |  1 +
 .../bookkeeper/client/LedgerRecovery2Test.java     |  1 +
 .../bookkeeper/client/MetadataUpdateLoopTest.java  | 57 ++++++-------
 .../bookkeeper/client/MockBookKeeperTestCase.java  |  1 +
 .../apache/bookkeeper/client/MockLedgerHandle.java |  1 +
 .../client/ParallelLedgerRecoveryTest.java         |  1 +
 .../client/ReadLastConfirmedAndEntryOpTest.java    |  1 +
 .../bookkeeper/client/TestBookieHealthCheck.java   | 14 ++--
 .../bookkeeper/client/TestDelayEnsembleChange.java |  9 +-
 .../org/apache/bookkeeper/client/TestFencing.java  |  4 +-
 .../client/TestGetBookieInfoTimeout.java           |  2 +-
 .../client/TestLedgerFragmentReplication.java      |  1 +
 .../apache/bookkeeper/client/TestParallelRead.java |  6 +-
 .../bookkeeper/client/TestReadEntryListener.java   |  4 +-
 .../client/TestReadLastConfirmedLongPoll.java      |  2 +-
 .../apache/bookkeeper/client/TestReadTimeout.java  |  6 +-
 .../bookkeeper/client/TestWatchEnsembleChange.java |  1 +
 .../bookkeeper/client/UpdateLedgerCmdTest.java     |  2 +-
 .../bookkeeper/client/UpdateLedgerOpTest.java      | 17 ++--
 .../client/api/BookKeeperBuildersTest.java         |  1 -
 .../meta/AbstractZkLedgerManagerTest.java          |  2 +-
 .../org/apache/bookkeeper/meta/GcLedgersTest.java  |  2 +-
 .../bookkeeper/meta/LedgerManagerIteratorTest.java |  2 +-
 .../apache/bookkeeper/meta/MockLedgerManager.java  |  2 +-
 .../replication/AuditorLedgerCheckerTest.java      |  2 +-
 .../replication/AuditorPeriodicCheckTest.java      |  6 +-
 .../replication/BookieAutoRecoveryTest.java        | 32 +++-----
 .../TestAutoRecoveryAlongWithBookieServers.java    |  4 +-
 .../java/org/apache/bookkeeper/tls/TestTLS.java    |  2 +-
 .../metadata/etcd/EtcdLedgerManager.java           |  2 +-
 .../metadata/etcd/LedgerMetadataConsumer.java      |  2 +-
 .../metadata/etcd/EtcdLedgerManagerTest.java       |  2 +-
 .../bookkeeper/client/BookKeeperAccessor.java      |  4 -
 .../org/apache/bookkeeper/client/LedgerReader.java |  4 +-
 .../distributedlog/TestAsyncReaderWriter.java      |  7 +-
 .../TestCompatRecoveryNoPassword.groovy            |  2 +-
 76 files changed, 308 insertions(+), 239 deletions(-)

diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java
index c58fa83..9108ef3 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java
@@ -90,8 +90,8 @@ import org.apache.bookkeeper.client.BookKeeperAdmin;
 import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo;
 import org.apache.bookkeeper.client.LedgerEntry;
 import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.client.LedgerMetadata;
 import org.apache.bookkeeper.client.UpdateLedgerOp;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.common.annotation.InterfaceAudience.Private;
 import org.apache.bookkeeper.common.util.OrderedExecutor;
 import org.apache.bookkeeper.conf.ClientConfiguration;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java
index a3c4007..b8f703d 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java
@@ -1041,8 +1041,7 @@ public class BookKeeper implements org.apache.bookkeeper.client.api.BookKeeper {
             throw BKException.create(BKException.Code.UnexpectedConditionException);
         }
 
-        LOG.info("Ensemble: {} for ledger: {}", lh.getLedgerMetadata().getEnsemble(0L),
-                lh.getId());
+        LOG.info("Ensemble: {} for ledger: {}", lh.getLedgerMetadata().getEnsembleAt(0L), lh.getId());
 
         return lh;
     }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java
index 3f7cee4..3f837db 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java
@@ -60,6 +60,7 @@ import org.apache.bookkeeper.client.AsyncCallback.RecoverCallback;
 import org.apache.bookkeeper.client.LedgerFragmentReplicator.SingleFragmentCallback;
 import org.apache.bookkeeper.client.SyncCallbackUtils.SyncOpenCallback;
 import org.apache.bookkeeper.client.SyncCallbackUtils.SyncReadCallback;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.RegistrationClient.RegistrationListener;
@@ -500,7 +501,8 @@ public class BookKeeperAdmin implements AutoCloseable {
                                 cb.processResult(BKException.getExceptionCode(exception), null, null);
                                 return;
                             }
-                            Set<BookieSocketAddress> bookiesInLedger = metadata.getValue().getBookiesInThisLedger();
+                            Set<BookieSocketAddress> bookiesInLedger =
+                                LedgerMetadataUtils.getBookiesInThisLedger(metadata.getValue());
                             Sets.SetView<BookieSocketAddress> intersection =
                                 Sets.intersection(bookiesInLedger, bookies);
                             if (!intersection.isEmpty()) {
@@ -739,7 +741,7 @@ public class BookKeeperAdmin implements AutoCloseable {
                 }
 
                 LedgerMetadata lm = lh.getLedgerMetadata();
-                if (skipOpenLedgers && !lm.isClosed() && !lm.isInRecovery()) {
+                if (skipOpenLedgers && lm.getState() == LedgerMetadata.State.OPEN) {
                     LOG.info("Skip recovering open ledger {}.", lId);
                     try {
                         lh.close();
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsembleUtils.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsembleUtils.java
index e4ab118..6995a06 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsembleUtils.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsembleUtils.java
@@ -28,6 +28,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.net.BookieSocketAddress;
 
 import org.slf4j.Logger;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java
index d95665c..eaa3163 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java
@@ -38,6 +38,7 @@ import org.apache.bookkeeper.client.SyncCallbackUtils.SyncCreateAdvCallback;
 import org.apache.bookkeeper.client.SyncCallbackUtils.SyncCreateCallback;
 import org.apache.bookkeeper.client.api.CreateAdvBuilder;
 import org.apache.bookkeeper.client.api.CreateBuilder;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.client.api.WriteAdvHandle;
 import org.apache.bookkeeper.client.api.WriteFlag;
 import org.apache.bookkeeper.client.api.WriteHandle;
@@ -216,7 +217,7 @@ class LedgerCreateOp {
                 return;
             }
 
-            List<BookieSocketAddress> curEns = lh.getLedgerMetadata().getEnsemble(0L);
+            List<BookieSocketAddress> curEns = lh.getLedgerMetadata().getEnsembleAt(0L);
             LOG.info("Ensemble: {} for ledger: {}", curEns, lh.getId());
 
             for (BookieSocketAddress bsa : curEns) {
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragment.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragment.java
index e3224fd..382fe4e 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragment.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragment.java
@@ -48,7 +48,7 @@ public class LedgerFragment {
         this.firstEntryId = firstEntryId;
         this.lastKnownEntryId = lastKnownEntryId;
         this.bookieIndexes = bookieIndexes;
-        this.ensemble = lh.getLedgerMetadata().getEnsemble(firstEntryId);
+        this.ensemble = lh.getLedgerMetadata().getEnsembleAt(firstEntryId);
         this.schedule = lh.getDistributionSchedule();
         SortedMap<Long, ? extends List<BookieSocketAddress>> ensembles = lh
                 .getLedgerMetadata().getAllEnsembles();
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 9af3271..075788b 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
@@ -67,6 +67,7 @@ import org.apache.bookkeeper.client.SyncCallbackUtils.SyncReadLastConfirmedCallb
 import org.apache.bookkeeper.client.api.BKException.Code;
 import org.apache.bookkeeper.client.api.LastConfirmedAndEntry;
 import org.apache.bookkeeper.client.api.LedgerEntries;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.client.api.WriteFlag;
 import org.apache.bookkeeper.client.api.WriteHandle;
 import org.apache.bookkeeper.client.impl.LedgerEntryImpl;
@@ -1688,7 +1689,7 @@ public class LedgerHandle implements WriteHandle {
 
     // close the ledger and send fails to all the adds in the pipeline
     void handleUnrecoverableErrorDuringAdd(int rc) {
-        if (getLedgerMetadata().isInRecovery()) {
+        if (getLedgerMetadata().getState() == LedgerMetadata.State.IN_RECOVERY) {
             // we should not close ledger if ledger is recovery mode
             // otherwise we may lose entry.
             errorOutPendingAdds(rc);
@@ -1854,16 +1855,17 @@ public class LedgerHandle implements WriteHandle {
         new MetadataUpdateLoop(
                 clientCtx.getLedgerManager(), getId(),
                 this::getVersionedLedgerMetadata,
-                (metadata) -> !metadata.isClosed() && !metadata.isInRecovery()
+                (metadata) -> metadata.getState() == LedgerMetadata.State.OPEN
                         && failedBookies.entrySet().stream().anyMatch(
-                                (e) -> metadata.getLastEnsembleValue().get(e.getKey()).equals(e.getValue())),
+                                e -> LedgerMetadataUtils.getLastEnsembleValue(metadata)
+                                             .get(e.getKey()).equals(e.getValue())),
                 (metadata) -> {
                     attempts.incrementAndGet();
 
                     List<BookieSocketAddress> currentEnsemble = getCurrentEnsemble();
                     List<BookieSocketAddress> newEnsemble = EnsembleUtils.replaceBookiesInEnsemble(
                             clientCtx.getBookieWatcher(), metadata, currentEnsemble, failedBookies, logContext);
-                    Long lastEnsembleKey = metadata.getLastEnsembleKey();
+                    Long lastEnsembleKey = LedgerMetadataUtils.getLastEnsembleKey(metadata);
                     LedgerMetadataBuilder builder = LedgerMetadataBuilder.from(metadata);
                     long newEnsembleStartEntry = getLastAddConfirmed() + 1;
                     checkState(lastEnsembleKey <= newEnsembleStartEntry,
@@ -1890,7 +1892,7 @@ public class LedgerHandle implements WriteHandle {
                                       + " Another client must have recovered the ledger.", logContext, attempts.get());
                         }
                         handleUnrecoverableErrorDuringAdd(BKException.Code.LedgerClosedException);
-                    } else if (metadata.getValue().isInRecovery()) {
+                    } else if (metadata.getValue().getState() == LedgerMetadata.State.IN_RECOVERY) {
                         if (LOG.isDebugEnabled()) {
                             LOG.debug("{}[attempt:{}] Metadata marked as in-recovery during attempt to replace bookie."
                                       + " Another client must be recovering the ledger.", logContext, attempts.get());
@@ -1963,6 +1965,6 @@ public class LedgerHandle implements WriteHandle {
         // Getting current ensemble from the metadata is only a temporary
         // thing until metadata is immutable. At that point, current ensemble
         // becomes a property of the LedgerHandle itself.
-        return versionedMetadata.getValue().getCurrentEnsemble();
+        return LedgerMetadataUtils.getCurrentEnsemble(versionedMetadata.getValue());
     }
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandleAdv.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandleAdv.java
index 6f38b8e..14317d0 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandleAdv.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandleAdv.java
@@ -34,6 +34,7 @@ import java.util.concurrent.RejectedExecutionException;
 import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
 import org.apache.bookkeeper.client.AsyncCallback.AddCallbackWithLatency;
 import org.apache.bookkeeper.client.SyncCallbackUtils.SyncAddCallback;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.client.api.WriteAdvHandle;
 import org.apache.bookkeeper.client.api.WriteFlag;
 import org.apache.bookkeeper.util.SafeRunnable;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java
index c5a329c..0c80315 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataBuilder.java
@@ -32,6 +32,7 @@ import java.util.Optional;
 import java.util.TreeMap;
 
 import org.apache.bookkeeper.client.api.DigestType;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.client.api.LedgerMetadata.State;
 import org.apache.bookkeeper.common.annotation.InterfaceAudience.LimitedPrivate;
 import org.apache.bookkeeper.common.annotation.InterfaceStability.Unstable;
@@ -87,7 +88,9 @@ public class LedgerMetadataBuilder {
         }
 
         builder.ctime = other.getCtime();
-        builder.storeCtime = other.storeCtime;
+
+        /** Hack to get around fact that ctime was never versioned correctly */
+        builder.storeCtime = LedgerMetadataUtils.shouldStoreCtime(other);
 
         builder.customMetadata = ImmutableMap.copyOf(other.getCustomMetadata());
 
@@ -182,11 +185,11 @@ public class LedgerMetadataBuilder {
         checkArgument(ensembleSize >= writeQuorumSize, "Write quorum must be less or equal to ensemble size");
         checkArgument(writeQuorumSize >= ackQuorumSize, "Write quorum must be greater or equal to ack quorum");
 
-        return new LedgerMetadata(metadataFormatVersion,
-                                  ensembleSize, writeQuorumSize, ackQuorumSize,
-                                  state, lastEntryId, length, ensembles,
-                                  digestType, password, ctime, storeCtime,
-                                  customMetadata);
+        return new LedgerMetadataImpl(metadataFormatVersion,
+                                      ensembleSize, writeQuorumSize, ackQuorumSize,
+                                      state, lastEntryId, length, ensembles,
+                                      digestType, password, ctime, storeCtime,
+                                      customMetadata);
     }
 
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataImpl.java
similarity index 76%
rename from bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java
rename to bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataImpl.java
index fa32f67..c0fcadd 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataImpl.java
@@ -18,26 +18,22 @@
 package org.apache.bookkeeper.client;
 
 import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkState;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.MoreObjects;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import java.util.Arrays;
 import java.util.Base64;
 import java.util.Collections;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
 import java.util.Optional;
-import java.util.Set;
-import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.stream.Collectors;
 import lombok.EqualsAndHashCode;
 import org.apache.bookkeeper.client.api.DigestType;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.client.api.LedgerMetadata.State;
 import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.slf4j.Logger;
@@ -50,8 +46,8 @@ import org.slf4j.LoggerFactory;
  * <p>It provides parsing and serialization methods of such metadata.
  */
 @EqualsAndHashCode
-public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMetadata {
-    static final Logger LOG = LoggerFactory.getLogger(LedgerMetadata.class);
+class LedgerMetadataImpl implements LedgerMetadata {
+    static final Logger LOG = LoggerFactory.getLogger(LedgerMetadataImpl.class);
 
     private final int metadataFormatVersion;
     private final int ensembleSize;
@@ -73,19 +69,19 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe
 
     private final Map<String, byte[]> customMetadata;
 
-    LedgerMetadata(int metadataFormatVersion,
-                   int ensembleSize,
-                   int writeQuorumSize,
-                   int ackQuorumSize,
-                   State state,
-                   Optional<Long> lastEntryId,
-                   Optional<Long> length,
-                   Map<Long, List<BookieSocketAddress>> ensembles,
-                   Optional<DigestType> digestType,
-                   Optional<byte[]> password,
-                   long ctime,
-                   boolean storeCtime,
-                   Map<String, byte[]> customMetadata) {
+    LedgerMetadataImpl(int metadataFormatVersion,
+                       int ensembleSize,
+                       int writeQuorumSize,
+                       int ackQuorumSize,
+                       State state,
+                       Optional<Long> lastEntryId,
+                       Optional<Long> length,
+                       Map<Long, List<BookieSocketAddress>> ensembles,
+                       Optional<DigestType> digestType,
+                       Optional<byte[]> password,
+                       long ctime,
+                       boolean storeCtime,
+                       Map<String, byte[]> customMetadata) {
         checkArgument(ensembles.size() > 0, "There must be at least one ensemble in the ledger");
         if (state == State.CLOSED) {
             checkArgument(length.isPresent(), "Closed ledger must have a length");
@@ -165,11 +161,12 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe
      *
      * @return whether the password has been stored in the metadata
      */
+    @Override
     public boolean hasPassword() {
         return hasPassword;
     }
 
-    @VisibleForTesting
+    @Override
     public byte[] getPassword() {
         if (!hasPassword()) {
             return new byte[0];
@@ -202,48 +199,19 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe
         return state == State.CLOSED;
     }
 
-    public boolean isInRecovery() {
-        return state == State.IN_RECOVERY;
-    }
-
     @Override
     public State getState() {
         return state;
     }
 
-    List<BookieSocketAddress> getCurrentEnsemble() {
-        return currentEnsemble;
-    }
-
-    List<BookieSocketAddress> getEnsemble(long entryId) {
+    @Override
+    public List<BookieSocketAddress> getEnsembleAt(long entryId) {
         // the head map cannot be empty, since we insert an ensemble for
         // entry-id 0, right when we start
         return ensembles.get(ensembles.headMap(entryId + 1).lastKey());
     }
 
     @Override
-    public List<BookieSocketAddress> getEnsembleAt(long entryId) {
-        return getEnsemble(entryId);
-    }
-
-    /**
-     * the entry id greater than the given entry-id at which the next ensemble change takes
-     * place.
-     *
-     * @param entryId
-     * @return the entry id of the next ensemble change (-1 if no further ensemble changes)
-     */
-    long getNextEnsembleChange(long entryId) {
-        SortedMap<Long, ? extends List<BookieSocketAddress>> tailMap = ensembles.tailMap(entryId + 1);
-
-        if (tailMap.isEmpty()) {
-            return -1;
-        } else {
-            return tailMap.firstKey();
-        }
-    }
-
-    @Override
     public Map<String, byte[]> getCustomMetadata() {
         return this.customMetadata;
     }
@@ -260,6 +228,7 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe
      * @return a string representation of the object without password field in
      *         it.
      */
+    @Override
     public String toSafeString() {
         return toStringRepresentation(false);
     }
@@ -291,30 +260,12 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe
         return helper.toString();
     }
 
-    Set<BookieSocketAddress> getBookiesInThisLedger() {
-        Set<BookieSocketAddress> bookies = new HashSet<BookieSocketAddress>();
-        for (List<BookieSocketAddress> ensemble : ensembles.values()) {
-            bookies.addAll(ensemble);
-        }
-        return bookies;
-    }
-
-    List<BookieSocketAddress> getLastEnsembleValue() {
-        checkState(!ensembles.isEmpty(), "Metadata should never be created with no ensembles");
-        return ensembles.lastEntry().getValue();
-    }
-
-    Long getLastEnsembleKey() {
-        checkState(!ensembles.isEmpty(), "Metadata should never be created with no ensembles");
-        return ensembles.lastKey();
-    }
-
+    @Override
     public int getMetadataFormatVersion() {
         return metadataFormatVersion;
     }
 
-    // temporarily method, until storeCtime is removed from the metadata object itself
-    public boolean shouldStoreCtime() {
+    boolean shouldStoreCtime() {
         return storeCtime;
     }
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataUtils.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataUtils.java
new file mode 100644
index 0000000..07d7340
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataUtils.java
@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.bookkeeper.client;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.SortedMap;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
+import org.apache.bookkeeper.net.BookieSocketAddress;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utilities for working with ledger metadata.
+ */
+public class LedgerMetadataUtils {
+    static final Logger LOG = LoggerFactory.getLogger(LedgerMetadataUtils.class);
+
+    static List<BookieSocketAddress> getCurrentEnsemble(LedgerMetadata metadata) {
+        return getLastEnsembleValue(metadata);
+    }
+
+    /**
+     * the entry id greater than the given entry-id at which the next ensemble change takes
+     * place.
+     *
+     * @param entryId
+     * @return the entry id of the next ensemble change (-1 if no further ensemble changes)
+     */
+    static long getNextEnsembleChange(LedgerMetadata metadata, long entryId) {
+        SortedMap<Long, ? extends List<BookieSocketAddress>> tailMap = metadata.getAllEnsembles().tailMap(entryId + 1);
+
+        if (tailMap.isEmpty()) {
+            return -1;
+        } else {
+            return tailMap.firstKey();
+        }
+    }
+
+    static Set<BookieSocketAddress> getBookiesInThisLedger(LedgerMetadata metadata) {
+        Set<BookieSocketAddress> bookies = new HashSet<BookieSocketAddress>();
+        for (List<BookieSocketAddress> ensemble : metadata.getAllEnsembles().values()) {
+            bookies.addAll(ensemble);
+        }
+        return bookies;
+    }
+
+    static List<BookieSocketAddress> getLastEnsembleValue(LedgerMetadata metadata) {
+        checkArgument(!metadata.getAllEnsembles().isEmpty(), "Metadata should never be created with no ensembles");
+        return metadata.getAllEnsembles().lastEntry().getValue();
+    }
+
+    static Long getLastEnsembleKey(LedgerMetadata metadata) {
+        checkArgument(!metadata.getAllEnsembles().isEmpty(), "Metadata should never be created with no ensembles");
+        return metadata.getAllEnsembles().lastKey();
+    }
+
+    public static boolean shouldStoreCtime(LedgerMetadata metadata) {
+        return metadata instanceof LedgerMetadataImpl && ((LedgerMetadataImpl) metadata).shouldStoreCtime();
+    }
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerOpenOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerOpenOp.java
index c2dc33a..2193b3e 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerOpenOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerOpenOp.java
@@ -34,6 +34,7 @@ import org.apache.bookkeeper.client.AsyncCallback.ReadLastConfirmedCallback;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
 import org.apache.bookkeeper.client.SyncCallbackUtils.SyncOpenCallback;
 import org.apache.bookkeeper.client.api.BKException.Code;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.client.api.ReadHandle;
 import org.apache.bookkeeper.client.impl.OpenBuilderBase;
 import org.apache.bookkeeper.stats.OpStatsLogger;
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 bc2985a..2dd6ea2 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
@@ -22,6 +22,7 @@ import com.google.common.annotations.VisibleForTesting;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryListener;
 import org.apache.bookkeeper.proto.checksum.DigestManager.RecoveryData;
 import org.slf4j.Logger;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/MetadataUpdateLoop.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/MetadataUpdateLoop.java
index 435ff69..5c8cb14 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/MetadataUpdateLoop.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/MetadataUpdateLoop.java
@@ -23,6 +23,7 @@ import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 import java.util.function.Supplier;
 
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.meta.LedgerManager;
 import org.apache.bookkeeper.versioning.Version;
 import org.apache.bookkeeper.versioning.Versioned;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java
index a66d889..aa89eaa 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java
@@ -35,6 +35,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.bookkeeper.client.BKException.BKDigestMatchException;
 import org.apache.bookkeeper.client.api.LedgerEntries;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.client.impl.LedgerEntriesImpl;
 import org.apache.bookkeeper.client.impl.LedgerEntryImpl;
 import org.apache.bookkeeper.common.util.SafeRunnable;
@@ -506,8 +507,8 @@ class PendingReadOp implements ReadEntryCallback, SafeRunnable {
         List<BookieSocketAddress> ensemble = null;
         do {
             if (i == nextEnsembleChange) {
-                ensemble = getLedgerMetadata().getEnsemble(i);
-                nextEnsembleChange = getLedgerMetadata().getNextEnsembleChange(i);
+                ensemble = getLedgerMetadata().getEnsembleAt(i);
+                nextEnsembleChange = LedgerMetadataUtils.getNextEnsembleChange(getLedgerMetadata(), i);
             }
             LedgerEntryRequest entry;
             if (parallelRead) {
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOp.java
index cb9de32..ed1bece 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOp.java
@@ -28,6 +28,7 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.client.impl.LedgerEntryImpl;
 import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.proto.BookieProtocol;
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 59ddd56..72930f7 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
@@ -37,6 +37,7 @@ import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
 import org.apache.bookkeeper.client.AsyncCallback.CloseCallback;
 import org.apache.bookkeeper.client.AsyncCallback.ReadCallback;
 import org.apache.bookkeeper.client.AsyncCallback.ReadLastConfirmedCallback;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.client.api.WriteFlag;
 import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback;
@@ -266,11 +267,11 @@ class ReadOnlyLedgerHandle extends LedgerHandle implements LedgerMetadataListene
             clientCtx.getClientStats().getRecoverOpLogger());
 
         MetadataUpdateLoop.NeedsUpdatePredicate needsUpdate =
-            (metadata) -> !(metadata.isClosed() || metadata.isInRecovery());
+            (metadata) -> metadata.getState() == LedgerMetadata.State.OPEN;
         if (forceRecovery) {
             // in the force recovery case, we want to update the metadata
             // to IN_RECOVERY, even if the ledger is already closed
-            needsUpdate = (metadata) -> !metadata.isInRecovery();
+            needsUpdate = (metadata) -> metadata.getState() != LedgerMetadata.State.IN_RECOVERY;
         }
         new MetadataUpdateLoop(
                 clientCtx.getLedgerManager(), getId(),
@@ -309,10 +310,10 @@ class ReadOnlyLedgerHandle extends LedgerHandle implements LedgerMetadataListene
         CompletableFuture<Versioned<LedgerMetadata>> f = new MetadataUpdateLoop(
                 clientCtx.getLedgerManager(), getId(),
                 this::getVersionedLedgerMetadata,
-                (metadata) -> metadata.isInRecovery(),
+                (metadata) -> metadata.getState() == LedgerMetadata.State.IN_RECOVERY,
                 (metadata) -> {
                     LedgerMetadataBuilder builder = LedgerMetadataBuilder.from(metadata);
-                    Long lastEnsembleKey = metadata.getLastEnsembleKey();
+                    Long lastEnsembleKey = LedgerMetadataUtils.getLastEnsembleKey(metadata);
                     synchronized (metadataLock) {
                         newEnsemblesFromRecovery.entrySet().forEach(
                                 (e) -> {
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java
index 98c544e..6fce7e1 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java
@@ -35,6 +35,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.stream.Collectors;
 
 import org.apache.bookkeeper.bookie.BookieShell.UpdateLedgerNotifier;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.meta.LedgerManager;
 import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.versioning.Versioned;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LedgerMetadata.java
index de86832..2ce1940 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LedgerMetadata.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/LedgerMetadata.java
@@ -75,7 +75,24 @@ public interface LedgerMetadata {
     long getLength();
 
     /**
+     * Whether the metadata contains the password and digest type for the ledger.
+     * Ledgers created with version 4.1.0 clients or older do not have this information.
+     *
+     * @return true if the metadata contains the password and digest type, false otherwise.
+     */
+    boolean hasPassword();
+
+    /**
+     * Get the password for the ledger.
+     * For ledgers created with version 4.1.0 or older, an empty byte array is returned.
+     *
+     * @return the password for the ledger.
+     */
+    byte[] getPassword();
+
+    /**
      * Returns the digest type used by this ledger.
+     * May return null if the ledger was created with version 4.1.0 or below.
      *
      * @return the digest type used by this ledger.
      */
@@ -143,4 +160,18 @@ public interface LedgerMetadata {
         */
         CLOSED;
     }
+
+    /**
+     * Similar to #toString(), but omits the password of the ledger, so that it is safe to log the output.
+     *
+     * @return a string representation of the metadata, omitting the password.
+     */
+    String toSafeString();
+
+    /**
+     * Get the format version which should be used to serialize the metadata.
+     *
+     * @return the format version.
+     */
+    int getMetadataFormatVersion();
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/AbstractConfiguration.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/AbstractConfiguration.java
index e395cb3..7f00d09 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/AbstractConfiguration.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/AbstractConfiguration.java
@@ -84,6 +84,7 @@ public abstract class AbstractConfiguration<T extends AbstractConfiguration>
     protected static final String REREPLICATION_ENTRY_BATCH_SIZE = "rereplicationEntryBatchSize";
     protected static final String STORE_SYSTEMTIME_AS_LEDGER_UNDERREPLICATED_MARK_TIME =
             "storeSystemTimeAsLedgerUnderreplicatedMarkTime";
+    protected static final String STORE_SYSTEMTIME_AS_LEDGER_CREATION_TIME = "storeSystemTimeAsLedgerCreationTime";
 
     // Metastore settings, only being used when LEDGER_MANAGER_FACTORY_CLASS is MSLedgerManagerFactory
     protected static final String METASTORE_IMPL_CLASS = "metastoreImplClass";
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ClientConfiguration.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ClientConfiguration.java
index 86ec444..7d0d319 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ClientConfiguration.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ClientConfiguration.java
@@ -154,9 +154,6 @@ public class ClientConfiguration extends AbstractConfiguration<ClientConfigurati
     protected static final String ENSEMBLE_PLACEMENT_POLICY_ORDER_SLOW_BOOKIES =
         "ensemblePlacementPolicyOrderSlowBookies";
 
-    // Ledger Metadata Parameters
-    protected static final String STORE_SYSTEMTIME_AS_LEDGER_CREATION_TIME = "storeSystemTimeAsLedgerCreationTime";
-
     // Stats
     protected static final String ENABLE_TASK_EXECUTION_STATS = "enableTaskExecutionStats";
     protected static final String TASK_EXECUTION_WARN_TIME_MICROS = "taskExecutionWarnTimeMicros";
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java
index 2bd9b13..5dbdd06 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java
@@ -36,7 +36,7 @@ import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.LedgerMetadata;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.conf.AbstractConfiguration;
 import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/CleanupLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/CleanupLedgerManager.java
index 311cb0a..25c5aca 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/CleanupLedgerManager.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/CleanupLedgerManager.java
@@ -28,7 +28,7 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.LedgerMetadata;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManager.java
index 8bcda06..cc7630b 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManager.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerManager.java
@@ -24,7 +24,7 @@ import java.util.SortedSet;
 import java.util.TreeSet;
 import java.util.concurrent.CompletableFuture;
 
-import org.apache.bookkeeper.client.LedgerMetadata;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor;
 import org.apache.bookkeeper.versioning.Version;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java
index 26b616b..6020a3b 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java
@@ -34,9 +34,10 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.stream.Collectors;
 
-import org.apache.bookkeeper.client.LedgerMetadata;
 import org.apache.bookkeeper.client.LedgerMetadataBuilder;
+import org.apache.bookkeeper.client.LedgerMetadataUtils;
 import org.apache.bookkeeper.client.api.DigestType;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.client.api.LedgerMetadata.State;
 import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat;
@@ -135,7 +136,8 @@ public class LedgerMetadataSerDe {
             break;
         }
 
-        if (metadata.shouldStoreCtime()) {
+        /** Hack to get around fact that ctime was never versioned correctly */
+        if (LedgerMetadataUtils.shouldStoreCtime(metadata)) {
             builder.setCtime(metadata.getCtime());
         }
 
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java
index 7e28217..fc87632 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MSLedgerManagerFactory.java
@@ -39,7 +39,7 @@ import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.LedgerMetadata;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.conf.AbstractConfiguration;
 import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookkeeperInternalCallbacks.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookkeeperInternalCallbacks.java
index 8412e06..2ade9e9 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookkeeperInternalCallbacks.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookkeeperInternalCallbacks.java
@@ -34,7 +34,7 @@ import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo;
 import org.apache.bookkeeper.client.LedgerEntry;
 import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.client.LedgerMetadata;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.stats.OpStatsLogger;
 import org.apache.bookkeeper.util.MathUtils;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java
index 6339810..eeaa96b 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java
@@ -51,7 +51,7 @@ import org.apache.bookkeeper.client.BookKeeperAdmin;
 import org.apache.bookkeeper.client.LedgerChecker;
 import org.apache.bookkeeper.client.LedgerFragment;
 import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.client.LedgerMetadata;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java
index 4225c08..7fcaeda 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/GetLedgerMetaService.java
@@ -23,7 +23,7 @@ import static com.google.common.base.Preconditions.checkNotNull;
 
 import com.google.common.collect.Maps;
 import java.util.Map;
-import org.apache.bookkeeper.client.LedgerMetadata;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.common.util.JsonUtil;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.http.HttpServer;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java
index 1df1b36..f553b64 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/ListLedgerService.java
@@ -25,7 +25,7 @@ import com.google.common.collect.Maps;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.concurrent.CompletableFuture;
-import org.apache.bookkeeper.client.LedgerMetadata;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.common.util.JsonUtil;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.http.HttpServer;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java
index 9f12e15..3867466 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieShellTest.java
@@ -43,7 +43,7 @@ import java.util.function.Function;
 import org.apache.bookkeeper.bookie.BookieShell.MyCommand;
 import org.apache.bookkeeper.bookie.BookieShell.RecoverCmd;
 import org.apache.bookkeeper.client.BookKeeperAdmin;
-import org.apache.bookkeeper.client.LedgerMetadata;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.discover.RegistrationManager;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java
index e35c309..c110eeb 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java
@@ -57,7 +57,7 @@ import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
 import org.apache.bookkeeper.client.LedgerEntry;
 import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.client.LedgerMetadata;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.conf.TestBKConfiguration;
 import org.apache.bookkeeper.meta.LedgerManager;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestGcOverreplicatedLedger.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestGcOverreplicatedLedger.java
index 8020f1f..bb39329 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestGcOverreplicatedLedger.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/TestGcOverreplicatedLedger.java
@@ -33,7 +33,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.bookkeeper.bookie.GarbageCollector.GarbageCleaner;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
 import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.client.LedgerMetadata;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.meta.HierarchicalLedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperDiskSpaceWeightedLedgerPlacementTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperDiskSpaceWeightedLedgerPlacementTest.java
index fec5dd8..33512ae 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperDiskSpaceWeightedLedgerPlacementTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperDiskSpaceWeightedLedgerPlacementTest.java
@@ -162,7 +162,7 @@ public class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
 
         for (int i = 0; i < 2000; i++) {
             LedgerHandle lh = client.createLedger(3, 3, DigestType.CRC32, "testPasswd".getBytes());
-            for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsemble(0)) {
+            for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsembleAt(0)) {
                 m.put(b, m.get(b) + 1);
             }
         }
@@ -212,7 +212,7 @@ public class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
 
         for (int i = 0; i < 2000; i++) {
             LedgerHandle lh = client.createLedger(3, 3, DigestType.CRC32, "testPasswd".getBytes());
-            for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsemble(0)) {
+            for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsembleAt(0)) {
                 m.put(b, m.get(b) + 1);
             }
         }
@@ -247,7 +247,7 @@ public class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
         }
         for (int i = 0; i < 2000; i++) {
             LedgerHandle lh = client.createLedger(3, 3, DigestType.CRC32, "testPasswd".getBytes());
-            for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsemble(0)) {
+            for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsembleAt(0)) {
                 m.put(b, m.get(b) + 1);
             }
         }
@@ -302,7 +302,7 @@ public class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
 
         for (int i = 0; i < 2000; i++) {
             LedgerHandle lh = client.createLedger(3, 3, DigestType.CRC32, "testPasswd".getBytes());
-            for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsemble(0)) {
+            for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsembleAt(0)) {
                 m.put(b, m.get(b) + 1);
             }
         }
@@ -330,7 +330,7 @@ public class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
 
         for (int i = 0; i < 2000; i++) {
             LedgerHandle lh = client.createLedger(3, 3, DigestType.CRC32, "testPasswd".getBytes());
-            for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsemble(0)) {
+            for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsembleAt(0)) {
                 m.put(b, m.get(b) + 1);
             }
         }
@@ -382,7 +382,7 @@ public class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
 
         for (int i = 0; i < 2000; i++) {
             LedgerHandle lh = client.createLedger(3, 3, DigestType.CRC32, "testPasswd".getBytes());
-            for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsemble(0)) {
+            for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsembleAt(0)) {
                 m.put(b, m.get(b) + 1);
             }
         }
@@ -406,7 +406,7 @@ public class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
         }
         for (int i = 0; i < 2000; i++) {
             LedgerHandle lh = client.createLedger(3, 3, DigestType.CRC32, "testPasswd".getBytes());
-            for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsemble(0)) {
+            for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsembleAt(0)) {
                 m.put(b, m.get(b) + 1);
             }
         }
@@ -462,7 +462,7 @@ public class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
 
         for (int i = 0; i < 2000; i++) {
             LedgerHandle lh = client.createLedger(3, 3, DigestType.CRC32, "testPasswd".getBytes());
-            for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsemble(0)) {
+            for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsembleAt(0)) {
                 m.put(b, m.get(b) + 1);
             }
         }
@@ -491,7 +491,7 @@ public class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
         }
         for (int i = 0; i < 2000; i++) {
             LedgerHandle lh = client.createLedger(3, 3, DigestType.CRC32, "testPasswd".getBytes());
-            for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsemble(0)) {
+            for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsembleAt(0)) {
                 m.put(b, m.get(b) + 1);
             }
         }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieRecoveryTest.java
index 506f512..2fc1d5f 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieRecoveryTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieRecoveryTest.java
@@ -41,6 +41,7 @@ import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.bookkeeper.client.AsyncCallback.RecoverCallback;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.net.BookieSocketAddress;
@@ -262,7 +263,7 @@ public class BookieRecoveryTest extends BookKeeperClusterTestCase {
         for (int i = 0; i < numEntries; i++) {
             lh.addEntry(data);
         }
-        BookieSocketAddress bookieToKill = lh.getLedgerMetadata().getEnsemble(numEntries - 1).get(1);
+        BookieSocketAddress bookieToKill = lh.getLedgerMetadata().getEnsembleAt(numEntries - 1).get(1);
         killBookie(bookieToKill);
         startNewBookie();
         for (int i = 0; i < numEntries; i++) {
@@ -270,7 +271,7 @@ public class BookieRecoveryTest extends BookKeeperClusterTestCase {
         }
         bkAdmin.recoverBookieData(bookieToKill);
         // fail another bookie to cause ensemble change again
-        bookieToKill = lh.getLedgerMetadata().getEnsemble(2 * numEntries - 1).get(1);
+        bookieToKill = lh.getLedgerMetadata().getEnsembleAt(2 * numEntries - 1).get(1);
         ServerConfiguration confOfKilledBookie = killBookie(bookieToKill);
         startNewBookie();
         for (int i = 0; i < numEntries; i++) {
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ClientUtil.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ClientUtil.java
index f56c350..68a6846 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ClientUtil.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ClientUtil.java
@@ -25,6 +25,7 @@ import io.netty.buffer.Unpooled;
 import java.security.GeneralSecurityException;
 import java.util.function.Function;
 
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.meta.LedgerManager;
 import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat.DigestType;
 import org.apache.bookkeeper.proto.checksum.DigestManager;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java
index 8fbb009..bb55d0c 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/GenericEnsemblePlacementPolicyTest.java
@@ -144,7 +144,7 @@ public class GenericEnsemblePlacementPolicyTest extends BookKeeperClusterTestCas
                 try (LedgerHandle lh = bk.createLedger(2, 2, 2, digestType, PASSWORD.getBytes(), customMetadata)) {
                     lh.addEntry(value);
                     long lId = lh.getId();
-                    List<BookieSocketAddress> ensembleAtFirstEntry = lh.getLedgerMetadata().getEnsemble(lId);
+                    List<BookieSocketAddress> ensembleAtFirstEntry = lh.getLedgerMetadata().getEnsembleAt(lId);
                     assertEquals(2, ensembleAtFirstEntry.size());
                     killBookie(ensembleAtFirstEntry.get(0));
                     lh.addEntry(value);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java
index 2f75e12..9f95a94 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java
@@ -33,6 +33,7 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.client.api.WriteFlag;
 import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.net.BookieSocketAddress;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java
index 541af0a..45b2713 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java
@@ -20,6 +20,7 @@ package org.apache.bookkeeper.client;
 import com.google.common.collect.Lists;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.client.api.WriteFlag;
 import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.net.BookieSocketAddress;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerHandleAdapter.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerHandleAdapter.java
index 7098f4e..9450e51 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerHandleAdapter.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerHandleAdapter.java
@@ -29,13 +29,6 @@ import org.apache.bookkeeper.util.ByteBufList;
  */
 public class LedgerHandleAdapter {
 
-    /**
-     * Get the ledger handle.
-     */
-    public static LedgerMetadata getLedgerMetadata(LedgerHandle lh) {
-        return lh.getLedgerMetadata();
-    }
-
     public static ByteBufList toSend(LedgerHandle lh, long entryId, ByteBuf data) {
         return lh.getDigestManager().computeDigestAndPackageForSending(entryId, lh.getLastAddConfirmed(),
                 lh.addToLength(data.readableBytes()), data);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java
index 80e1c75..a9ffc2d 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerMetadataTest.java
@@ -29,6 +29,7 @@ import java.util.Base64;
 import java.util.Collections;
 import java.util.List;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.meta.LedgerMetadataSerDe;
 import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java
index d9c5d35..2e17836 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecovery2Test.java
@@ -26,6 +26,7 @@ import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import org.apache.bookkeeper.client.api.DigestType;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallbackFuture;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java
index 2544a20..ffacb21 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MetadataUpdateLoopTest.java
@@ -42,6 +42,7 @@ import java.util.stream.IntStream;
 import lombok.AllArgsConstructor;
 import lombok.Data;
 
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.meta.LedgerManager;
 import org.apache.bookkeeper.meta.MockLedgerManager;
 import org.apache.bookkeeper.net.BookieSocketAddress;
@@ -86,7 +87,7 @@ public class MetadataUpdateLoopTest {
                     reference::get,
                     (currentMetadata) -> true,
                     (currentMetadata) -> {
-                        List<BookieSocketAddress> ensemble = Lists.newArrayList(currentMetadata.getEnsemble(0L));
+                        List<BookieSocketAddress> ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L));
                         ensemble.set(0, newAddress);
                         return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build();
                     },
@@ -94,7 +95,7 @@ public class MetadataUpdateLoopTest {
             loop.run().get();
 
             Assert.assertNotEquals(reference.get(), writtenMetadata);
-            Assert.assertEquals(reference.get().getValue().getEnsemble(0L).get(0), newAddress);
+            Assert.assertEquals(reference.get().getValue().getEnsembleAt(0L).get(0), newAddress);
         }
     }
 
@@ -123,9 +124,9 @@ public class MetadataUpdateLoopTest {
                     lm,
                     ledgerId,
                     reference1::get,
-                    (currentMetadata) -> currentMetadata.getEnsemble(0L).contains(b0),
+                    (currentMetadata) -> currentMetadata.getEnsembleAt(0L).contains(b0),
                     (currentMetadata) -> {
-                        List<BookieSocketAddress> ensemble = Lists.newArrayList(currentMetadata.getEnsemble(0L));
+                        List<BookieSocketAddress> ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L));
                         ensemble.set(0, b2);
                         return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build();
                     },
@@ -136,9 +137,9 @@ public class MetadataUpdateLoopTest {
                     lm,
                     ledgerId,
                     reference2::get,
-                    (currentMetadata) -> currentMetadata.getEnsemble(0L).contains(b1),
+                    (currentMetadata) -> currentMetadata.getEnsembleAt(0L).contains(b1),
                     (currentMetadata) -> {
-                        List<BookieSocketAddress> ensemble = Lists.newArrayList(currentMetadata.getEnsemble(0L));
+                        List<BookieSocketAddress> ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L));
                         ensemble.set(1, b3);
                         return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build();
                     },
@@ -154,11 +155,11 @@ public class MetadataUpdateLoopTest {
 
             Assert.assertEquals(l1meta.getVersion().compare(l2meta.getVersion()), Version.Occurred.BEFORE);
 
-            Assert.assertEquals(l1meta.getValue().getEnsemble(0L).get(0), b2);
-            Assert.assertEquals(l1meta.getValue().getEnsemble(0L).get(1), b1);
+            Assert.assertEquals(l1meta.getValue().getEnsembleAt(0L).get(0), b2);
+            Assert.assertEquals(l1meta.getValue().getEnsembleAt(0L).get(1), b1);
 
-            Assert.assertEquals(l2meta.getValue().getEnsemble(0L).get(0), b2);
-            Assert.assertEquals(l2meta.getValue().getEnsemble(0L).get(1), b3);
+            Assert.assertEquals(l2meta.getValue().getEnsembleAt(0L).get(0), b2);
+            Assert.assertEquals(l2meta.getValue().getEnsembleAt(0L).get(1), b3);
 
             verify(lm, times(3)).writeLedgerMetadata(anyLong(), any(), any());
         }
@@ -188,9 +189,9 @@ public class MetadataUpdateLoopTest {
                     lm,
                     ledgerId,
                     reference::get,
-                    (currentMetadata) -> currentMetadata.getEnsemble(0L).contains(b0),
+                    (currentMetadata) -> currentMetadata.getEnsembleAt(0L).contains(b0),
                     (currentMetadata) -> {
-                        List<BookieSocketAddress> ensemble = Lists.newArrayList(currentMetadata.getEnsemble(0L));
+                        List<BookieSocketAddress> ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L));
                         ensemble.set(0, b2);
                         return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build();
                     },
@@ -199,9 +200,9 @@ public class MetadataUpdateLoopTest {
                     lm,
                     ledgerId,
                     reference::get,
-                    (currentMetadata) -> currentMetadata.getEnsemble(0L).contains(b0),
+                    (currentMetadata) -> currentMetadata.getEnsembleAt(0L).contains(b0),
                     (currentMetadata) -> {
-                        List<BookieSocketAddress> ensemble = Lists.newArrayList(currentMetadata.getEnsemble(0L));
+                        List<BookieSocketAddress> ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L));
                         ensemble.set(0, b2);
                         return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build();
                     },
@@ -212,8 +213,8 @@ public class MetadataUpdateLoopTest {
             Assert.assertEquals(loop1.get(), loop2.get());
             Assert.assertEquals(loop1.get(), reference.get());
 
-            Assert.assertEquals(reference.get().getValue().getEnsemble(0L).get(0), b2);
-            Assert.assertEquals(reference.get().getValue().getEnsemble(0L).get(1), b1);
+            Assert.assertEquals(reference.get().getValue().getEnsembleAt(0L).get(0), b2);
+            Assert.assertEquals(reference.get().getValue().getEnsembleAt(0L).get(1), b1);
 
             verify(lm, times(2)).writeLedgerMetadata(anyLong(), any(), any());
         }
@@ -241,9 +242,9 @@ public class MetadataUpdateLoopTest {
                     lm,
                     ledgerId,
                     reference::get,
-                    (currentMetadata) -> currentMetadata.getEnsemble(0L).contains(b0),
+                    (currentMetadata) -> currentMetadata.getEnsembleAt(0L).contains(b0),
                     (currentMetadata) -> {
-                        List<BookieSocketAddress> ensemble = Lists.newArrayList(currentMetadata.getEnsemble(0L));
+                        List<BookieSocketAddress> ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L));
                         ensemble.set(0, b2);
                         return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build();
                     },
@@ -254,9 +255,9 @@ public class MetadataUpdateLoopTest {
                     lm,
                     ledgerId,
                     reference::get,
-                    (currentMetadata) -> currentMetadata.getEnsemble(0L).contains(b1),
+                    (currentMetadata) -> currentMetadata.getEnsembleAt(0L).contains(b1),
                     (currentMetadata) -> {
-                        List<BookieSocketAddress> ensemble = Lists.newArrayList(currentMetadata.getEnsemble(0L));
+                        List<BookieSocketAddress> ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L));
                         ensemble.set(1, b3);
                         return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build();
                     },
@@ -267,8 +268,8 @@ public class MetadataUpdateLoopTest {
 
             Assert.assertEquals(loop1.get(), reference.get());
 
-            Assert.assertEquals(reference.get().getValue().getEnsemble(0L).get(0), b2);
-            Assert.assertEquals(reference.get().getValue().getEnsemble(0L).get(1), b3);
+            Assert.assertEquals(reference.get().getValue().getEnsembleAt(0L).get(0), b2);
+            Assert.assertEquals(reference.get().getValue().getEnsembleAt(0L).get(1), b3);
 
             verify(lm, times(3)).writeLedgerMetadata(anyLong(), any(), any());
         }
@@ -312,9 +313,9 @@ public class MetadataUpdateLoopTest {
                     lm,
                     ledgerId,
                     reference::get,
-                    (currentMetadata) -> currentMetadata.getEnsemble(0L).contains(initialEnsemble.get(i)),
+                    (currentMetadata) -> currentMetadata.getEnsembleAt(0L).contains(initialEnsemble.get(i)),
                     (currentMetadata) -> {
-                        List<BookieSocketAddress> ensemble = Lists.newArrayList(currentMetadata.getEnsemble(0L));
+                        List<BookieSocketAddress> ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L));
                         ensemble.set(i, replacementBookies.get(i));
                         return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build();
                     },
@@ -323,7 +324,7 @@ public class MetadataUpdateLoopTest {
 
             loops.forEach((l) -> l.join());
 
-            Assert.assertEquals(reference.get().getValue().getEnsemble(0L), replacementBookies);
+            Assert.assertEquals(reference.get().getValue().getEnsembleAt(0L), replacementBookies);
         }
     }
 
@@ -364,11 +365,11 @@ public class MetadataUpdateLoopTest {
                         if (currentMetadata.isClosed()) {
                             throw new BKException.BKLedgerClosedException();
                         } else {
-                            return currentMetadata.getEnsemble(0L).contains(b0);
+                            return currentMetadata.getEnsembleAt(0L).contains(b0);
                         }
                     },
                     (currentMetadata) -> {
-                        List<BookieSocketAddress> ensemble = Lists.newArrayList(currentMetadata.getEnsemble(0L));
+                        List<BookieSocketAddress> ensemble = Lists.newArrayList(currentMetadata.getEnsembleAt(0L));
                         ensemble.set(0, b1);
                         return LedgerMetadataBuilder.from(currentMetadata).replaceEnsembleEntry(0L, ensemble).build();
                     },
@@ -383,7 +384,7 @@ public class MetadataUpdateLoopTest {
                 Assert.assertEquals(ee.getCause().getClass(), BKException.BKLedgerClosedException.class);
             }
             Assert.assertEquals(l1meta, reference.get());
-            Assert.assertEquals(l1meta.getValue().getEnsemble(0L).get(0), b0);
+            Assert.assertEquals(l1meta.getValue().getEnsembleAt(0L).get(0), b0);
             Assert.assertTrue(l1meta.getValue().isClosed());
 
             verify(lm, times(2)).writeLedgerMetadata(anyLong(), any(), any());
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java
index a88f356..6d6e4d1 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java
@@ -53,6 +53,7 @@ import org.apache.bookkeeper.client.BKException.BKDigestMatchException;
 import org.apache.bookkeeper.client.BKException.Code;
 import org.apache.bookkeeper.client.api.CreateBuilder;
 import org.apache.bookkeeper.client.api.DeleteBuilder;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.client.api.OpenBuilder;
 import org.apache.bookkeeper.common.util.OrderedExecutor;
 import org.apache.bookkeeper.common.util.OrderedScheduler;
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 b7cb9e1..3aaeedb 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
@@ -39,6 +39,7 @@ import org.apache.bookkeeper.client.AsyncCallback.ReadCallback;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
 import org.apache.bookkeeper.client.api.LastConfirmedAndEntry;
 import org.apache.bookkeeper.client.api.LedgerEntries;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.client.api.ReadHandle;
 import org.apache.bookkeeper.client.api.WriteFlag;
 import org.apache.bookkeeper.client.impl.LedgerEntryImpl;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java
index 9571d1e..ab966fb 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java
@@ -43,6 +43,7 @@ import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieException;
 import org.apache.bookkeeper.bookie.InterleavedLedgerStorage;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.client.api.WriteFlag;
 import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java
index 527ce8d..a1032a1 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java
@@ -47,6 +47,7 @@ import org.apache.bookkeeper.client.BKException.Code;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
 import org.apache.bookkeeper.client.ReadLastConfirmedAndEntryOp.LastConfirmedAndEntryCallback;
 import org.apache.bookkeeper.client.api.LastConfirmedAndEntry;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.client.impl.LastConfirmedAndEntryImpl;
 import org.apache.bookkeeper.client.impl.LedgerEntryImpl;
 import org.apache.bookkeeper.common.concurrent.FutureUtils;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestBookieHealthCheck.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestBookieHealthCheck.java
index ad35450..cbee711 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestBookieHealthCheck.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestBookieHealthCheck.java
@@ -55,7 +55,7 @@ public class TestBookieHealthCheck extends BookKeeperClusterTestCase {
             lh.addEntry(msg);
         }
 
-        BookieSocketAddress bookieToQuarantine = lh.getLedgerMetadata().getEnsemble(numEntries).get(0);
+        BookieSocketAddress bookieToQuarantine = lh.getLedgerMetadata().getEnsembleAt(numEntries).get(0);
         sleepBookie(bookieToQuarantine, baseClientConf.getAddEntryTimeout() * 2).await();
 
         byte[] tempMsg = "temp-msg".getBytes();
@@ -79,12 +79,12 @@ public class TestBookieHealthCheck extends BookKeeperClusterTestCase {
         // the bookie to be left out of the ensemble should always be the quarantined bookie
         LedgerHandle lh1 = bkc.createLedger(2, 2, 2, BookKeeper.DigestType.CRC32, new byte[] {});
         LedgerHandle lh2 = bkc.createLedger(3, 3, 3, BookKeeper.DigestType.CRC32, new byte[] {});
-        Assert.assertFalse(lh1.getLedgerMetadata().getEnsemble(0).contains(bookieToQuarantine));
-        Assert.assertFalse(lh2.getLedgerMetadata().getEnsemble(0).contains(bookieToQuarantine));
+        Assert.assertFalse(lh1.getLedgerMetadata().getEnsembleAt(0).contains(bookieToQuarantine));
+        Assert.assertFalse(lh2.getLedgerMetadata().getEnsembleAt(0).contains(bookieToQuarantine));
 
         // the quarantined bookie can still be in the ensemble if we do not have enough healthy bookies
         LedgerHandle lh3 = bkc.createLedger(4, 4, 4, BookKeeper.DigestType.CRC32, new byte[] {});
-        Assert.assertTrue(lh3.getLedgerMetadata().getEnsemble(0).contains(bookieToQuarantine));
+        Assert.assertTrue(lh3.getLedgerMetadata().getEnsembleAt(0).contains(bookieToQuarantine));
 
         // make sure faulty bookie is out of quarantine
         Thread.sleep(baseClientConf.getBookieQuarantineTimeSeconds() * 1000);
@@ -97,7 +97,7 @@ public class TestBookieHealthCheck extends BookKeeperClusterTestCase {
     public void testNoQuarantineOnBkRestart() throws Exception {
         final LedgerHandle lh = bkc.createLedger(2, 2, 2, BookKeeper.DigestType.CRC32, new byte[] {});
         final int numEntries = 20;
-        BookieSocketAddress bookieToRestart = lh.getLedgerMetadata().getEnsemble(0).get(0);
+        BookieSocketAddress bookieToRestart = lh.getLedgerMetadata().getEnsembleAt(0).get(0);
 
         // we add entries on a separate thread so that we can restart a bookie on the current thread
         Thread addEntryThread = new Thread() {
@@ -132,8 +132,8 @@ public class TestBookieHealthCheck extends BookKeeperClusterTestCase {
             byte[] msg = ("msg-" + i).getBytes();
             lh.addEntry(msg);
         }
-        BookieSocketAddress bookie1 = lh.getLedgerMetadata().getEnsemble(0).get(0);
-        BookieSocketAddress bookie2 = lh.getLedgerMetadata().getEnsemble(0).get(1);
+        BookieSocketAddress bookie1 = lh.getLedgerMetadata().getEnsembleAt(0).get(0);
+        BookieSocketAddress bookie2 = lh.getLedgerMetadata().getEnsembleAt(0).get(1);
         try {
             // we read an entry that is not added
             lh.readEntries(10, 10);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDelayEnsembleChange.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDelayEnsembleChange.java
index c76a75c..f8c5be5 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDelayEnsembleChange.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDelayEnsembleChange.java
@@ -37,6 +37,7 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback;
@@ -103,7 +104,7 @@ public class TestDelayEnsembleChange extends BookKeeperClusterTestCase {
         LedgerMetadata md = lh.getLedgerMetadata();
 
         for (long eid = startEntry; eid < untilEntry; eid++) {
-            List<BookieSocketAddress> addresses = md.getEnsemble(eid);
+            List<BookieSocketAddress> addresses = md.getEnsembleAt(eid);
             VerificationCallback callback = new VerificationCallback(addresses.size());
             for (BookieSocketAddress addr : addresses) {
                 bkc.getBookieClient().readEntry(addr, lh.getId(), eid,
@@ -121,7 +122,7 @@ public class TestDelayEnsembleChange extends BookKeeperClusterTestCase {
         LedgerMetadata md = lh.getLedgerMetadata();
 
         for (long eid = startEntry; eid < untilEntry; eid++) {
-            List<BookieSocketAddress> addresses = md.getEnsemble(eid);
+            List<BookieSocketAddress> addresses = md.getEnsembleAt(eid);
             VerificationCallback callback = new VerificationCallback(addresses.size());
             for (BookieSocketAddress addr : addresses) {
                 bkc.getBookieClient().readEntry(addr, lh.getId(), eid,
@@ -257,8 +258,8 @@ public class TestDelayEnsembleChange extends BookKeeperClusterTestCase {
                         CLIENT_SCOPE + "." + WATCHER_SCOPE + "." + REPLACE_BOOKIE_TIME)
                         .getSuccessCount() > 0);
 
-        List<BookieSocketAddress> firstFragment = lh.getLedgerMetadata().getEnsemble(0);
-        List<BookieSocketAddress> secondFragment = lh.getLedgerMetadata().getEnsemble(3 * numEntries);
+        List<BookieSocketAddress> firstFragment = lh.getLedgerMetadata().getEnsembleAt(0);
+        List<BookieSocketAddress> secondFragment = lh.getLedgerMetadata().getEnsembleAt(3 * numEntries);
         assertFalse(firstFragment.get(0).equals(secondFragment.get(0)));
         assertFalse(firstFragment.get(1).equals(secondFragment.get(1)));
         assertFalse(firstFragment.get(2).equals(secondFragment.get(2)));
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java
index cfe9979..aa2dd6b 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java
@@ -266,7 +266,7 @@ public class TestFencing extends BookKeeperClusterTestCase {
             writelh.addEntry(tmp.getBytes());
         }
 
-        BookieSocketAddress bookieToKill = writelh.getLedgerMetadata().getEnsemble(numEntries).get(0);
+        BookieSocketAddress bookieToKill = writelh.getLedgerMetadata().getEnsembleAt(numEntries).get(0);
         killBookie(bookieToKill);
 
         // write entries to change ensemble
@@ -318,7 +318,7 @@ public class TestFencing extends BookKeeperClusterTestCase {
         LedgerHandle readlh = bkc.openLedger(writelh.getId(),
                                              digestType, "testPasswd".getBytes());
         // should be fenced by now
-        BookieSocketAddress bookieToKill = writelh.getLedgerMetadata().getEnsemble(numEntries).get(0);
+        BookieSocketAddress bookieToKill = writelh.getLedgerMetadata().getEnsembleAt(numEntries).get(0);
         killBookie(bookieToKill);
         admin.recoverBookieData(bookieToKill);
 
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestGetBookieInfoTimeout.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestGetBookieInfoTimeout.java
index 6291413..4ec5992 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestGetBookieInfoTimeout.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestGetBookieInfoTimeout.java
@@ -100,7 +100,7 @@ public class TestGetBookieInfoTimeout extends BookKeeperClusterTestCase {
         ClientConfiguration cConf = new ClientConfiguration();
         cConf.setGetBookieInfoTimeout(2);
 
-        final BookieSocketAddress bookieToSleep = writelh.getLedgerMetadata().getEnsemble(0).get(0);
+        final BookieSocketAddress bookieToSleep = writelh.getLedgerMetadata().getEnsembleAt(0).get(0);
         int sleeptime = cConf.getBookieInfoTimeout() * 3;
         CountDownLatch latch = sleepBookie(bookieToSleep, sleeptime);
         latch.await();
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerFragmentReplication.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerFragmentReplication.java
index 4b8d06c..68580ed 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerFragmentReplication.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerFragmentReplication.java
@@ -34,6 +34,7 @@ import java.util.SortedMap;
 import java.util.concurrent.CountDownLatch;
 
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.client.api.WriteFlag;
 import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestParallelRead.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestParallelRead.java
index 68fd29c..84edc62 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestParallelRead.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestParallelRead.java
@@ -156,7 +156,7 @@ public class TestParallelRead extends BookKeeperClusterTestCase {
 
         LedgerHandle lh = bkc.openLedger(id, digestType, passwd);
 
-        List<BookieSocketAddress> ensemble = lh.getLedgerMetadata().getEnsemble(10);
+        List<BookieSocketAddress> ensemble = lh.getLedgerMetadata().getEnsembleAt(10);
         CountDownLatch latch1 = new CountDownLatch(1);
         CountDownLatch latch2 = new CountDownLatch(1);
         // sleep two bookie
@@ -187,7 +187,7 @@ public class TestParallelRead extends BookKeeperClusterTestCase {
 
         LedgerHandle lh = bkc.openLedger(id, digestType, passwd);
 
-        List<BookieSocketAddress> ensemble = lh.getLedgerMetadata().getEnsemble(5);
+        List<BookieSocketAddress> ensemble = lh.getLedgerMetadata().getEnsembleAt(5);
         // kill two bookies
         killBookie(ensemble.get(0));
         killBookie(ensemble.get(1));
@@ -223,7 +223,7 @@ public class TestParallelRead extends BookKeeperClusterTestCase {
 
         LedgerHandle lh = bkc.openLedger(id, digestType, passwd);
 
-        List<BookieSocketAddress> ensemble = lh.getLedgerMetadata().getEnsemble(5);
+        List<BookieSocketAddress> ensemble = lh.getLedgerMetadata().getEnsembleAt(5);
         // kill two bookies
         killBookie(ensemble.get(0));
         killBookie(ensemble.get(1));
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadEntryListener.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadEntryListener.java
index 7f28cc3..2ef72ff 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadEntryListener.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadEntryListener.java
@@ -231,7 +231,7 @@ public class TestReadEntryListener extends BookKeeperClusterTestCase {
         LedgerHandle lh = bkc.openLedger(id, digestType, passwd);
 
         List<BookieSocketAddress> ensemble =
-                lh.getLedgerMetadata().getEnsemble(5);
+                lh.getLedgerMetadata().getEnsembleAt(5);
         // kill two bookies
         killBookie(ensemble.get(0));
         killBookie(ensemble.get(1));
@@ -270,7 +270,7 @@ public class TestReadEntryListener extends BookKeeperClusterTestCase {
         LedgerHandle lh = bkc.openLedger(id, digestType, passwd);
 
         List<BookieSocketAddress> ensemble =
-            lh.getLedgerMetadata().getEnsemble(5);
+            lh.getLedgerMetadata().getEnsembleAt(5);
         // kill bookies
         killBookie(ensemble.get(0));
         killBookie(ensemble.get(1));
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastConfirmedLongPoll.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastConfirmedLongPoll.java
index 48f638c..c4ec8f7 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastConfirmedLongPoll.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastConfirmedLongPoll.java
@@ -156,7 +156,7 @@ public class TestReadLastConfirmedLongPoll extends BookKeeperClusterTestCase {
             ServerConfiguration[] confs = new ServerConfiguration[numEntries - 1];
             for (int j = 0; j < numEntries - 1; j++) {
                 int idx = (i + 1 + j) % numEntries;
-                confs[j] = killBookie(lh.getLedgerMetadata().getLastEnsembleValue().get(idx));
+                confs[j] = killBookie(LedgerMetadataUtils.getLastEnsembleValue(lh.getLedgerMetadata()).get(idx));
             }
 
             final AtomicBoolean entryAsExpected = new AtomicBoolean(false);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadTimeout.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadTimeout.java
index d244218..1e2f17e 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadTimeout.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadTimeout.java
@@ -62,9 +62,9 @@ public class TestReadTimeout extends BookKeeperClusterTestCase {
         }
 
         Set<BookieSocketAddress> beforeSet = new HashSet<BookieSocketAddress>();
-        beforeSet.addAll(writelh.getLedgerMetadata().getEnsemble(numEntries));
+        beforeSet.addAll(writelh.getLedgerMetadata().getEnsembleAt(numEntries));
 
-        final BookieSocketAddress bookieToSleep = writelh.getLedgerMetadata().getEnsemble(numEntries).get(0);
+        final BookieSocketAddress bookieToSleep = writelh.getLedgerMetadata().getEnsembleAt(numEntries).get(0);
         int sleeptime = baseClientConf.getReadTimeout() * 3;
         CountDownLatch latch = sleepBookie(bookieToSleep, sleeptime);
         latch.await();
@@ -80,7 +80,7 @@ public class TestReadTimeout extends BookKeeperClusterTestCase {
         Assert.assertTrue("Write request did not finish", completed.get());
 
         Set<BookieSocketAddress> afterSet = new HashSet<BookieSocketAddress>();
-        afterSet.addAll(writelh.getLedgerMetadata().getEnsemble(numEntries + 1));
+        afterSet.addAll(writelh.getLedgerMetadata().getEnsembleAt(numEntries + 1));
         beforeSet.removeAll(afterSet);
         Assert.assertTrue("Bookie set should not match", beforeSet.size() != 0);
 
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java
index 2590666..5c6a8c7 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java
@@ -34,6 +34,7 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import lombok.Cleanup;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.meta.HierarchicalLedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerIdGenerator;
 import org.apache.bookkeeper.meta.LedgerManager;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java
index fa18497..4e8d892 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java
@@ -99,7 +99,7 @@ public class UpdateLedgerCmdTest extends BookKeeperClusterTestCase {
         for (LedgerHandle lh : ledgers) {
             lh.close();
             LedgerHandle openLedger = bk.openLedger(lh.getId(), digestType, PASSWORD.getBytes());
-            ensemble = openLedger.getLedgerMetadata().getEnsemble(0);
+            ensemble = openLedger.getLedgerMetadata().getEnsembleAt(0);
             if (ensemble.contains(toBookieAddr)) {
                 updatedLedgersCount++;
             }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java
index ad5c450..c2b0d75 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java
@@ -35,6 +35,7 @@ import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieShell.UpdateLedgerNotifier;
 import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.proto.BookieServer;
@@ -98,7 +99,7 @@ public class UpdateLedgerOpTest extends BookKeeperClusterTestCase {
                 ledgers.add(createLedgerWithEntries(bk, 0));
             }
 
-            List<BookieSocketAddress> ensemble = lh1.getLedgerMetadata().getEnsemble(0);
+            List<BookieSocketAddress> ensemble = lh1.getLedgerMetadata().getEnsembleAt(0);
 
             BookieSocketAddress curBookieAddr = ensemble.get(0);
             baseConf.setUseHostNameAsBookieID(true);
@@ -112,7 +113,7 @@ public class UpdateLedgerOpTest extends BookKeeperClusterTestCase {
             for (LedgerHandle lh : ledgers) {
                 lh.close();
                 LedgerHandle openLedger = bk.openLedger(lh.getId(), digestType, PASSWORD.getBytes());
-                ensemble = openLedger.getLedgerMetadata().getEnsemble(0);
+                ensemble = openLedger.getLedgerMetadata().getEnsembleAt(0);
                 assertTrue("Failed to update the ledger metadata to use bookie host name",
                         ensemble.contains(toBookieAddr));
                 assertFalse("Failed to update the ledger metadata to use bookie host name",
@@ -137,7 +138,7 @@ public class UpdateLedgerOpTest extends BookKeeperClusterTestCase {
                 ledgers.add(createLedgerWithEntries(bk, 0));
             }
 
-            List<BookieSocketAddress> ensemble = lh1.getLedgerMetadata().getEnsemble(0);
+            List<BookieSocketAddress> ensemble = lh1.getLedgerMetadata().getEnsembleAt(0);
 
             BookieSocketAddress curBookieAddr = ensemble.get(0);
             baseConf.setUseHostNameAsBookieID(true);
@@ -193,7 +194,7 @@ public class UpdateLedgerOpTest extends BookKeeperClusterTestCase {
             LedgerHandle lh = createLedgerWithEntries(bk, 100);
 
             BookieServer bookieServer = bs.get(0);
-            List<BookieSocketAddress> ensemble = lh.getLedgerMetadata().getEnsemble(0);
+            List<BookieSocketAddress> ensemble = lh.getLedgerMetadata().getEnsembleAt(0);
             BookieSocketAddress curBookieAddr = null;
             for (BookieSocketAddress bookieSocketAddress : ensemble) {
                 if (bookieServer.getLocalAddress().equals(bookieSocketAddress)) {
@@ -234,7 +235,7 @@ public class UpdateLedgerOpTest extends BookKeeperClusterTestCase {
             LedgerHandle openLedger = bk.openLedger(lh.getId(), digestType, PASSWORD.getBytes());
             final LedgerMetadata ledgerMetadata = openLedger.getLedgerMetadata();
             assertEquals("Failed to reform ensemble!", 2, ledgerMetadata.getAllEnsembles().size());
-            ensemble = ledgerMetadata.getEnsemble(0);
+            ensemble = ledgerMetadata.getEnsembleAt(0);
             assertTrue("Failed to update the ledger metadata to use bookie host name",
                     ensemble.contains(toBookieAddr));
         }
@@ -273,7 +274,7 @@ public class UpdateLedgerOpTest extends BookKeeperClusterTestCase {
                 }
             };
             th.start();
-            List<BookieSocketAddress> ensemble = lh.getLedgerMetadata().getEnsemble(0);
+            List<BookieSocketAddress> ensemble = lh.getLedgerMetadata().getEnsembleAt(0);
             BookieSocketAddress curBookieAddr = ensemble.get(0);
             BookieSocketAddress toBookieAddr = new BookieSocketAddress("localhost:" + curBookieAddr.getPort());
             UpdateLedgerOp updateLedgerOp = new UpdateLedgerOp(bk, bkadmin);
@@ -287,7 +288,7 @@ public class UpdateLedgerOpTest extends BookKeeperClusterTestCase {
             }
             lh.close();
             LedgerHandle openLedger = bk.openLedger(lh.getId(), digestType, PASSWORD.getBytes());
-            ensemble = openLedger.getLedgerMetadata().getEnsemble(0);
+            ensemble = openLedger.getLedgerMetadata().getEnsembleAt(0);
             assertTrue("Failed to update the ledger metadata to use bookie host name",
                     ensemble.contains(toBookieAddr));
         }
@@ -300,7 +301,7 @@ public class UpdateLedgerOpTest extends BookKeeperClusterTestCase {
         for (LedgerHandle lh : ledgers) {
             lh.close();
             LedgerHandle openLedger = bk.openLedger(lh.getId(), digestType, PASSWORD.getBytes());
-            ensemble = openLedger.getLedgerMetadata().getEnsemble(0);
+            ensemble = openLedger.getLedgerMetadata().getEnsembleAt(0);
             if (ensemble.contains(toBookieAddr)) {
                 updatedLedgersCount++;
             }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java
index 0316b7f..85cb727 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/BookKeeperBuildersTest.java
@@ -33,7 +33,6 @@ import org.apache.bookkeeper.client.BKException.BKIncorrectParameterException;
 import org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsException;
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.client.LedgerMetadata;
 import org.apache.bookkeeper.client.LedgerMetadataBuilder;
 import org.apache.bookkeeper.client.MockBookKeeperTestCase;
 import org.apache.bookkeeper.conf.ClientConfiguration;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java
index 6e37fee..37f55b3 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/AbstractZkLedgerManagerTest.java
@@ -50,8 +50,8 @@ import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BKException.Code;
-import org.apache.bookkeeper.client.LedgerMetadata;
 import org.apache.bookkeeper.client.LedgerMetadataBuilder;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.common.testing.executors.MockExecutorController;
 import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java
index 182ffa1..d273565 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java
@@ -63,8 +63,8 @@ import org.apache.bookkeeper.bookie.LedgerDirsManager;
 import org.apache.bookkeeper.bookie.ScanAndCompareGarbageCollector;
 import org.apache.bookkeeper.bookie.StateManager;
 import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.LedgerMetadata;
 import org.apache.bookkeeper.client.LedgerMetadataBuilder;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.common.util.Watcher;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.meta.LedgerManager.LedgerRange;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java
index 03e1358..bdb2902 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java
@@ -46,8 +46,8 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BookKeeper;
-import org.apache.bookkeeper.client.LedgerMetadata;
 import org.apache.bookkeeper.client.LedgerMetadataBuilder;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.meta.LedgerManager.LedgerRangeIterator;
 import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
 import org.apache.bookkeeper.net.BookieSocketAddress;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java
index ff0126a..398bb07 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java
@@ -29,7 +29,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 
 import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.LedgerMetadata;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java
index ef57d6b..f25aa88 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java
@@ -51,8 +51,8 @@ import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
 import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.client.LedgerMetadata;
 import org.apache.bookkeeper.client.LedgerMetadataBuilder;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.common.util.OrderedScheduler;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.meta.LedgerManager;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java
index 23b3378..fd97da3 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java
@@ -55,7 +55,6 @@ import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
 import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.client.LedgerHandleAdapter;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerUnderreplicationManager;
@@ -480,9 +479,8 @@ public class AuditorPeriodicCheckTest extends BookKeeperClusterTestCase {
 
     private BookieSocketAddress replaceBookieWithWriteFailingBookie(LedgerHandle lh) throws Exception {
         int bookieIdx = -1;
-        Long entryId = LedgerHandleAdapter.getLedgerMetadata(lh).getAllEnsembles().firstKey();
-        List<BookieSocketAddress> curEnsemble = LedgerHandleAdapter
-                .getLedgerMetadata(lh).getAllEnsembles().get(entryId);
+        Long entryId = lh.getLedgerMetadata().getAllEnsembles().firstKey();
+        List<BookieSocketAddress> curEnsemble = lh.getLedgerMetadata().getAllEnsembles().get(entryId);
 
         // Identify a bookie in the current ledger ensemble to be replaced
         BookieSocketAddress replacedBookie = null;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java
index 07b9375..45cd2eb 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java
@@ -36,7 +36,6 @@ import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
 import org.apache.bookkeeper.client.BookKeeperTestClient;
 import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.client.LedgerHandleAdapter;
 import org.apache.bookkeeper.common.util.OrderedScheduler;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.meta.LedgerManager;
@@ -150,8 +149,7 @@ public class BookieAutoRecoveryTest extends BookKeeperClusterTestCase {
         List<LedgerHandle> listOfLedgerHandle = createLedgersAndAddEntries(1, 5);
         LedgerHandle lh = listOfLedgerHandle.get(0);
         int ledgerReplicaIndex = 0;
-        BookieSocketAddress replicaToKillAddr = LedgerHandleAdapter
-                .getLedgerMetadata(lh).getAllEnsembles().get(0L).get(0);
+        BookieSocketAddress replicaToKillAddr = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0);
 
         final String urLedgerZNode = getUrLedgerZNode(lh);
         ledgerReplicaIndex = getReplicaIndexInLedger(lh, replicaToKillAddr);
@@ -199,8 +197,7 @@ public class BookieAutoRecoveryTest extends BookKeeperClusterTestCase {
         closeLedgers(listOfLedgerHandle);
         LedgerHandle lhandle = listOfLedgerHandle.get(0);
         int ledgerReplicaIndex = 0;
-        BookieSocketAddress replicaToKillAddr = LedgerHandleAdapter
-                .getLedgerMetadata(lhandle).getAllEnsembles().get(0L).get(0);
+        BookieSocketAddress replicaToKillAddr = lhandle.getLedgerMetadata().getAllEnsembles().get(0L).get(0);
 
         CountDownLatch latch = new CountDownLatch(listOfLedgerHandle.size());
         for (LedgerHandle lh : listOfLedgerHandle) {
@@ -261,8 +258,7 @@ public class BookieAutoRecoveryTest extends BookKeeperClusterTestCase {
                 numberOfLedgers, 5);
         closeLedgers(listOfLedgerHandle);
         LedgerHandle handle = listOfLedgerHandle.get(0);
-        BookieSocketAddress replicaToKillAddr = LedgerHandleAdapter
-                .getLedgerMetadata(handle).getAllEnsembles().get(0L).get(0);
+        BookieSocketAddress replicaToKillAddr = handle.getLedgerMetadata().getAllEnsembles().get(0L).get(0);
         LOG.info("Killing Bookie:" + replicaToKillAddr);
 
         // Each ledger, there will be two events : create urLedger and after
@@ -340,13 +336,13 @@ public class BookieAutoRecoveryTest extends BookKeeperClusterTestCase {
             assertNull("UrLedger already exists!",
                     watchUrLedgerNode(getUrLedgerZNode(lh), latch));
         }
-        BookieSocketAddress replicaToKillAddr = LedgerHandleAdapter
-                .getLedgerMetadata(listOfLedgerHandle.get(0)).getAllEnsembles()
-                .get(0L).get(0);
+        BookieSocketAddress replicaToKillAddr = listOfLedgerHandle.get(0)
+            .getLedgerMetadata().getAllEnsembles()
+            .get(0L).get(0);
         killBookie(replicaToKillAddr);
-        replicaToKillAddr = LedgerHandleAdapter
-                .getLedgerMetadata(listOfLedgerHandle.get(0)).getAllEnsembles()
-                .get(0L).get(0);
+        replicaToKillAddr = listOfLedgerHandle.get(0)
+            .getLedgerMetadata().getAllEnsembles()
+            .get(0L).get(0);
         killBookie(replicaToKillAddr);
         // waiting to publish urLedger znode by Auditor
         latch.await();
@@ -383,10 +379,9 @@ public class BookieAutoRecoveryTest extends BookKeeperClusterTestCase {
         String urZNode = getUrLedgerZNode(lh);
         watchUrLedgerNode(urZNode, latch);
 
-        BookieSocketAddress replicaToKill = LedgerHandleAdapter
-            .getLedgerMetadata(lh).getAllEnsembles().get(0L).get(2);
+        BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(2);
         LOG.info("Killing last bookie, {}, in ensemble {}", replicaToKill,
-                 LedgerHandleAdapter.getLedgerMetadata(lh).getAllEnsembles().get(0L));
+                 lh.getLedgerMetadata().getAllEnsembles().get(0L));
         killBookie(replicaToKill);
 
         getAuditor(10, TimeUnit.SECONDS).submitAuditTask().get(); // ensure auditor runs
@@ -398,10 +393,9 @@ public class BookieAutoRecoveryTest extends BookKeeperClusterTestCase {
             assertTrue("Should be marked as replicated", latch.await(10, TimeUnit.SECONDS));
         }
 
-        replicaToKill = LedgerHandleAdapter
-            .getLedgerMetadata(lh).getAllEnsembles().get(0L).get(1);
+        replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(1);
         LOG.info("Killing second bookie, {}, in ensemble {}", replicaToKill,
-                 LedgerHandleAdapter.getLedgerMetadata(lh).getAllEnsembles().get(0L));
+                 lh.getLedgerMetadata().getAllEnsembles().get(0L));
         killBookie(replicaToKill);
 
         getAuditor(10, TimeUnit.SECONDS).submitAuditTask().get(); // ensure auditor runs
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestAutoRecoveryAlongWithBookieServers.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestAutoRecoveryAlongWithBookieServers.java
index 8131999..0e6f927 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestAutoRecoveryAlongWithBookieServers.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestAutoRecoveryAlongWithBookieServers.java
@@ -29,7 +29,6 @@ import java.util.Map.Entry;
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.LedgerEntry;
 import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.client.LedgerHandleAdapter;
 import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
 import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.test.BookKeeperClusterTestCase;
@@ -74,8 +73,7 @@ public class TestAutoRecoveryAlongWithBookieServers extends
             lh.addEntry(testData);
         }
         lh.close();
-        BookieSocketAddress replicaToKill = LedgerHandleAdapter
-                .getLedgerMetadata(lh).getAllEnsembles().get(0L).get(0);
+        BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0);
 
         killBookie(replicaToKill);
 
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java
index 5d92979..306a7f6 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java
@@ -48,7 +48,7 @@ import org.apache.bookkeeper.client.BookKeeperClientStats;
 import org.apache.bookkeeper.client.BookKeeperTestClient;
 import org.apache.bookkeeper.client.LedgerEntry;
 import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.client.LedgerMetadata;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.net.BookieSocketAddress;
diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java
index 5155fad..d571bf8 100644
--- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java
+++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManager.java
@@ -44,7 +44,7 @@ import java.util.function.Consumer;
 import java.util.function.Function;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.LedgerMetadata;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.meta.LedgerManager;
 import org.apache.bookkeeper.meta.LedgerMetadataSerDe;
 import org.apache.bookkeeper.metadata.etcd.helpers.KeyIterator;
diff --git a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/LedgerMetadataConsumer.java b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/LedgerMetadataConsumer.java
index d466b34..292d973 100644
--- a/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/LedgerMetadataConsumer.java
+++ b/metadata-drivers/etcd/src/main/java/org/apache/bookkeeper/metadata/etcd/LedgerMetadataConsumer.java
@@ -16,7 +16,7 @@ package org.apache.bookkeeper.metadata.etcd;
 
 import java.util.Objects;
 import java.util.function.Consumer;
-import org.apache.bookkeeper.client.LedgerMetadata;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener;
 import org.apache.bookkeeper.versioning.Versioned;
 
diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java
index a7df94f..984224c 100644
--- a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java
+++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/EtcdLedgerManagerTest.java
@@ -50,8 +50,8 @@ import lombok.extern.slf4j.Slf4j;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BKException.Code;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
-import org.apache.bookkeeper.client.LedgerMetadata;
 import org.apache.bookkeeper.client.LedgerMetadataBuilder;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.meta.LedgerManager.LedgerRange;
 import org.apache.bookkeeper.meta.LedgerManager.LedgerRangeIterator;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/BookKeeperAccessor.java b/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/BookKeeperAccessor.java
index 3152a74..2ca0268 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/BookKeeperAccessor.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/BookKeeperAccessor.java
@@ -37,8 +37,4 @@ public class BookKeeperAccessor {
                       "Recovery can only run on ReadOnlyLedgerHandle");
         ((ReadOnlyLedgerHandle) lh).recover(cb, null, true);
     }
-
-    public static LedgerMetadata getLedgerMetadata(LedgerHandle lh) {
-        return lh.getLedgerMetadata();
-    }
 }
diff --git a/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java b/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java
index c06bf84..b7a6ba9 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java
@@ -119,7 +119,7 @@ public class LedgerReader {
             }
         };
 
-        List<BookieSocketAddress> ensemble = lh.getLedgerMetadata().getEnsemble(eid);
+        List<BookieSocketAddress> ensemble = lh.getLedgerMetadata().getEnsembleAt(eid);
         for (int i = 0; i < writeSet.size(); i++) {
             int idx = writeSet.get(i);
             clientCtx.getBookieClient().readEntry(ensemble.get(idx), lh.getId(), eid, readEntryCallback,
@@ -224,7 +224,7 @@ public class LedgerReader {
             }
         };
 
-        List<BookieSocketAddress> ensemble = lh.getLedgerMetadata().getEnsemble(eid);
+        List<BookieSocketAddress> ensemble = lh.getLedgerMetadata().getEnsembleAt(eid);
         for (int i = 0; i < writeSet.size(); i++) {
             int idx = writeSet.get(i);
             clientCtx.getBookieClient().readEntry(ensemble.get(idx), lh.getId(), eid, readEntryCallback,
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java
index 75e22ac..b04c27b 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java
@@ -42,9 +42,8 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.bookkeeper.client.BookKeeper;
-import org.apache.bookkeeper.client.BookKeeperAccessor;
 import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.client.LedgerMetadata;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.common.concurrent.FutureEventListener;
 import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.feature.FixedValueFeature;
@@ -1992,7 +1991,7 @@ public class TestAsyncReaderWriter extends TestDistributedLogBase {
         long ledgerId = segments.get(0).getLogSegmentId();
         LedgerHandle lh = ((BKNamespaceDriver) namespace.getNamespaceDriver()).getReaderBKC().get()
                 .openLedgerNoRecovery(ledgerId, BookKeeper.DigestType.CRC32, confLocal.getBKDigestPW().getBytes(UTF_8));
-        LedgerMetadata metadata = BookKeeperAccessor.getLedgerMetadata(lh);
+        LedgerMetadata metadata = lh.getLedgerMetadata();
         assertEquals(DistributedLogConfiguration.BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT, metadata.getEnsembleSize());
         lh.close();
         Utils.close(writer);
@@ -2011,7 +2010,7 @@ public class TestAsyncReaderWriter extends TestDistributedLogBase {
         ledgerId = segments.get(0).getLogSegmentId();
         lh = ((BKNamespaceDriver) namespace.getNamespaceDriver()).getReaderBKC().get()
                 .openLedgerNoRecovery(ledgerId, BookKeeper.DigestType.CRC32, confLocal.getBKDigestPW().getBytes(UTF_8));
-        metadata = BookKeeperAccessor.getLedgerMetadata(lh);
+        metadata = lh.getLedgerMetadata();
         assertEquals(DistributedLogConfiguration.BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT - 1, metadata.getEnsembleSize());
         lh.close();
         Utils.close(writer);
diff --git a/tests/backward-compat/recovery-no-password/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatRecoveryNoPassword.groovy b/tests/backward-compat/recovery-no-password/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatRecoveryNoPassword.groovy
index 7f229e7..b7987a8 100644
--- a/tests/backward-compat/recovery-no-password/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatRecoveryNoPassword.groovy
+++ b/tests/backward-compat/recovery-no-password/src/test/groovy/org/apache/bookkeeper/tests/backwardcompat/TestCompatRecoveryNoPassword.groovy
@@ -30,7 +30,7 @@ import org.apache.bookkeeper.client.BKException
 import org.apache.bookkeeper.client.BookKeeper
 import org.apache.bookkeeper.client.BookKeeperAdmin
 import org.apache.bookkeeper.client.LedgerHandle
-import org.apache.bookkeeper.client.LedgerMetadata
+import org.apache.bookkeeper.client.api.LedgerMetadata
 import org.apache.bookkeeper.conf.ClientConfiguration
 import org.apache.bookkeeper.net.BookieSocketAddress
 import org.apache.bookkeeper.proto.BookieProtocol