You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by iv...@apache.org on 2012/06/28 19:19:47 UTC
svn commit: r1355079 - in /zookeeper/bookkeeper/trunk: ./
hedwig-server/src/main/java/org/apache/hedwig/server/persistence/
hedwig-server/src/test/java/org/apache/hedwig/server/persistence/
Author: ivank
Date: Thu Jun 28 17:19:46 2012
New Revision: 1355079
URL: http://svn.apache.org/viewvc?rev=1355079&view=rev
Log:
BOOKKEEPER-302: No more messages delivered when hub server scans messages over two ledgers. (sijie via ivank)
Added:
zookeeper/bookkeeper/trunk/hedwig-server/src/test/java/org/apache/hedwig/server/persistence/TestBookKeeperPersistenceManager.java
Modified:
zookeeper/bookkeeper/trunk/CHANGES.txt
zookeeper/bookkeeper/trunk/hedwig-server/src/main/java/org/apache/hedwig/server/persistence/BookkeeperPersistenceManager.java
Modified: zookeeper/bookkeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/CHANGES.txt?rev=1355079&r1=1355078&r2=1355079&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/CHANGES.txt (original)
+++ zookeeper/bookkeeper/trunk/CHANGES.txt Thu Jun 28 17:19:46 2012
@@ -30,6 +30,10 @@ Trunk (unreleased changes)
BOOKKEEPER-274: Hedwig cpp client library should not link to cppunit which is just used for test. (sijie via ivank)
+ hedwig-server:
+
+ BOOKKEEPER-302: No more messages delivered when hub server scans messages over two ledgers. (sijie via ivank)
+
IMPROVEMENTS:
Release 4.1.0 - 2012-06-07
Modified: zookeeper/bookkeeper/trunk/hedwig-server/src/main/java/org/apache/hedwig/server/persistence/BookkeeperPersistenceManager.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/hedwig-server/src/main/java/org/apache/hedwig/server/persistence/BookkeeperPersistenceManager.java?rev=1355079&r1=1355078&r2=1355079&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/hedwig-server/src/main/java/org/apache/hedwig/server/persistence/BookkeeperPersistenceManager.java (original)
+++ zookeeper/bookkeeper/trunk/hedwig-server/src/main/java/org/apache/hedwig/server/persistence/BookkeeperPersistenceManager.java Thu Jun 28 17:19:46 2012
@@ -173,13 +173,15 @@ public class BookkeeperPersistenceManage
long startSeqIdToScan;
public RangeScanOp(RangeScanRequest request) {
- this(request, -1L);
+ this(request, -1L, 0, 0L);
}
- public RangeScanOp(RangeScanRequest request, long startSeqId) {
+ public RangeScanOp(RangeScanRequest request, long startSeqId, int numMessagesRead, long totalSizeRead) {
queuer.super(request.topic);
this.request = request;
this.startSeqIdToScan = startSeqId;
+ this.numMessagesRead = numMessagesRead;
+ this.totalSizeRead = totalSizeRead;
}
@Override
@@ -283,7 +285,7 @@ public class BookkeeperPersistenceManage
}
// continue scanning messages
- scanMessages(request, imlr.startSeqIdIncluded + entry.getEntryId() + 1);
+ scanMessages(request, imlr.startSeqIdIncluded + entry.getEntryId() + 1, numMessagesRead, totalSizeRead);
}
}, request.ctx);
}
@@ -317,8 +319,8 @@ public class BookkeeperPersistenceManage
queuer.pushAndMaybeRun(request.topic, new RangeScanOp(request));
}
- protected void scanMessages(RangeScanRequest request, long scanSeqId) {
- queuer.pushAndMaybeRun(request.topic, new RangeScanOp(request, scanSeqId));
+ protected void scanMessages(RangeScanRequest request, long scanSeqId, int numMsgsRead, long totalSizeRead) {
+ queuer.pushAndMaybeRun(request.topic, new RangeScanOp(request, scanSeqId, numMsgsRead, totalSizeRead));
}
public void deliveredUntil(ByteString topic, Long seqId) {
Added: zookeeper/bookkeeper/trunk/hedwig-server/src/test/java/org/apache/hedwig/server/persistence/TestBookKeeperPersistenceManager.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/hedwig-server/src/test/java/org/apache/hedwig/server/persistence/TestBookKeeperPersistenceManager.java?rev=1355079&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/hedwig-server/src/test/java/org/apache/hedwig/server/persistence/TestBookKeeperPersistenceManager.java (added)
+++ zookeeper/bookkeeper/trunk/hedwig-server/src/test/java/org/apache/hedwig/server/persistence/TestBookKeeperPersistenceManager.java Thu Jun 28 17:19:46 2012
@@ -0,0 +1,243 @@
+/**
+ * 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.hedwig.server.persistence;
+
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+
+import junit.framework.TestCase;
+
+import org.apache.hedwig.HelperMethods;
+import org.apache.hedwig.exceptions.PubSubException;
+import org.apache.hedwig.protocol.PubSubProtocol.Message;
+import org.apache.hedwig.server.common.ServerConfiguration;
+import org.apache.hedwig.server.topics.TopicManager;
+import org.apache.hedwig.server.topics.TrivialOwnAllTopicManager;
+import org.apache.hedwig.util.Callback;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.protobuf.ByteString;
+
+public class TestBookKeeperPersistenceManager extends TestCase {
+ static Logger logger = LoggerFactory.getLogger(TestPersistenceManagerBlackBox.class);
+
+ BookKeeperTestBase bktb;
+ private final int numBookies = 3;
+ private final long readDelay = 2000L;
+
+ ServerConfiguration conf;
+ ScheduledExecutorService scheduler;
+
+ TopicManager tm;
+ BookkeeperPersistenceManager manager;
+ PubSubException failureException = null;
+
+ @Override
+ @Before
+ protected void setUp() throws Exception {
+ super.setUp();
+
+ // delay read response for 2s.
+ bktb = new BookKeeperTestBase(numBookies, readDelay);
+ bktb.setUp();
+
+ conf = new ServerConfiguration();
+ org.apache.bookkeeper.conf.ClientConfiguration bkClientConf =
+ new org.apache.bookkeeper.conf.ClientConfiguration();
+ bkClientConf.setNumWorkerThreads(1).setReadTimeout(9999)
+ .setThrottleValue(3);
+ conf.addConf(bkClientConf);
+
+ scheduler = Executors.newScheduledThreadPool(1);
+ tm = new TrivialOwnAllTopicManager(conf, scheduler);
+ manager = new BookkeeperPersistenceManager(bktb.bk, bktb.getZooKeeperClient(), tm, conf, scheduler);
+ }
+
+ @Override
+ @After
+ protected void tearDown() throws Exception {
+ tm.stop();
+ scheduler.shutdown();
+ bktb.tearDown();
+ super.tearDown();
+ }
+
+ class RangeScanVerifier implements ScanCallback {
+ LinkedList<Message> pubMsgs;
+ boolean runNextScan = false;
+ RangeScanRequest nextScan = null;
+
+ public RangeScanVerifier(LinkedList<Message> pubMsgs, RangeScanRequest nextScan) {
+ this.pubMsgs = pubMsgs;
+ this.nextScan = nextScan;
+ }
+
+ @Override
+ public void messageScanned(Object ctx, Message recvMessage) {
+ if (null != nextScan && !runNextScan) {
+ runNextScan = true;
+ manager.scanMessages(nextScan);
+ }
+
+ if (pubMsgs.size() == 0) {
+ return;
+ }
+
+ Message pubMsg = pubMsgs.removeFirst();
+ if (!HelperMethods.areEqual(recvMessage, pubMsg)) {
+ fail("Scanned message not equal to expected");
+ }
+ }
+
+ @Override
+ public void scanFailed(Object ctx, Exception exception) {
+ fail("Failed to scan messages.");
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public void scanFinished(Object ctx, ReasonForFinish reason) {
+ LinkedBlockingQueue<Boolean> statusQueue = (LinkedBlockingQueue<Boolean>) ctx;
+ try {
+ statusQueue.put(true);
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ }
+
+ private LinkedList<Message> subMessages(List<Message> msgs, int start, int end) {
+ LinkedList<Message> result = new LinkedList<Message>();
+ for (int i=start; i<=end; i++) {
+ result.add(msgs.get(i));
+ }
+ return result;
+ }
+
+ @Test
+ public void testScanMessagesOnTwoLedgers() throws Exception {
+ ByteString topic = ByteString.copyFromUtf8("TestScanMessagesOnTwoLedgers");
+
+ List<Message> msgs = new ArrayList<Message>();
+
+ acquireTopic(topic);
+ msgs.addAll(publishMessages(topic, 1));
+ releaseTopic(topic);
+
+ // acquire topic again to force a new ledger
+ acquireTopic(topic);
+ msgs.addAll(publishMessages(topic, 3));
+
+ // scan messages
+ LinkedBlockingQueue<Boolean> statusQueue = new LinkedBlockingQueue<Boolean>();
+ RangeScanRequest nextScan = new RangeScanRequest(topic, 3, 2, Long.MAX_VALUE,
+ new RangeScanVerifier(subMessages(msgs, 2, 3), null), statusQueue);
+ manager.scanMessages(new RangeScanRequest(topic, 1, 2, Long.MAX_VALUE,
+ new RangeScanVerifier(subMessages(msgs, 0, 1), nextScan), statusQueue));
+ Boolean b = statusQueue.poll(10 * readDelay, TimeUnit.MILLISECONDS);
+ if (b == null) {
+ fail("One scan request doesn't finish");
+ }
+ b = statusQueue.poll(10 * readDelay, TimeUnit.MILLISECONDS);
+ if (b == null) {
+ fail("One scan request doesn't finish");
+ }
+ }
+
+ class TestCallback implements Callback<Long> {
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public void operationFailed(Object ctx, PubSubException exception) {
+ LinkedBlockingQueue<Boolean> statusQueue = (LinkedBlockingQueue<Boolean>) ctx;
+ try {
+ statusQueue.put(false);
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public void operationFinished(Object ctx, Long resultOfOperation) {
+ LinkedBlockingQueue<Boolean> statusQueue = (LinkedBlockingQueue<Boolean>) ctx;
+ try {
+ statusQueue.put(true);
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ }
+
+ protected List<Message> publishMessages(ByteString topic, int numMsgs) throws Exception {
+ List<Message> msgs = HelperMethods.getRandomPublishedMessages(numMsgs, 1024);
+ LinkedBlockingQueue<Boolean> statusQueue = new LinkedBlockingQueue<Boolean>();
+ for (Message msg : msgs) {
+
+ try {
+ manager.persistMessage(new PersistRequest(topic, msg, new TestCallback(), statusQueue));
+ // wait a maximum of a minute
+ Boolean b = statusQueue.poll(60, TimeUnit.SECONDS);
+ if (b == null) {
+ throw new RuntimeException("Publish timed out");
+ }
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ return msgs;
+ }
+
+ protected void acquireTopic(ByteString topic) throws Exception {
+ Semaphore latch = new Semaphore(1);
+ latch.acquire();
+ manager.acquiredTopic(topic, new Callback<Void>() {
+ @Override
+ public void operationFinished(Object ctx, Void resultOfOperation) {
+ failureException = null;
+ ((Semaphore)ctx).release();
+ }
+ @Override
+ public void operationFailed(Object ctx, PubSubException exception) {
+ failureException = exception;
+ ((Semaphore)ctx).release();
+ }
+ }, latch);
+ latch.acquire();
+ latch.release();
+ if (null != failureException) {
+ throw failureException;
+ }
+ }
+
+ protected void releaseTopic(ByteString topic) throws Exception {
+ manager.lostTopic(topic);
+ }
+
+}