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/11/23 00:04:48 UTC
[bookkeeper] branch master updated: Remove LedgerMetadata methods
modifying ensembles
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 32ae783 Remove LedgerMetadata methods modifying ensembles
32ae783 is described below
commit 32ae78311c1a284551e2a6b6941d11d05fec097e
Author: Ivan Kelly <iv...@apache.org>
AuthorDate: Fri Nov 23 00:04:44 2018 +0000
Remove LedgerMetadata methods modifying ensembles
Remove any methods in LedgerMetadata that modify the ensemble. With
this change the ensembles are 100% immutable.
Master issue: #281
Reviewers: Enrico Olivelli <eo...@gmail.com>, Sijie Guo <si...@apache.org>
This closes #1830 from ivankelly/no-mod-ensemble
---
.../apache/bookkeeper/client/LedgerMetadata.java | 68 +++++++---------------
1 file changed, 22 insertions(+), 46 deletions(-)
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java
index 7b40e73..bf5e9f5 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java
@@ -32,6 +32,7 @@ import java.io.StringReader;
import java.nio.CharBuffer;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@@ -76,14 +77,15 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe
private final int ensembleSize;
private final int writeQuorumSize;
private final int ackQuorumSize;
+
+ private LedgerMetadataFormat.State state;
private long length;
private long lastEntryId;
private final long ctime;
final boolean storeCtime; // non-private so builder can access for copy
- private LedgerMetadataFormat.State state;
- private TreeMap<Long, ImmutableList<BookieSocketAddress>> ensembles = new TreeMap<>();
- private List<BookieSocketAddress> currentEnsemble;
+ private final NavigableMap<Long, ImmutableList<BookieSocketAddress>> ensembles;
+ private final ImmutableList<BookieSocketAddress> currentEnsemble;
private final boolean hasPassword; // IKTODO other things should be optionals instead
private final LedgerMetadataFormat.DigestType digestType;
@@ -117,10 +119,17 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe
this.lastEntryId = LedgerHandle.INVALID_ENTRY_ID;
}
length.ifPresent((l) -> this.length = l);
- setEnsembles(ensembles);
+
+ this.ensembles = Collections.unmodifiableNavigableMap(
+ ensembles.entrySet().stream().collect(TreeMap::new,
+ (m, e) -> m.put(e.getKey(),
+ ImmutableList.copyOf(e.getValue())),
+ TreeMap::putAll));
if (state != LedgerMetadataFormat.State.CLOSED) {
currentEnsemble = this.ensembles.lastEntry().getValue();
+ } else {
+ currentEnsemble = null;
}
this.digestType = digestType.equals(DigestType.MAC)
@@ -156,9 +165,15 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe
this.storeCtime = other.storeCtime;
this.password = new byte[other.password.length];
System.arraycopy(other.password, 0, this.password, 0, other.password.length);
- // copy the ensembles
- for (Entry<Long, ? extends List<BookieSocketAddress>> entry : other.ensembles.entrySet()) {
- this.addEnsemble(entry.getKey(), entry.getValue());
+ this.ensembles = Collections.unmodifiableNavigableMap(
+ other.ensembles.entrySet().stream().collect(TreeMap::new,
+ (m, e) -> m.put(e.getKey(),
+ ImmutableList.copyOf(e.getValue())),
+ TreeMap::putAll));
+ if (state != LedgerMetadataFormat.State.CLOSED) {
+ currentEnsemble = this.ensembles.lastEntry().getValue();
+ } else {
+ currentEnsemble = null;
}
this.customMetadata = other.customMetadata;
}
@@ -168,13 +183,6 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe
return ensembles;
}
- void setEnsembles(Map<Long, ? extends List<BookieSocketAddress>> newEnsembles) {
- this.ensembles = newEnsembles.entrySet().stream()
- .collect(TreeMap::new,
- (m, e) -> m.put(e.getKey(), ImmutableList.copyOf(e.getValue())),
- TreeMap::putAll);
- }
-
@Override
public int getEnsembleSize() {
return ensembleSize;
@@ -270,22 +278,10 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe
state = LedgerMetadataFormat.State.CLOSED;
}
- public void addEnsemble(long startEntryId, List<BookieSocketAddress> ensemble) {
- checkArgument(ensembles.isEmpty() || startEntryId >= ensembles.lastKey());
-
- ensembles.put(startEntryId, ImmutableList.copyOf(ensemble));
- currentEnsemble = ensemble;
- }
-
List<BookieSocketAddress> getCurrentEnsemble() {
return currentEnsemble;
}
- public void updateEnsemble(long startEntryId, List<BookieSocketAddress> ensemble) {
- checkArgument(ensembles.containsKey(startEntryId));
- ensembles.put(startEntryId, ImmutableList.copyOf(ensemble));
- }
-
List<BookieSocketAddress> getEnsemble(long entryId) {
// the head map cannot be empty, since we insert an ensemble for
// entry-id 0, right when we start
@@ -575,26 +571,6 @@ public class LedgerMetadata implements org.apache.bookkeeper.client.api.LedgerMe
return sb.toString();
}
- void mergeEnsembles(SortedMap<Long, ? extends List<BookieSocketAddress>> newEnsembles) {
- // allow new metadata to be one ensemble less than current metadata
- // since ensemble change might kick in when recovery changed metadata
- int diff = ensembles.size() - newEnsembles.size();
- if (0 != diff && 1 != diff) {
- return;
- }
- int i = 0;
- for (Entry<Long, ? extends List<BookieSocketAddress>> entry : newEnsembles.entrySet()) {
- ++i;
- if (ensembles.size() != i) {
- // we should use last ensemble from current metadata
- // not the new metadata read from zookeeper
- long key = entry.getKey();
- List<BookieSocketAddress> ensemble = entry.getValue();
- ensembles.put(key, ImmutableList.copyOf(ensemble));
- }
- }
- }
-
Set<BookieSocketAddress> getBookiesInThisLedger() {
Set<BookieSocketAddress> bookies = new HashSet<BookieSocketAddress>();
for (List<BookieSocketAddress> ensemble : ensembles.values()) {