Browse Source

ZOOKEEPER-121. SyncRequestProcessor is not closing log stream during shutdown (3rd attempt)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/zookeeper/trunk@923984 13f79535-47bb-0310-9956-ffa450edef68
Patrick D. Hunt 15 years ago
parent
commit
f4138613a1

+ 5 - 2
CHANGES.txt

@@ -279,8 +279,11 @@ BUGFIXES:
   ZOOKEEPER-698. intermittent JMX test failures due to not verifying QuorumPeer
   shutdown (phunt)
 
-  ZOOKEEPER-121. SyncRequestProcessor is not closing log stream during shutdown
-  (breed via mahadev)
+  ZOOKEEPER-121_2. SyncRequestProcessor is not closing log stream during
+  shutdown (breed via mahadev)
+
+  ZOOKEEPER-121_3. SyncRequestProcessor is not closing log stream during
+  shutdown (mahadev via phunt)
 
 IMPROVEMENTS:
   ZOOKEEPER-473. cleanup junit tests to eliminate false positives due to

+ 0 - 5
src/java/main/org/apache/zookeeper/server/SyncRequestProcessor.java

@@ -172,11 +172,6 @@ public class SyncRequestProcessor extends Thread implements RequestProcessor {
         } catch(InterruptedException e) {
             LOG.warn("Interrupted while wating for " + this + " to finish");
         }
-        try {
-            zks.getZKDatabase().close();
-        } catch (IOException ie) {
-            LOG.warn("Error closing logs ", ie);
-        }
         nextProcessor.shutdown();
     }
 

+ 1 - 1
src/java/main/org/apache/zookeeper/server/persistence/FileTxnLog.java

@@ -156,7 +156,7 @@ public class FileTxnLog implements TxnLog {
      * rollover the current log file to a new one.
      * @throws IOException
      */
-    public void rollLog() throws IOException {
+    public synchronized void rollLog() throws IOException {
         if (logStream != null) {
             this.logStream.flush();
             this.logStream = null;

+ 5 - 0
src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java

@@ -672,6 +672,11 @@ public class QuorumPeer extends Thread implements QuorumStats.Provider {
 
     public void shutdown() {
         running = false;
+        try {
+            zkDb.close();
+        } catch (IOException ie) {
+            LOG.warn("Error closing logs ", ie);
+        }
         if (leader != null) {
             leader.shutdown("quorum Peer shutdown");
         }

+ 9 - 2
src/java/test/org/apache/zookeeper/test/ClientBase.java

@@ -50,6 +50,7 @@ import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.apache.zookeeper.server.NIOServerCnxn;
+import org.apache.zookeeper.server.ZKDatabase;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.persistence.FileTxnLog;
 
@@ -344,9 +345,15 @@ public abstract class ClientBase extends TestCase {
 
     static void shutdownServerInstance(NIOServerCnxn.Factory factory,
             String hostPort)
-    {
-        if (factory != null) {
+    {    	
+    	if (factory != null) {
+    	    ZKDatabase zkDb = factory.getZooKeeperServer().getZKDatabase();
             factory.shutdown();
+            try {
+                zkDb.close();
+            } catch (IOException ie) {
+                LOG.warn("Error closing logs ", ie);
+       	 	}
             final int PORT = getPort(hostPort);
 
             assertTrue("waiting for server down",

+ 8 - 1
src/java/test/org/apache/zookeeper/test/TruncateTest.java

@@ -34,6 +34,7 @@ import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.server.NIOServerCnxn;
+import org.apache.zookeeper.server.ZKDatabase;
 import org.apache.zookeeper.server.quorum.QuorumPeer;
 import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer;
 import org.junit.After;
@@ -75,8 +76,14 @@ public class TruncateTest extends TestCase {
         for(int i = 0; i < 50; i++) {
             zk.create("/" + i, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
         }
-        factory.shutdown();
         zk.close();
+        ZKDatabase zkDb = factory.getZooKeeperServer().getZKDatabase();
+        factory.shutdown();
+        try {
+            zkDb.close();
+        } catch (IOException ie) {
+            LOG.warn("Error closing logs ", ie);
+        }
         int tickTime = 2000;
         int initLimit = 3;
         int syncLimit = 3;