You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@bookkeeper.apache.org by GitBox <gi...@apache.org> on 2020/04/06 11:19:32 UTC

[GitHub] [bookkeeper] ivankelly opened a new pull request #2303: QuorumCoverage should only count unknown nodes

ivankelly opened a new pull request #2303: QuorumCoverage should only count unknown nodes
URL: https://github.com/apache/bookkeeper/pull/2303
 
 
   Quorum coverage checks if we have heard from enough nodes to know that
   there is no entry that can have been written to enough nodes that we
   haven't heard from to have formed an ack quorum.
   
   The coverage algorithm was correct pre-5e399df.
   
   5e399df(BOOKKEEPER-759: Delay Ensemble Change & Disable Ensemble
   Change) broke this, but it still seems to have worked because they had
   a broken else statement at the end. Why a change which is 100% about
   the write-path changed something in the read-path is a mystery.
   
   dcdd1e(Small fix wrong nodesUninitialized count when checkCovered)
   went on to fix the broken fix, so the whole thing ended up broke.
   
   The change also modifies ReadLastConfirmedOp to make it testable.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [bookkeeper] rdhabalia commented on a change in pull request #2303: QuorumCoverage should only count unknown nodes

Posted by GitBox <gi...@apache.org>.
rdhabalia commented on a change in pull request #2303: QuorumCoverage should only count unknown nodes
URL: https://github.com/apache/bookkeeper/pull/2303#discussion_r410969659
 
 

 ##########
 File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java
 ##########
 @@ -140,17 +149,15 @@ public synchronized void readEntryComplete(final int rc, final long ledgerId, fi
         }
 
         if (numResponsesPending == 0 && !completed) {
-            int totalExepctedResponse = lh.getLedgerMetadata().getWriteQuorumSize()
 
 Review comment:
   yes, change looks good, we don't need this logic if `coverageSet.checkCovered()` checks correct quorum response.
   However, as I explained in another comment: I think `coverageSet.checkCovered()` doesn't return true if `E=3, W=2, A=2` and if client receives ack from one of the bookie.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [bookkeeper] eolivelli commented on issue #2303: QuorumCoverage should only count unknown nodes

Posted by GitBox <gi...@apache.org>.
eolivelli commented on issue #2303: QuorumCoverage should only count unknown nodes
URL: https://github.com/apache/bookkeeper/pull/2303#issuecomment-610908509
 
 
   @sijie PTAL

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [bookkeeper] rdhabalia commented on a change in pull request #2303: QuorumCoverage should only count unknown nodes

Posted by GitBox <gi...@apache.org>.
rdhabalia commented on a change in pull request #2303: QuorumCoverage should only count unknown nodes
URL: https://github.com/apache/bookkeeper/pull/2303#discussion_r410841315
 
 

 ##########
 File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RoundRobinDistributionSchedule.java
 ##########
 @@ -373,29 +373,43 @@ public synchronized void addBookie(int bookieIndexHeardFrom, int rc) {
         public synchronized boolean checkCovered() {
             // now check if there are any write quorums, with |ackQuorum| nodes available
             for (int i = 0; i < ensembleSize; i++) {
-                int nodesNotCovered = 0;
-                int nodesOkay = 0;
-                int nodesUninitialized = 0;
+                /* Nodes which have either responded with an error other than NoSuch{Entry,Ledger},
 
 Review comment:
   I think I am missing something here So, I have a question. While doing a ledger recovery, bk client waits for response from [(Qw - Qa) + 1 bookies](https://bookkeeper.apache.org/docs/4.10.0/development/protocol/)
   So, if we have ledger with `E=3, W=2, A=2` and if bk-client receives ack from one of the bookie then: Qw-Qa+1 = (2-2+1) = 1 >= Response (1).
   So, `checkCovered()` should return true.
   However, with this change it fails on such useacase:
   eg.
   ```
   RoundRobinDistributionSchedule schedule = new RoundRobinDistributionSchedule(
           2, 2, 3);
   Set<Integer> resultSet = Sets.newHashSet(BKException.Code.OK,
           BKException.Code.UNINITIALIZED, BKException.Code.UNINITIALIZED);
   DistributionSchedule.QuorumCoverageSet covSet = schedule.getCoverageSet();
   int index =0;
   for (Integer i : resultSet) {
       covSet.addBookie(index++, i);
   }
   boolean covSetSays = covSet.checkCovered();
   assertTrue(covSetSays);
   ```
   
   So, can you please confirm the above assumption is correct or am I missing anything here?
   and can't we just check Qw-Qa+1 in this method:
   ```
   public synchronized boolean checkCovered() {
   int nodesUnknown = 0;
   for (int i = 0; i < covered.length; i++) {
       if (covered[i] != BKException.Code.OK
               && covered[i] != BKException.Code.NoSuchEntryException
               && covered[i] != BKException.Code.NoSuchLedgerExistsException) {
               nodesUnknown++;
           }
   }
   int expectedKnownNodes = (writeQuorumSize - ackQuorumSize) + 1;
   return (ensembleSize - nodesUnknown) >= expectedKnownNodes;
   }
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [bookkeeper] ivankelly commented on a change in pull request #2303: QuorumCoverage should only count unknown nodes

Posted by GitBox <gi...@apache.org>.
ivankelly commented on a change in pull request #2303: QuorumCoverage should only count unknown nodes
URL: https://github.com/apache/bookkeeper/pull/2303#discussion_r405474696
 
 

 ##########
 File path: bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedOpTest.java
 ##########
 @@ -0,0 +1,108 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.bookkeeper.client;
+
+import io.netty.buffer.UnpooledByteBufAllocator;
+import com.google.common.collect.Lists;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.net.BookieSocketAddress;
+import org.apache.bookkeeper.proto.MockBookieClient;
+import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat.DigestType;
+import org.apache.bookkeeper.proto.checksum.DigestManager;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReadLastConfirmedOpTest {
+    private static final Logger log = LoggerFactory.getLogger(ReadLastConfirmedOpTest.class);
+    private final BookieSocketAddress bookie1 = new BookieSocketAddress("bookie1", 3181);
+    private final BookieSocketAddress bookie2 = new BookieSocketAddress("bookie2", 3181);
+
+    OrderedExecutor executor = null;
+
+    @Before
+    public void setup() throws Exception {
+        executor = OrderedExecutor.newBuilder()
+                .name("BookKeeperClientWorker")
+                .numThreads(1)
+                .build();
+    }
+
+    @After
+    public void teardown() throws Exception {
+        if (executor != null) {
+            executor.shutdown();
+        }
+    }
+
+    /**
+     * Test for specific bug that was introduced with dcdd1e88
+     */
+    @Test
+    public void testBookieFailsAfterLedgerMissingOnFirst() throws Exception {
+        long ledgerId = 0xf00b;
+        List<BookieSocketAddress> ensemble = Lists.newArrayList(bookie1, bookie2);
+        byte[] ledgerKey = new byte[0];
+
+        MockBookieClient bookieClient = new MockBookieClient(executor);
+        DistributionSchedule schedule = new RoundRobinDistributionSchedule(2, 2, 2);
+        DigestManager digestManager = DigestManager.instantiate(ledgerId, ledgerKey,
+                                                                DigestType.CRC32C,
+                                                                UnpooledByteBufAllocator.DEFAULT,
+                                                                true /* useV2 */);
 
 Review comment:
   We only use V2 internally, so that's what I tested with originally.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [bookkeeper] eolivelli commented on a change in pull request #2303: QuorumCoverage should only count unknown nodes

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #2303: QuorumCoverage should only count unknown nodes
URL: https://github.com/apache/bookkeeper/pull/2303#discussion_r405490852
 
 

 ##########
 File path: bookkeeper-server/src/test/java/org/apache/bookkeeper/client/RoundRobinDistributionScheduleTest.java
 ##########
 @@ -74,16 +74,6 @@ public void testCoverageSets() {
             }
         }
         assertEquals("Should be no errors", 0, errors);
-
-        RoundRobinDistributionSchedule schedule = new RoundRobinDistributionSchedule(
-            5, 3, 5);
-        DistributionSchedule.QuorumCoverageSet covSet = schedule.getCoverageSet();
-        covSet.addBookie(0, BKException.Code.NoSuchLedgerExistsException);
-        covSet.addBookie(1, BKException.Code.NoSuchEntryException);
-        covSet.addBookie(2, BKException.Code.NoSuchLedgerExistsException);
-        covSet.addBookie(3, BKException.Code.UNINITIALIZED);
-        covSet.addBookie(4, BKException.Code.UNINITIALIZED);
-        assertFalse(covSet.checkCovered());
 
 Review comment:
   ok

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [bookkeeper] eolivelli commented on a change in pull request #2303: QuorumCoverage should only count unknown nodes

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #2303: QuorumCoverage should only count unknown nodes
URL: https://github.com/apache/bookkeeper/pull/2303#discussion_r405463513
 
 

 ##########
 File path: bookkeeper-server/src/test/java/org/apache/bookkeeper/client/RoundRobinDistributionScheduleTest.java
 ##########
 @@ -74,16 +74,6 @@ public void testCoverageSets() {
             }
         }
         assertEquals("Should be no errors", 0, errors);
-
-        RoundRobinDistributionSchedule schedule = new RoundRobinDistributionSchedule(
-            5, 3, 5);
-        DistributionSchedule.QuorumCoverageSet covSet = schedule.getCoverageSet();
-        covSet.addBookie(0, BKException.Code.NoSuchLedgerExistsException);
-        covSet.addBookie(1, BKException.Code.NoSuchEntryException);
-        covSet.addBookie(2, BKException.Code.NoSuchLedgerExistsException);
-        covSet.addBookie(3, BKException.Code.UNINITIALIZED);
-        covSet.addBookie(4, BKException.Code.UNINITIALIZED);
-        assertFalse(covSet.checkCovered());
 
 Review comment:
   why dropping this test ?
   can't we move this to *assertTrue*
   and maybe add other cases ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [bookkeeper] ivankelly commented on issue #2303: QuorumCoverage should only count unknown nodes

Posted by GitBox <gi...@apache.org>.
ivankelly commented on issue #2303: QuorumCoverage should only count unknown nodes
URL: https://github.com/apache/bookkeeper/pull/2303#issuecomment-610921425
 
 
   @eolivelli I need to take a look at those test failures. it doesn't fail on our internal branch, so either the failing tests have been added since, or they're flakes which we've disabled internally.
   
   I'll try to get to it before end of week, but pretty low on cycles.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [bookkeeper] eolivelli commented on issue #2303: QuorumCoverage should only count unknown nodes

Posted by GitBox <gi...@apache.org>.
eolivelli commented on issue #2303: QuorumCoverage should only count unknown nodes
URL: https://github.com/apache/bookkeeper/pull/2303#issuecomment-612372727
 
 
   @fpj @sijie @jvrao @merlimat @jiazhai @reddycharan   to you want to take a look and comment ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [bookkeeper] eolivelli commented on a change in pull request #2303: QuorumCoverage should only count unknown nodes

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #2303: QuorumCoverage should only count unknown nodes
URL: https://github.com/apache/bookkeeper/pull/2303#discussion_r405462057
 
 

 ##########
 File path: bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedOpTest.java
 ##########
 @@ -0,0 +1,108 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.bookkeeper.client;
+
+import io.netty.buffer.UnpooledByteBufAllocator;
+import com.google.common.collect.Lists;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.net.BookieSocketAddress;
+import org.apache.bookkeeper.proto.MockBookieClient;
+import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat.DigestType;
+import org.apache.bookkeeper.proto.checksum.DigestManager;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReadLastConfirmedOpTest {
+    private static final Logger log = LoggerFactory.getLogger(ReadLastConfirmedOpTest.class);
+    private final BookieSocketAddress bookie1 = new BookieSocketAddress("bookie1", 3181);
+    private final BookieSocketAddress bookie2 = new BookieSocketAddress("bookie2", 3181);
+
+    OrderedExecutor executor = null;
+
+    @Before
+    public void setup() throws Exception {
+        executor = OrderedExecutor.newBuilder()
+                .name("BookKeeperClientWorker")
+                .numThreads(1)
+                .build();
+    }
+
+    @After
+    public void teardown() throws Exception {
+        if (executor != null) {
+            executor.shutdown();
+        }
+    }
+
+    /**
+     * Test for specific bug that was introduced with dcdd1e88
+     */
+    @Test
+    public void testBookieFailsAfterLedgerMissingOnFirst() throws Exception {
+        long ledgerId = 0xf00b;
+        List<BookieSocketAddress> ensemble = Lists.newArrayList(bookie1, bookie2);
+        byte[] ledgerKey = new byte[0];
+
+        MockBookieClient bookieClient = new MockBookieClient(executor);
+        DistributionSchedule schedule = new RoundRobinDistributionSchedule(2, 2, 2);
+        DigestManager digestManager = DigestManager.instantiate(ledgerId, ledgerKey,
+                                                                DigestType.CRC32C,
+                                                                UnpooledByteBufAllocator.DEFAULT,
+                                                                true /* useV2 */);
 
 Review comment:
   is is necessary to use V2 ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [bookkeeper] ivankelly commented on a change in pull request #2303: QuorumCoverage should only count unknown nodes

Posted by GitBox <gi...@apache.org>.
ivankelly commented on a change in pull request #2303: QuorumCoverage should only count unknown nodes
URL: https://github.com/apache/bookkeeper/pull/2303#discussion_r405474303
 
 

 ##########
 File path: bookkeeper-server/src/test/java/org/apache/bookkeeper/client/RoundRobinDistributionScheduleTest.java
 ##########
 @@ -74,16 +74,6 @@ public void testCoverageSets() {
             }
         }
         assertEquals("Should be no errors", 0, errors);
-
-        RoundRobinDistributionSchedule schedule = new RoundRobinDistributionSchedule(
-            5, 3, 5);
-        DistributionSchedule.QuorumCoverageSet covSet = schedule.getCoverageSet();
-        covSet.addBookie(0, BKException.Code.NoSuchLedgerExistsException);
-        covSet.addBookie(1, BKException.Code.NoSuchEntryException);
-        covSet.addBookie(2, BKException.Code.NoSuchLedgerExistsException);
-        covSet.addBookie(3, BKException.Code.UNINITIALIZED);
-        covSet.addBookie(4, BKException.Code.UNINITIALIZED);
-        assertFalse(covSet.checkCovered());
 
 Review comment:
   Because, it's completely incorrect.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [bookkeeper] eolivelli commented on issue #2303: QuorumCoverage should only count unknown nodes

Posted by GitBox <gi...@apache.org>.
eolivelli commented on issue #2303: QuorumCoverage should only count unknown nodes
URL: https://github.com/apache/bookkeeper/pull/2303#issuecomment-610934263
 
 
   btw Overall I am +1 with this change

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [bookkeeper] eolivelli commented on a change in pull request #2303: QuorumCoverage should only count unknown nodes

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #2303: QuorumCoverage should only count unknown nodes
URL: https://github.com/apache/bookkeeper/pull/2303#discussion_r405462893
 
 

 ##########
 File path: bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedOpTest.java
 ##########
 @@ -0,0 +1,108 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.bookkeeper.client;
+
+import io.netty.buffer.UnpooledByteBufAllocator;
+import com.google.common.collect.Lists;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.net.BookieSocketAddress;
+import org.apache.bookkeeper.proto.MockBookieClient;
+import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat.DigestType;
+import org.apache.bookkeeper.proto.checksum.DigestManager;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReadLastConfirmedOpTest {
+    private static final Logger log = LoggerFactory.getLogger(ReadLastConfirmedOpTest.class);
+    private final BookieSocketAddress bookie1 = new BookieSocketAddress("bookie1", 3181);
+    private final BookieSocketAddress bookie2 = new BookieSocketAddress("bookie2", 3181);
+
+    OrderedExecutor executor = null;
+
+    @Before
+    public void setup() throws Exception {
+        executor = OrderedExecutor.newBuilder()
+                .name("BookKeeperClientWorker")
+                .numThreads(1)
+                .build();
+    }
+
+    @After
+    public void teardown() throws Exception {
+        if (executor != null) {
+            executor.shutdown();
+        }
+    }
+
+    /**
+     * Test for specific bug that was introduced with dcdd1e88
 
 Review comment:
   Can we describe the scenario ?
   
   Maybe referring dcdd1e88 is not useful

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [bookkeeper] eolivelli commented on issue #2303: QuorumCoverage should only count unknown nodes

Posted by GitBox <gi...@apache.org>.
eolivelli commented on issue #2303: QuorumCoverage should only count unknown nodes
URL: https://github.com/apache/bookkeeper/pull/2303#issuecomment-610909172
 
 
   @ivankelly  I see several tests failing on CI and IMHO they are due to this patch
   Something like:
   ERROR] Errors: 
   [ERROR] org.apache.bookkeeper.test.BookieFailureTest.testLedgerNoRecoveryOpenAfterBKCrashed(org.apache.bookkeeper.test.BookieFailureTest)
   [ERROR]   Run 1: BookieFailureTest.testLedgerNoRecoveryOpenAfterBKCrashed » BKRead Error while ...
   [ERROR]   Run 2: BookieFailureTest.testLedgerNoRecoveryOpenAfterBKCrashed » BKRead Error while ...
   [ERROR]   Run 3: BookieFailureTest.testLedgerNoRecoveryOpenAfterBKCrashed » BKRead Error while ...
   [ERROR] org.apache.bookkeeper.test.BookieFailureTest.testLedgerOpenAfterBKCrashed(org.apache.bookkeeper.test.BookieFailureTest)
   [ERROR]   Run 1: BookieFailureTest.testLedgerOpenAfterBKCrashed » BKLedgerRecovery Error while ...
   [ERROR]   Run 2: BookieFailureTest.testLedgerOpenAfterBKCrashed » BKLedgerRecovery Error while ...
   [ERROR]   Run 3: BookieFailureTest.testLedgerOpenAfterBKCrashed » BKLedgerRecovery Error while ...
   [ERROR] Tests run: 517, Failures: 0, Errors: 2, Skipped: 7

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [bookkeeper] eolivelli commented on a change in pull request #2303: QuorumCoverage should only count unknown nodes

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #2303: QuorumCoverage should only count unknown nodes
URL: https://github.com/apache/bookkeeper/pull/2303#discussion_r405491443
 
 

 ##########
 File path: bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedOpTest.java
 ##########
 @@ -0,0 +1,108 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.bookkeeper.client;
+
+import io.netty.buffer.UnpooledByteBufAllocator;
+import com.google.common.collect.Lists;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.net.BookieSocketAddress;
+import org.apache.bookkeeper.proto.MockBookieClient;
+import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat.DigestType;
+import org.apache.bookkeeper.proto.checksum.DigestManager;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReadLastConfirmedOpTest {
+    private static final Logger log = LoggerFactory.getLogger(ReadLastConfirmedOpTest.class);
+    private final BookieSocketAddress bookie1 = new BookieSocketAddress("bookie1", 3181);
+    private final BookieSocketAddress bookie2 = new BookieSocketAddress("bookie2", 3181);
+
+    OrderedExecutor executor = null;
+
+    @Before
+    public void setup() throws Exception {
+        executor = OrderedExecutor.newBuilder()
+                .name("BookKeeperClientWorker")
+                .numThreads(1)
+                .build();
+    }
+
+    @After
+    public void teardown() throws Exception {
+        if (executor != null) {
+            executor.shutdown();
+        }
+    }
+
+    /**
+     * Test for specific bug that was introduced with dcdd1e88
+     */
+    @Test
+    public void testBookieFailsAfterLedgerMissingOnFirst() throws Exception {
+        long ledgerId = 0xf00b;
+        List<BookieSocketAddress> ensemble = Lists.newArrayList(bookie1, bookie2);
+        byte[] ledgerKey = new byte[0];
+
+        MockBookieClient bookieClient = new MockBookieClient(executor);
+        DistributionSchedule schedule = new RoundRobinDistributionSchedule(2, 2, 2);
+        DigestManager digestManager = DigestManager.instantiate(ledgerId, ledgerKey,
+                                                                DigestType.CRC32C,
+                                                                UnpooledByteBufAllocator.DEFAULT,
+                                                                true /* useV2 */);
 
 Review comment:
   can we change it to v3 ? 
   there is no particular reason to use v2, it does not affect code coverage but it is "legacy"
   so in my opinion new code/tests should use latest version

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [bookkeeper] ivankelly commented on a change in pull request #2303: QuorumCoverage should only count unknown nodes

Posted by GitBox <gi...@apache.org>.
ivankelly commented on a change in pull request #2303: QuorumCoverage should only count unknown nodes
URL: https://github.com/apache/bookkeeper/pull/2303#discussion_r405474303
 
 

 ##########
 File path: bookkeeper-server/src/test/java/org/apache/bookkeeper/client/RoundRobinDistributionScheduleTest.java
 ##########
 @@ -74,16 +74,6 @@ public void testCoverageSets() {
             }
         }
         assertEquals("Should be no errors", 0, errors);
-
-        RoundRobinDistributionSchedule schedule = new RoundRobinDistributionSchedule(
-            5, 3, 5);
-        DistributionSchedule.QuorumCoverageSet covSet = schedule.getCoverageSet();
-        covSet.addBookie(0, BKException.Code.NoSuchLedgerExistsException);
-        covSet.addBookie(1, BKException.Code.NoSuchEntryException);
-        covSet.addBookie(2, BKException.Code.NoSuchLedgerExistsException);
-        covSet.addBookie(3, BKException.Code.UNINITIALIZED);
-        covSet.addBookie(4, BKException.Code.UNINITIALIZED);
-        assertFalse(covSet.checkCovered());
 
 Review comment:
   Because it's completely incorrect.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [bookkeeper] eolivelli commented on a change in pull request #2303: QuorumCoverage should only count unknown nodes

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #2303: QuorumCoverage should only count unknown nodes
URL: https://github.com/apache/bookkeeper/pull/2303#discussion_r405461349
 
 

 ##########
 File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java
 ##########
 @@ -140,17 +149,15 @@ public synchronized void readEntryComplete(final int rc, final long ledgerId, fi
         }
 
         if (numResponsesPending == 0 && !completed) {
-            int totalExepctedResponse = lh.getLedgerMetadata().getWriteQuorumSize()
 
 Review comment:
   @rdhabalia PTAL

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [bookkeeper] rdhabalia commented on a change in pull request #2303: QuorumCoverage should only count unknown nodes

Posted by GitBox <gi...@apache.org>.
rdhabalia commented on a change in pull request #2303: QuorumCoverage should only count unknown nodes
URL: https://github.com/apache/bookkeeper/pull/2303#discussion_r410841315
 
 

 ##########
 File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RoundRobinDistributionSchedule.java
 ##########
 @@ -373,29 +373,43 @@ public synchronized void addBookie(int bookieIndexHeardFrom, int rc) {
         public synchronized boolean checkCovered() {
             // now check if there are any write quorums, with |ackQuorum| nodes available
             for (int i = 0; i < ensembleSize; i++) {
-                int nodesNotCovered = 0;
-                int nodesOkay = 0;
-                int nodesUninitialized = 0;
+                /* Nodes which have either responded with an error other than NoSuch{Entry,Ledger},
 
 Review comment:
   I think I am missing something here So, I have a question. While doing a ledger recovery, bk client waits for response from [(Qw - Qa) + 1 bookies](https://bookkeeper.apache.org/docs/4.10.0/development/protocol/)
   So, if we have ledger with `E=3, W=2, A=3` and if bk-client receives ack from one of the bookie then: Qw-Qa+1 = (2-2+1) = 1 >= Response (1).
   So, `checkCovered()` should return true.
   However, with this change it fails on such useacase:
   eg.
   ```
   RoundRobinDistributionSchedule schedule = new RoundRobinDistributionSchedule(
           2, 2, 3);
   Set<Integer> resultSet = Sets.newHashSet(BKException.Code.OK,
           BKException.Code.UNINITIALIZED, BKException.Code.UNINITIALIZED);
   DistributionSchedule.QuorumCoverageSet covSet = schedule.getCoverageSet();
   int index =0;
   for (Integer i : resultSet) {
       covSet.addBookie(index++, i);
   }
   boolean covSetSays = covSet.checkCovered();
   assertTrue(covSetSays);
   ```
   
   So, can you please confirm the above assumption is correct or am I missing anything here?
   and can't we just check Qw-Qa+1 in this method:
   ```
   public synchronized boolean checkCovered() {
   int nodesUnknown = 0;
   for (int i = 0; i < covered.length; i++) {
       if (covered[i] != BKException.Code.OK
               && covered[i] != BKException.Code.NoSuchEntryException
               && covered[i] != BKException.Code.NoSuchLedgerExistsException) {
               nodesUnknown++;
           }
   }
   int expectedKnownNodes = (writeQuorumSize - ackQuorumSize) + 1;
   return (ensembleSize - nodesUnknown) >= expectedKnownNodes;
   }
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services