Skip to content

Commit

Permalink
[transaction][acknowledge] Introduce in-memory PENDING_ACK state in a…
Browse files Browse the repository at this point in the history
…cknowledgement path (apache#4265)

Master Issue: apache#2664

Motivation:
Add acknowledgeMessage, commit, abort for transaction in PersistentSubscription.

Changes:
Will put message in Pending_ACK status when acknowledgeMessage is called with TxnID.
No real status class introduced, only added collection to hold messages in Pending_ACK status.
Current PR only keep Pending_ACK state in memory, in subsequent PR will also persistent these pending acks so we can recover from broker failure.

Add commitTxn to put message to Deleted status.
Add abortTxn to put message to Pending status.

For normal acknowledgeMessage and redeliverUnacknowledgedMessages, will check to see if 
message if message is in Pending_ACK first. If true, will **ignore** those acks/redeliverys.

Add unit test.
  • Loading branch information
MarvinCai authored and sijie committed Jun 17, 2019
1 parent 9eb7cc6 commit e66ba27
Show file tree
Hide file tree
Showing 10 changed files with 706 additions and 17 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -2518,6 +2518,12 @@ public LongPairRangeSet<PositionImpl> getIndividuallyDeletedMessagesSet() {
return individualDeletedMessages;
}

public boolean isMessageDeleted(Position position) {
checkArgument(position instanceof PositionImpl);
return individualDeletedMessages.contains(((PositionImpl) position).getLedgerId(),
((PositionImpl) position).getEntryId()) || ((PositionImpl) position).compareTo(markDeletePosition) <= 0 ;
}

/**
* Checks given position is part of deleted-range and returns next position of upper-end as all the messages are
* deleted up to that point.
Expand Down
6 changes: 6 additions & 0 deletions pulsar-broker/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -108,6 +108,12 @@
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>pulsar-transaction-common</artifactId>
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>managed-ledger-original</artifactId>
Expand Down

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import com.google.common.collect.Sets;

import java.lang.reflect.Field;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Optional;
Expand Down Expand Up @@ -1307,7 +1308,8 @@ public void testMessageRedelivery() throws Exception {
final String subName = "sub2";

Message<String> msg;
int totalMessages = 10;
List<Message<String>> unackedMessages = new ArrayList<>();
int totalMessages = 20;

Consumer<String> consumer = pulsarClient.newConsumer(Schema.STRING)
.topic(topicName)
Expand All @@ -1326,26 +1328,29 @@ public void testMessageRedelivery() throws Exception {
producer.send("my-message-" + i);
}

// (2) Consume and ack messages except first message
Message<String> unAckedMsg = null;
// (2) Consume and only ack last 10 messages
for (int i = 0; i < totalMessages; i++) {
msg = consumer.receive();
if (i == 0) {
unAckedMsg = msg;
if (i >= 10) {
unackedMessages.add(msg);
} else {
consumer.acknowledge(msg);
}
}

consumer.redeliverUnacknowledgedMessages();

// Verify: msg [L:0] must be redelivered
try {
msg = consumer.receive(1, TimeUnit.SECONDS);
assertEquals(msg.getValue(), unAckedMsg.getValue());
} catch (Exception e) {
fail("msg should be redelivered ", e);
for (int i = 0; i < 10; i++) {
// Verify: msg [L:0] must be redelivered
try {
final Message<String> redeliveredMsg = consumer.receive(1, TimeUnit.SECONDS);
unackedMessages.removeIf(unackedMessage -> unackedMessage.getValue().equals(redeliveredMsg.getValue()));
} catch (Exception e) {
fail("msg should be redelivered ", e);
}
}
// Make sure that first 10 messages that we didn't acknowledge get redelivered.
assertTrue(unackedMessages.size() == 0);

// Verify no other messages are redelivered
msg = consumer.receive(100, TimeUnit.MILLISECONDS);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,284 @@
/**
* 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.pulsar.broker.service.persistent;

import org.apache.bookkeeper.mledger.AsyncCallbacks;
import org.apache.bookkeeper.mledger.ManagedCursor;
import org.apache.bookkeeper.mledger.ManagedLedger;
import org.apache.bookkeeper.mledger.ManagedLedgerFactory;
import org.apache.bookkeeper.mledger.Position;
import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl;
import org.apache.bookkeeper.mledger.impl.PositionImpl;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.cache.ConfigurationCacheService;
import org.apache.pulsar.broker.cache.LocalZooKeeperCacheService;
import org.apache.pulsar.broker.service.BrokerService;
import org.apache.pulsar.broker.service.BrokerServiceException;
import org.apache.pulsar.broker.service.Consumer;
import org.apache.pulsar.broker.service.PersistentTopicTest;
import org.apache.pulsar.client.api.SubscriptionType;
import org.apache.pulsar.common.api.proto.PulsarApi;
import org.apache.pulsar.common.policies.data.Policies;
import org.apache.pulsar.compaction.Compactor;
import org.apache.pulsar.transaction.common.exception.TransactionConflictException;
import org.apache.pulsar.transaction.impl.common.TxnID;
import org.apache.pulsar.zookeeper.ZooKeeperCache;
import org.apache.pulsar.zookeeper.ZooKeeperDataCache;
import org.apache.zookeeper.ZooKeeper;
import org.powermock.core.classloader.annotations.PrepareForTest;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testng.annotations.AfterMethod;
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.Test;

import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;

import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.createMockBookKeeper;
import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.createMockZooKeeper;
import static org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyList;
import static org.mockito.Matchers.anyMap;
import static org.mockito.Matchers.anyObject;
import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertTrue;
import static org.testng.Assert.fail;

@PrepareForTest({ ZooKeeperDataCache.class, BrokerService.class })
public class PersistentSubscriptionTest {

private PulsarService pulsarMock;
private BrokerService brokerMock;
private ManagedLedgerFactory mlFactoryMock;
private ManagedLedger ledgerMock;
private ManagedCursorImpl cursorMock;
private ConfigurationCacheService configCacheServiceMock;
private PersistentTopic topic;
private PersistentSubscription persistentSubscription;
private Consumer consumerMock;

final String successTopicName = "persistent://prop/use/ns-abc/successTopic";
final String subName = "subscriptionName";

final TxnID txnID1 = new TxnID(1,1);
final TxnID txnID2 = new TxnID(1,2);

private static final Logger log = LoggerFactory.getLogger(PersistentTopicTest.class);

@BeforeMethod
public void setup() throws Exception {
ServiceConfiguration svcConfig = spy(new ServiceConfiguration());
pulsarMock = spy(new PulsarService(svcConfig));
doReturn(svcConfig).when(pulsarMock).getConfiguration();
doReturn(mock(Compactor.class)).when(pulsarMock).getCompactor();

mlFactoryMock = mock(ManagedLedgerFactory.class);
doReturn(mlFactoryMock).when(pulsarMock).getManagedLedgerFactory();

ZooKeeper zkMock = createMockZooKeeper();
doReturn(zkMock).when(pulsarMock).getZkClient();
doReturn(createMockBookKeeper(zkMock, pulsarMock.getOrderedExecutor().chooseThread(0)))
.when(pulsarMock).getBookKeeperClient();

ZooKeeperCache cache = mock(ZooKeeperCache.class);
doReturn(30).when(cache).getZkOperationTimeoutSeconds();
CompletableFuture getDataFuture = new CompletableFuture();
getDataFuture.complete(Optional.empty());
doReturn(getDataFuture).when(cache).getDataAsync(anyString(), any(), any());
doReturn(cache).when(pulsarMock).getLocalZkCache();

configCacheServiceMock = mock(ConfigurationCacheService.class);
@SuppressWarnings("unchecked")
ZooKeeperDataCache<Policies> zkPoliciesDataCacheMock = mock(ZooKeeperDataCache.class);
doReturn(zkPoliciesDataCacheMock).when(configCacheServiceMock).policiesCache();
doReturn(configCacheServiceMock).when(pulsarMock).getConfigurationCache();
doReturn(Optional.empty()).when(zkPoliciesDataCacheMock).get(anyString());

LocalZooKeeperCacheService zkCacheMock = mock(LocalZooKeeperCacheService.class);
doReturn(CompletableFuture.completedFuture(Optional.empty())).when(zkPoliciesDataCacheMock).getAsync(any());
doReturn(zkPoliciesDataCacheMock).when(zkCacheMock).policiesCache();
doReturn(zkCacheMock).when(pulsarMock).getLocalZkCacheService();

brokerMock = spy(new BrokerService(pulsarMock));
doNothing().when(brokerMock).unloadNamespaceBundlesGracefully();
doReturn(brokerMock).when(pulsarMock).getBrokerService();

ledgerMock = mock(ManagedLedger.class);
cursorMock = mock(ManagedCursorImpl.class);
doReturn(new ArrayList<Object>()).when(ledgerMock).getCursors();
doReturn("mockCursor").when(cursorMock).getName();
doReturn(new PositionImpl(1, 50)).when(cursorMock).getMarkDeletedPosition();

topic = new PersistentTopic(successTopicName, ledgerMock, brokerMock);

consumerMock = mock(Consumer.class);

persistentSubscription = new PersistentSubscription(topic, subName, cursorMock, false);
}

@AfterMethod
public void teardown() throws Exception {
brokerMock.close(); //to clear pulsarStats
try {
pulsarMock.close();
} catch (Exception e) {
log.warn("Failed to close pulsar service", e);
throw e;
}
}

@Test
public void testCanAcknowledgeAndCommitForTransaction() throws TransactionConflictException {
List<Position> expectedSinglePositions = new ArrayList<>();
expectedSinglePositions.add(new PositionImpl(1, 1));
expectedSinglePositions.add(new PositionImpl(1, 3));
expectedSinglePositions.add(new PositionImpl(1, 5));

doAnswer((invocationOnMock) -> {
assertTrue(((List)invocationOnMock.getArguments()[0]).containsAll(expectedSinglePositions));
((AsyncCallbacks.DeleteCallback) invocationOnMock.getArguments()[1])
.deleteComplete(invocationOnMock.getArguments()[2]);
return null;
}).when(cursorMock).asyncDelete(anyList(), any(AsyncCallbacks.DeleteCallback.class), anyObject());

doAnswer((invocationOnMock) -> {
assertTrue(((PositionImpl)invocationOnMock.getArguments()[0]).compareTo(new PositionImpl(3, 100)) == 0);
((AsyncCallbacks.MarkDeleteCallback) invocationOnMock.getArguments()[2])
.markDeleteComplete(invocationOnMock.getArguments()[3]);
return null;
}).when(cursorMock).asyncMarkDelete(anyObject(), anyObject(), any(AsyncCallbacks.MarkDeleteCallback.class), anyObject());

List<Position> positions = new ArrayList<>();
positions.add(new PositionImpl(1, 1));
positions.add(new PositionImpl(1, 3));
positions.add(new PositionImpl(1, 5));

// Single ack for txn
persistentSubscription.acknowledgeMessage(txnID1, positions, AckType.Individual);

positions.clear();
positions.add(new PositionImpl(3, 100));

// Cumulative ack for txn
persistentSubscription.acknowledgeMessage(txnID1, positions, AckType.Cumulative);

// Commit txn
persistentSubscription.commitTxn(txnID1, Collections.emptyMap());

// Verify corresponding ledger method was called with expected args.
verify(cursorMock, times(1)).asyncDelete(anyList(), any(), any());
verify(cursorMock, times(1)).asyncMarkDelete(any(), anyMap(), anyObject(), any());
}

@Test
public void testCanAcknowledgeAndAbortForTransaction() throws TransactionConflictException, BrokerServiceException {
List<Position> positions = new ArrayList<>();
positions.add(new PositionImpl(2, 1));
positions.add(new PositionImpl(2, 3));
positions.add(new PositionImpl(2, 5));

Position[] expectedSinglePositions = {new PositionImpl(3, 1),
new PositionImpl(3, 3), new PositionImpl(3, 5)};

doAnswer((invocationOnMock) -> {
assertTrue(Arrays.deepEquals(((List)invocationOnMock.getArguments()[0]).toArray(), expectedSinglePositions));
((AsyncCallbacks.DeleteCallback) invocationOnMock.getArguments()[1])
.deleteComplete(invocationOnMock.getArguments()[2]);
return null;
}).when(cursorMock).asyncDelete(anyList(), any(AsyncCallbacks.DeleteCallback.class), anyObject());

doReturn(PulsarApi.CommandSubscribe.SubType.Exclusive).when(consumerMock).subType();

persistentSubscription.addConsumer(consumerMock);

// Single ack for txn1
persistentSubscription.acknowledgeMessage(txnID1, positions, AckType.Individual);

positions.clear();
positions.add(new PositionImpl(1, 100));

// Cumulative ack for txn1
persistentSubscription.acknowledgeMessage(txnID1, positions, AckType.Cumulative);

positions.clear();
positions.add(new PositionImpl(2, 1));

// Can not single ack message already acked.
try {
persistentSubscription.acknowledgeMessage(txnID2, positions, AckType.Individual);
fail("Single acknowledge for transaction2 should fail. ");
} catch (TransactionConflictException e) {
assertEquals(e.getMessage(),"[persistent://prop/use/ns-abc/successTopic][subscriptionName] " +
"Transaction:(1,2) try to ack message:2:1 in pending ack status.");
}

positions.clear();
positions.add(new PositionImpl(2, 50));

// Can not cumulative ack message for another txn.
try {
persistentSubscription.acknowledgeMessage(txnID2, positions, AckType.Cumulative);
fail("Cumulative acknowledge for transaction2 should fail. ");
} catch (TransactionConflictException e) {
System.out.println(e.getMessage());
assertEquals(e.getMessage(),"[persistent://prop/use/ns-abc/successTopic][subscriptionName] " +
"Transaction:(1,2) try to cumulative ack message while transaction:(1,1) already cumulative acked messages.");
}

positions.clear();
positions.add(new PositionImpl(1, 1));
positions.add(new PositionImpl(1, 3));
positions.add(new PositionImpl(1, 5));
positions.add(new PositionImpl(3, 1));
positions.add(new PositionImpl(3, 3));
positions.add(new PositionImpl(3, 5));

// Acknowledge from normal consumer will succeed ignoring message acked by ongoing transaction.
persistentSubscription.acknowledgeMessage(positions, AckType.Individual, Collections.emptyMap());

//Abort txn.
persistentSubscription.abortTxn(txnID1, consumerMock);

positions.clear();
positions.add(new PositionImpl(2, 50));

// Retry above ack, will succeed. As abort has clear pending_ack for those messages.
persistentSubscription.acknowledgeMessage(txnID2, positions, AckType.Cumulative);

positions.clear();
positions.add(new PositionImpl(2, 1));

persistentSubscription.acknowledgeMessage(txnID2, positions, AckType.Individual);
}
}
2 changes: 1 addition & 1 deletion pulsar-transaction/common/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -69,4 +69,4 @@
</plugin>
</plugins>
</build>
</project>
</project>
Loading

0 comments on commit e66ba27

Please sign in to comment.