Skip to content

Commit

Permalink
Kill LedgerMetadata#isConflictWith
Browse files Browse the repository at this point in the history
It doesn't make sense anymore, as local copies of the metadata are
never modified, only updated with the latest version from the metadata
store.

In effect, this logic has been broken out to the places where we try
to update the metadata store copy. Each time we try to update the
metadata store, we ensure that the update we are applying still makes
sense with regard to the copy of the metadata we are updating.

Master issue: #281


Reviewers: Sijie Guo <sijie@apache.org>, Enrico Olivelli <eolivelli@gmail.com>

This closes #1760 from ivankelly/kill-is-conflict-with
  • Loading branch information
ivankelly authored and sijie committed Oct 28, 2018
1 parent 86fd295 commit fcd63f9
Show file tree
Hide file tree
Showing 4 changed files with 0 additions and 129 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,6 @@
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
Expand Down Expand Up @@ -672,66 +671,6 @@ public static boolean areByteArrayValMapsEqual(Map<String, byte[]> first, Map<St
return true;
}

/**
* Is the metadata conflict with new updated metadata.
*
* @param newMeta
* Re-read metadata
* @return true if the metadata is conflict.
*/
boolean isConflictWith(LedgerMetadata newMeta) {
/*
* if length & close have changed, then another client has
* opened the ledger, can't resolve this conflict.
*/

if (metadataFormatVersion != newMeta.metadataFormatVersion
|| ensembleSize != newMeta.ensembleSize
|| writeQuorumSize != newMeta.writeQuorumSize
|| ackQuorumSize != newMeta.ackQuorumSize
|| length != newMeta.length
|| state != newMeta.state
|| !digestType.equals(newMeta.digestType)
|| !Arrays.equals(password, newMeta.password)
|| !LedgerMetadata.areByteArrayValMapsEqual(customMetadata, newMeta.customMetadata)) {
return true;
}

// verify the ctime
if (storeSystemtimeAsLedgerCreationTime != newMeta.storeSystemtimeAsLedgerCreationTime) {
return true;
} else if (storeSystemtimeAsLedgerCreationTime) {
return ctime != newMeta.ctime;
}

if (state == LedgerMetadataFormat.State.CLOSED
&& lastEntryId != newMeta.lastEntryId) {
return true;
}
// if ledger is closed, we can just take the new ensembles
if (newMeta.state != LedgerMetadataFormat.State.CLOSED) {
// 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() - newMeta.ensembles.size();
if (0 != diff && 1 != diff) {
return true;
}
// ensemble distribution should be same
// we don't check the detail ensemble, since new bookie will be set
// using recovery tool.
Iterator<Long> keyIter = ensembles.keySet().iterator();
Iterator<Long> newMetaKeyIter = newMeta.ensembles.keySet().iterator();
for (int i = 0; i < newMeta.ensembles.size(); i++) {
Long curKey = keyIter.next();
Long newMetaKey = newMetaKeyIter.next();
if (!curKey.equals(newMetaKey)) {
return true;
}
}
}
return false;
}

@Override
public String toString() {
return toStringRepresentation(true);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -99,62 +99,6 @@ public void testStoreSystemtimeAsLedgerCtimeDisabled()
assertFalse(format.hasCtime());
}

@Test
public void testIsConflictWithStoreSystemtimeAsLedgerCtimeDisabled() {
LedgerMetadata lm1 = new LedgerMetadata(
3,
3,
2,
DigestType.CRC32,
passwd,
Collections.emptyMap(),
false);
LedgerMetadata lm2 = new LedgerMetadata(lm1);

lm1.setCtime(1L);
lm2.setCtime(2L);
assertFalse(lm1.isConflictWith(lm2));
}

@Test
public void testIsConflictWithStoreSystemtimeAsLedgerCtimeEnabled() {
LedgerMetadata lm1 = new LedgerMetadata(
3,
3,
2,
DigestType.CRC32,
passwd,
Collections.emptyMap(),
true);
LedgerMetadata lm2 = new LedgerMetadata(lm1);

lm1.setCtime(1L);
lm2.setCtime(2L);
assertTrue(lm1.isConflictWith(lm2));
}

@Test
public void testIsConflictWithDifferentStoreSystemtimeAsLedgerCtimeFlags() {
LedgerMetadata lm1 = new LedgerMetadata(
3,
3,
2,
DigestType.CRC32,
passwd,
Collections.emptyMap(),
true);
LedgerMetadata lm2 = new LedgerMetadata(
3,
3,
2,
DigestType.CRC32,
passwd,
Collections.emptyMap(),
false);

assertTrue(lm1.isConflictWith(lm2));
}

@Test
public void testToString() {
LedgerMetadata lm1 = new LedgerMetadata(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -97,9 +97,6 @@ private int getUpdatedLedgersCount(BookKeeper bk, List<LedgerHandle> ledgers, Bo
List<BookieSocketAddress> ensemble;
int updatedLedgersCount = 0;
for (LedgerHandle lh : ledgers) {
// ledger#close() would hit BadVersion exception as rename
// increments cversion. But LedgerMetadata#isConflictWith()
// gracefully handles this conflicts.
lh.close();
LedgerHandle openLedger = bk.openLedger(lh.getId(), digestType, PASSWORD.getBytes());
ensemble = openLedger.getLedgerMetadata().getEnsemble(0);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -110,9 +110,6 @@ public void testManyLedgers(boolean useShortHostName) throws Exception {
updateLedgerOp.updateBookieIdInLedgers(curBookieAddr, toBookieAddr, 5, Integer.MIN_VALUE, progressable);

for (LedgerHandle lh : ledgers) {
// ledger#close() would hit BadVersion exception as rename
// increments cversion. But LedgerMetadata#isConflictWith()
// gracefully handles this conflicts.
lh.close();
LedgerHandle openLedger = bk.openLedger(lh.getId(), digestType, PASSWORD.getBytes());
ensemble = openLedger.getLedgerMetadata().getEnsemble(0);
Expand Down Expand Up @@ -218,9 +215,6 @@ public void testChangeEnsembleAfterRenaming(boolean useShortHostName) throws Exc
bsConfs.add(serverConf1);
bs.add(startBookie(serverConf1));

// ledger#asyncAddEntry() would hit BadVersion exception as rename incr
// cversion. But LedgerMetadata#isConflictWith() gracefully handles
// this conflicts.
final CountDownLatch latch = new CountDownLatch(1);
final AtomicInteger rc = new AtomicInteger(BKException.Code.OK);
lh.asyncAddEntry("foobar".getBytes(), new AddCallback() {
Expand Down Expand Up @@ -304,9 +298,6 @@ private int getUpdatedLedgersCount(BookKeeper bk, List<LedgerHandle> ledgers, Bo
List<BookieSocketAddress> ensemble;
int updatedLedgersCount = 0;
for (LedgerHandle lh : ledgers) {
// ledger#close() would hit BadVersion exception as rename
// increments cversion. But LedgerMetadata#isConflictWith()
// gracefully handles this conflicts.
lh.close();
LedgerHandle openLedger = bk.openLedger(lh.getId(), digestType, PASSWORD.getBytes());
ensemble = openLedger.getLedgerMetadata().getEnsemble(0);
Expand Down

0 comments on commit fcd63f9

Please sign in to comment.