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

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


The following commit(s) were added to refs/heads/branch-4.6 by this push:
     new b59dab5  ISSUE #1229: PendingAddOp can get recycled before it gets 
executed
b59dab5 is described below

commit b59dab59020e11ff2fa6fab342081a3f90c22b03
Author: Sijie Guo <[email protected]>
AuthorDate: Mon Mar 5 19:13:14 2018 -0800

    ISSUE #1229: PendingAddOp can get recycled before it gets executed
    
    Descriptions of the changes in this PR:
    *Problem*
    
    The PendingAddOp can be recycled when it is cancelled before it is 
executed. so it will hit NPE when it is actually executed. This is a bug 
introduced by #1091
    
    *Solution*
    
    Only recycle PendingAddOp after it has been run.
    
    Master Issue: #1229
    
    Author: Sijie Guo <[email protected]>
    
    Reviewers: Andrey Yegorov <None>, Matteo Merli <[email protected]>, 
Venkateswararao Jujjuri (JV) <None>
    
    This closes #1230 from sijie/fix_npe, closes #1229
---
 .../org/apache/bookkeeper/client/LedgerHandle.java |  4 ++
 .../org/apache/bookkeeper/client/PendingAddOp.java | 11 ++-
 .../apache/bookkeeper/client/PendingAddOpTest.java | 78 ++++++++++++++++++++++
 3 files changed, 87 insertions(+), 6 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 e523b37..e0b7e72 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
@@ -184,6 +184,10 @@ public class LedgerHandle implements WriteHandle {
         initializeExplicitLacFlushPolicy();
     }
 
+    BookKeeper getBk() {
+        return bk;
+    }
+
     protected void initializeExplicitLacFlushPolicy() {
         if (!metadata.isClosed() && bk.getExplicitLacInterval() > 0) {
             explicitLacFlushPolicy = new 
ExplicitLacFlushPolicy.ExplicitLacFlushPolicyImpl(this);
diff --git 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java
 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java
index 297b32a..9c50379 100644
--- 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java
+++ 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java
@@ -25,7 +25,6 @@ import io.netty.util.ReferenceCountUtil;
 import io.netty.util.Timeout;
 import io.netty.util.TimerTask;
 
-import java.util.Arrays;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
@@ -37,7 +36,6 @@ import 
org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
 import org.apache.bookkeeper.stats.OpStatsLogger;
 import org.apache.bookkeeper.util.MathUtils;
 import org.apache.bookkeeper.util.SafeRunnable;
-import org.apache.commons.lang3.ArrayUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import java.util.concurrent.RejectedExecutionException;
@@ -89,13 +87,13 @@ class PendingAddOp extends SafeRunnable implements 
WriteCallback, TimerTask {
         op.entryLength = payload.readableBytes();
 
         op.completed = false;
-        op.ackSet = lh.distributionSchedule.getAckSet();
-        op.addOpLogger = lh.bk.getAddOpLogger();
+        op.ackSet = lh.getDistributionSchedule().getAckSet();
+        op.addOpLogger = lh.getBk().getAddOpLogger();
         if (op.timeout != null) {
             op.timeout.cancel();
         }
         op.timeout = null;
-        op.timeoutSec = lh.bk.getConf().getAddEntryQuorumTimeout();
+        op.timeoutSec = lh.getBk().getConf().getAddEntryQuorumTimeout();
         op.pendingWriteRequests = 0;
         op.callbackTriggered = false;
         op.hasRun = false;
@@ -420,7 +418,8 @@ class PendingAddOp extends SafeRunnable implements 
WriteCallback, TimerTask {
             ReferenceCountUtil.release(toSend);
             toSend = null;
         }
-        if (toSend == null && pendingWriteRequests == 0) {
+        // only recycle a pending add op after it has been run.
+        if (hasRun && toSend == null && pendingWriteRequests == 0) {
             recyclePendAddOpObject();
         }
     }
diff --git 
a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/PendingAddOpTest.java
 
b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/PendingAddOpTest.java
new file mode 100644
index 0000000..57e7d6b
--- /dev/null
+++ 
b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/PendingAddOpTest.java
@@ -0,0 +1,78 @@
+/*
+ * 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 static java.nio.charset.StandardCharsets.UTF_8;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertSame;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.bookkeeper.client.BKException.Code;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Unit test of {@link PendingAddOp}.
+ */
+public class PendingAddOpTest {
+
+    private BookKeeper bk;
+    private LedgerHandle lh;
+    private ByteBuf payload;
+
+    @Before
+    public void setup() {
+        bk = mock(BookKeeper.class);
+        
when(bk.getAddOpLogger()).thenReturn(NullStatsLogger.INSTANCE.getOpStatsLogger("test"));
+        when(bk.getConf()).thenReturn(new ClientConfiguration());
+        lh = mock(LedgerHandle.class);
+        when(lh.getBk()).thenReturn(bk);
+        when(lh.getDistributionSchedule())
+            .thenReturn(new RoundRobinDistributionSchedule(3, 3, 2));
+        byte[] data = "test-pending-add-op".getBytes(UTF_8);
+        payload = Unpooled.wrappedBuffer(data);
+        payload.writerIndex(data.length);
+    }
+
+    @Test
+    public void testExecuteAfterCancelled() {
+        AtomicInteger rcHolder = new AtomicInteger(-0xdead);
+        PendingAddOp op = PendingAddOp.create(
+            lh, payload, (rc, handle, entryId, ctx) -> rcHolder.set(rc), null);
+        assertSame(lh, op.lh);
+
+        // cancel the op.
+        op.submitCallback(Code.NotEnoughBookiesException);
+        // if a op is cancelled, it is not recycled until it has been run.
+        assertSame(lh, op.lh);
+        assertEquals(Code.NotEnoughBookiesException, rcHolder.get());
+
+        op.run();
+        // after the op is run, the object is recycled.
+        assertNull(op.lh);
+    }
+
+}

-- 
To stop receiving notification emails like this one, please contact
[email protected].

Reply via email to