@ -37,10 +37,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecords
@@ -37,10 +37,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecords
import org.apache.kafka.clients.consumer. { ConsumerConfig , KafkaConsumer }
import org.apache.kafka.clients.producer.KafkaProducer
import org.apache.kafka.clients.producer.ProducerRecord
import org.apache.kafka.common.ConsumerGroupState
import org.apache.kafka.common.ElectionType
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.TopicPartitionReplica
import org.apache.kafka.common. { ConsumerGroupState , ElectionType , TopicPartition , TopicPartitionInfo , TopicPartitionReplica }
import org.apache.kafka.common.acl._
import org.apache.kafka.common.config. { ConfigResource , LogLevelConfig }
import org.apache.kafka.common.errors._
@ -296,15 +293,14 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
@@ -296,15 +293,14 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
waitForTopics ( client , expectedPresent = Seq ( topic ) , expectedMissing = List ( ) )
// without includeAuthorizedOperations flag
var topicResult = client . describeTopics ( Seq ( topic ) . asJava ) . values
assertEquals ( Set ( ) . asJava , topicResult . get ( topic ) . get ( ) . authorizedOperations ( ) )
var topicResult = getTopicMetadata ( client , topic )
assertEquals ( Set ( ) . asJava , topicResult . authorizedOperations )
// with includeAuthorizedOperations flag
topicResult = client . describeTopics ( Seq ( topic ) . asJava ,
new DescribeTopicsOptions ( ) . includeAuthorizedOperations ( true ) ) . values
topicResult = getTopicMetadata ( client , topic , new DescribeTopicsOptions ( ) . includeAuthorizedOperations ( true ) )
expectedOperations = Topic . supportedOperations
. map ( operation => operation . toJava ) . asJava
assertEquals ( expectedOperations , topicResult . get ( topic ) . get ( ) . authorizedOperations ( ) )
assertEquals ( expectedOperations , topicResult . authorizedOperations )
}
def configuredClusterPermissions ( ) : Set [ AclOperation ] = {
@ -559,17 +555,19 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
@@ -559,17 +555,19 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
createTopic ( topic2 , numPartitions = 1 , replicationFactor = 2 )
// assert that both the topics have 1 partition
assertEquals ( 1 , client . describeTopics ( Set ( topic1 ) . asJava ) . values . get ( topic1 ) . get . partitions . size )
assertEquals ( 1 , client . describeTopics ( Set ( topic2 ) . asJava ) . values . get ( topic2 ) . get . partitions . size )
val topic1_metadata = getTopicMetadata ( client , topic1 )
val topic2_metadata = getTopicMetadata ( client , topic2 )
assertEquals ( 1 , topic1_metadata . partitions . size )
assertEquals ( 1 , topic2_metadata . partitions . size )
val validateOnly = new CreatePartitionsOptions ( ) . validateOnly ( true )
val actuallyDoIt = new CreatePartitionsOptions ( ) . validateOnly ( false )
def partitions ( topic : String ) =
client . describeTopics ( Set ( topic ) . asJava ) . values . get ( topic ) . get . partitions
def partitions ( topic : String , expectedNumPartitionsOpt : Option [ Int ] = None ) : util.List [ TopicPartitionInfo ] = {
getTopicMetadata ( client , topic , expectedNumPartitionsOpt = expectedNumPartitionsOpt ) . partitions
}
def numPartitions ( topic : String ) =
partitions ( topic ) . size
def numPartitions ( topic : String ) : Int = partitions ( topic ) . size
// validateOnly : try creating a new partition ( no assignments ) , to bring the total to 3 partitions
var alterResult = client . createPartitions ( Map ( topic1 ->
@ -581,7 +579,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
@@ -581,7 +579,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
alterResult = client . createPartitions ( Map ( topic1 ->
NewPartitions . increaseTo ( 3 ) ) . asJava , actuallyDoIt )
altered = alterResult . values . get ( topic1 ) . get
assertEquals ( 3 , numPartitions ( topic1 ) )
TestUtils . waitUntilTrue ( ( ) => numPartitions ( topic1 ) == 3 , "Timed out waiting for new partitions to appear" )
// validateOnly : now try creating a new partition ( with assignments ) , to bring the total to 3 partitions
val newPartition2Assignments = asList [ util . List [ Integer ] ] ( asList ( 0 , 1 ) , asList ( 1 , 2 ) )
@ -594,7 +592,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
@@ -594,7 +592,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
alterResult = client . createPartitions ( Map ( topic2 ->
NewPartitions . increaseTo ( 3 , newPartition2Assignments ) ) . asJava , actuallyDoIt )
altered = alterResult . values . get ( topic2 ) . get
val actualPartitions2 = partitions ( topic2 )
val actualPartitions2 = partitions ( topic2 , expectedNumPartitionsOpt = Some ( 3 ) )
assertEquals ( 3 , actualPartitions2 . size )
assertEquals ( Seq ( 0 , 1 ) , actualPartitions2 . get ( 1 ) . replicas . asScala . map ( _ . id ) . toList )
assertEquals ( Seq ( 1 , 2 ) , actualPartitions2 . get ( 2 ) . replicas . asScala . map ( _ . id ) . toList )
@ -782,7 +780,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
@@ -782,7 +780,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
topic2 -> NewPartitions . increaseTo ( 2 ) ) . asJava , actuallyDoIt )
// assert that the topic1 now has 4 partitions
altered = alterResult . values . get ( topic1 ) . get
assertEquals ( 4 , numPartitions ( topic1 ) )
TestUtils . waitUntilTrue ( ( ) => numPartitions ( topic1 ) == 4 , "Timed out waiting for new partitions to appear" )
try {
altered = alterResult . values . get ( topic2 ) . get
} catch {
@ -1452,15 +1450,17 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
@@ -1452,15 +1450,17 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
val partition2 = new TopicPartition ( "elect-preferred-leaders-topic-2" , 0 )
TestUtils . createTopic ( zkClient , partition2 . topic , Map [ Int , Seq [ Int ] ] ( partition2 . partition -> prefer0 ) , servers )
def preferredLeader ( topicPartition : TopicPartition ) =
client . describeTopics ( asList ( topicPartition . topic ) ) . values . get ( topicPartition . topic ) .
get . partitions . get ( topicPartition . partition ) . replicas . get ( 0 ) . id
def preferredLeader ( topicPartition : TopicPartition ) : Int = {
val partitionMetadata = getTopicMetadata ( client , topicPartition . topic ) . partitions . get ( topicPartition . partition )
val preferredLeaderMetadata = partitionMetadata . replicas . get ( 0 )
preferredLeaderMetadata . id
}
/* * Changes the <i>preferred< / i> leader without changing the <i>current< / i> leader. */
def changePreferredLeader ( newAssignment : Seq [ Int ] ) = {
val preferred = newAssignment . head
val prior1 = TestUtils . currentLeader ( client , partition1 ) . get
val prior2 = TestUtils . currentLeader ( client , partition2 ) . get
val prior1 = zkClient . getLeaderForPartition ( partition1 ) . get
val prior2 = zkClient . getLeaderForPartition ( partition2 ) . get
var m = Map . empty [ TopicPartition , Seq [ Int ] ]
@ -1475,26 +1475,26 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
@@ -1475,26 +1475,26 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
s" Expected preferred leader to become $preferred , but is ${ preferredLeader ( partition1 ) } and ${ preferredLeader ( partition2 ) } " ,
10000 )
// Check the leader hasn 't moved
assertEquals ( Some ( prior1 ) , TestUtils . curren tLeader( client , partition1 ) )
assertEquals ( Some ( prior2 ) , TestUtils . curren tLeader( client , partition2 ) )
TestUtils . asser tLeader( client , partition1 , prior1 )
TestUtils . asser tLeader( client , partition2 , prior2 )
}
// Check current leaders are 0
assertEquals ( Some ( 0 ) , TestUtils . curren tLeader( client , partition1 ) )
assertEquals ( Some ( 0 ) , TestUtils . curren tLeader( client , partition2 ) )
TestUtils . asser tLeader( client , partition1 , 0 )
TestUtils . asser tLeader( client , partition2 , 0 )
// Noop election
var electResult = client . electLeaders ( ElectionType . PREFERRED , Set ( partition1 ) . asJava )
var exception = electResult . partitions . get . get ( partition1 ) . get
assertEquals ( classOf [ ElectionNotNeededException ] , exception . getClass )
assertEquals ( "Leader election not needed for topic partition" , exception . getMessage )
assertEquals ( Some ( 0 ) , TestUtils . curren tLeader( client , partition1 ) )
TestUtils . asser tLeader( client , partition1 , 0 )
// Noop election with null partitions
electResult = client . electLeaders ( ElectionType . PREFERRED , null )
assertTrue ( electResult . partitions . get . isEmpty )
assertEquals ( Some ( 0 ) , TestUtils . curren tLeader( client , partition1 ) )
assertEquals ( Some ( 0 ) , TestUtils . curren tLeader( client , partition2 ) )
TestUtils . asser tLeader( client , partition1 , 0 )
TestUtils . asser tLeader( client , partition2 , 0 )
// Now change the preferred leader to 1
changePreferredLeader ( prefer1 )
@ -1503,17 +1503,17 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
@@ -1503,17 +1503,17 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
electResult = client . electLeaders ( ElectionType . PREFERRED , Set ( partition1 ) . asJava )
assertEquals ( Set ( partition1 ) . asJava , electResult . partitions . get . keySet )
assertFalse ( electResult . partitions . get . get ( partition1 ) . isPresent )
TestUtils . waitForLeaderToBecome ( client , partition1 , Some ( 1 ) )
TestUtils . assertLeader ( client , partition1 , 1 )
// topic 2 unchanged
assertFalse ( electResult . partitions . get . containsKey ( partition2 ) )
assertEquals ( Some ( 0 ) , TestUtils . curren tLeader( client , partition2 ) )
TestUtils . asser tLeader( client , partition2 , 0 )
// meaningful election with null partitions
electResult = client . electLeaders ( ElectionType . PREFERRED , null )
assertEquals ( Set ( partition2 ) , electResult . partitions . get . keySet . asScala )
assertFalse ( electResult . partitions . get . get ( partition2 ) . isPresent )
TestUtils . waitForLeaderToBecome ( client , partition2 , Some ( 1 ) )
TestUtils . assertLeader ( client , partition2 , 1 )
// unknown topic
val unknownPartition = new TopicPartition ( "topic-does-not-exist" , 0 )
@ -1522,8 +1522,8 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
@@ -1522,8 +1522,8 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
exception = electResult . partitions . get . get ( unknownPartition ) . get
assertEquals ( classOf [ UnknownTopicOrPartitionException ] , exception . getClass )
assertEquals ( "The partition does not exist." , exception . getMessage )
assertEquals ( Some ( 1 ) , TestUtils . curren tLeader( client , partition1 ) )
assertEquals ( Some ( 1 ) , TestUtils . curren tLeader( client , partition2 ) )
TestUtils . asser tLeader( client , partition1 , 1 )
TestUtils . asser tLeader( client , partition2 , 1 )
// Now change the preferred leader to 2
changePreferredLeader ( prefer2 )
@ -1531,8 +1531,8 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
@@ -1531,8 +1531,8 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
// mixed results
electResult = client . electLeaders ( ElectionType . PREFERRED , Set ( unknownPartition , partition1 ) . asJava )
assertEquals ( Set ( unknownPartition , partition1 ) . asJava , electResult . partitions . get . keySet )
TestUtils . waitForLeaderToBecome ( client , partition1 , Some ( 2 ) )
assertEquals ( Some ( 1 ) , TestUtils . curren tLeader( client , partition2 ) )
TestUtils . assertLeader ( client , partition1 , 2 )
TestUtils . asser tLeader( client , partition2 , 1 )
exception = electResult . partitions . get . get ( unknownPartition ) . get
assertEquals ( classOf [ UnknownTopicOrPartitionException ] , exception . getClass )
assertEquals ( "The partition does not exist." , exception . getMessage )
@ -1541,7 +1541,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
@@ -1541,7 +1541,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
electResult = client . electLeaders ( ElectionType . PREFERRED , Set ( partition2 ) . asJava )
assertEquals ( Set ( partition2 ) . asJava , electResult . partitions . get . keySet )
assertFalse ( electResult . partitions . get . get ( partition2 ) . isPresent )
TestUtils . waitForLeaderToBecome ( client , partition2 , Some ( 2 ) )
TestUtils . assertLeader ( client , partition2 , 2 )
// Now change the preferred leader to 1
changePreferredLeader ( prefer1 )
@ -1557,7 +1557,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
@@ -1557,7 +1557,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
assertEquals ( classOf [ PreferredLeaderNotAvailableException ] , exception . getClass )
assertTrue ( s" Wrong message ${ exception . getMessage } " , exception . getMessage . contains (
"Failed to elect leader for partition elect-preferred-leaders-topic-1-0 under strategy PreferredReplicaPartitionLeaderElectionStrategy" ) )
assertEquals ( Some ( 2 ) , TestUtils . curren tLeader( client , partition1 ) )
TestUtils . asser tLeader( client , partition1 , 2 )
// preferred leader unavailable with null argument
electResult = client . electLeaders ( ElectionType . PREFERRED , null , shortTimeout )
@ -1572,8 +1572,8 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
@@ -1572,8 +1572,8 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
assertTrue ( s" Wrong message ${ exception . getMessage } " , exception . getMessage . contains (
"Failed to elect leader for partition elect-preferred-leaders-topic-2-0 under strategy PreferredReplicaPartitionLeaderElectionStrategy" ) )
assertEquals ( Some ( 2 ) , TestUtils . curren tLeader( client , partition1 ) )
assertEquals ( Some ( 2 ) , TestUtils . curren tLeader( client , partition2 ) )
TestUtils . asser tLeader( client , partition1 , 2 )
TestUtils . asser tLeader( client , partition2 , 2 )
}
@Test
@ -1588,17 +1588,17 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
@@ -1588,17 +1588,17 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
val partition1 = new TopicPartition ( "unclean-test-topic-1" , 0 )
TestUtils . createTopic ( zkClient , partition1 . topic , Map [ Int , Seq [ Int ] ] ( partition1 . partition -> assignment1 ) , servers )
TestUtils . waitForLeaderToBecome ( client , partition1 , Option ( broker1 ) )
TestUtils . assertLeader ( client , partition1 , broker1 )
servers ( broker2 ) . shutdown ( )
TestUtils . waitForBrokersOutOfIsr ( client , Set ( partition1 ) , Set ( broker2 ) )
servers ( broker1 ) . shutdown ( )
TestUtils . waitForLeaderToBecome ( client , partition1 , None )
TestUtils . assertNoLeader ( client , partition1 )
servers ( broker2 ) . startup ( )
val electResult = client . electLeaders ( ElectionType . UNCLEAN , Set ( partition1 ) . asJava )
assertFalse ( electResult . partitions . get . get ( partition1 ) . isPresent )
assertEquals ( Option ( broker2 ) , TestUtils . curren tLeader( client , partition1 ) )
TestUtils . asser tLeader( client , partition1 , broker2 )
}
@Test
@ -1622,21 +1622,21 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
@@ -1622,21 +1622,21 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
servers
)
TestUtils . waitForLeaderToBecome ( client , partition1 , Option ( broker1 ) )
TestUtils . waitForLeaderToBecome ( client , partition2 , Option ( broker1 ) )
TestUtils . assertLeader ( client , partition1 , broker1 )
TestUtils . assertLeader ( client , partition2 , broker1 )
servers ( broker2 ) . shutdown ( )
TestUtils . waitForBrokersOutOfIsr ( client , Set ( partition1 , partition2 ) , Set ( broker2 ) )
servers ( broker1 ) . shutdown ( )
TestUtils . waitForLeaderToBecome ( client , partition1 , None )
TestUtils . waitForLeaderToBecome ( client , partition2 , None )
TestUtils . assertNoLeader ( client , partition1 )
TestUtils . assertNoLeader ( client , partition2 )
servers ( broker2 ) . startup ( )
val electResult = client . electLeaders ( ElectionType . UNCLEAN , Set ( partition1 , partition2 ) . asJava )
assertFalse ( electResult . partitions . get . get ( partition1 ) . isPresent )
assertFalse ( electResult . partitions . get . get ( partition2 ) . isPresent )
assertEquals ( Option ( broker2 ) , TestUtils . curren tLeader( client , partition1 ) )
assertEquals ( Option ( broker2 ) , TestUtils . curren tLeader( client , partition2 ) )
TestUtils . asser tLeader( client , partition1 , broker2 )
TestUtils . asser tLeader( client , partition2 , broker2 )
}
@Test
@ -1661,21 +1661,21 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
@@ -1661,21 +1661,21 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
servers
)
TestUtils . waitForLeaderToBecome ( client , partition1 , Option ( broker1 ) )
TestUtils . waitForLeaderToBecome ( client , partition2 , Option ( broker1 ) )
TestUtils . assertLeader ( client , partition1 , broker1 )
TestUtils . assertLeader ( client , partition2 , broker1 )
servers ( broker2 ) . shutdown ( )
TestUtils . waitForBrokersOutOfIsr ( client , Set ( partition1 ) , Set ( broker2 ) )
servers ( broker1 ) . shutdown ( )
TestUtils . waitForLeaderToBecome ( client , partition1 , None )
TestUtils . waitForLeaderToBecome ( client , partition2 , Some ( broker3 ) )
TestUtils . assertNoLeader ( client , partition1 )
TestUtils . assertLeader ( client , partition2 , broker3 )
servers ( broker2 ) . startup ( )
val electResult = client . electLeaders ( ElectionType . UNCLEAN , null )
assertFalse ( electResult . partitions . get . get ( partition1 ) . isPresent )
assertFalse ( electResult . partitions . get . containsKey ( partition2 ) )
assertEquals ( Option ( broker2 ) , TestUtils . curren tLeader( client , partition1 ) )
assertEquals ( Option ( broker3 ) , TestUtils . curren tLeader( client , partition2 ) )
TestUtils . asser tLeader( client , partition1 , broker2 )
TestUtils . asser tLeader( client , partition2 , broker3 )
}
@Test
@ -1698,7 +1698,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
@@ -1698,7 +1698,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
servers
)
TestUtils . waitForLeaderToBecome ( client , new TopicPartition ( topic , 0 ) , Option ( broker1 ) )
TestUtils . assertLeader ( client , new TopicPartition ( topic , 0 ) , broker1 )
val electResult = client . electLeaders ( ElectionType . UNCLEAN , Set ( unknownPartition , unknownTopic ) . asJava )
assertTrue ( electResult . partitions . get . get ( unknownPartition ) . get . isInstanceOf [ UnknownTopicOrPartitionException ] )
@ -1724,12 +1724,12 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
@@ -1724,12 +1724,12 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
servers
)
TestUtils . waitForLeaderToBecome ( client , partition1 , Option ( broker1 ) )
TestUtils . assertLeader ( client , partition1 , broker1 )
servers ( broker2 ) . shutdown ( )
TestUtils . waitForBrokersOutOfIsr ( client , Set ( partition1 ) , Set ( broker2 ) )
servers ( broker1 ) . shutdown ( )
TestUtils . waitForLeaderToBecome ( client , partition1 , None )
TestUtils . assertNoLeader ( client , partition1 )
val electResult = client . electLeaders ( ElectionType . UNCLEAN , Set ( partition1 ) . asJava )
assertTrue ( electResult . partitions . get . get ( partition1 ) . get . isInstanceOf [ EligibleLeadersNotAvailableException ] )
@ -1754,10 +1754,10 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
@@ -1754,10 +1754,10 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
servers
)
TestUtils . waitForLeaderToBecome ( client , partition1 , Option ( broker1 ) )
TestUtils . assertLeader ( client , partition1 , broker1 )
servers ( broker1 ) . shutdown ( )
TestUtils . waitForLeaderToBecome ( client , partition1 , Some ( broker2 ) )
TestUtils . assertLeader ( client , partition1 , broker2 )
servers ( broker1 ) . startup ( )
val electResult = client . electLeaders ( ElectionType . UNCLEAN , Set ( partition1 ) . asJava )
@ -1786,21 +1786,21 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
@@ -1786,21 +1786,21 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
servers
)
TestUtils . waitForLeaderToBecome ( client , partition1 , Option ( broker1 ) )
TestUtils . waitForLeaderToBecome ( client , partition2 , Option ( broker1 ) )
TestUtils . assertLeader ( client , partition1 , broker1 )
TestUtils . assertLeader ( client , partition2 , broker1 )
servers ( broker2 ) . shutdown ( )
TestUtils . waitForBrokersOutOfIsr ( client , Set ( partition1 ) , Set ( broker2 ) )
servers ( broker1 ) . shutdown ( )
TestUtils . waitForLeaderToBecome ( client , partition1 , None )
TestUtils . waitForLeaderToBecome ( client , partition2 , Some ( broker3 ) )
TestUtils . assertNoLeader ( client , partition1 )
TestUtils . assertLeader ( client , partition2 , broker3 )
servers ( broker2 ) . startup ( )
val electResult = client . electLeaders ( ElectionType . UNCLEAN , Set ( partition1 , partition2 ) . asJava )
assertFalse ( electResult . partitions . get . get ( partition1 ) . isPresent )
assertTrue ( electResult . partitions . get . get ( partition2 ) . get . isInstanceOf [ ElectionNotNeededException ] )
assertEquals ( Option ( broker2 ) , TestUtils . curren tLeader( client , partition1 ) )
assertEquals ( Option ( broker3 ) , TestUtils . curren tLeader( client , partition2 ) )
TestUtils . asser tLeader( client , partition1 , broker2 )
TestUtils . asser tLeader( client , partition2 , broker3 )
}
@Test
@ -2428,4 +2428,23 @@ object AdminClientIntegrationTest {
@@ -2428,4 +2428,23 @@ object AdminClientIntegrationTest {
assertEquals ( Defaults . CompressionType . toString , configs . get ( brokerResource ) . get ( KafkaConfig . CompressionTypeProp ) . value )
}
private def getTopicMetadata ( client : Admin ,
topic : String ,
describeOptions : DescribeTopicsOptions = new DescribeTopicsOptions ,
expectedNumPartitionsOpt : Option [ Int ] = None ) : TopicDescription = {
var result : TopicDescription = null
TestUtils . waitUntilTrue ( ( ) => {
val topicResult = client . describeTopics ( Set ( topic ) . asJava , describeOptions ) . values . get ( topic )
try {
result = topicResult . get
expectedNumPartitionsOpt . map ( _ == result . partitions . size ) . getOrElse ( true )
} catch {
case e : ExecutionException if e.getCause.isInstanceOf [ UnknownTopicOrPartitionException ] => false // metadata may not have propagated yet , so retry
}
} , s" Timed out waiting for metadata for $topic " )
result
}
}