@ -89,6 +89,7 @@ import org.apache.kafka.common.message.DescribeConfigsResponseData;
@@ -89,6 +89,7 @@ import org.apache.kafka.common.message.DescribeConfigsResponseData;
import org.apache.kafka.common.message.DescribeGroupsResponseData ;
import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroupMember ;
import org.apache.kafka.common.message.DescribeLogDirsResponseData ;
import org.apache.kafka.common.message.DescribeLogDirsResponseData.DescribeLogDirsTopic ;
import org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult ;
import org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult ;
import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData ;
@ -182,6 +183,7 @@ import java.util.stream.Stream;
@@ -182,6 +183,7 @@ import java.util.stream.Stream;
import static java.util.Arrays.asList ;
import static java.util.Collections.emptyList ;
import static java.util.Collections.emptySet ;
import static java.util.Collections.singletonList ;
import static org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.ReassignablePartitionResponse ;
import static org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.ReassignableTopicResponse ;
@ -1350,6 +1352,263 @@ public class KafkaAdminClientTest {
@@ -1350,6 +1352,263 @@ public class KafkaAdminClientTest {
}
}
private static DescribeLogDirsResponse prepareDescribeLogDirsResponse ( Errors error , String logDir , TopicPartition tp , long partitionSize , long offsetLag ) {
return prepareDescribeLogDirsResponse ( error , logDir ,
prepareDescribeLogDirsTopics ( partitionSize , offsetLag , tp . topic ( ) , tp . partition ( ) , false ) ) ;
}
private static List < DescribeLogDirsTopic > prepareDescribeLogDirsTopics (
long partitionSize , long offsetLag , String topic , int partition , boolean isFuture ) {
return singletonList ( new DescribeLogDirsTopic ( )
. setName ( topic )
. setPartitions ( singletonList ( new DescribeLogDirsResponseData . DescribeLogDirsPartition ( )
. setPartitionIndex ( partition )
. setPartitionSize ( partitionSize )
. setIsFutureKey ( isFuture )
. setOffsetLag ( offsetLag ) ) ) ) ;
}
private static DescribeLogDirsResponse prepareDescribeLogDirsResponse ( Errors error , String logDir ,
List < DescribeLogDirsTopic > topics ) {
return new DescribeLogDirsResponse (
new DescribeLogDirsResponseData ( ) . setResults ( singletonList ( new DescribeLogDirsResponseData . DescribeLogDirsResult ( )
. setErrorCode ( error . code ( ) )
. setLogDir ( logDir )
. setTopics ( topics )
) ) ) ;
}
@Test
public void testDescribeLogDirs ( ) throws ExecutionException , InterruptedException {
Set < Integer > brokers = Collections . singleton ( 0 ) ;
String logDir = "/var/data/kafka" ;
TopicPartition tp = new TopicPartition ( "topic" , 12 ) ;
long partitionSize = 1234567890 ;
long offsetLag = 24 ;
try ( AdminClientUnitTestEnv env = mockClientEnv ( ) ) {
env . kafkaClient ( ) . setNodeApiVersions ( NodeApiVersions . create ( ) ) ;
env . kafkaClient ( ) . prepareResponseFrom (
prepareDescribeLogDirsResponse ( Errors . NONE , logDir , tp , partitionSize , offsetLag ) ,
env . cluster ( ) . nodeById ( 0 ) ) ;
DescribeLogDirsResult result = env . adminClient ( ) . describeLogDirs ( brokers ) ;
Map < Integer , KafkaFuture < Map < String , LogDirDescription > > > descriptions = result . descriptions ( ) ;
assertEquals ( brokers , descriptions . keySet ( ) ) ;
assertNotNull ( descriptions . get ( 0 ) ) ;
assertDescriptionContains ( descriptions . get ( 0 ) . get ( ) , logDir , tp , partitionSize , offsetLag ) ;
Map < Integer , Map < String , LogDirDescription > > allDescriptions = result . allDescriptions ( ) . get ( ) ;
assertEquals ( brokers , allDescriptions . keySet ( ) ) ;
assertDescriptionContains ( allDescriptions . get ( 0 ) , logDir , tp , partitionSize , offsetLag ) ;
}
}
private static void assertDescriptionContains ( Map < String , LogDirDescription > descriptionsMap , String logDir ,
TopicPartition tp , long partitionSize , long offsetLag ) {
assertNotNull ( descriptionsMap ) ;
assertEquals ( Collections . singleton ( logDir ) , descriptionsMap . keySet ( ) ) ;
assertNull ( descriptionsMap . get ( logDir ) . error ( ) ) ;
Map < TopicPartition , ReplicaInfo > descriptionsReplicaInfos = descriptionsMap . get ( logDir ) . replicaInfos ( ) ;
assertEquals ( Collections . singleton ( tp ) , descriptionsReplicaInfos . keySet ( ) ) ;
assertEquals ( partitionSize , descriptionsReplicaInfos . get ( tp ) . size ( ) ) ;
assertEquals ( offsetLag , descriptionsReplicaInfos . get ( tp ) . offsetLag ( ) ) ;
assertFalse ( descriptionsReplicaInfos . get ( tp ) . isFuture ( ) ) ;
}
@SuppressWarnings ( "deprecation" )
@Test
public void testDescribeLogDirsDeprecated ( ) throws ExecutionException , InterruptedException {
Set < Integer > brokers = Collections . singleton ( 0 ) ;
TopicPartition tp = new TopicPartition ( "topic" , 12 ) ;
String logDir = "/var/data/kafka" ;
Errors error = Errors . NONE ;
int offsetLag = 24 ;
long partitionSize = 1234567890 ;
try ( AdminClientUnitTestEnv env = mockClientEnv ( ) ) {
env . kafkaClient ( ) . setNodeApiVersions ( NodeApiVersions . create ( ) ) ;
env . kafkaClient ( ) . prepareResponseFrom (
prepareDescribeLogDirsResponse ( error , logDir , tp , partitionSize , offsetLag ) ,
env . cluster ( ) . nodeById ( 0 ) ) ;
DescribeLogDirsResult result = env . adminClient ( ) . describeLogDirs ( brokers ) ;
Map < Integer , KafkaFuture < Map < String , DescribeLogDirsResponse . LogDirInfo > > > deprecatedValues = result . values ( ) ;
assertEquals ( brokers , deprecatedValues . keySet ( ) ) ;
assertNotNull ( deprecatedValues . get ( 0 ) ) ;
assertDescriptionContains ( deprecatedValues . get ( 0 ) . get ( ) , logDir , tp , error , offsetLag , partitionSize ) ;
Map < Integer , Map < String , DescribeLogDirsResponse . LogDirInfo > > deprecatedAll = result . all ( ) . get ( ) ;
assertEquals ( brokers , deprecatedAll . keySet ( ) ) ;
assertDescriptionContains ( deprecatedAll . get ( 0 ) , logDir , tp , error , offsetLag , partitionSize ) ;
}
}
@SuppressWarnings ( "deprecation" )
private static void assertDescriptionContains ( Map < String , DescribeLogDirsResponse . LogDirInfo > descriptionsMap ,
String logDir , TopicPartition tp , Errors error ,
int offsetLag , long partitionSize ) {
assertNotNull ( descriptionsMap ) ;
assertEquals ( Collections . singleton ( logDir ) , descriptionsMap . keySet ( ) ) ;
assertEquals ( error , descriptionsMap . get ( logDir ) . error ) ;
Map < TopicPartition , DescribeLogDirsResponse . ReplicaInfo > allReplicaInfos =
descriptionsMap . get ( logDir ) . replicaInfos ;
assertEquals ( Collections . singleton ( tp ) , allReplicaInfos . keySet ( ) ) ;
assertEquals ( partitionSize , allReplicaInfos . get ( tp ) . size ) ;
assertEquals ( offsetLag , allReplicaInfos . get ( tp ) . offsetLag ) ;
assertFalse ( allReplicaInfos . get ( tp ) . isFuture ) ;
}
@Test
public void testDescribeLogDirsOfflineDir ( ) throws ExecutionException , InterruptedException {
Set < Integer > brokers = Collections . singleton ( 0 ) ;
String logDir = "/var/data/kafka" ;
Errors error = Errors . KAFKA_STORAGE_ERROR ;
try ( AdminClientUnitTestEnv env = mockClientEnv ( ) ) {
env . kafkaClient ( ) . setNodeApiVersions ( NodeApiVersions . create ( ) ) ;
env . kafkaClient ( ) . prepareResponseFrom (
prepareDescribeLogDirsResponse ( error , logDir , emptyList ( ) ) ,
env . cluster ( ) . nodeById ( 0 ) ) ;
DescribeLogDirsResult result = env . adminClient ( ) . describeLogDirs ( brokers ) ;
Map < Integer , KafkaFuture < Map < String , LogDirDescription > > > descriptions = result . descriptions ( ) ;
assertEquals ( brokers , descriptions . keySet ( ) ) ;
assertNotNull ( descriptions . get ( 0 ) ) ;
Map < String , LogDirDescription > descriptionsMap = descriptions . get ( 0 ) . get ( ) ;
assertEquals ( Collections . singleton ( logDir ) , descriptionsMap . keySet ( ) ) ;
assertEquals ( error . exception ( ) . getClass ( ) , descriptionsMap . get ( logDir ) . error ( ) . getClass ( ) ) ;
assertEquals ( emptySet ( ) , descriptionsMap . get ( logDir ) . replicaInfos ( ) . keySet ( ) ) ;
Map < Integer , Map < String , LogDirDescription > > allDescriptions = result . allDescriptions ( ) . get ( ) ;
assertEquals ( brokers , allDescriptions . keySet ( ) ) ;
Map < String , LogDirDescription > allMap = allDescriptions . get ( 0 ) ;
assertNotNull ( allMap ) ;
assertEquals ( Collections . singleton ( logDir ) , allMap . keySet ( ) ) ;
assertEquals ( error . exception ( ) . getClass ( ) , allMap . get ( logDir ) . error ( ) . getClass ( ) ) ;
assertEquals ( emptySet ( ) , allMap . get ( logDir ) . replicaInfos ( ) . keySet ( ) ) ;
}
}
@SuppressWarnings ( "deprecation" )
@Test
public void testDescribeLogDirsOfflineDirDeprecated ( ) throws ExecutionException , InterruptedException {
Set < Integer > brokers = Collections . singleton ( 0 ) ;
String logDir = "/var/data/kafka" ;
Errors error = Errors . KAFKA_STORAGE_ERROR ;
try ( AdminClientUnitTestEnv env = mockClientEnv ( ) ) {
env . kafkaClient ( ) . setNodeApiVersions ( NodeApiVersions . create ( ) ) ;
env . kafkaClient ( ) . prepareResponseFrom (
prepareDescribeLogDirsResponse ( error , logDir , emptyList ( ) ) ,
env . cluster ( ) . nodeById ( 0 ) ) ;
DescribeLogDirsResult result = env . adminClient ( ) . describeLogDirs ( brokers ) ;
Map < Integer , KafkaFuture < Map < String , DescribeLogDirsResponse . LogDirInfo > > > deprecatedValues = result . values ( ) ;
assertEquals ( brokers , deprecatedValues . keySet ( ) ) ;
assertNotNull ( deprecatedValues . get ( 0 ) ) ;
Map < String , DescribeLogDirsResponse . LogDirInfo > valuesMap = deprecatedValues . get ( 0 ) . get ( ) ;
assertEquals ( Collections . singleton ( logDir ) , valuesMap . keySet ( ) ) ;
assertEquals ( error , valuesMap . get ( logDir ) . error ) ;
assertEquals ( emptySet ( ) , valuesMap . get ( logDir ) . replicaInfos . keySet ( ) ) ;
Map < Integer , Map < String , DescribeLogDirsResponse . LogDirInfo > > deprecatedAll = result . all ( ) . get ( ) ;
assertEquals ( brokers , deprecatedAll . keySet ( ) ) ;
Map < String , DescribeLogDirsResponse . LogDirInfo > allMap = deprecatedAll . get ( 0 ) ;
assertNotNull ( allMap ) ;
assertEquals ( Collections . singleton ( logDir ) , allMap . keySet ( ) ) ;
assertEquals ( error , allMap . get ( logDir ) . error ) ;
assertEquals ( emptySet ( ) , allMap . get ( logDir ) . replicaInfos . keySet ( ) ) ;
}
}
@Test
public void testDescribeReplicaLogDirs ( ) throws ExecutionException , InterruptedException {
TopicPartitionReplica tpr1 = new TopicPartitionReplica ( "topic" , 12 , 1 ) ;
TopicPartitionReplica tpr2 = new TopicPartitionReplica ( "topic" , 12 , 2 ) ;
try ( AdminClientUnitTestEnv env = mockClientEnv ( ) ) {
env . kafkaClient ( ) . setNodeApiVersions ( NodeApiVersions . create ( ) ) ;
String broker1log0 = "/var/data/kafka0" ;
String broker1log1 = "/var/data/kafka1" ;
String broker2log0 = "/var/data/kafka2" ;
int broker1Log0OffsetLag = 24 ;
int broker1Log0PartitionSize = 987654321 ;
int broker1Log1PartitionSize = 123456789 ;
int broker1Log1OffsetLag = 4321 ;
env . kafkaClient ( ) . prepareResponseFrom (
new DescribeLogDirsResponse (
new DescribeLogDirsResponseData ( ) . setResults ( asList (
prepareDescribeLogDirsResult ( tpr1 , broker1log0 , broker1Log0PartitionSize , broker1Log0OffsetLag , false ) ,
prepareDescribeLogDirsResult ( tpr1 , broker1log1 , broker1Log1PartitionSize , broker1Log1OffsetLag , true ) ) ) ) ,
env . cluster ( ) . nodeById ( tpr1 . brokerId ( ) ) ) ;
env . kafkaClient ( ) . prepareResponseFrom (
prepareDescribeLogDirsResponse ( Errors . KAFKA_STORAGE_ERROR , broker2log0 ) ,
env . cluster ( ) . nodeById ( tpr2 . brokerId ( ) ) ) ;
DescribeReplicaLogDirsResult result = env . adminClient ( ) . describeReplicaLogDirs ( asList ( tpr1 , tpr2 ) ) ;
Map < TopicPartitionReplica , KafkaFuture < DescribeReplicaLogDirsResult . ReplicaLogDirInfo > > values = result . values ( ) ;
assertEquals ( TestUtils . toSet ( asList ( tpr1 , tpr2 ) ) , values . keySet ( ) ) ;
assertNotNull ( values . get ( tpr1 ) ) ;
assertEquals ( broker1log0 , values . get ( tpr1 ) . get ( ) . getCurrentReplicaLogDir ( ) ) ;
assertEquals ( broker1Log0OffsetLag , values . get ( tpr1 ) . get ( ) . getCurrentReplicaOffsetLag ( ) ) ;
assertEquals ( broker1log1 , values . get ( tpr1 ) . get ( ) . getFutureReplicaLogDir ( ) ) ;
assertEquals ( broker1Log1OffsetLag , values . get ( tpr1 ) . get ( ) . getFutureReplicaOffsetLag ( ) ) ;
assertNotNull ( values . get ( tpr2 ) ) ;
assertNull ( values . get ( tpr2 ) . get ( ) . getCurrentReplicaLogDir ( ) ) ;
assertEquals ( - 1 , values . get ( tpr2 ) . get ( ) . getCurrentReplicaOffsetLag ( ) ) ;
assertNull ( values . get ( tpr2 ) . get ( ) . getFutureReplicaLogDir ( ) ) ;
assertEquals ( - 1 , values . get ( tpr2 ) . get ( ) . getFutureReplicaOffsetLag ( ) ) ;
}
}
private static DescribeLogDirsResponseData . DescribeLogDirsResult prepareDescribeLogDirsResult ( TopicPartitionReplica tpr , String logDir , int partitionSize , int offsetLag , boolean isFuture ) {
return new DescribeLogDirsResponseData . DescribeLogDirsResult ( )
. setErrorCode ( Errors . NONE . code ( ) )
. setLogDir ( logDir )
. setTopics ( prepareDescribeLogDirsTopics ( partitionSize , offsetLag , tpr . topic ( ) , tpr . partition ( ) , isFuture ) ) ;
}
@Test
public void testDescribeReplicaLogDirsUnexpected ( ) throws ExecutionException , InterruptedException {
TopicPartitionReplica expected = new TopicPartitionReplica ( "topic" , 12 , 1 ) ;
TopicPartitionReplica unexpected = new TopicPartitionReplica ( "topic" , 12 , 2 ) ;
try ( AdminClientUnitTestEnv env = mockClientEnv ( ) ) {
env . kafkaClient ( ) . setNodeApiVersions ( NodeApiVersions . create ( ) ) ;
String broker1log0 = "/var/data/kafka0" ;
String broker1log1 = "/var/data/kafka1" ;
int broker1Log0PartitionSize = 987654321 ;
int broker1Log0OffsetLag = 24 ;
int broker1Log1PartitionSize = 123456789 ;
int broker1Log1OffsetLag = 4321 ;
env . kafkaClient ( ) . prepareResponseFrom (
new DescribeLogDirsResponse (
new DescribeLogDirsResponseData ( ) . setResults ( asList (
prepareDescribeLogDirsResult ( expected , broker1log0 , broker1Log0PartitionSize , broker1Log0OffsetLag , false ) ,
prepareDescribeLogDirsResult ( unexpected , broker1log1 , broker1Log1PartitionSize , broker1Log1OffsetLag , true ) ) ) ) ,
env . cluster ( ) . nodeById ( expected . brokerId ( ) ) ) ;
DescribeReplicaLogDirsResult result = env . adminClient ( ) . describeReplicaLogDirs ( asList ( expected ) ) ;
Map < TopicPartitionReplica , KafkaFuture < DescribeReplicaLogDirsResult . ReplicaLogDirInfo > > values = result . values ( ) ;
assertEquals ( TestUtils . toSet ( asList ( expected ) ) , values . keySet ( ) ) ;
assertNotNull ( values . get ( expected ) ) ;
assertEquals ( broker1log0 , values . get ( expected ) . get ( ) . getCurrentReplicaLogDir ( ) ) ;
assertEquals ( broker1Log0OffsetLag , values . get ( expected ) . get ( ) . getCurrentReplicaOffsetLag ( ) ) ;
assertEquals ( broker1log1 , values . get ( expected ) . get ( ) . getFutureReplicaLogDir ( ) ) ;
assertEquals ( broker1Log1OffsetLag , values . get ( expected ) . get ( ) . getFutureReplicaOffsetLag ( ) ) ;
}
}
@Test
public void testCreatePartitions ( ) throws Exception {
try ( AdminClientUnitTestEnv env = mockClientEnv ( ) ) {
@ -4187,8 +4446,8 @@ public class KafkaAdminClientTest {
@@ -4187,8 +4446,8 @@ public class KafkaAdminClientTest {
DescribeLogDirsResult result = env . adminClient ( ) . describeLogDirs ( Arrays . asList ( 0 , 1 ) ) ;
TestUtils . assertFutureThrows ( result . value s( ) . get ( 0 ) , ApiException . class ) ;
assertNotNull ( result . value s( ) . get ( 1 ) . get ( ) ) ;
TestUtils . assertFutureThrows ( result . description s( ) . get ( 0 ) , ApiException . class ) ;
assertNotNull ( result . description s( ) . get ( 1 ) . get ( ) ) ;
}
}