Repository: kafka
Updated Branches:
  refs/heads/trunk 04b0d870b -> 78685dc16


http://git-wip-us.apache.org/repos/asf/kafka/blob/78685dc1/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 
b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala
index 39a6852..f846698 100755
--- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala
+++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala
@@ -25,7 +25,6 @@ import kafka.utils.TestUtils._
 import kafka.zk.ZooKeeperTestHarness
 
 import org.junit.{After, Before, Test}
-import org.scalatest.junit.JUnit3Suite
 
 import java.util.Properties
 import java.io.File
@@ -33,9 +32,9 @@ import java.io.File
 import scala.util.Random
 import scala.collection._
 
-import junit.framework.Assert._
+import org.junit.Assert._
 
-class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness {
+class OffsetCommitTest extends ZooKeeperTestHarness {
   val random: Random = new Random()
   val group = "test-group"
   val retentionCheckInterval: Long = 100L

http://git-wip-us.apache.org/repos/asf/kafka/blob/78685dc1/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala 
b/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala
index a3a03db..dead087 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala
@@ -17,7 +17,7 @@
 
 package kafka.server
 
-import org.scalatest.junit.JUnit3Suite
+import org.junit.{After, Before}
 import kafka.zk.ZooKeeperTestHarness
 import kafka.utils.TestUtils._
 import kafka.producer.KeyedMessage
@@ -25,11 +25,12 @@ import kafka.serializer.StringEncoder
 import kafka.utils.{TestUtils}
 import kafka.common._
 
-class ReplicaFetchTest extends JUnit3Suite with ZooKeeperTestHarness  {
+class ReplicaFetchTest extends ZooKeeperTestHarness  {
   var brokers: Seq[KafkaServer] = null
   val topic1 = "foo"
   val topic2 = "bar"
 
+  @Before
   override def setUp() {
     super.setUp()
     brokers = createBrokerConfigs(2, zkConnect, false)
@@ -37,6 +38,7 @@ class ReplicaFetchTest extends JUnit3Suite with 
ZooKeeperTestHarness  {
       .map(config => TestUtils.createServer(config))
   }
 
+  @After
   override def tearDown() {
     brokers.foreach(_.shutdown())
     super.tearDown()

http://git-wip-us.apache.org/repos/asf/kafka/blob/78685dc1/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala 
b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
index 00d5933..3770cb4 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
@@ -27,12 +27,11 @@ import java.io.File
 import org.apache.kafka.common.protocol.Errors
 import org.easymock.EasyMock
 import org.I0Itec.zkclient.ZkClient
-import org.scalatest.junit.JUnit3Suite
 import org.junit.Test
 
 import scala.collection.Map
 
-class ReplicaManagerTest extends JUnit3Suite {
+class ReplicaManagerTest {
 
   val topic = "test-topic"
 
@@ -84,7 +83,7 @@ class ReplicaManagerTest extends JUnit3Suite {
 
     rm.appendMessages(timeout = 0, requiredAcks = 3, internalTopicsAllowed = 
false, messagesPerPartition = produceRequest.data, responseCallback = callback)
 
-    rm.shutdown(false);
+    rm.shutdown(false)
 
     TestUtils.verifyNonDaemonThreadsStatus
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/78685dc1/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala
----------------------------------------------------------------------
diff --git 
a/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala 
b/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala
index 12269cd..1185a6f 100755
--- a/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala
@@ -20,18 +20,18 @@ import java.util.Properties
 
 import kafka.zk.ZooKeeperTestHarness
 import kafka.utils.{TestUtils, CoreUtils}
-import org.junit.Test
-import org.scalatest.junit.JUnit3Suite
-import junit.framework.Assert._
+import org.junit.{Before, Test}
+import org.junit.Assert._
 import java.io.File
 
-class ServerGenerateBrokerIdTest extends JUnit3Suite with ZooKeeperTestHarness 
{
+class ServerGenerateBrokerIdTest extends ZooKeeperTestHarness {
   var props1: Properties = null
   var config1: KafkaConfig = null
   var props2: Properties = null
   var config2: KafkaConfig = null
   val brokerMetaPropsFile = "meta.properties"
 
+  @Before
   override def setUp() {
     super.setUp()
     props1 = TestUtils.createBrokerConfig(-1, zkConnect)

http://git-wip-us.apache.org/repos/asf/kafka/blob/78685dc1/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala 
b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala
index 95534e3..2a8da0c 100755
--- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala
@@ -27,18 +27,18 @@ import kafka.serializer.StringEncoder
 
 import java.io.File
 
-import org.junit.Test
-import org.scalatest.junit.JUnit3Suite
-import junit.framework.Assert._
+import org.junit.{Before, Test}
+import org.junit.Assert._
 
-class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness {
+class ServerShutdownTest extends ZooKeeperTestHarness {
   var config: KafkaConfig = null
   val host = "localhost"
   val topic = "test"
   val sent1 = List("hello", "there")
   val sent2 = List("more", "messages")
 
-  override def setUp(): Unit = {
+  @Before
+  override def setUp() {
     super.setUp()
     val props = TestUtils.createBrokerConfig(0, zkConnect)
     config = KafkaConfig.fromProps(props)

http://git-wip-us.apache.org/repos/asf/kafka/blob/78685dc1/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala 
b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala
index 60e10b3..7d986ad 100755
--- a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala
@@ -17,15 +17,14 @@
 
 package kafka.server
 
-import org.scalatest.junit.JUnit3Suite
 import kafka.utils.ZkUtils
 import kafka.utils.CoreUtils
 import kafka.utils.TestUtils
 
 import kafka.zk.ZooKeeperTestHarness
-import junit.framework.Assert._
+import org.junit.Assert._
 
-class ServerStartupTest extends JUnit3Suite with ZooKeeperTestHarness {
+class ServerStartupTest extends ZooKeeperTestHarness {
 
   def testBrokerCreatesZKChroot {
     val brokerId = 0

http://git-wip-us.apache.org/repos/asf/kafka/blob/78685dc1/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 
b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
index 09a0961..d950665 100644
--- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
+++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
@@ -22,18 +22,17 @@ import kafka.cluster.Replica
 import kafka.common.TopicAndPartition
 import kafka.log.Log
 import kafka.message.{MessageSet, ByteBufferMessageSet, Message}
+import org.junit.{After, Before}
 
-import scala.Some
 import java.util.{Properties, Collections}
 import java.util.concurrent.atomic.AtomicBoolean
 import collection.JavaConversions._
 
 import org.easymock.EasyMock
 import org.I0Itec.zkclient.ZkClient
-import org.scalatest.junit.JUnit3Suite
-import junit.framework.Assert._
+import org.junit.Assert._
 
-class SimpleFetchTest extends JUnit3Suite {
+class SimpleFetchTest {
 
   val replicaLagTimeMaxMs = 100L
   val replicaFetchWaitMaxMs = 100
@@ -63,9 +62,8 @@ class SimpleFetchTest extends JUnit3Suite {
 
   var replicaManager: ReplicaManager = null
 
-  override def setUp() {
-    super.setUp()
-
+  @Before
+  def setUp() {
     // create nice mock since we don't particularly care about zkclient calls
     val zkClient = EasyMock.createNiceMock(classOf[ZkClient])
     EasyMock.replay(zkClient)
@@ -117,9 +115,9 @@ class SimpleFetchTest extends JUnit3Suite {
     partition.inSyncReplicas = allReplicas.toSet
   }
 
-  override def tearDown() {
+  @After
+  def tearDown() {
     replicaManager.shutdown(false)
-    super.tearDown()
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/kafka/blob/78685dc1/core/src/test/scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala
----------------------------------------------------------------------
diff --git 
a/core/src/test/scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala 
b/core/src/test/scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala
index fd8cf7b..4a070bd 100644
--- a/core/src/test/scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala
+++ b/core/src/test/scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala
@@ -19,7 +19,7 @@ package kafka.utils
 
 import java.util.concurrent.TimeUnit
 
-import junit.framework.Assert._
+import org.junit.Assert._
 import org.junit.Test
 
 class ByteBoundedBlockingQueueTest {

http://git-wip-us.apache.org/repos/asf/kafka/blob/78685dc1/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala 
b/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala
index 6380b6e..068526e 100644
--- a/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala
+++ b/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala
@@ -17,7 +17,7 @@
 
 package kafka.utils
 
-import junit.framework.Assert._
+import org.junit.Assert._
 import org.junit.Test
 
 class CommandLineUtilsTest {

http://git-wip-us.apache.org/repos/asf/kafka/blob/78685dc1/core/src/test/scala/unit/kafka/utils/IteratorTemplateTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/utils/IteratorTemplateTest.scala 
b/core/src/test/scala/unit/kafka/utils/IteratorTemplateTest.scala
index fbd245c..56f5905 100644
--- a/core/src/test/scala/unit/kafka/utils/IteratorTemplateTest.scala
+++ b/core/src/test/scala/unit/kafka/utils/IteratorTemplateTest.scala
@@ -16,7 +16,7 @@
  */
 package kafka.utils
 
-import junit.framework.Assert._
+import org.junit.Assert._
 import org.scalatest.Assertions
 import org.junit.{Test, After, Before}
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/78685dc1/core/src/test/scala/unit/kafka/utils/JsonTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/utils/JsonTest.scala 
b/core/src/test/scala/unit/kafka/utils/JsonTest.scala
index 93550e8..6c8ed97 100644
--- a/core/src/test/scala/unit/kafka/utils/JsonTest.scala
+++ b/core/src/test/scala/unit/kafka/utils/JsonTest.scala
@@ -16,7 +16,7 @@
  */
 package kafka.utils
 
-import junit.framework.Assert._
+import org.junit.Assert._
 import org.junit.{Test, After, Before}
 
 class JsonTest {

http://git-wip-us.apache.org/repos/asf/kafka/blob/78685dc1/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala 
b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala
index b9de8d6..b3835f0 100644
--- a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala
+++ b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala
@@ -22,13 +22,12 @@ import kafka.server.{ReplicaFetcherManager, KafkaConfig}
 import kafka.api.LeaderAndIsr
 import kafka.zk.ZooKeeperTestHarness
 import kafka.common.TopicAndPartition
-import org.scalatest.junit.JUnit3Suite
 import org.junit.Assert._
-import org.junit.Test
+import org.junit.{Before, Test}
 import org.easymock.EasyMock
 
 
-class ReplicationUtilsTest extends JUnit3Suite with ZooKeeperTestHarness {
+class ReplicationUtilsTest extends ZooKeeperTestHarness {
   val topic = "my-topic-test"
   val partitionId = 0
   val brokerId = 1
@@ -45,7 +44,7 @@ class ReplicationUtilsTest extends JUnit3Suite with 
ZooKeeperTestHarness {
 
   val topicDataLeaderIsrAndControllerEpoch = 
LeaderIsrAndControllerEpoch(LeaderAndIsr(1,leaderEpoch,List(1,2),0), 
controllerEpoch)
 
-
+  @Before
   override def setUp() {
     super.setUp()
     ZkUtils.createPersistentPath(zkClient,topicPath,topicData)

http://git-wip-us.apache.org/repos/asf/kafka/blob/78685dc1/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala 
b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala
index cfea63b..7c131fc 100644
--- a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala
+++ b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala
@@ -16,7 +16,7 @@
  */
 package kafka.utils
 
-import junit.framework.Assert._
+import org.junit.Assert._
 import java.util.concurrent.atomic._
 import org.junit.{Test, After, Before}
 import kafka.utils.TestUtils.retry

http://git-wip-us.apache.org/repos/asf/kafka/blob/78685dc1/core/src/test/scala/unit/kafka/utils/TestUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala 
b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
index eb169d8..f00f00a 100755
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -43,8 +43,7 @@ import kafka.admin.AdminUtils
 import kafka.producer.ProducerConfig
 import kafka.log._
 
-import junit.framework.AssertionFailedError
-import junit.framework.Assert._
+import org.junit.Assert._
 import org.apache.kafka.clients.producer.KafkaProducer
 
 import scala.collection.Map
@@ -595,7 +594,7 @@ object TestUtils extends Logging {
         block
         return
       } catch {
-        case e: AssertionFailedError =>
+        case e: AssertionError =>
           val ellapsed = System.currentTimeMillis - startTime
           if(ellapsed > maxWaitMs) {
             throw e

http://git-wip-us.apache.org/repos/asf/kafka/blob/78685dc1/core/src/test/scala/unit/kafka/utils/timer/TimerTaskListTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/utils/timer/TimerTaskListTest.scala 
b/core/src/test/scala/unit/kafka/utils/timer/TimerTaskListTest.scala
index 052aecd..a018dde 100644
--- a/core/src/test/scala/unit/kafka/utils/timer/TimerTaskListTest.scala
+++ b/core/src/test/scala/unit/kafka/utils/timer/TimerTaskListTest.scala
@@ -16,7 +16,7 @@
  */
 package kafka.utils.timer
 
-import junit.framework.Assert._
+import org.junit.Assert._
 import java.util.concurrent.atomic._
 import org.junit.{Test, After, Before}
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/78685dc1/core/src/test/scala/unit/kafka/utils/timer/TimerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/utils/timer/TimerTest.scala 
b/core/src/test/scala/unit/kafka/utils/timer/TimerTest.scala
index 8507592..95de378 100644
--- a/core/src/test/scala/unit/kafka/utils/timer/TimerTest.scala
+++ b/core/src/test/scala/unit/kafka/utils/timer/TimerTest.scala
@@ -18,7 +18,7 @@ package kafka.utils.timer
 
 import java.util.concurrent.{CountDownLatch, ExecutorService, Executors, 
TimeUnit}
 
-import junit.framework.Assert._
+import org.junit.Assert._
 import java.util.concurrent.atomic._
 import org.junit.{Test, After, Before}
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/78685dc1/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala 
b/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala
index 2be1619..247aa6e 100644
--- a/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala
+++ b/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala
@@ -18,13 +18,11 @@
 package kafka.zk
 
 import kafka.consumer.ConsumerConfig
-import org.I0Itec.zkclient.ZkClient
 import kafka.utils.ZkUtils
 import kafka.utils.TestUtils
 import org.junit.Assert
-import org.scalatest.junit.JUnit3Suite
 
-class ZKEphemeralTest extends JUnit3Suite with ZooKeeperTestHarness {
+class ZKEphemeralTest extends ZooKeeperTestHarness {
   var zkSessionTimeoutMs = 1000
 
   def testEphemeralNodeCleanup = {

http://git-wip-us.apache.org/repos/asf/kafka/blob/78685dc1/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala 
b/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala
index d3e44c6..35c635a 100644
--- a/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala
+++ b/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala
@@ -17,13 +17,12 @@
 
 package kafka.zk
 
-import junit.framework.Assert
 import kafka.consumer.ConsumerConfig
 import kafka.utils.{ZkPath, TestUtils, ZkUtils}
 import org.apache.kafka.common.config.ConfigException
-import org.scalatest.junit.JUnit3Suite
+import org.junit.Assert._
 
-class ZKPathTest extends JUnit3Suite with ZooKeeperTestHarness {
+class ZKPathTest extends ZooKeeperTestHarness {
 
   val path: String = "/some_dir"
   val zkSessionTimeoutMs = 1000
@@ -54,7 +53,7 @@ class ZKPathTest extends JUnit3Suite with 
ZooKeeperTestHarness {
       case exception: Throwable => fail("Failed to create persistent path")
     }
 
-    Assert.assertTrue("Failed to create persistent path", 
ZkUtils.pathExists(zkClient, path))
+    assertTrue("Failed to create persistent path", 
ZkUtils.pathExists(zkClient, path))
   }
 
   def testMakeSurePersistsPathExistsThrowsException {
@@ -82,7 +81,7 @@ class ZKPathTest extends JUnit3Suite with 
ZooKeeperTestHarness {
       case exception: Throwable => fail("Failed to create persistent path")
     }
 
-    Assert.assertTrue("Failed to create persistent path", 
ZkUtils.pathExists(zkClient, path))
+    assertTrue("Failed to create persistent path", 
ZkUtils.pathExists(zkClient, path))
   }
 
   def testCreateEphemeralPathThrowsException {
@@ -110,7 +109,7 @@ class ZKPathTest extends JUnit3Suite with 
ZooKeeperTestHarness {
       case exception: Throwable => fail("Failed to create ephemeral path")
     }
 
-    Assert.assertTrue("Failed to create ephemeral path", 
ZkUtils.pathExists(zkClient, path))
+    assertTrue("Failed to create ephemeral path", ZkUtils.pathExists(zkClient, 
path))
   }
 
   def testCreatePersistentSequentialThrowsException {
@@ -140,6 +139,6 @@ class ZKPathTest extends JUnit3Suite with 
ZooKeeperTestHarness {
       case exception: Throwable => fail("Failed to create persistent path")
     }
 
-    Assert.assertTrue("Failed to create persistent path", 
ZkUtils.pathExists(zkClient, actualPath))
+    assertTrue("Failed to create persistent path", 
ZkUtils.pathExists(zkClient, actualPath))
   }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/78685dc1/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala 
b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala
index 1f4d10d..e4bfb48 100755
--- a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala
+++ b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala
@@ -17,11 +17,12 @@
 
 package kafka.zk
 
-import org.scalatest.junit.JUnit3Suite
 import org.I0Itec.zkclient.ZkClient
 import kafka.utils.{ZkUtils, CoreUtils}
+import org.junit.{After, Before}
+import org.scalatest.junit.JUnitSuite
 
-trait ZooKeeperTestHarness extends JUnit3Suite {
+trait ZooKeeperTestHarness extends JUnitSuite {
   var zkPort: Int = -1
   var zookeeper: EmbeddedZookeeper = null
   var zkClient: ZkClient = null
@@ -30,17 +31,17 @@ trait ZooKeeperTestHarness extends JUnit3Suite {
 
   def zkConnect: String = "127.0.0.1:" + zkPort
 
-  override def setUp() {
-    super.setUp
+  @Before
+  def setUp() {
     zookeeper = new EmbeddedZookeeper()
     zkPort = zookeeper.port
     zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeout, 
zkConnectionTimeout)
   }
 
-  override def tearDown() {
+  @After
+  def tearDown() {
     CoreUtils.swallow(zkClient.close())
     CoreUtils.swallow(zookeeper.shutdown())
-    super.tearDown
   }
 
 }

Reply via email to