Author: phunt
Date: Fri Apr 17 18:11:43 2009
New Revision: 766107
URL: http://svn.apache.org/viewvc?rev=766107&view=rev
Log:
ZOOKEEPER-367. RecoveryTest failure - 'unreasonable length' IOException
Added:
hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/
hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/
hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/log.1 (with props)
hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/log.274 (with props)
hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/log.63b (with props)
hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/snapshot.0 (with props)
hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/snapshot.272 (with props)
hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/snapshot.639 (with props)
hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/snapshot.83f (with props)
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/InvalidSnapshotTest.java
(with props)
Modified:
hadoop/zookeeper/trunk/CHANGES.txt
hadoop/zookeeper/trunk/build.xml
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/BinaryInputArchive.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/persistence/FileSnap.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/util/SerializeUtils.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/CRCTest.java
Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=766107&r1=766106&r2=766107&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Fri Apr 17 18:11:43 2009
@@ -37,22 +37,26 @@
ZOOKEEPER-360. WeakHashMap in Bookie.java causes NPE (flavio via mahadev)
ZOOKEEPER-362. Issues with FLENewEpochTest. (fix bug in Fast leader election)
-(flavio via mahadev)
+ (flavio via mahadev)
ZOOKEEPER-363. NPE when recovering ledger with no hint. (flavio via mahadev)
- ZOOKEEPER-370. Fix critical problems reported by findbugs. (flavio via mahadev)
+ ZOOKEEPER-370. Fix critical problems reported by findbugs.
+ (flavio via mahadev)
ZOOKEEPER-347. zkfuse uses non-standard String. (patrick hunt via mahadev)
- ZOOKEEPER-355. make validatePath non public in Zookeeper client api. (phunt
-via mahadev)
+ ZOOKEEPER-355. make validatePath non public in Zookeeper client api.
+ (phunt via mahadev)
ZOOKEEPER-374. Uninitialized struct variable in C causes warning which
-is treated as an error (phunt via mahadev)
+ is treated as an error (phunt via mahadev)
ZOOKEEPER-337. improve logging in leader election lookForLeader method when
-address resolution fails (phunt via mahadev)
+ address resolution fails (phunt via mahadev)
+
+ ZOOKEEPER-367. RecoveryTest failure - "unreasonable length" IOException
+ (mahadev via phunt)
IMPROVEMENTS:
ZOOKEEPER-308. improve the atomic broadcast performance 3x.
@@ -66,14 +70,14 @@
ZOOKEEPER-351. to run checkstyle (giridharan kesavan via mahadev)
- ZOOKEEPER-350. to run rats for releaseaudit. (giridharan kesavan via
-mahadev)
+ ZOOKEEPER-350. to run rats for releaseaudit.
+ (giridharan kesavan via mahadev)
ZOOKEEPER-352. to add standard ant targets required by test-patch.sh script
-(giridharan kesavan via mahadev)
+ (giridharan kesavan via mahadev)
- ZOOKEEPER-353. javadoc warnings needs to be fixed. (giridharan kesavan via
-mahadev)
+ ZOOKEEPER-353. javadoc warnings needs to be fixed.
+ (giridharan kesavan via mahadev)
ZOOKEEPER-354. to fix javadoc warning in the source files. (mahadev)
@@ -82,16 +86,16 @@
ZOOKEEPER-288. Cleanup and fixes to BookKeeper (flavio via mahadev)
ZOOKEEPER-305. Replace timers with semaphores in FLENewEpochTest.
-(flavio via mahadev)
+ (flavio via mahadev)
- ZOOKEEPER-60. Get cppunit tests running as part of Hudson CI. (girish via
-mahadev)
+ ZOOKEEPER-60. Get cppunit tests running as part of Hudson CI.
+ (girish via mahadev)
ZOOKEEPER-343. add tests that specifically verify the zkmain and
-qpmain classes. (phunt via mahadev)
+ qpmain classes. (phunt via mahadev)
- ZOOKEEPER-361. integrate cppunit testing as part of hudson patch
-process. (giri via mahadev)
+ ZOOKEEPER-361. integrate cppunit testing as part of hudson patch process.
+ (giri via mahadev)
NEW FEATURES:
Modified: hadoop/zookeeper/trunk/build.xml
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/build.xml?rev=766107&r1=766106&r2=766107&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/build.xml (original)
+++ hadoop/zookeeper/trunk/build.xml Fri Apr 17 18:11:43 2009
@@ -55,6 +55,7 @@
<property name="test.log.dir" value="${test.build.dir}/logs" />
<property name="test.data.dir" value="${test.build.dir}/data" />
<property name="test.data.upgrade.dir" value="${test.data.dir}/upgrade" />
+ <property name="test.data.invalid.dir" value="${test.data.dir}/invalidsnap" />
<property name="test.cppunit.dir" value="${test.build.dir}/test-cppunit"/>
<property name="test.tmp.dir" value="${test.build.dir}/tmp" />
<property name="test.output" value="no" />
@@ -534,14 +535,20 @@
<delete dir="${test.log.dir}" />
<delete dir="${test.tmp.dir}" />
<delete dir="${test.data.upgrade.dir}" />
+ <delete dir="${test.data.invalid.dir}" />
<delete dir="${test.data.dir}" />
<mkdir dir="${test.log.dir}" />
<mkdir dir="${test.tmp.dir}" />
<mkdir dir="${test.data.dir}" />
<mkdir dir="${test.data.upgrade.dir}" />
+ <mkdir dir="${test.data.invalid.dir}" />
<copy todir="${test.data.upgrade.dir}">
<fileset dir="${basedir}/src/java/test/data/upgrade"/>
- </copy>
+ </copy>
+ <copy todir="${test.data.invalid.dir}">
+ <fileset dir="${basedir}/src/java/test/data/invalidsnap"/>
+ </copy>
+
</target>
<target name="junit.run">
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/jute/BinaryInputArchive.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/jute/BinaryInputArchive.java?rev=766107&r1=766106&r2=766107&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/jute/BinaryInputArchive.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/jute/BinaryInputArchive.java Fri Apr 17
18:11:43 2009
@@ -98,7 +98,7 @@
int len = readInt(tag);
if (len == -1) return null;
if (len < 0 || len > maxBuffer) {
- throw new RuntimeException("Unreasonable length = " + len);
+ throw new IOException("Unreasonable length = " + len);
}
byte[] arr = new byte[len];
in.readFully(arr);
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/persistence/FileSnap.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/persistence/FileSnap.java?rev=766107&r1=766106&r2=766107&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/persistence/FileSnap.java
(original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/persistence/FileSnap.java
Fri Apr 17 18:11:43 2009
@@ -64,22 +64,44 @@
*/
public long deserialize(DataTree dt, Map<Long, Integer> sessions)
throws IOException {
- File snap = findMostRecentSnapshot();
- if (snap == null) {
+ // we run through 100 snapshots (not all of them)
+ // if we cannot get it running within 100 snapshots
+ // we should give up
+ List<File> snapList = findNValidSnapshots(100);
+ if (snapList.size() == 0) {
return -1L;
}
- LOG.info("Reading snapshot " + snap);
- InputStream snapIS = new BufferedInputStream(new FileInputStream(snap));
- CheckedInputStream crcIn = new CheckedInputStream(snapIS, new Adler32());
- InputArchive ia=BinaryInputArchive.getArchive(crcIn);
- deserialize(dt,sessions, ia);
- long checkSum = crcIn.getChecksum().getValue();
- long val = ia.readLong("val");
- if (val != checkSum) {
- throw new IOException("CRC corruption in snapshot : " + snap);
+ File snap = null;
+ boolean foundValid = false;
+ for (int i = 0; i < snapList.size(); i++) {
+ snap = snapList.get(i);
+ InputStream snapIS = null;
+ CheckedInputStream crcIn = null;
+ try {
+ LOG.info("Reading snapshot " + snap);
+ snapIS = new BufferedInputStream(new FileInputStream(snap));
+ crcIn = new CheckedInputStream(snapIS, new Adler32());
+ InputArchive ia = BinaryInputArchive.getArchive(crcIn);
+ deserialize(dt,sessions, ia);
+ long checkSum = crcIn.getChecksum().getValue();
+ long val = ia.readLong("val");
+ if (val != checkSum) {
+ throw new IOException("CRC corruption in snapshot : " + snap);
+ }
+ foundValid = true;
+ break;
+ } catch(IOException e) {
+ LOG.warn("problem reading snap file " + snap, e);
+ } finally {
+ if (snapIS != null)
+ snapIS.close();
+ if (crcIn != null)
+ crcIn.close();
+ }
+ }
+ if (!foundValid) {
+ throw new IOException("Not able to find valid snapshots in " + snapDir);
}
- snapIS.close();
- crcIn.close();
dt.lastProcessedZxid = Util.getZxidFromName(snap.getName(), "snapshot");
return dt.lastProcessedZxid;
}
@@ -91,7 +113,7 @@
* @param ia the input archive to restore from
* @throws IOException
*/
- protected void deserialize(DataTree dt, Map<Long, Integer> sessions,
+ public void deserialize(DataTree dt, Map<Long, Integer> sessions,
InputArchive ia) throws IOException {
FileHeader header = new FileHeader();
header.deserialize(ia, "fileheader");
@@ -108,23 +130,51 @@
* @return the file containing the most recent snapshot
*/
public File findMostRecentSnapshot() throws IOException {
- List<File> files = Util.sortDataDir(snapDir.listFiles(), "snapshot", false);
+ List<File> files = findNValidSnapshots(1);
+ if (files.size() == 0) {
+ return null;
+ }
+ return files.get(0);
+ }
+
+ /**
+ * find the last (maybe) valid n snapshots. this does some
+ * minor checks on the validity of the snapshots. It just
+ * checks for / at the end of the snapshot. This does
+ * not mean that the snapshot is truly valid but is
+ * valid with a high probability. also, the most recent
+ * will be first on the list.
+ * @param n the number of most recent snapshots
+ * @return the last n snapshots (the number might be
+ * less than n in case enough snapshots are not available).
+ * @throws IOException
+ */
+ private List<File> findNValidSnapshots(int n) throws IOException {
+ List<File> files = Util.sortDataDir(snapDir.listFiles(),"snapshot", false);
+ int count = 0;
+ List<File> list = new ArrayList<File>();
for (File f : files) {
- // we should catch the exceptions from
- // the valid snapshot and continue
+ // we should catch the exceptions
+ // from the valid snapshot and continue
// until we find a valid one
try {
- if(Util.isValidSnapshot(f))
- return f;
- } catch(IOException e) {
+ if (Util.isValidSnapshot(f)) {
+ list.add(f);
+ count++;
+ if (count == n) {
+ break;
+ }
+ }
+ } catch (IOException e) {
LOG.info("invalid snapshot " + f, e);
}
}
- return null;
+ return list;
}
-
+
/**
- * find the last n snapshots.
+ * find the last n snapshots. this does not have
+ * any checks if the snapshot might be valid or not
* @param the number of most recent snapshots
* @return the last n snapshots
* @throws IOException
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/util/SerializeUtils.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/util/SerializeUtils.java?rev=766107&r1=766106&r2=766107&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/util/SerializeUtils.java
(original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/util/SerializeUtils.java
Fri Apr 17 18:11:43 2009
@@ -78,25 +78,17 @@
public static void deserializeSnapshot(DataTree dt,InputArchive ia,
Map<Long, Integer> sessions) throws IOException {
- try {
- int count = ia.readInt("count");
- while (count > 0) {
- long id = ia.readLong("id");
- int to = ia.readInt("timeout");
- sessions.put(id, to);
- ZooTrace.logTraceMessage(LOG, ZooTrace.SESSION_TRACE_MASK,
- "loadData --- session in archive: " + id
- + " with timeout: " + to);
- count--;
- }
- dt.deserialize(ia, "tree");
- } catch(IOException e) {
- throw e;
- } catch(Exception e) {
- IOException ioe = new IOException(e.getMessage());
- ioe.initCause(e);
- throw ioe;
+ int count = ia.readInt("count");
+ while (count > 0) {
+ long id = ia.readLong("id");
+ int to = ia.readInt("timeout");
+ sessions.put(id, to);
+ ZooTrace.logTraceMessage(LOG, ZooTrace.SESSION_TRACE_MASK,
+ "loadData --- session in archive: " + id
+ + " with timeout: " + to);
+ count--;
}
+ dt.deserialize(ia, "tree");
}
public static void serializeSnapshot(DataTree dt,OutputArchive oa,
Added: hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/log.1
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/log.1?rev=766107&view=auto
==============================================================================
Binary file - no diff available.
Propchange: hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/log.1
------------------------------------------------------------------------------
svn:mime-type = application/octet-stream
Added: hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/log.274
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/log.274?rev=766107&view=auto
==============================================================================
Binary file - no diff available.
Propchange: hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/log.274
------------------------------------------------------------------------------
svn:mime-type = application/octet-stream
Added: hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/log.63b
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/log.63b?rev=766107&view=auto
==============================================================================
Binary file - no diff available.
Propchange: hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/log.63b
------------------------------------------------------------------------------
svn:mime-type = application/octet-stream
Added: hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/snapshot.0
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/snapshot.0?rev=766107&view=auto
==============================================================================
Binary file - no diff available.
Propchange: hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/snapshot.0
------------------------------------------------------------------------------
svn:mime-type = application/octet-stream
Added: hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/snapshot.272
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/snapshot.272?rev=766107&view=auto
==============================================================================
Binary file - no diff available.
Propchange: hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/snapshot.272
------------------------------------------------------------------------------
svn:mime-type = application/octet-stream
Added: hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/snapshot.639
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/snapshot.639?rev=766107&view=auto
==============================================================================
Binary file - no diff available.
Propchange: hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/snapshot.639
------------------------------------------------------------------------------
svn:mime-type = application/octet-stream
Added: hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/snapshot.83f
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/snapshot.83f?rev=766107&view=auto
==============================================================================
Binary file - no diff available.
Propchange: hadoop/zookeeper/trunk/src/java/test/data/invalidsnap/version-2/snapshot.83f
------------------------------------------------------------------------------
svn:mime-type = application/octet-stream
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/CRCTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/CRCTest.java?rev=766107&r1=766106&r2=766107&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/CRCTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/CRCTest.java Fri Apr
17 18:11:43 2009
@@ -20,15 +20,22 @@
import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
+import java.io.BufferedInputStream;
import java.io.File;
+import java.io.FileInputStream;
import java.io.IOException;
+import java.io.InputStream;
import java.io.RandomAccessFile;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CountDownLatch;
+import java.util.zip.Adler32;
+import java.util.zip.CheckedInputStream;
import junit.framework.TestCase;
+import org.apache.jute.BinaryInputArchive;
+import org.apache.jute.InputArchive;
import org.apache.log4j.Logger;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.WatchedEvent;
@@ -133,13 +140,15 @@
DataTree dt = new DataTree();
Map<Long, Integer> sessions =
new ConcurrentHashMap<Long, Integer>();
- try {
- snap.deserialize(dt, sessions);
- assertTrue(false);
- } catch(IOException ie) {
- LOG.info("checksu failure in snapshot", ie);
- }
-
+ InputStream snapIS = new BufferedInputStream(new FileInputStream(snapFile));
+ CheckedInputStream crcIn = new CheckedInputStream(snapIS, new Adler32());
+ InputArchive ia=BinaryInputArchive.getArchive(crcIn);
+ snap.deserialize(dt,sessions, ia);
+ long checkSum = crcIn.getChecksum().getValue();
+ long val = ia.readLong("val");
+ snapIS.close();
+ crcIn.close();
+ assertTrue(val != checkSum);
}
public void process(WatchedEvent event) {
Added: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/InvalidSnapshotTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/InvalidSnapshotTest.java?rev=766107&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/InvalidSnapshotTest.java
(added)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/InvalidSnapshotTest.java
Fri Apr 17 18:11:43 2009
@@ -0,0 +1,88 @@
+/**
+ * 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.zookeeper.test;
+
+import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
+
+import java.io.File;
+import java.util.concurrent.CountDownLatch;
+
+import junit.framework.TestCase;
+
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.WatchedEvent;
+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.SyncRequestProcessor;
+import org.apache.zookeeper.server.ZooKeeperServer;
+
+public class InvalidSnapshotTest extends TestCase implements Watcher {
+ private final static Logger LOG = Logger.getLogger(UpgradeTest.class);
+ private static String HOSTPORT = "127.0.0.1:2359";
+ ZooKeeperServer zks;
+ private static final File testData = new File(
+ System.getProperty("test.data.dir", "build/test/data"));
+ private CountDownLatch startSignal;
+
+ @Override
+ protected void setUp() throws Exception {
+ LOG.info("STARTING " + getName());
+ }
+ @Override
+ protected void tearDown() throws Exception {
+ LOG.info("FINISHED " + getName());
+ }
+
+ /**
+ * test the snapshot
+ * @throws Exception
+ */
+ public void testSnapshot() throws Exception {
+ File snapDir = new File(testData, "invalidsnap");
+ zks = new ZooKeeperServer(snapDir, snapDir, 3000);
+ SyncRequestProcessor.snapCount = 1000;
+ final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
+ NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(PORT);
+ f.startup(zks);
+ LOG.info("starting up the zookeeper server .. waiting");
+ assertTrue("waiting for server being up",
+ ClientBase.waitForServerUp(HOSTPORT,CONNECTION_TIMEOUT));
+ ZooKeeper zk = new ZooKeeper(HOSTPORT, 20000, this);
+ // we know this from the data files
+ // this node is the last node in the snapshot
+
+ assertTrue(zk.exists("/9/9/8", false) != null);
+ f.shutdown();
+ assertTrue("waiting for server down",
+ ClientBase.waitForServerDown(HOSTPORT,
+ ClientBase.CONNECTION_TIMEOUT));
+
+ }
+
+ public void process(WatchedEvent event) {
+ LOG.info("Event:" + event.getState() + " " + event.getType() + " " + event.getPath());
+ if (event.getState() == KeeperState.SyncConnected
+ && startSignal != null && startSignal.getCount() > 0)
+ {
+ startSignal.countDown();
+ }
+ }
+}
\ No newline at end of file
Propchange: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/InvalidSnapshotTest.java
------------------------------------------------------------------------------
svn:eol-style = native
|