You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by si...@apache.org on 2018/03/06 03:13:21 UTC
[bookkeeper] branch master updated: ISSUE #1229: PendingAddOp can
get recycled before it gets executed
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/bookkeeper.git
The following commit(s) were added to refs/heads/master by this push:
new fbc06ef ISSUE #1229: PendingAddOp can get recycled before it gets executed
fbc06ef is described below
commit fbc06ef3139ca857c60a2edbc209c8b088e979df
Author: Sijie Guo <si...@apache.org>
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 <si...@apache.org>
Reviewers: Andrey Yegorov <None>, Matteo Merli <mm...@apache.org>, Venkateswararao Jujjuri (JV) <None>
This closes #1230 from sijie/fix_npe, closes #1229
---
.../org/apache/bookkeeper/client/BookKeeper.java | 4 ++
.../org/apache/bookkeeper/client/LedgerHandle.java | 4 ++
.../org/apache/bookkeeper/client/PendingAddOp.java | 11 +--
.../apache/bookkeeper/client/PendingAddOpTest.java | 80 ++++++++++++++++++++++
4 files changed, 94 insertions(+), 5 deletions(-)
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java
index d6f88e9..b657147 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java
@@ -573,6 +573,10 @@ public class BookKeeper implements org.apache.bookkeeper.client.api.BookKeeper {
addEntryQuorumTimeoutNanos = 0;
}
+ long getAddEntryQuorumTimeoutNanos() {
+ return addEntryQuorumTimeoutNanos;
+ }
+
public int getExplicitLacInterval() {
return explicitLacInterval;
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 bdc547f..eebe43c 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
@@ -223,6 +223,10 @@ public class LedgerHandle implements WriteHandle {
}
}
+ BookKeeper getBk() {
+ return bk;
+ }
+
protected void initializeExplicitLacFlushPolicy() {
if (!metadata.isClosed() && !(this instanceof ReadOnlyLedgerHandle) && 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 67334aa..dd5058e 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
@@ -92,10 +92,10 @@ class PendingAddOp extends SafeRunnable implements WriteCallback {
op.entryLength = payload.readableBytes();
op.completed = false;
- op.ackSet = lh.distributionSchedule.getAckSet();
- op.addOpLogger = lh.bk.getAddOpLogger();
- op.addOpUrCounter = lh.bk.getAddOpUrCounter();
- op.timeoutNanos = lh.bk.addEntryQuorumTimeoutNanos;
+ op.ackSet = lh.getDistributionSchedule().getAckSet();
+ op.addOpLogger = lh.getBk().getAddOpLogger();
+ op.addOpUrCounter = lh.getBk().getAddOpUrCounter();
+ op.timeoutNanos = lh.getBk().getAddEntryQuorumTimeoutNanos();
op.pendingWriteRequests = 0;
op.callbackTriggered = false;
op.hasRun = false;
@@ -422,7 +422,8 @@ class PendingAddOp extends SafeRunnable implements WriteCallback {
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..e9af76e
--- /dev/null
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/PendingAddOpTest.java
@@ -0,0 +1,80 @@
+/*
+ * 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.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.getAddEntryQuorumTimeoutNanos()).thenReturn(1000L);
+ when(bk.getAddOpLogger()).thenReturn(NullStatsLogger.INSTANCE.getOpStatsLogger("test"));
+ when(bk.getAddOpUrCounter()).thenReturn(NullStatsLogger.INSTANCE.getCounter("test"));
+ 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, qwcLatency, 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
sijie@apache.org.