Date: Tue, 19 Mar 2024 10:48:01 +0000 (UTC) Message-ID: <1245434730.56267.1710845281528@cwiki-he-fi.apache.org> Subject: Exported From Confluence MIME-Version: 1.0 Content-Type: multipart/related; boundary="----=_Part_56266_1686656073.1710845281528" ------=_Part_56266_1686656073.1710845281528 Content-Type: text/html; charset=UTF-8 Content-Transfer-Encoding: quoted-printable Content-Location: file:///C:/exported.html
Current state: Under Discussion
Discussion thread: here<=
em>
JIRA: https://issues.apache.org=
/jira/browse/KAFKA-10490
Please keep the discussion on the mailing list rather than commenting on= the wiki (wiki discussions get unwieldy fast).
A common strate=
gy in the testing of software components that interact with an external sys=
tem is to construct a mock representing the behaviour of the external syste=
m. There are several JVM based solutions that provide mocking functionality=
such as Mockito, JMockit, and EasyMock.
When devising test strategi=
es for a system that interacts with a Kafka cluster, employing mocks that b=
ehave as the KafkaAdmin has a lot of benefits. It makes tests more robust, =
simpler and faster compared to interacting with a real cluster. However, co=
nstructing the value objects that such mocks would return is currently need=
lessly complicated because value object classes that the Kafka clients retu=
rns lack public constructors.
For example, consider a unit test for a piece of software =
that uses the AdminClient.deleteTopics()
method. Setting up a =
mock that conforms to the contract is easy enough, but creating a Del=
eteTopicsResult
instance to return from invoking the method on the m=
ock is not straight forward as
=
the constructor is=
declared with default access and can not be instantiated by code outside t=
he org.apache.kafka.clients.admin
package.
While there are ways to wor=
k around this limitation, for example by creating a mock of the value objec=
t to return, this adds complexity to any user that would want to mock any o=
f the kafka clients while testing their code. As a community, we should str=
ive for making good testing easy.
Mocking AdminCl= ient.deleteTopics() currently might look something like this:
DeleteTopicsResult deleteTopicsResult =3D mock(DeleteTopicsResult.clas= s);
when(deleteTopicsResult.values()).thenReturn(singletonMap(TOPIC_NAME= , KafkaFuture.completedFuture(null)));
AdminClient mockAdminClient = =3D mock(AdminClient.class);
when(mockAdminClient.deleteTopics(singleton= (TOPIC_NAME))).thenReturn(deleteTopicsResult);
Whereas making = constructors public would enable test writers to instead write:
AdminClient mockAdminClient =3D mock(AdminClient.class);
when(mockA= dminClient.deleteTopics(singleton(TOPIC_NAME))).thenReturn(
new Dele= teTopicsResult(singletonMap(TOPIC_NAME, KafkaFuture.completedFuture(null)))= )
);
Make the constr=
uctors of the following top level classes and their inner classes public: <=
/span>CreateT=
opicResult
DeleteTopicsResult<=
/code>
ListTopicsResult
DescribeTopicsResult
Descr=
ibeClusterResult
DescribeAclsResult=
span>CreateAclsResult
DeleteAclsResult
Des=
cribeConfigsResult
=
AlterConfigsResult=
AlterReplicaLogDirsResult
DescribeLogDirsResult
DescribeReplicaLogsDirsResult
CreatePartitionsResult
DeleteRe=
cordsResult
CreateDelegationTokenResu=
lt
RenewDelegationTokenResult<=
/code>
ExpireDelegationTokenResult
DescribeDelegationTokenResult
ListConsumerGroupsResult
ListConsumerGroupOffsetsResult
El=
ectLeadersResult
AlterPartitionReassi=
gnmentResultListPartitonReassignment=
sResult
RemoveMembersFromConsumerResu=
lt
AlterConsumerGroupOffsetsResult
ListOffsetsResult
DescribeClientQuotasResult
AlterClientQuotasResult
Dscrib=
eUserScramCredentialsResult
AlterUser=
ScramCredentialsResult
UpdateFeaturesResult<=
/code>
This KIP prop=
oses no change in functionality, just a change in the access modifiers for =
the mentioned constructors to make available the already existing functiona=
lity to users outside of the Apache Kafka codebase.
There should be=
no compatibility and migration neccessary for this change. Some Kafka test=
cases in for example ConfigCommandTest
could be simplified to=
not use mock instances of value objects when it makes sense, but this is p=
urely optional.
Besides leaving= this as it is, one might envision some sort of factory setup where constru= ction of value objects would be delegated to a separate class. This would a= dd indirection and complexity with the questionable gain of having a slight= ly smaller public footprint in the Apache Kafka admin client.