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

ivank pushed a commit to branch branch-4.8
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


The following commit(s) were added to refs/heads/branch-4.8 by this push:
     new 2c421e9  Delayed write ensemble change may cause dataloss
2c421e9 is described below

commit 2c421e9ed6cee7f5259f2a7ea433466a5424edf6
Author: JV Jujjuri <[email protected]>
AuthorDate: Mon Aug 13 15:00:02 2018 +0200

    Delayed write ensemble change may cause dataloss
    
    Descriptions of the changes in this PR:
    
    The Original intent of this change is to do a best-effort ensemble change.
    But this is not possible until the local metadata is completely immutable.
    Until the feature "Make LedgerMetadata Immutable #610" Is complete we will 
use
    handleBookieFailure() to handle delayed writes as regular bookie failures.
    
    Signed-off-by: Venkateswararao Jujjuri (JV) <vjujjurisalesforce.com>
    
    Master Issue: #1591
    Relate Issue: #1395
    
    Author: JV Jujjuri <[email protected]>
    Author: Ivan Kelly <[email protected]>
    
    Reviewers: Ivan Kelly <[email protected]>, Sijie Guo <[email protected]>
    
    This closes #1592 from jvrao/datalossbug
    
    (cherry picked from commit 3ab6e9253555d013a8c6eb3b90cddffa090fcaee)
    Signed-off-by: Ivan Kelly <[email protected]>
---
 .../org/apache/bookkeeper/client/LedgerHandle.java | 78 +++++-----------------
 1 file changed, 17 insertions(+), 61 deletions(-)

diff --git 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
index 611b182..a646651 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
@@ -1867,48 +1867,15 @@ public class LedgerHandle implements WriteHandle {
     }
 
     void handleDelayedWriteBookieFailure() {
-        int curBlockAddCompletions = blockAddCompletions.get();
-        if (bk.getDisableEnsembleChangeFeature().isAvailable()) {
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("Ensemble change is disabled. Failed bookies {} for 
ledger {}.",
-                        delayedWriteFailedBookies, ledgerId);
-            }
-            return;
-        }
-        int curNumEnsembleChanges = numEnsembleChanges.incrementAndGet();
-        if (curNumEnsembleChanges > maxAllowedEnsembleChanges) {
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("Exceeding maxAllowedEnsembeChanges {}. Failed 
bookies {} for ledger {}.",
-                        maxAllowedEnsembleChanges, delayedWriteFailedBookies, 
ledgerId);
-            }
-            return;
-        }
-        if (writeFlags.contains(WriteFlag.DEFERRED_SYNC)) {
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("Cannot perform ensemble change with writeflags {}."
-                        + "Failed bookies {} for ledger {}.",
-                        writeFlags, delayedWriteFailedBookies, ledgerId);
-            }
-            return;
-        }
-        LedgerMetadata metadata = getLedgerMetadata();
-        synchronized (metadata) {
-            try {
-                EnsembleInfo ensembleInfo = 
replaceBookieInMetadata(delayedWriteFailedBookies, curNumEnsembleChanges);
-                if (ensembleInfo.replacedBookies.isEmpty()) {
-                    return;
-                }
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("[EnsembleChange-L{}-{}] : writing new ensemble 
info = {}",
-                            getId(), curNumEnsembleChanges, ensembleInfo);
-                }
-                writeLedgerConfig(new ChangeEnsembleCb(ensembleInfo, 
curBlockAddCompletions,
-                        curNumEnsembleChanges, false));
-            } catch (BKException.BKNotEnoughBookiesException e) {
-                LOG.error("Could not get additional bookie to remake ensemble: 
{}", ledgerId);
-            }
-            delayedWriteFailedBookies.clear();
-        }
+        final Map<Integer, BookieSocketAddress> copyDelayedWriteFailedBookies =
+                new HashMap<Integer, 
BookieSocketAddress>(delayedWriteFailedBookies);
+        delayedWriteFailedBookies.clear();
+
+        // Original intent of this change is to do a best-effort ensemble 
change.
+        // But this is not possible until the local metadata is completely 
immutable.
+        // Until the feature "Make LedgerMetadata Immutable #610" Is complete 
we will use
+        // handleBookieFailure() to handle delayed writes as regular bookie 
failures.
+        handleBookieFailure(copyDelayedWriteFailedBookies);
     }
 
     void handleBookieFailure(final Map<Integer, BookieSocketAddress> 
failedBookies) {
@@ -1958,7 +1925,7 @@ public class LedgerHandle implements WriteHandle {
                             getId(), curNumEnsembleChanges, ensembleInfo, 
curBlockAddCompletions);
                 }
                 writeLedgerConfig(new ChangeEnsembleCb(ensembleInfo, 
curBlockAddCompletions,
-                        curNumEnsembleChanges, true));
+                        curNumEnsembleChanges));
                 // clear if there are any delayed write failures were recorded.
                 delayedWriteFailedBookies.clear();
             } catch (BKException.BKNotEnoughBookiesException e) {
@@ -2002,17 +1969,14 @@ public class LedgerHandle implements WriteHandle {
         private final EnsembleInfo ensembleInfo;
         private final int curBlockAddCompletions;
         private final int ensembleChangeIdx;
-        private final boolean addEntryFailureRecovery;
 
         ChangeEnsembleCb(EnsembleInfo ensembleInfo,
                          int curBlockAddCompletions,
-                         int ensembleChangeIdx,
-                         boolean addEntryFailureRecovery) {
+                         int ensembleChangeIdx) {
             super(bk.getMainWorkerPool(), ledgerId);
             this.ensembleInfo = ensembleInfo;
             this.curBlockAddCompletions = curBlockAddCompletions;
             this.ensembleChangeIdx = ensembleChangeIdx;
-            this.addEntryFailureRecovery = addEntryFailureRecovery;
         }
 
         @Override
@@ -2033,17 +1997,11 @@ public class LedgerHandle implements WriteHandle {
             } else if (rc != BKException.Code.OK) {
                 LOG.error("[EnsembleChange-L{}-{}] : could not persist ledger 
metadata : info = {}, "
                         + "closing ledger : {}.", getId(), ensembleChangeIdx, 
ensembleInfo, rc);
-                if (addEntryFailureRecovery) {
-                    handleUnrecoverableErrorDuringAdd(rc);
-                }
+                handleUnrecoverableErrorDuringAdd(rc);
                 return;
             }
-            int newBlockAddCompletions;
-            if (addEntryFailureRecovery) {
-                newBlockAddCompletions = blockAddCompletions.decrementAndGet();
-            } else {
-                newBlockAddCompletions = blockAddCompletions.get();
-            }
+            int newBlockAddCompletions = blockAddCompletions.decrementAndGet();
+
 
             if (LOG.isDebugEnabled()) {
                 LOG.info("[EnsembleChange-L{}-{}] : completed ensemble change, 
block add completion {} => {}",
@@ -2054,10 +2012,8 @@ public class LedgerHandle implements WriteHandle {
             ensembleChangeCounter.inc();
             LOG.info("New Ensemble: {} for ledger: {}", 
ensembleInfo.newEnsemble, ledgerId);
 
-            if (addEntryFailureRecovery) {
-                // the failed bookie has been replaced
-                unsetSuccessAndSendWriteRequest(ensembleInfo.replacedBookies);
-            }
+            // the failed bookie has been replaced
+            unsetSuccessAndSendWriteRequest(ensembleInfo.replacedBookies);
         }
 
         @Override
@@ -2232,7 +2188,7 @@ public class LedgerHandle implements WriteHandle {
             // since they might be modified by recovery tool.
             metadata.mergeEnsembles(newMeta.getEnsembles());
             writeLedgerConfig(new ChangeEnsembleCb(ensembleInfo, 
curBlockAddCompletions,
-                    ensembleChangeIdx, true));
+                    ensembleChangeIdx));
             return true;
         }
 

Reply via email to