@ -19,7 +19,8 @@ package kafka.zookeeper
@@ -19,7 +19,8 @@ package kafka.zookeeper
import java.net.UnknownHostException
import java.nio.charset.StandardCharsets
import java.util.UUID
import java.util.concurrent. { CountDownLatch , TimeUnit }
import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent. { ArrayBlockingQueue , CountDownLatch , TimeUnit }
import javax.security.auth.login.Configuration
import kafka.zk.ZooKeeperTestHarness
@ -41,23 +42,23 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@@ -41,23 +42,23 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@Test ( expected = classOf [ UnknownHostException ] )
def testUnresolvableConnectString ( ) : Unit = {
new ZooKeeperClient ( "some.invalid.hostname.foo.bar.local" , - 1 , - 1 , null )
new ZooKeeperClient ( "some.invalid.hostname.foo.bar.local" , - 1 , - 1 , Int . MaxValue , null )
}
@Test ( expected = classOf [ ZooKeeperClientTimeoutException ] )
def testConnectionTimeout ( ) : Unit = {
zookeeper . shutdown ( )
new ZooKeeperClient ( zkConnect , zkSessionTimeout , connectionTimeoutMs = 100 , null )
new ZooKeeperClient ( zkConnect , zkSessionTimeout , connectionTimeoutMs = 100 , Int . MaxValue , null )
}
@Test
def testConnection ( ) : Unit = {
new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , null )
new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , Int . MaxValue , null )
}
@Test
def testDeleteNonExistentZNode ( ) : Unit = {
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , null )
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , Int . MaxValue , null )
val deleteResponse = zooKeeperClient . handleRequest ( DeleteRequest ( mockPath , - 1 ) )
assertEquals ( "Response code should be NONODE" , Code . NONODE , deleteResponse . resultCode )
}
@ -65,7 +66,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@@ -65,7 +66,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@Test
def testDeleteExistingZNode ( ) : Unit = {
import scala.collection.JavaConverters._
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , null )
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , Int . MaxValue , null )
val createResponse = zooKeeperClient . handleRequest ( CreateRequest ( mockPath , Array . empty [ Byte ] , ZooDefs . Ids . OPEN_ACL_UNSAFE . asScala , CreateMode . PERSISTENT ) )
assertEquals ( "Response code for create should be OK" , Code . OK , createResponse . resultCode )
val deleteResponse = zooKeeperClient . handleRequest ( DeleteRequest ( mockPath , - 1 ) )
@ -74,7 +75,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@@ -74,7 +75,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@Test
def testExistsNonExistentZNode ( ) : Unit = {
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , null )
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , Int . MaxValue , null )
val existsResponse = zooKeeperClient . handleRequest ( ExistsRequest ( mockPath ) )
assertEquals ( "Response code should be NONODE" , Code . NONODE , existsResponse . resultCode )
}
@ -82,7 +83,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@@ -82,7 +83,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@Test
def testExistsExistingZNode ( ) : Unit = {
import scala.collection.JavaConverters._
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , null )
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , Int . MaxValue , null )
val createResponse = zooKeeperClient . handleRequest ( CreateRequest ( mockPath , Array . empty [ Byte ] , ZooDefs . Ids . OPEN_ACL_UNSAFE . asScala , CreateMode . PERSISTENT ) )
assertEquals ( "Response code for create should be OK" , Code . OK , createResponse . resultCode )
val existsResponse = zooKeeperClient . handleRequest ( ExistsRequest ( mockPath ) )
@ -91,7 +92,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@@ -91,7 +92,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@Test
def testGetDataNonExistentZNode ( ) : Unit = {
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , null )
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , Int . MaxValue , null )
val getDataResponse = zooKeeperClient . handleRequest ( GetDataRequest ( mockPath ) )
assertEquals ( "Response code should be NONODE" , Code . NONODE , getDataResponse . resultCode )
}
@ -100,7 +101,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@@ -100,7 +101,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
def testGetDataExistingZNode ( ) : Unit = {
import scala.collection.JavaConverters._
val data = bytes
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , null )
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , Int . MaxValue , null )
val createResponse = zooKeeperClient . handleRequest ( CreateRequest ( mockPath , data , ZooDefs . Ids . OPEN_ACL_UNSAFE . asScala ,
CreateMode . PERSISTENT ) )
assertEquals ( "Response code for create should be OK" , Code . OK , createResponse . resultCode )
@ -111,7 +112,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@@ -111,7 +112,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@Test
def testSetDataNonExistentZNode ( ) : Unit = {
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , null )
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , Int . MaxValue , null )
val setDataResponse = zooKeeperClient . handleRequest ( SetDataRequest ( mockPath , Array . empty [ Byte ] , - 1 ) )
assertEquals ( "Response code should be NONODE" , Code . NONODE , setDataResponse . resultCode )
}
@ -120,7 +121,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@@ -120,7 +121,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
def testSetDataExistingZNode ( ) : Unit = {
import scala.collection.JavaConverters._
val data = bytes
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , null )
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , Int . MaxValue , null )
val createResponse = zooKeeperClient . handleRequest ( CreateRequest ( mockPath , Array . empty [ Byte ] ,
ZooDefs . Ids . OPEN_ACL_UNSAFE . asScala , CreateMode . PERSISTENT ) )
assertEquals ( "Response code for create should be OK" , Code . OK , createResponse . resultCode )
@ -133,7 +134,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@@ -133,7 +134,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@Test
def testGetAclNonExistentZNode ( ) : Unit = {
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , null )
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , Int . MaxValue , null )
val getAclResponse = zooKeeperClient . handleRequest ( GetAclRequest ( mockPath ) )
assertEquals ( "Response code should be NONODE" , Code . NONODE , getAclResponse . resultCode )
}
@ -141,7 +142,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@@ -141,7 +142,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@Test
def testGetAclExistingZNode ( ) : Unit = {
import scala.collection.JavaConverters._
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , null )
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , Int . MaxValue , null )
val createResponse = zooKeeperClient . handleRequest ( CreateRequest ( mockPath , Array . empty [ Byte ] , ZooDefs . Ids . OPEN_ACL_UNSAFE . asScala , CreateMode . PERSISTENT ) )
assertEquals ( "Response code for create should be OK" , Code . OK , createResponse . resultCode )
val getAclResponse = zooKeeperClient . handleRequest ( GetAclRequest ( mockPath ) )
@ -152,14 +153,14 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@@ -152,14 +153,14 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@Test
def testSetAclNonExistentZNode ( ) : Unit = {
import scala.collection.JavaConverters._
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , null )
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , Int . MaxValue , null )
val setAclResponse = zooKeeperClient . handleRequest ( SetAclRequest ( mockPath , ZooDefs . Ids . OPEN_ACL_UNSAFE . asScala , - 1 ) )
assertEquals ( "Response code should be NONODE" , Code . NONODE , setAclResponse . resultCode )
}
@Test
def testGetChildrenNonExistentZNode ( ) : Unit = {
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , null )
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , Int . MaxValue , null )
val getChildrenResponse = zooKeeperClient . handleRequest ( GetChildrenRequest ( mockPath ) )
assertEquals ( "Response code should be NONODE" , Code . NONODE , getChildrenResponse . resultCode )
}
@ -167,7 +168,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@@ -167,7 +168,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@Test
def testGetChildrenExistingZNode ( ) : Unit = {
import scala.collection.JavaConverters._
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , null )
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , Int . MaxValue , null )
val createResponse = zooKeeperClient . handleRequest ( CreateRequest ( mockPath , Array . empty [ Byte ] ,
ZooDefs . Ids . OPEN_ACL_UNSAFE . asScala , CreateMode . PERSISTENT ) )
assertEquals ( "Response code for create should be OK" , Code . OK , createResponse . resultCode )
@ -183,7 +184,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@@ -183,7 +184,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
val child2 = "child2"
val child1Path = mockPath + "/" + child1
val child2Path = mockPath + "/" + child2
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , null )
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , Int . MaxValue , null )
val createResponse = zooKeeperClient . handleRequest ( CreateRequest ( mockPath , Array . empty [ Byte ] ,
ZooDefs . Ids . OPEN_ACL_UNSAFE . asScala , CreateMode . PERSISTENT ) )
assertEquals ( "Response code for create should be OK" , Code . OK , createResponse . resultCode )
@ -202,7 +203,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@@ -202,7 +203,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@Test
def testPipelinedGetData ( ) : Unit = {
import scala.collection.JavaConverters._
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , null )
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , Int . MaxValue , null )
val createRequests = ( 1 to 3 ) . map ( x => CreateRequest ( "/" + x , ( x * 2 ) . toString . getBytes , ZooDefs . Ids . OPEN_ACL_UNSAFE . asScala , CreateMode . PERSISTENT ) )
val createResponses = createRequests . map ( zooKeeperClient . handleRequest )
createResponses . foreach ( createResponse => assertEquals ( "Response code for create should be OK" , Code . OK , createResponse . resultCode ) )
@ -219,7 +220,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@@ -219,7 +220,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@Test
def testMixedPipeline ( ) : Unit = {
import scala.collection.JavaConverters._
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , null )
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , Int . MaxValue , null )
val createResponse = zooKeeperClient . handleRequest ( CreateRequest ( mockPath , Array . empty [ Byte ] ,
ZooDefs . Ids . OPEN_ACL_UNSAFE . asScala , CreateMode . PERSISTENT ) )
assertEquals ( "Response code for create should be OK" , Code . OK , createResponse . resultCode )
@ -234,7 +235,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@@ -234,7 +235,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@Test
def testZNodeChangeHandlerForCreation ( ) : Unit = {
import scala.collection.JavaConverters._
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , null )
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , Int . MaxValue , null )
val znodeChangeHandlerCountDownLatch = new CountDownLatch ( 1 )
val zNodeChangeHandler = new ZNodeChangeHandler {
override def handleCreation ( ) : Unit = {
@ -255,7 +256,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@@ -255,7 +256,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@Test
def testZNodeChangeHandlerForDeletion ( ) : Unit = {
import scala.collection.JavaConverters._
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , null )
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , Int . MaxValue , null )
val znodeChangeHandlerCountDownLatch = new CountDownLatch ( 1 )
val zNodeChangeHandler = new ZNodeChangeHandler {
override def handleDeletion ( ) : Unit = {
@ -278,7 +279,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@@ -278,7 +279,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@Test
def testZNodeChangeHandlerForDataChange ( ) : Unit = {
import scala.collection.JavaConverters._
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , null )
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , Int . MaxValue , null )
val znodeChangeHandlerCountDownLatch = new CountDownLatch ( 1 )
val zNodeChangeHandler = new ZNodeChangeHandler {
override def handleDataChange ( ) : Unit = {
@ -301,7 +302,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@@ -301,7 +302,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@Test
def testZNodeChildChangeHandlerForChildChange ( ) : Unit = {
import scala.collection.JavaConverters._
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , null )
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , Int . MaxValue , null )
val zNodeChildChangeHandlerCountDownLatch = new CountDownLatch ( 1 )
val zNodeChildChangeHandler = new ZNodeChildChangeHandler {
override def handleChildChange ( ) : Unit = {
@ -331,9 +332,42 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
@@ -331,9 +332,42 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
stateChangeHandlerCountDownLatch . countDown ( )
}
}
new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , stateChangeHandler )
new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , Int . MaxValue , stateChangeHandler )
assertTrue ( "Failed to receive auth failed notification" , stateChangeHandlerCountDownLatch . await ( 5 , TimeUnit . SECONDS ) )
}
@Test
def testConnectionLossRequestTermination ( ) : Unit = {
val batchSize = 10
val zooKeeperClient = new ZooKeeperClient ( zkConnect , zkSessionTimeout , zkConnectionTimeout , 2 , null )
zookeeper . shutdown ( )
val requests = ( 1 to batchSize ) . map ( i => GetDataRequest ( s" / $i " ) )
val countDownLatch = new CountDownLatch ( 1 )
val running = new AtomicBoolean ( true )
val unexpectedResponses = new ArrayBlockingQueue [ GetDataResponse ] ( batchSize )
val requestThread = new Thread {
override def run ( ) : Unit = {
while ( running . get ( ) ) {
val responses = zooKeeperClient . handleRequests ( requests )
val suffix = responses . dropWhile ( response => response . resultCode != Code . CONNECTIONLOSS )
if ( ! suffix . forall ( response => response . resultCode == Code . CONNECTIONLOSS ) )
responses . foreach ( unexpectedResponses . add )
if ( ! unexpectedResponses . isEmpty || suffix . nonEmpty )
running . set ( false )
}
countDownLatch . countDown ( )
}
}
requestThread . start ( )
val requestThreadTerminated = countDownLatch . await ( 30 , TimeUnit . SECONDS )
if ( ! requestThreadTerminated ) {
running . set ( false )
requestThread . join ( 5000 )
fail ( "Failed to receive a CONNECTIONLOSS response code after zookeeper has shutdown." )
} else if ( ! unexpectedResponses . isEmpty ) {
fail ( s" Received an unexpected non-CONNECTIONLOSS response code after a CONNECTIONLOSS response code from a single batch: $unexpectedResponses " )
}
}
private def bytes = UUID . randomUUID ( ) . toString . getBytes ( StandardCharsets . UTF_8 )
}