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 fb8b58e  Update distributedlog library to use 
NoSuchLedgerExistsOnMetadataServer
fb8b58e is described below

commit fb8b58e9dff3fcdb30cf8faabff442d683922232
Author: Sijie Guo <[email protected]>
AuthorDate: Thu Jun 6 01:58:43 2019 -0700

    Update distributedlog library to use NoSuchLedgerExistsOnMetadataServer
    
    *Motivation*
    
    Issue #2066 introduced `NoSuchLedgerExistsOnMetadataServer`. But the 
distributedlog
    library is not updated to reflect to this change.
    
    *Modifications*
    
    Change to use `NoSuchLedgerExistsOnMetadataServer` in the right place.
    
    Reviewers: Ivan Kelly <[email protected]>, Enrico Olivelli 
<[email protected]>
    
    This closes #2102 from sijie/fix_failure_test
---
 .../core/src/main/java/org/apache/distributedlog/BookKeeperClient.java | 3 ++-
 .../apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java  | 3 ++-
 .../main/java/org/apache/distributedlog/tools/DistributedLogTool.java  | 2 +-
 .../java/org/apache/distributedlog/TestBKDistributedLogManager.java    | 2 +-
 4 files changed, 6 insertions(+), 4 deletions(-)

diff --git 
a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java
 
b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java
index 366178a..8878135 100644
--- 
a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java
+++ 
b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java
@@ -27,6 +27,7 @@ import java.util.Collections;
 import java.util.concurrent.CompletableFuture;
 import org.apache.bookkeeper.client.AsyncCallback;
 import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BKException.Code;
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.LedgerHandle;
 import org.apache.bookkeeper.client.RegionAwareEnsemblePlacementPolicy;
@@ -238,7 +239,7 @@ public class BookKeeperClient {
             public void deleteComplete(int rc, Object ctx) {
                 if (BKException.Code.OK == rc) {
                     promise.complete(null);
-                } else if (BKException.Code.NoSuchLedgerExistsException == rc) 
{
+                } else if (Code.NoSuchLedgerExistsOnMetadataServerException == 
rc) {
                     if (ignoreNonExistentLedger) {
                         promise.complete(null);
                     } else {
diff --git 
a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java
 
b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java
index aaff62e..b164a39 100644
--- 
a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java
+++ 
b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.util.concurrent.CompletableFuture;
 import org.apache.bookkeeper.client.AsyncCallback;
 import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BKException.Code;
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.LedgerHandle;
 import org.apache.bookkeeper.common.concurrent.FutureUtils;
@@ -134,7 +135,7 @@ public class BKLogSegmentEntryStore implements
     @Override
     public void deleteComplete(int rc, Object ctx) {
         DeleteLogSegmentRequest deleteRequest = (DeleteLogSegmentRequest) ctx;
-        if (BKException.Code.NoSuchLedgerExistsException == rc) {
+        if (Code.NoSuchLedgerExistsOnMetadataServerException == rc) {
             logger.warn("No ledger {} found to delete for {}.",
                     deleteRequest.segment.getLogSegmentId(), 
deleteRequest.segment);
         } else if (BKException.Code.OK != rc) {
diff --git 
a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java
 
b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java
index d19e768..7448996 100644
--- 
a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java
+++ 
b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java
@@ -1277,7 +1277,7 @@ import org.slf4j.LoggerFactory;
                                     if (numLedgersDeleted % 1000 == 0) {
                                         System.out.println("Deleted " + 
numLedgersDeleted + " ledgers.");
                                     }
-                                } catch 
(BKException.BKNoSuchLedgerExistsException e) {
+                                } catch 
(BKException.BKNoSuchLedgerExistsOnMetadataServerException e) {
                                     int numLedgersDeleted = 
numLedgers.incrementAndGet();
                                     if (numLedgersDeleted % 1000 == 0) {
                                         System.out.println("Deleted " + 
numLedgersDeleted + " ledgers.");
diff --git 
a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java
 
b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java
index 14c751e..db06891 100644
--- 
a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java
+++ 
b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java
@@ -1237,7 +1237,7 @@ public class TestBKDistributedLogManager extends 
TestDistributedLogBase {
             driver.getReaderBKC().get().openLedgerNoRecovery(ledgerId,
                 BookKeeper.DigestType.CRC32, 
conf.getBKDigestPW().getBytes(UTF_8));
             fail("Should fail to open ledger after we delete the log");
-        } catch (BKException.BKNoSuchLedgerExistsException e) {
+        } catch (BKException.BKNoSuchLedgerExistsOnMetadataServerException e) {
             // ignore
         }
         // delete again should not throw any exception

Reply via email to