-
Notifications
You must be signed in to change notification settings - Fork 907
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Fix NPE issue during recovery add #3620
base: master
Are you sure you want to change the base?
Changes from 6 commits
ffa2458
636d8bd
96a2f82
d554c64
1836930
1cf0ba7
7848138
69a4879
ddc33e7
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,45 @@ | ||
/* | ||
* 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 java.util.List; | ||
import java.util.Map; | ||
import java.util.Set; | ||
import org.apache.bookkeeper.conf.ClientConfiguration; | ||
import org.apache.bookkeeper.discover.RegistrationClient; | ||
import org.apache.bookkeeper.net.BookieId; | ||
import org.apache.bookkeeper.proto.BookieAddressResolver; | ||
import org.apache.bookkeeper.stats.StatsLogger; | ||
|
||
public class MockBookieWatcher extends BookieWatcherImpl { | ||
|
||
public MockBookieWatcher(ClientConfiguration conf, EnsemblePlacementPolicy placementPolicy, | ||
RegistrationClient registrationClient, BookieAddressResolver bookieAddressResolver, | ||
StatsLogger statsLogger) { | ||
super(conf, placementPolicy, registrationClient, bookieAddressResolver, statsLogger); | ||
} | ||
|
||
@Override | ||
public BookieId replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, | ||
Map<String, byte[]> customMetadata, List<BookieId> existingBookies, int bookieIdx, | ||
Set<BookieId> excludeBookies) throws BKException.BKNotEnoughBookiesException { | ||
throw new BKException.BKNotEnoughBookiesException(); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -25,14 +25,26 @@ | |
import static org.mockito.Mockito.mock; | ||
import static org.mockito.Mockito.when; | ||
|
||
import com.google.common.collect.Lists; | ||
import io.netty.buffer.ByteBuf; | ||
import io.netty.buffer.Unpooled; | ||
import java.util.EnumSet; | ||
import java.util.HashMap; | ||
import java.util.Map; | ||
import java.util.concurrent.Callable; | ||
import java.util.concurrent.atomic.AtomicInteger; | ||
import org.apache.bookkeeper.client.BKException.Code; | ||
import org.apache.bookkeeper.client.api.LedgerMetadata; | ||
import org.apache.bookkeeper.client.api.WriteFlag; | ||
import org.apache.bookkeeper.common.util.OrderedExecutor; | ||
import org.apache.bookkeeper.conf.ClientConfiguration; | ||
import org.apache.bookkeeper.discover.MockRegistrationClient; | ||
import org.apache.bookkeeper.net.BookieId; | ||
import org.apache.bookkeeper.net.BookieSocketAddress; | ||
import org.apache.bookkeeper.proto.BookieClient; | ||
import org.apache.bookkeeper.proto.MockBookies; | ||
import org.apache.bookkeeper.stats.NullStatsLogger; | ||
import org.apache.bookkeeper.versioning.Versioned; | ||
import org.junit.Before; | ||
import org.junit.Test; | ||
|
||
|
@@ -60,6 +72,10 @@ public void setup() { | |
lh = mock(LedgerHandle.class); | ||
when(lh.getDistributionSchedule()) | ||
.thenReturn(new RoundRobinDistributionSchedule(3, 3, 2)); | ||
when(lh.getWriteFlags()) | ||
.thenReturn(EnumSet.of(WriteFlag.DEFERRED_SYNC)); | ||
Map<Integer, BookieId> failedBookies = new HashMap<>(); | ||
failedBookies.put(1, BookieId.parse("0.0.0.0:3181")); | ||
byte[] data = "test-pending-add-op".getBytes(UTF_8); | ||
payload = Unpooled.wrappedBuffer(data); | ||
payload.writerIndex(data.length); | ||
|
@@ -87,4 +103,69 @@ public void testExecuteAfterCancelled() { | |
assertNull(op.lh); | ||
} | ||
|
||
@Test | ||
public void testLedgerHandleWithDeferredSyncDuringRecoveryAdd() throws Exception { | ||
final BookieId b1 = new BookieSocketAddress("b1", 3181).toBookieId(); | ||
final BookieId b2 = new BookieSocketAddress("b2", 3181).toBookieId(); | ||
final BookieId b3 = new BookieSocketAddress("b3", 3181).toBookieId(); | ||
MockClientContext clientCtx = MockClientContext.create(); | ||
Versioned<LedgerMetadata> md = ClientUtil.setupLedger(clientCtx, 0, | ||
LedgerMetadataBuilder.create().withInRecoveryState().newEnsembleEntry(0L, | ||
Lists.newArrayList(b1, b2, b3))); | ||
LedgerHandle lh = new LedgerHandle(clientCtx, 0, md, BookKeeper.DigestType.CRC32C, | ||
ClientUtil.PASSWD, EnumSet.of(WriteFlag.DEFERRED_SYNC)); | ||
lh.notifyWriteFailed(0, b1); | ||
AtomicInteger rcHolder = new AtomicInteger(-0xdead); | ||
PendingAddOp op = PendingAddOp.create( | ||
lh, mockClientContext, lh.getCurrentEnsemble(), | ||
payload, EnumSet.of(WriteFlag.DEFERRED_SYNC), | ||
(rc, handle, entryId, qwcLatency, ctx) -> { | ||
rcHolder.set(rc); | ||
}, null).enableRecoveryAdd(); | ||
assertSame(lh, op.lh); | ||
op.setEntryId(0); | ||
lh.pendingAddOps.add(op); | ||
lh.clientCtx.getMainWorkerPool().submitOrdered(lh.ledgerId, (Callable<Void>) () -> { | ||
op.run(); | ||
return null; | ||
}).get(); | ||
} | ||
|
||
@Test | ||
public void testReadOnlyLedgerHandleWithNotEnoughBookiesExceptionDuringRecoveryAdd() throws Exception { | ||
final BookieId b1 = new BookieSocketAddress("b1", 3181).toBookieId(); | ||
final BookieId b2 = new BookieSocketAddress("b2", 3181).toBookieId(); | ||
final BookieId b3 = new BookieSocketAddress("b3", 3181).toBookieId(); | ||
MockBookies mockBookies = new MockBookies(); | ||
ClientConfiguration conf = new ClientConfiguration(); | ||
MockRegistrationClient regClient = new MockRegistrationClient(); | ||
EnsemblePlacementPolicy placementPolicy = new DefaultEnsemblePlacementPolicy(); | ||
BookieWatcher bookieWatcher = new MockBookieWatcher(conf, placementPolicy, | ||
regClient, | ||
new DefaultBookieAddressResolver(regClient), | ||
NullStatsLogger.INSTANCE); | ||
ClientContext clientCtx = MockClientContext.create(mockBookies, conf, regClient, placementPolicy, | ||
bookieWatcher); | ||
Versioned<LedgerMetadata> md = ClientUtil.setupLedger(clientCtx, 0, | ||
LedgerMetadataBuilder.create().withInRecoveryState().newEnsembleEntry(0L, | ||
Lists.newArrayList(b1, b2, b3))); | ||
ReadOnlyLedgerHandle lh = new ReadOnlyLedgerHandle(clientCtx, 0, md, BookKeeper.DigestType.CRC32C, | ||
ClientUtil.PASSWD, true); | ||
lh.notifyWriteFailed(0, b1); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. In the normal case, it can't happen. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I try to give a scenario where the problem might occur. It is a little troublesome to construct the above scenario, so I ensure that delayedWriteFailedBookies is not empty (by lh.notifyWriteFailed(0, b1)) from the beginning of my test case, so that I can quickly verify the npe issue. It should be noted that this is a scenario I deduced from the error log. In the log, only 946 write failures caused by BKNotEnoughBookiesException can be seen. |
||
AtomicInteger rcHolder = new AtomicInteger(-0xdead); | ||
PendingAddOp op = PendingAddOp.create( | ||
lh, mockClientContext, lh.getCurrentEnsemble(), | ||
payload, WriteFlag.NONE, | ||
(rc, handle, entryId, qwcLatency, ctx) -> { | ||
rcHolder.set(rc); | ||
}, null).enableRecoveryAdd(); | ||
assertSame(lh, op.lh); | ||
op.setEntryId(0); | ||
lh.pendingAddOps.add(op); | ||
lh.clientCtx.getMainWorkerPool().submitOrdered(lh.ledgerId, (Callable<Void>) () -> { | ||
op.run(); | ||
return null; | ||
}).get(); | ||
} | ||
|
||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@horizonzy I may not have explained it clearly. Although this is changed to an async operation, this code block will never be executed, because when in IN_RECOVERY mode, the method of ReadOnlyLedgerHandle must be called.
What do you think if I roll back the changes to LedgerHandle and only keep the changes to ReadOnlyLedgerHandle
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I see.