You are viewing a plain text version of this content. The canonical link for it is here.
Posted to distributedlog-commits@bookkeeper.apache.org by si...@apache.org on 2017/11/01 16:06:42 UTC

[distributedlog] branch master updated (11df8ef -> 1485241)

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

sijie pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/distributedlog.git.


    from 11df8ef  DLFS - A FileSystem API wrapper over dlog API
     new 24abc9e  Fix DistributedSchedule.WriteSet compilation issue
     new 1485241  Update exception handling in constructing bookkeeper client

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../org/apache/bookkeeper/client/LedgerReader.java | 11 +++++----
 .../apache/distributedlog/BookKeeperClient.java    | 26 +++++++++++-----------
 2 files changed, 20 insertions(+), 17 deletions(-)

-- 
To stop receiving notification emails like this one, please contact
['"distributedlog-commits@bookkeeper.apache.org" <di...@bookkeeper.apache.org>'].

[distributedlog] 01/02: Fix DistributedSchedule.WriteSet compilation issue

Posted by si...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

sijie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/distributedlog.git

commit 24abc9e7ff67b3c7291435c18133dac9d42293a3
Author: Sijie Guo <si...@apache.org>
AuthorDate: Tue Oct 31 01:31:51 2017 -0700

    Fix DistributedSchedule.WriteSet compilation issue
    
    DisributedSchedule writeset structure is changed in recent 4.6.0-SNAPSHOT version
---
 .../main/java/org/apache/bookkeeper/client/LedgerReader.java  | 11 +++++++----
 1 file changed, 7 insertions(+), 4 deletions(-)

diff --git a/distributedlog-core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java b/distributedlog-core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java
index 63c18d4..ccdc52b 100644
--- a/distributedlog-core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java
+++ b/distributedlog-core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java
@@ -27,6 +27,7 @@ import java.util.List;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.bookkeeper.client.DistributionSchedule.WriteSet;
 import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.proto.BookieClient;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback;
@@ -86,7 +87,7 @@ public class LedgerReader {
 
     public void readEntriesFromAllBookies(final LedgerHandle lh, long eid,
                                           final GenericCallback<Set<ReadResult<ByteBuf>>> callback) {
-        List<Integer> writeSet = lh.distributionSchedule.getWriteSet(eid);
+        WriteSet writeSet = lh.distributionSchedule.getWriteSet(eid);
         final AtomicInteger numBookies = new AtomicInteger(writeSet.size());
         final Set<ReadResult<ByteBuf>> readResults = new HashSet<>();
         ReadEntryCallback readEntryCallback = new ReadEntryCallback() {
@@ -118,7 +119,8 @@ public class LedgerReader {
         };
 
         ArrayList<BookieSocketAddress> ensemble = lh.getLedgerMetadata().getEnsemble(eid);
-        for (Integer idx : writeSet) {
+        for (int i = 0; i < writeSet.size(); i++) {
+            int idx = writeSet.get(i);
             bookieClient.readEntry(ensemble.get(idx), lh.getId(), eid, readEntryCallback, ensemble.get(idx));
         }
     }
@@ -181,7 +183,7 @@ public class LedgerReader {
 
     public void readLacs(final LedgerHandle lh, long eid,
                          final GenericCallback<Set<ReadResult<Long>>> callback) {
-        List<Integer> writeSet = lh.distributionSchedule.getWriteSet(eid);
+        WriteSet writeSet = lh.distributionSchedule.getWriteSet(eid);
         final AtomicInteger numBookies = new AtomicInteger(writeSet.size());
         final Set<ReadResult<Long>> readResults = new HashSet<ReadResult<Long>>();
         ReadEntryCallback readEntryCallback = (rc, lid, eid1, buffer, ctx) -> {
@@ -204,7 +206,8 @@ public class LedgerReader {
         };
 
         ArrayList<BookieSocketAddress> ensemble = lh.getLedgerMetadata().getEnsemble(eid);
-        for (Integer idx : writeSet) {
+        for (int i = 0; i < writeSet.size(); i++) {
+            int idx = writeSet.get(i);
             bookieClient.readEntry(ensemble.get(idx), lh.getId(), eid, readEntryCallback, ensemble.get(idx));
         }
     }

-- 
To stop receiving notification emails like this one, please contact
"distributedlog-commits@bookkeeper.apache.org" <di...@bookkeeper.apache.org>.

[distributedlog] 02/02: Update exception handling in constructing bookkeeper client

Posted by si...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

sijie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/distributedlog.git

commit 1485241ff1bebc3bad0d100beac9a61076f0265d
Author: Jia Zhai <zh...@apache.org>
AuthorDate: Wed Nov 1 01:39:24 2017 -0700

    Update exception handling in constructing bookkeeper client
---
 .../apache/distributedlog/BookKeeperClient.java    | 26 +++++++++++-----------
 1 file changed, 13 insertions(+), 13 deletions(-)

diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClient.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClient.java
index d1c1d6f..33baa9d 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClient.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClient.java
@@ -41,10 +41,8 @@ import org.apache.distributedlog.ZooKeeperClient.DigestCredentials;
 import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.AlreadyClosedException;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
-import org.apache.distributedlog.exceptions.ZKException;
 import org.apache.distributedlog.net.NetUtils;
 import org.apache.distributedlog.util.ConfUtils;
-import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -85,7 +83,7 @@ public class BookKeeperClient {
             String ledgersPath,
             EventLoopGroup eventLoopGroup,
             StatsLogger statsLogger, HashedWheelTimer requestTimer)
-        throws IOException, InterruptedException, KeeperException {
+        throws IOException, InterruptedException {
         ClientConfiguration bkConfig = new ClientConfiguration();
         bkConfig.setAddEntryTimeout(conf.getBKClientWriteTimeout());
         bkConfig.setReadTimeout(conf.getBKClientReadTimeout());
@@ -109,14 +107,18 @@ public class BookKeeperClient {
         final DNSToSwitchMapping dnsResolver =
                 NetUtils.getDNSResolver(dnsResolverCls, conf.getBkDNSResolverOverrides());
 
-        this.bkc = BookKeeper.forConfig(bkConfig)
-            .setZookeeper(zkc.get())
-            .setEventLoopGroup(eventLoopGroup)
-            .setStatsLogger(statsLogger)
-            .dnsResolver(dnsResolver)
-            .requestTimer(requestTimer)
-            .featureProvider(featureProvider.orNull())
-            .build();
+        try {
+            this.bkc = BookKeeper.forConfig(bkConfig)
+                .setZookeeper(zkc.get())
+                .setEventLoopGroup(eventLoopGroup)
+                .setStatsLogger(statsLogger)
+                .dnsResolver(dnsResolver)
+                .requestTimer(requestTimer)
+                .featureProvider(featureProvider.orNull())
+                .build();
+        } catch (BKException bke) {
+            throw new IOException(bke);
+        }
     }
 
     BookKeeperClient(DistributedLogConfiguration conf,
@@ -167,8 +169,6 @@ public class BookKeeperClient {
             commonInitialization(conf, ledgersPath, eventLoopGroup, statsLogger, requestTimer);
         } catch (InterruptedException e) {
             throw new DLInterruptedException("Interrupted on creating bookkeeper client " + name + " : ", e);
-        } catch (KeeperException e) {
-            throw new ZKException("Error on creating bookkeeper client " + name + " : ", e);
         }
 
         if (ownZK) {

-- 
To stop receiving notification emails like this one, please contact
"distributedlog-commits@bookkeeper.apache.org" <di...@bookkeeper.apache.org>.