/
AuthorizerIntegrationTest.scala
2658 lines (2304 loc) · 132 KB
/
AuthorizerIntegrationTest.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/**
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
* file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
* to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
* License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*/
package kafka.api
import java.lang.{Byte => JByte}
import java.time.Duration
import java.util
import java.util.concurrent.ExecutionException
import java.util.regex.Pattern
import java.util.{Collections, Optional, Properties}
import kafka.admin.ConsumerGroupCommand.{ConsumerGroupCommandOptions, ConsumerGroupService}
import kafka.security.authorizer.{AclAuthorizer, AclEntry}
import kafka.security.authorizer.AclEntry.WildcardHost
import kafka.server.{BaseRequestTest, KafkaConfig}
import kafka.utils.{TestInfoUtils, TestUtils}
import kafka.utils.TestUtils.waitUntilTrue
import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, NewTopic}
import org.apache.kafka.clients.consumer._
import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener
import org.apache.kafka.clients.producer._
import org.apache.kafka.common.acl.AclOperation._
import org.apache.kafka.common.acl.AclPermissionType.{ALLOW, DENY}
import org.apache.kafka.common.acl.{AccessControlEntry, AccessControlEntryFilter, AclBindingFilter, AclOperation, AclPermissionType}
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
import org.apache.kafka.common.config.{ConfigResource, LogLevelConfig, TopicConfig}
import org.apache.kafka.common.errors._
import org.apache.kafka.common.internals.Topic.GROUP_METADATA_TOPIC_NAME
import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic
import org.apache.kafka.common.message.CreateTopicsRequestData.{CreatableTopic, CreatableTopicCollection}
import org.apache.kafka.common.message.IncrementalAlterConfigsRequestData.{AlterConfigsResource, AlterableConfig, AlterableConfigCollection}
import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocolCollection
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity
import org.apache.kafka.common.message.ListOffsetsRequestData.{ListOffsetsPartition, ListOffsetsTopic}
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.{OffsetForLeaderPartition, OffsetForLeaderTopic, OffsetForLeaderTopicCollection}
import org.apache.kafka.common.message.StopReplicaRequestData.{StopReplicaPartitionState, StopReplicaTopicState}
import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadataBroker, UpdateMetadataEndpoint, UpdateMetadataPartitionState}
import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AlterPartitionReassignmentsRequestData, AlterReplicaLogDirsRequestData, ControlledShutdownRequestData, CreateAclsRequestData, CreatePartitionsRequestData, CreateTopicsRequestData, DeleteAclsRequestData, DeleteGroupsRequestData, DeleteRecordsRequestData, DeleteTopicsRequestData, DescribeClusterRequestData, DescribeConfigsRequestData, DescribeGroupsRequestData, DescribeLogDirsRequestData, DescribeProducersRequestData, DescribeTransactionsRequestData, FindCoordinatorRequestData, HeartbeatRequestData, IncrementalAlterConfigsRequestData, JoinGroupRequestData, ListPartitionReassignmentsRequestData, ListTransactionsRequestData, MetadataRequestData, OffsetCommitRequestData, ProduceRequestData, SyncGroupRequestData}
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, SimpleRecord}
import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData
import org.apache.kafka.common.requests._
import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED}
import org.apache.kafka.common.resource.ResourceType._
import org.apache.kafka.common.resource.{PatternType, Resource, ResourcePattern, ResourcePatternFilter, ResourceType}
import org.apache.kafka.common.security.auth.{AuthenticationContext, KafkaPrincipal, SecurityProtocol}
import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder
import org.apache.kafka.common.utils.Utils
import org.apache.kafka.common.{ElectionType, IsolationLevel, KafkaException, Node, TopicPartition, Uuid, requests}
import org.apache.kafka.metadata.authorizer.StandardAuthorizer
import org.apache.kafka.test.{TestUtils => JTestUtils}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.{CsvSource, ValueSource}
import java.util.Collections.singletonList
import org.apache.kafka.common.message.MetadataRequestData.MetadataRequestTopic
import org.junit.jupiter.api.function.Executable
import scala.annotation.nowarn
import scala.collection.mutable
import scala.jdk.CollectionConverters._
object AuthorizerIntegrationTest {
val BrokerPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "broker")
val ClientPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "client")
val BrokerListenerName = "BROKER"
val ClientListenerName = "CLIENT"
val ControllerListenerName = "CONTROLLER"
class PrincipalBuilder extends DefaultKafkaPrincipalBuilder(null, null) {
override def build(context: AuthenticationContext): KafkaPrincipal = {
context.listenerName match {
case BrokerListenerName | ControllerListenerName => BrokerPrincipal
case ClientListenerName => ClientPrincipal
case listenerName => throw new IllegalArgumentException(s"No principal mapped to listener $listenerName")
}
}
}
}
class AuthorizerIntegrationTest extends BaseRequestTest {
import AuthorizerIntegrationTest._
override def interBrokerListenerName: ListenerName = new ListenerName(BrokerListenerName)
override def listenerName: ListenerName = new ListenerName(ClientListenerName)
override def brokerCount: Int = 1
def clientPrincipal: KafkaPrincipal = ClientPrincipal
def brokerPrincipal: KafkaPrincipal = BrokerPrincipal
val clientPrincipalString: String = clientPrincipal.toString
val brokerId: Integer = 0
val topic = "topic"
val topicPattern = "topic.*"
val transactionalId = "transactional.id"
val producerId = 83392L
val part = 0
val correlationId = 0
val clientId = "client-Id"
val tp = new TopicPartition(topic, part)
val logDir = "logDir"
val group = "my-group"
val protocolType = "consumer"
val protocolName = "consumer-range"
val clusterResource = new ResourcePattern(CLUSTER, Resource.CLUSTER_NAME, LITERAL)
val topicResource = new ResourcePattern(TOPIC, topic, LITERAL)
val groupResource = new ResourcePattern(GROUP, group, LITERAL)
val transactionalIdResource = new ResourcePattern(TRANSACTIONAL_ID, transactionalId, LITERAL)
val groupReadAcl = Map(groupResource -> Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)))
val groupDescribeAcl = Map(groupResource -> Set(new AccessControlEntry(clientPrincipalString, WildcardHost, DESCRIBE, ALLOW)))
val groupDeleteAcl = Map(groupResource -> Set(new AccessControlEntry(clientPrincipalString, WildcardHost, DELETE, ALLOW)))
val clusterAcl = Map(clusterResource -> Set(new AccessControlEntry(clientPrincipalString, WildcardHost, CLUSTER_ACTION, ALLOW)))
val clusterCreateAcl = Map(clusterResource -> Set(new AccessControlEntry(clientPrincipalString, WildcardHost, CREATE, ALLOW)))
val clusterAlterAcl = Map(clusterResource -> Set(new AccessControlEntry(clientPrincipalString, WildcardHost, ALTER, ALLOW)))
val clusterDescribeAcl = Map(clusterResource -> Set(new AccessControlEntry(clientPrincipalString, WildcardHost, DESCRIBE, ALLOW)))
val clusterAlterConfigsAcl = Map(clusterResource -> Set(new AccessControlEntry(clientPrincipalString, WildcardHost, ALTER_CONFIGS, ALLOW)))
val clusterIdempotentWriteAcl = Map(clusterResource -> Set(new AccessControlEntry(clientPrincipalString, WildcardHost, IDEMPOTENT_WRITE, ALLOW)))
val topicCreateAcl = Map(topicResource -> Set(new AccessControlEntry(clientPrincipalString, WildcardHost, CREATE, ALLOW)))
val topicReadAcl = Map(topicResource -> Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)))
val topicWriteAcl = Map(topicResource -> Set(new AccessControlEntry(clientPrincipalString, WildcardHost, WRITE, ALLOW)))
val topicDescribeAcl = Map(topicResource -> Set(new AccessControlEntry(clientPrincipalString, WildcardHost, DESCRIBE, ALLOW)))
val topicAlterAcl = Map(topicResource -> Set(new AccessControlEntry(clientPrincipalString, WildcardHost, ALTER, ALLOW)))
val topicDeleteAcl = Map(topicResource -> Set(new AccessControlEntry(clientPrincipalString, WildcardHost, DELETE, ALLOW)))
val topicDescribeConfigsAcl = Map(topicResource -> Set(new AccessControlEntry(clientPrincipalString, WildcardHost, DESCRIBE_CONFIGS, ALLOW)))
val topicAlterConfigsAcl = Map(topicResource -> Set(new AccessControlEntry(clientPrincipalString, WildcardHost, ALTER_CONFIGS, ALLOW)))
val transactionIdWriteAcl = Map(transactionalIdResource -> Set(new AccessControlEntry(clientPrincipalString, WildcardHost, WRITE, ALLOW)))
val transactionalIdDescribeAcl = Map(transactionalIdResource -> Set(new AccessControlEntry(clientPrincipalString, WildcardHost, DESCRIBE, ALLOW)))
val numRecords = 1
producerConfig.setProperty(ProducerConfig.ACKS_CONFIG, "1")
producerConfig.setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "false")
producerConfig.setProperty(ProducerConfig.MAX_BLOCK_MS_CONFIG, "50000")
consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, group)
override def brokerPropertyOverrides(properties: Properties): Unit = {
properties.put(KafkaConfig.BrokerIdProp, brokerId.toString)
addNodeProperties(properties)
}
override def kraftControllerConfigs(): collection.Seq[Properties] = {
val controllerConfigs = super.kraftControllerConfigs()
controllerConfigs.foreach(addNodeProperties)
controllerConfigs
}
private def addNodeProperties(properties: Properties): Unit = {
if (isKRaftTest()) {
properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName)
properties.put(StandardAuthorizer.SUPER_USERS_CONFIG, BrokerPrincipal.toString)
} else {
properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[AclAuthorizer].getName)
}
properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
properties.put(KafkaConfig.TransactionsTopicPartitionsProp, "1")
properties.put(KafkaConfig.TransactionsTopicReplicationFactorProp, "1")
properties.put(KafkaConfig.TransactionsTopicMinISRProp, "1")
properties.put(KafkaConfig.UnstableApiVersionsEnableProp, "true")
properties.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[PrincipalBuilder].getName)
}
val requestKeyToError = (topicNames: Map[Uuid, String], version: Short) => Map[ApiKeys, Nothing => Errors](
ApiKeys.METADATA -> ((resp: requests.MetadataResponse) => resp.errors.asScala.find(_._1 == topic).getOrElse(("test", Errors.NONE))._2),
ApiKeys.PRODUCE -> ((resp: requests.ProduceResponse) => {
Errors.forCode(
resp.data
.responses.find(topic)
.partitionResponses.asScala.find(_.index == part).get
.errorCode
)
}),
// We may need to get the top level error if the topic does not exist in the response
ApiKeys.FETCH -> ((resp: requests.FetchResponse) => Errors.forCode(resp.responseData(topicNames.asJava, version).asScala.find {
case (topicPartition, _) => topicPartition == tp}.map { case (_, data) => data.errorCode }.getOrElse(resp.error.code()))),
ApiKeys.LIST_OFFSETS -> ((resp: ListOffsetsResponse) => {
Errors.forCode(
resp.data
.topics.asScala.find(_.name == topic).get
.partitions.asScala.find(_.partitionIndex == part).get
.errorCode
)
}),
ApiKeys.OFFSET_COMMIT -> ((resp: requests.OffsetCommitResponse) => Errors.forCode(
resp.data.topics().get(0).partitions().get(0).errorCode)),
ApiKeys.OFFSET_FETCH -> ((resp: requests.OffsetFetchResponse) => resp.groupLevelError(group)),
ApiKeys.FIND_COORDINATOR -> ((resp: FindCoordinatorResponse) => {
Errors.forCode(resp.data.coordinators.asScala.find(g => group == g.key).head.errorCode)
}),
ApiKeys.UPDATE_METADATA -> ((resp: requests.UpdateMetadataResponse) => resp.error),
ApiKeys.JOIN_GROUP -> ((resp: JoinGroupResponse) => resp.error),
ApiKeys.SYNC_GROUP -> ((resp: SyncGroupResponse) => Errors.forCode(resp.data.errorCode)),
ApiKeys.DESCRIBE_GROUPS -> ((resp: DescribeGroupsResponse) => {
Errors.forCode(resp.data.groups.asScala.find(g => group == g.groupId).head.errorCode)
}),
ApiKeys.HEARTBEAT -> ((resp: HeartbeatResponse) => resp.error),
ApiKeys.LEAVE_GROUP -> ((resp: LeaveGroupResponse) => resp.error),
ApiKeys.DELETE_GROUPS -> ((resp: DeleteGroupsResponse) => resp.get(group)),
ApiKeys.LEADER_AND_ISR -> ((resp: requests.LeaderAndIsrResponse) => Errors.forCode(
resp.topics.asScala.find(t => topicNames(t.topicId) == tp.topic).get.partitionErrors.asScala.find(
p => p.partitionIndex == tp.partition).get.errorCode)),
ApiKeys.STOP_REPLICA -> ((resp: requests.StopReplicaResponse) => Errors.forCode(
resp.partitionErrors.asScala.find(pe => pe.topicName == tp.topic && pe.partitionIndex == tp.partition).get.errorCode)),
ApiKeys.CONTROLLED_SHUTDOWN -> ((resp: requests.ControlledShutdownResponse) => resp.error),
ApiKeys.CREATE_TOPICS -> ((resp: CreateTopicsResponse) => Errors.forCode(resp.data.topics.find(topic).errorCode)),
ApiKeys.DELETE_TOPICS -> ((resp: requests.DeleteTopicsResponse) => Errors.forCode(resp.data.responses.find(topic).errorCode)),
ApiKeys.DELETE_RECORDS -> ((resp: requests.DeleteRecordsResponse) => Errors.forCode(
resp.data.topics.find(tp.topic).partitions.find(tp.partition).errorCode)),
ApiKeys.OFFSET_FOR_LEADER_EPOCH -> ((resp: OffsetsForLeaderEpochResponse) => Errors.forCode(
resp.data.topics.find(tp.topic).partitions.asScala.find(_.partition == tp.partition).get.errorCode)),
ApiKeys.DESCRIBE_CONFIGS -> ((resp: DescribeConfigsResponse) =>
Errors.forCode(resp.resultMap.get(new ConfigResource(ConfigResource.Type.TOPIC, tp.topic)).errorCode)),
ApiKeys.ALTER_CONFIGS -> ((resp: AlterConfigsResponse) =>
resp.errors.get(new ConfigResource(ConfigResource.Type.TOPIC, tp.topic)).error),
ApiKeys.INIT_PRODUCER_ID -> ((resp: InitProducerIdResponse) => resp.error),
ApiKeys.WRITE_TXN_MARKERS -> ((resp: WriteTxnMarkersResponse) => resp.errorsByProducerId.get(producerId).get(tp)),
ApiKeys.ADD_PARTITIONS_TO_TXN -> ((resp: AddPartitionsToTxnResponse) => resp.errors.get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID).get(tp)),
ApiKeys.ADD_OFFSETS_TO_TXN -> ((resp: AddOffsetsToTxnResponse) => Errors.forCode(resp.data.errorCode)),
ApiKeys.END_TXN -> ((resp: EndTxnResponse) => resp.error),
ApiKeys.TXN_OFFSET_COMMIT -> ((resp: TxnOffsetCommitResponse) => resp.errors.get(tp)),
ApiKeys.CREATE_ACLS -> ((resp: CreateAclsResponse) => Errors.forCode(resp.results.asScala.head.errorCode)),
ApiKeys.DESCRIBE_ACLS -> ((resp: DescribeAclsResponse) => resp.error.error),
ApiKeys.DELETE_ACLS -> ((resp: DeleteAclsResponse) => Errors.forCode(resp.filterResults.asScala.head.errorCode)),
ApiKeys.ALTER_REPLICA_LOG_DIRS -> ((resp: AlterReplicaLogDirsResponse) => Errors.forCode(resp.data.results.asScala
.find(x => x.topicName == tp.topic).get.partitions.asScala
.find(p => p.partitionIndex == tp.partition).get.errorCode)),
ApiKeys.DESCRIBE_LOG_DIRS -> ((resp: DescribeLogDirsResponse) =>
Errors.forCode(if (resp.data.results.size > 0) resp.data.results.get(0).errorCode else resp.data.errorCode)),
ApiKeys.CREATE_PARTITIONS -> ((resp: CreatePartitionsResponse) => Errors.forCode(resp.data.results.asScala.head.errorCode)),
ApiKeys.ELECT_LEADERS -> ((resp: ElectLeadersResponse) => Errors.forCode(resp.data.errorCode)),
ApiKeys.INCREMENTAL_ALTER_CONFIGS -> ((resp: IncrementalAlterConfigsResponse) => {
val topicResourceError = IncrementalAlterConfigsResponse.fromResponseData(resp.data).get(new ConfigResource(ConfigResource.Type.TOPIC, tp.topic))
if (topicResourceError == null)
IncrementalAlterConfigsResponse.fromResponseData(resp.data).get(new ConfigResource(ConfigResource.Type.BROKER_LOGGER, brokerId.toString)).error
else
topicResourceError.error()
}),
ApiKeys.ALTER_PARTITION_REASSIGNMENTS -> ((resp: AlterPartitionReassignmentsResponse) => Errors.forCode(resp.data.errorCode)),
ApiKeys.LIST_PARTITION_REASSIGNMENTS -> ((resp: ListPartitionReassignmentsResponse) => Errors.forCode(resp.data.errorCode)),
ApiKeys.OFFSET_DELETE -> ((resp: OffsetDeleteResponse) => {
Errors.forCode(
resp.data
.topics.asScala.find(_.name == topic).get
.partitions.asScala.find(_.partitionIndex == part).get
.errorCode
)
}),
ApiKeys.DESCRIBE_PRODUCERS -> ((resp: DescribeProducersResponse) => {
Errors.forCode(
resp.data
.topics.asScala.find(_.name == topic).get
.partitions.asScala.find(_.partitionIndex == part).get
.errorCode
)
}),
ApiKeys.DESCRIBE_TRANSACTIONS -> ((resp: DescribeTransactionsResponse) => {
Errors.forCode(
resp.data
.transactionStates.asScala.find(_.transactionalId == transactionalId).get
.errorCode
)
})
)
def findErrorForTopicId(id: Uuid, response: AbstractResponse): Errors = {
response match {
case res: DeleteTopicsResponse =>
Errors.forCode(res.data.responses.asScala.find(_.topicId == id).get.errorCode)
case _ =>
fail(s"Unexpected response type $response")
}
}
val requestKeysToAcls = Map[ApiKeys, Map[ResourcePattern, Set[AccessControlEntry]]](
ApiKeys.METADATA -> topicDescribeAcl,
ApiKeys.PRODUCE -> (topicWriteAcl ++ transactionIdWriteAcl ++ clusterIdempotentWriteAcl),
ApiKeys.FETCH -> topicReadAcl,
ApiKeys.LIST_OFFSETS -> topicDescribeAcl,
ApiKeys.OFFSET_COMMIT -> (topicReadAcl ++ groupReadAcl),
ApiKeys.OFFSET_FETCH -> (topicReadAcl ++ groupDescribeAcl),
ApiKeys.FIND_COORDINATOR -> (topicReadAcl ++ groupDescribeAcl ++ transactionalIdDescribeAcl),
ApiKeys.UPDATE_METADATA -> clusterAcl,
ApiKeys.JOIN_GROUP -> groupReadAcl,
ApiKeys.SYNC_GROUP -> groupReadAcl,
ApiKeys.DESCRIBE_GROUPS -> groupDescribeAcl,
ApiKeys.HEARTBEAT -> groupReadAcl,
ApiKeys.LEAVE_GROUP -> groupReadAcl,
ApiKeys.DELETE_GROUPS -> groupDeleteAcl,
ApiKeys.LEADER_AND_ISR -> clusterAcl,
ApiKeys.STOP_REPLICA -> clusterAcl,
ApiKeys.CONTROLLED_SHUTDOWN -> clusterAcl,
ApiKeys.CREATE_TOPICS -> topicCreateAcl,
ApiKeys.DELETE_TOPICS -> topicDeleteAcl,
ApiKeys.DELETE_RECORDS -> topicDeleteAcl,
ApiKeys.OFFSET_FOR_LEADER_EPOCH -> topicDescribeAcl,
ApiKeys.DESCRIBE_CONFIGS -> topicDescribeConfigsAcl,
ApiKeys.ALTER_CONFIGS -> topicAlterConfigsAcl,
ApiKeys.INIT_PRODUCER_ID -> (transactionIdWriteAcl ++ clusterIdempotentWriteAcl),
ApiKeys.WRITE_TXN_MARKERS -> clusterAcl,
ApiKeys.ADD_PARTITIONS_TO_TXN -> (topicWriteAcl ++ transactionIdWriteAcl),
ApiKeys.ADD_OFFSETS_TO_TXN -> (groupReadAcl ++ transactionIdWriteAcl),
ApiKeys.END_TXN -> transactionIdWriteAcl,
ApiKeys.TXN_OFFSET_COMMIT -> (groupReadAcl ++ transactionIdWriteAcl),
ApiKeys.CREATE_ACLS -> clusterAlterAcl,
ApiKeys.DESCRIBE_ACLS -> clusterDescribeAcl,
ApiKeys.DELETE_ACLS -> clusterAlterAcl,
ApiKeys.ALTER_REPLICA_LOG_DIRS -> clusterAlterAcl,
ApiKeys.DESCRIBE_LOG_DIRS -> clusterDescribeAcl,
ApiKeys.CREATE_PARTITIONS -> topicAlterAcl,
ApiKeys.ELECT_LEADERS -> clusterAlterAcl,
ApiKeys.INCREMENTAL_ALTER_CONFIGS -> topicAlterConfigsAcl,
ApiKeys.ALTER_PARTITION_REASSIGNMENTS -> clusterAlterAcl,
ApiKeys.LIST_PARTITION_REASSIGNMENTS -> clusterDescribeAcl,
ApiKeys.OFFSET_DELETE -> groupReadAcl,
ApiKeys.DESCRIBE_PRODUCERS -> topicReadAcl,
ApiKeys.DESCRIBE_TRANSACTIONS -> transactionalIdDescribeAcl
)
@BeforeEach
override def setUp(testInfo: TestInfo): Unit = {
doSetup(testInfo, createOffsetsTopic = false)
// Allow inter-broker communication
addAndVerifyAcls(Set(new AccessControlEntry(brokerPrincipal.toString, WildcardHost, CLUSTER_ACTION, ALLOW)), clusterResource)
createOffsetsTopic(listenerName = interBrokerListenerName)
}
@AfterEach
override def tearDown(): Unit = {
removeAllClientAcls()
super.tearDown()
}
private def createMetadataRequest(allowAutoTopicCreation: Boolean) = {
new requests.MetadataRequest.Builder(List(topic).asJava, allowAutoTopicCreation).build()
}
private def createProduceRequest =
requests.ProduceRequest.forCurrentMagic(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(
Collections.singletonList(new ProduceRequestData.TopicProduceData()
.setName(tp.topic).setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData()
.setIndex(tp.partition)
.setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes))))))
.iterator))
.setAcks(1.toShort)
.setTimeoutMs(5000))
.build()
private def createFetchRequest = {
val partitionMap = new util.LinkedHashMap[TopicPartition, requests.FetchRequest.PartitionData]
partitionMap.put(tp, new requests.FetchRequest.PartitionData(getTopicIds().getOrElse(tp.topic, Uuid.ZERO_UUID),
0, 0, 100, Optional.of(27)))
requests.FetchRequest.Builder.forConsumer(ApiKeys.FETCH.latestVersion, 100, Int.MaxValue, partitionMap).build()
}
private def createFetchRequestWithUnknownTopic(id: Uuid, version: Short) = {
val partitionMap = new util.LinkedHashMap[TopicPartition, requests.FetchRequest.PartitionData]
partitionMap.put(tp,
new requests.FetchRequest.PartitionData(id, 0, 0, 100, Optional.of(27)))
requests.FetchRequest.Builder.forConsumer(version, 100, Int.MaxValue, partitionMap).build()
}
private def createFetchFollowerRequest = {
val partitionMap = new util.LinkedHashMap[TopicPartition, requests.FetchRequest.PartitionData]
partitionMap.put(tp, new requests.FetchRequest.PartitionData(getTopicIds().getOrElse(tp.topic, Uuid.ZERO_UUID),
0, 0, 100, Optional.of(27)))
val version = ApiKeys.FETCH.latestVersion
requests.FetchRequest.Builder.forReplica(version, 5000, -1, 100, Int.MaxValue, partitionMap).build()
}
private def createListOffsetsRequest = {
requests.ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false).setTargetTimes(
List(new ListOffsetsTopic()
.setName(tp.topic)
.setPartitions(List(new ListOffsetsPartition()
.setPartitionIndex(tp.partition)
.setTimestamp(0L)
.setCurrentLeaderEpoch(27)).asJava)).asJava
).
build()
}
private def offsetsForLeaderEpochRequest: OffsetsForLeaderEpochRequest = {
val epochs = new OffsetForLeaderTopicCollection()
epochs.add(new OffsetForLeaderTopic()
.setTopic(tp.topic)
.setPartitions(List(new OffsetForLeaderPartition()
.setPartition(tp.partition)
.setLeaderEpoch(7)
.setCurrentLeaderEpoch(27)).asJava))
OffsetsForLeaderEpochRequest.Builder.forConsumer(epochs).build()
}
private def createOffsetFetchRequest: OffsetFetchRequest = {
new requests.OffsetFetchRequest.Builder(group, false, List(tp).asJava, false).build()
}
private def createOffsetFetchRequestAllPartitions: OffsetFetchRequest = {
new requests.OffsetFetchRequest.Builder(group, false, null, false).build()
}
private def createOffsetFetchRequest(groupToPartitionMap: util.Map[String, util.List[TopicPartition]]): OffsetFetchRequest = {
new requests.OffsetFetchRequest.Builder(groupToPartitionMap, false, false).build()
}
private def createFindCoordinatorRequest = {
new FindCoordinatorRequest.Builder(
new FindCoordinatorRequestData()
.setKeyType(FindCoordinatorRequest.CoordinatorType.GROUP.id)
.setCoordinatorKeys(Collections.singletonList(group))).build()
}
private def createUpdateMetadataRequest = {
val partitionStates = Seq(new UpdateMetadataPartitionState()
.setTopicName(tp.topic)
.setPartitionIndex(tp.partition)
.setControllerEpoch(Int.MaxValue)
.setLeader(brokerId)
.setLeaderEpoch(Int.MaxValue)
.setIsr(List(brokerId).asJava)
.setZkVersion(2)
.setReplicas(Seq(brokerId).asJava)).asJava
val securityProtocol = SecurityProtocol.PLAINTEXT
val brokers = Seq(new UpdateMetadataBroker()
.setId(brokerId)
.setEndpoints(Seq(new UpdateMetadataEndpoint()
.setHost("localhost")
.setPort(0)
.setSecurityProtocol(securityProtocol.id)
.setListener(ListenerName.forSecurityProtocol(securityProtocol).value)).asJava)).asJava
val version = ApiKeys.UPDATE_METADATA.latestVersion
new requests.UpdateMetadataRequest.Builder(version, brokerId, Int.MaxValue, Long.MaxValue, partitionStates,
brokers, Collections.emptyMap()).build()
}
private def createJoinGroupRequest = {
val protocolSet = new JoinGroupRequestProtocolCollection(
Collections.singletonList(new JoinGroupRequestData.JoinGroupRequestProtocol()
.setName(protocolName)
.setMetadata("test".getBytes())
).iterator())
new JoinGroupRequest.Builder(
new JoinGroupRequestData()
.setGroupId(group)
.setSessionTimeoutMs(10000)
.setMemberId(JoinGroupRequest.UNKNOWN_MEMBER_ID)
.setGroupInstanceId(null)
.setProtocolType(protocolType)
.setProtocols(protocolSet)
.setRebalanceTimeoutMs(60000)
).build()
}
private def createSyncGroupRequest = {
new SyncGroupRequest.Builder(
new SyncGroupRequestData()
.setGroupId(group)
.setGenerationId(1)
.setMemberId(JoinGroupRequest.UNKNOWN_MEMBER_ID)
.setProtocolType(protocolType)
.setProtocolName(protocolName)
.setAssignments(Collections.emptyList())
).build()
}
private def createDescribeGroupsRequest = {
new DescribeGroupsRequest.Builder(new DescribeGroupsRequestData().setGroups(List(group).asJava)).build()
}
private def createOffsetCommitRequest = {
new requests.OffsetCommitRequest.Builder(
new OffsetCommitRequestData()
.setGroupId(group)
.setMemberId(JoinGroupRequest.UNKNOWN_MEMBER_ID)
.setGenerationIdOrMemberEpoch(1)
.setTopics(Collections.singletonList(
new OffsetCommitRequestData.OffsetCommitRequestTopic()
.setName(topic)
.setPartitions(Collections.singletonList(
new OffsetCommitRequestData.OffsetCommitRequestPartition()
.setPartitionIndex(part)
.setCommittedOffset(0)
.setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
.setCommitTimestamp(OffsetCommitRequest.DEFAULT_TIMESTAMP)
.setCommittedMetadata("metadata")
)))
)
).build()
}
private def createPartitionsRequest = {
val partitionTopic = new CreatePartitionsTopic()
.setName(topic)
.setCount(10)
.setAssignments(null)
val data = new CreatePartitionsRequestData()
.setTimeoutMs(10000)
.setValidateOnly(true)
data.topics().add(partitionTopic)
new CreatePartitionsRequest.Builder(data).build(0.toShort)
}
private def heartbeatRequest = new HeartbeatRequest.Builder(
new HeartbeatRequestData()
.setGroupId(group)
.setGenerationId(1)
.setMemberId(JoinGroupRequest.UNKNOWN_MEMBER_ID)).build()
private def leaveGroupRequest = new LeaveGroupRequest.Builder(
group, Collections.singletonList(
new MemberIdentity()
.setMemberId(JoinGroupRequest.UNKNOWN_MEMBER_ID)
)).build()
private def deleteGroupsRequest = new DeleteGroupsRequest.Builder(
new DeleteGroupsRequestData()
.setGroupsNames(Collections.singletonList(group))
).build()
private def leaderAndIsrRequest: LeaderAndIsrRequest = {
new requests.LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, brokerId, Int.MaxValue, Long.MaxValue,
Seq(new LeaderAndIsrPartitionState()
.setTopicName(tp.topic)
.setPartitionIndex(tp.partition)
.setControllerEpoch(Int.MaxValue)
.setLeader(brokerId)
.setLeaderEpoch(Int.MaxValue)
.setIsr(List(brokerId).asJava)
.setPartitionEpoch(2)
.setReplicas(Seq(brokerId).asJava)
.setIsNew(false)).asJava,
getTopicIds().asJava,
Set(new Node(brokerId, "localhost", 0)).asJava).build()
}
private def stopReplicaRequest: StopReplicaRequest = {
val topicStates = Seq(
new StopReplicaTopicState()
.setTopicName(tp.topic)
.setPartitionStates(Seq(new StopReplicaPartitionState()
.setPartitionIndex(tp.partition)
.setLeaderEpoch(LeaderAndIsr.InitialLeaderEpoch + 2)
.setDeletePartition(true)).asJava)
).asJava
new StopReplicaRequest.Builder(ApiKeys.STOP_REPLICA.latestVersion, brokerId, Int.MaxValue,
Long.MaxValue, false, topicStates).build()
}
private def controlledShutdownRequest: ControlledShutdownRequest = {
new ControlledShutdownRequest.Builder(
new ControlledShutdownRequestData()
.setBrokerId(brokerId)
.setBrokerEpoch(Long.MaxValue),
ApiKeys.CONTROLLED_SHUTDOWN.latestVersion).build()
}
private def createTopicsRequest: CreateTopicsRequest = {
new CreateTopicsRequest.Builder(new CreateTopicsRequestData().setTopics(
new CreatableTopicCollection(Collections.singleton(new CreatableTopic().
setName(topic).setNumPartitions(1).
setReplicationFactor(1.toShort)).iterator))).build()
}
private def deleteTopicsRequest: DeleteTopicsRequest = {
new DeleteTopicsRequest.Builder(
new DeleteTopicsRequestData()
.setTopicNames(Collections.singletonList(topic))
.setTimeoutMs(5000)).build()
}
private def deleteTopicsWithIdsRequest(topicId: Uuid): DeleteTopicsRequest = {
new DeleteTopicsRequest.Builder(
new DeleteTopicsRequestData()
.setTopics(Collections.singletonList(
new DeleteTopicsRequestData.DeleteTopicState()
.setTopicId(topicId)))
.setTimeoutMs(5000)).build()
}
private def deleteRecordsRequest = new DeleteRecordsRequest.Builder(
new DeleteRecordsRequestData()
.setTimeoutMs(5000)
.setTopics(Collections.singletonList(new DeleteRecordsRequestData.DeleteRecordsTopic()
.setName(tp.topic)
.setPartitions(Collections.singletonList(new DeleteRecordsRequestData.DeleteRecordsPartition()
.setPartitionIndex(tp.partition)
.setOffset(0L)))))).build()
private def describeConfigsRequest =
new DescribeConfigsRequest.Builder(new DescribeConfigsRequestData().setResources(Collections.singletonList(
new DescribeConfigsRequestData.DescribeConfigsResource().setResourceType(ConfigResource.Type.TOPIC.id)
.setResourceName(tp.topic)))).build()
private def alterConfigsRequest =
new AlterConfigsRequest.Builder(
Collections.singletonMap(new ConfigResource(ConfigResource.Type.TOPIC, tp.topic),
new AlterConfigsRequest.Config(Collections.singleton(
new AlterConfigsRequest.ConfigEntry(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, "1000000")
))), true).build()
private def incrementalAlterConfigsRequest = {
val data = new IncrementalAlterConfigsRequestData
val alterableConfig = new AlterableConfig
alterableConfig.setName(TopicConfig.MAX_MESSAGE_BYTES_CONFIG).
setValue("1000000").setConfigOperation(AlterConfigOp.OpType.SET.id())
val alterableConfigSet = new AlterableConfigCollection
alterableConfigSet.add(alterableConfig)
data.resources().add(new AlterConfigsResource().
setResourceName(tp.topic).setResourceType(ConfigResource.Type.TOPIC.id()).
setConfigs(alterableConfigSet))
new IncrementalAlterConfigsRequest.Builder(data).build()
}
private def describeAclsRequest = new DescribeAclsRequest.Builder(AclBindingFilter.ANY).build()
private def createAclsRequest: CreateAclsRequest = new CreateAclsRequest.Builder(
new CreateAclsRequestData().setCreations(Collections.singletonList(
new CreateAclsRequestData.AclCreation()
.setResourceType(ResourceType.TOPIC.code)
.setResourceName("mytopic")
.setResourcePatternType(PatternType.LITERAL.code)
.setPrincipal(clientPrincipalString)
.setHost("*")
.setOperation(AclOperation.WRITE.code)
.setPermissionType(AclPermissionType.DENY.code)))
).build()
private def deleteAclsRequest: DeleteAclsRequest = new DeleteAclsRequest.Builder(
new DeleteAclsRequestData().setFilters(Collections.singletonList(
new DeleteAclsRequestData.DeleteAclsFilter()
.setResourceTypeFilter(ResourceType.TOPIC.code)
.setResourceNameFilter(null)
.setPatternTypeFilter(PatternType.LITERAL.code)
.setPrincipalFilter(clientPrincipalString)
.setHostFilter("*")
.setOperation(AclOperation.ANY.code)
.setPermissionType(AclPermissionType.DENY.code)))
).build()
private def alterReplicaLogDirsRequest = {
val dir = new AlterReplicaLogDirsRequestData.AlterReplicaLogDir()
.setPath(logDir)
dir.topics.add(new AlterReplicaLogDirsRequestData.AlterReplicaLogDirTopic()
.setName(tp.topic)
.setPartitions(Collections.singletonList(tp.partition)))
val data = new AlterReplicaLogDirsRequestData();
data.dirs.add(dir)
new AlterReplicaLogDirsRequest.Builder(data).build()
}
private def describeLogDirsRequest = new DescribeLogDirsRequest.Builder(new DescribeLogDirsRequestData().setTopics(new DescribeLogDirsRequestData.DescribableLogDirTopicCollection(Collections.singleton(
new DescribeLogDirsRequestData.DescribableLogDirTopic().setTopic(tp.topic).setPartitions(Collections.singletonList(tp.partition))).iterator()))).build()
private def addPartitionsToTxnRequest = AddPartitionsToTxnRequest.Builder.forClient(transactionalId, 1, 1, Collections.singletonList(tp)).build()
private def addOffsetsToTxnRequest = new AddOffsetsToTxnRequest.Builder(
new AddOffsetsToTxnRequestData()
.setTransactionalId(transactionalId)
.setProducerId(1)
.setProducerEpoch(1)
.setGroupId(group)
).build()
private def electLeadersRequest = new ElectLeadersRequest.Builder(
ElectionType.PREFERRED,
Collections.singleton(tp),
10000
).build()
private def describeProducersRequest: DescribeProducersRequest = new DescribeProducersRequest.Builder(
new DescribeProducersRequestData()
.setTopics(List(
new DescribeProducersRequestData.TopicRequest()
.setName(tp.topic)
.setPartitionIndexes(List(Int.box(tp.partition)).asJava)
).asJava)
).build()
private def describeTransactionsRequest: DescribeTransactionsRequest = new DescribeTransactionsRequest.Builder(
new DescribeTransactionsRequestData().setTransactionalIds(List(transactionalId).asJava)
).build()
private def alterPartitionReassignmentsRequest = new AlterPartitionReassignmentsRequest.Builder(
new AlterPartitionReassignmentsRequestData().setTopics(
List(new AlterPartitionReassignmentsRequestData.ReassignableTopic()
.setName(topic)
.setPartitions(
List(new AlterPartitionReassignmentsRequestData.ReassignablePartition().setPartitionIndex(tp.partition)).asJava
)).asJava
)
).build()
private def listPartitionReassignmentsRequest = new ListPartitionReassignmentsRequest.Builder(
new ListPartitionReassignmentsRequestData().setTopics(
List(new ListPartitionReassignmentsRequestData.ListPartitionReassignmentsTopics()
.setName(topic)
.setPartitionIndexes(
List(Integer.valueOf(tp.partition)).asJava
)).asJava
)
).build()
private def sendRequests(requestKeyToRequest: mutable.Map[ApiKeys, AbstractRequest], topicExists: Boolean = true,
topicNames: Map[Uuid, String] = getTopicNames()) = {
for ((key, request) <- requestKeyToRequest) {
removeAllClientAcls()
val resources = requestKeysToAcls(key).map(_._1.resourceType).toSet
sendRequestAndVerifyResponseError(request, resources, isAuthorized = false, topicExists = topicExists, topicNames = topicNames)
val resourceToAcls = requestKeysToAcls(key)
resourceToAcls.get(topicResource).foreach { acls =>
val describeAcls = topicDescribeAcl(topicResource)
val isAuthorized = if (isKRaftTest() && key.equals(ApiKeys.DELETE_TOPICS) && topicExists == false) {
// In KRaft mode, trying to delete a topic that doesn't exist but that you do have
// describe permission for will give UNKNOWN_TOPIC_OR_PARTITION. In ZK mode it gives
// TOPIC_AUTHORIZATION_FAILED unless you have DELETE permission.
true
} else {
describeAcls == acls
}
addAndVerifyAcls(describeAcls, topicResource)
sendRequestAndVerifyResponseError(request, resources, isAuthorized = isAuthorized, topicExists = topicExists, topicNames = topicNames)
removeAllClientAcls()
}
for ((resource, acls) <- resourceToAcls)
addAndVerifyAcls(acls, resource)
sendRequestAndVerifyResponseError(request, resources, isAuthorized = true, topicExists = topicExists, topicNames = topicNames)
}
}
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
@ValueSource(strings = Array("zk", "kraft"))
def testAuthorizationWithTopicExisting(quorum: String): Unit = {
//First create the topic so we have a valid topic ID
sendRequests(mutable.Map(ApiKeys.CREATE_TOPICS -> createTopicsRequest))
val requestKeyToRequest = mutable.LinkedHashMap[ApiKeys, AbstractRequest](
ApiKeys.METADATA -> createMetadataRequest(allowAutoTopicCreation = true),
ApiKeys.PRODUCE -> createProduceRequest,
ApiKeys.FETCH -> createFetchRequest,
ApiKeys.LIST_OFFSETS -> createListOffsetsRequest,
ApiKeys.OFFSET_FETCH -> createOffsetFetchRequest,
ApiKeys.FIND_COORDINATOR -> createFindCoordinatorRequest,
ApiKeys.JOIN_GROUP -> createJoinGroupRequest,
ApiKeys.SYNC_GROUP -> createSyncGroupRequest,
ApiKeys.DESCRIBE_GROUPS -> createDescribeGroupsRequest,
ApiKeys.OFFSET_COMMIT -> createOffsetCommitRequest,
ApiKeys.HEARTBEAT -> heartbeatRequest,
ApiKeys.LEAVE_GROUP -> leaveGroupRequest,
ApiKeys.DELETE_RECORDS -> deleteRecordsRequest,
ApiKeys.OFFSET_FOR_LEADER_EPOCH -> offsetsForLeaderEpochRequest,
ApiKeys.DESCRIBE_CONFIGS -> describeConfigsRequest,
ApiKeys.ALTER_CONFIGS -> alterConfigsRequest,
ApiKeys.CREATE_ACLS -> createAclsRequest,
ApiKeys.DELETE_ACLS -> deleteAclsRequest,
ApiKeys.DESCRIBE_ACLS -> describeAclsRequest,
ApiKeys.ALTER_REPLICA_LOG_DIRS -> alterReplicaLogDirsRequest,
ApiKeys.DESCRIBE_LOG_DIRS -> describeLogDirsRequest,
ApiKeys.CREATE_PARTITIONS -> createPartitionsRequest,
ApiKeys.ADD_PARTITIONS_TO_TXN -> addPartitionsToTxnRequest,
ApiKeys.ADD_OFFSETS_TO_TXN -> addOffsetsToTxnRequest,
ApiKeys.ELECT_LEADERS -> electLeadersRequest,
ApiKeys.INCREMENTAL_ALTER_CONFIGS -> incrementalAlterConfigsRequest,
ApiKeys.ALTER_PARTITION_REASSIGNMENTS -> alterPartitionReassignmentsRequest,
ApiKeys.LIST_PARTITION_REASSIGNMENTS -> listPartitionReassignmentsRequest,
ApiKeys.DESCRIBE_PRODUCERS -> describeProducersRequest,
ApiKeys.DESCRIBE_TRANSACTIONS -> describeTransactionsRequest,
)
if (!isKRaftTest()) {
// Inter-broker APIs use an invalid broker epoch, so does not affect the test case
requestKeyToRequest += ApiKeys.UPDATE_METADATA -> createUpdateMetadataRequest
requestKeyToRequest += ApiKeys.LEADER_AND_ISR -> leaderAndIsrRequest
requestKeyToRequest += ApiKeys.STOP_REPLICA -> stopReplicaRequest
requestKeyToRequest += ApiKeys.CONTROLLED_SHUTDOWN -> controlledShutdownRequest
}
// Delete the topic last
requestKeyToRequest += ApiKeys.DELETE_TOPICS -> deleteTopicsRequest
sendRequests(requestKeyToRequest, true)
}
/*
* even if the topic doesn't exist, request APIs should not leak the topic name
*/
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
@ValueSource(strings = Array("zk", "kraft"))
def testAuthorizationWithTopicNotExisting(quorum: String): Unit = {
val id = Uuid.randomUuid()
val topicNames = Map(id -> "topic")
val requestKeyToRequest = mutable.LinkedHashMap[ApiKeys, AbstractRequest](
ApiKeys.METADATA -> createMetadataRequest(allowAutoTopicCreation = false),
ApiKeys.PRODUCE -> createProduceRequest,
ApiKeys.FETCH -> createFetchRequestWithUnknownTopic(id, ApiKeys.FETCH.latestVersion()),
ApiKeys.LIST_OFFSETS -> createListOffsetsRequest,
ApiKeys.OFFSET_COMMIT -> createOffsetCommitRequest,
ApiKeys.OFFSET_FETCH -> createOffsetFetchRequest,
ApiKeys.DELETE_TOPICS -> deleteTopicsRequest,
ApiKeys.DELETE_RECORDS -> deleteRecordsRequest,
ApiKeys.ADD_PARTITIONS_TO_TXN -> addPartitionsToTxnRequest,
ApiKeys.ADD_OFFSETS_TO_TXN -> addOffsetsToTxnRequest,
ApiKeys.CREATE_PARTITIONS -> createPartitionsRequest,
ApiKeys.DELETE_GROUPS -> deleteGroupsRequest,
ApiKeys.OFFSET_FOR_LEADER_EPOCH -> offsetsForLeaderEpochRequest,
ApiKeys.ELECT_LEADERS -> electLeadersRequest
)
sendRequests(requestKeyToRequest, false, topicNames)
}
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
@CsvSource(value = Array("zk,false", "zk,true", "kraft,false", "kraft,true"))
def testTopicIdAuthorization(quorum: String, withTopicExisting: Boolean): Unit = {
val topicId = if (withTopicExisting) {
createTopicWithBrokerPrincipal(topic)
getTopicIds()(topic)
} else {
Uuid.randomUuid()
}
val requestKeyToRequest = mutable.LinkedHashMap[ApiKeys, AbstractRequest](
ApiKeys.DELETE_TOPICS -> deleteTopicsWithIdsRequest(topicId)
)
def sendAndVerify(
request: AbstractRequest,
isAuthorized: Boolean,
isDescribeAuthorized: Boolean
): Unit = {
val response = connectAndReceive[AbstractResponse](request)
val error = findErrorForTopicId(topicId, response)
if (!withTopicExisting) {
assertEquals(Errors.UNKNOWN_TOPIC_ID, error)
} else if (!isDescribeAuthorized || !isAuthorized) {
assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED, error)
}
}
for ((key, request) <- requestKeyToRequest) {
removeAllClientAcls()
sendAndVerify(request, isAuthorized = false, isDescribeAuthorized = false)
val describeAcls = topicDescribeAcl(topicResource)
addAndVerifyAcls(describeAcls, topicResource)
val resourceToAcls = requestKeysToAcls(key)
resourceToAcls.get(topicResource).foreach { acls =>
val isAuthorized = describeAcls == acls
sendAndVerify(request, isAuthorized = isAuthorized, isDescribeAuthorized = true)
}
removeAllClientAcls()
for ((resource, acls) <- resourceToAcls) {
addAndVerifyAcls(acls, resource)
}
sendAndVerify(request, isAuthorized = true, isDescribeAuthorized = true)
}
}
/*
* even if the topic doesn't exist, request APIs should not leak the topic name
*/
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
@ValueSource(strings = Array("zk", "kraft"))
def testAuthorizationFetchV12WithTopicNotExisting(quorum: String): Unit = {
val id = Uuid.ZERO_UUID
val topicNames = Map(id -> "topic")
val requestKeyToRequest = mutable.LinkedHashMap[ApiKeys, AbstractRequest](
ApiKeys.FETCH -> createFetchRequestWithUnknownTopic(id, 12),
)
sendRequests(requestKeyToRequest, false, topicNames)
}
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
@ValueSource(strings = Array("zk", "kraft"))
def testCreateTopicAuthorizationWithClusterCreate(quorum: String): Unit = {
removeAllClientAcls()
val resources = Set[ResourceType](TOPIC)
sendRequestAndVerifyResponseError(createTopicsRequest, resources, isAuthorized = false)
for ((resource, acls) <- clusterCreateAcl)
addAndVerifyAcls(acls, resource)
sendRequestAndVerifyResponseError(createTopicsRequest, resources, isAuthorized = true)
}
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
@ValueSource(strings = Array("zk", "kraft"))
def testFetchFollowerRequest(quorum: String): Unit = {
createTopicWithBrokerPrincipal(topic)
val request = createFetchFollowerRequest
removeAllClientAcls()
val resources = Set(topicResource.resourceType, clusterResource.resourceType)
sendRequestAndVerifyResponseError(request, resources, isAuthorized = false)
val readAcls = topicReadAcl(topicResource)
addAndVerifyAcls(readAcls, topicResource)
sendRequestAndVerifyResponseError(request, resources, isAuthorized = false)
val clusterAcls = clusterAcl(clusterResource)
addAndVerifyAcls(clusterAcls, clusterResource)
sendRequestAndVerifyResponseError(request, resources, isAuthorized = true)
}
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
@ValueSource(strings = Array("zk", "kraft"))
def testIncrementalAlterConfigsRequestRequiresClusterPermissionForBrokerLogger(quorum: String): Unit = {
createTopicWithBrokerPrincipal(topic)
val data = new IncrementalAlterConfigsRequestData
val alterableConfig = new AlterableConfig().setName("kafka.controller.KafkaController").
setValue(LogLevelConfig.DEBUG_LOG_LEVEL).setConfigOperation(AlterConfigOp.OpType.DELETE.id())
val alterableConfigSet = new AlterableConfigCollection
alterableConfigSet.add(alterableConfig)
data.resources().add(new AlterConfigsResource().
setResourceName(brokerId.toString).setResourceType(ConfigResource.Type.BROKER_LOGGER.id()).
setConfigs(alterableConfigSet))
val request = new IncrementalAlterConfigsRequest.Builder(data).build()
removeAllClientAcls()
val resources = Set(topicResource.resourceType, clusterResource.resourceType)
sendRequestAndVerifyResponseError(request, resources, isAuthorized = false)
val clusterAcls = clusterAlterConfigsAcl(clusterResource)
addAndVerifyAcls(clusterAcls, clusterResource)
sendRequestAndVerifyResponseError(request, resources, isAuthorized = true)
}
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
@ValueSource(strings = Array("zk", "kraft"))
def testOffsetsForLeaderEpochClusterPermission(quorum: String): Unit = {
createTopicWithBrokerPrincipal(topic)
val request = offsetsForLeaderEpochRequest
removeAllClientAcls()
val resources = Set(topicResource.resourceType, clusterResource.resourceType)
sendRequestAndVerifyResponseError(request, resources, isAuthorized = false)
// Although the OffsetsForLeaderEpoch API now accepts topic describe, we should continue
// allowing cluster action for backwards compatibility
val clusterAcls = clusterAcl(clusterResource)
addAndVerifyAcls(clusterAcls, clusterResource)
sendRequestAndVerifyResponseError(request, resources, isAuthorized = true)
}
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
@ValueSource(strings = Array("zk", "kraft"))
def testProduceWithNoTopicAccess(quorum: String): Unit = {
createTopicWithBrokerPrincipal(topic)
val producer = createProducer()
assertThrows(classOf[TopicAuthorizationException], () => sendRecords(producer, numRecords, tp))
}
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
@ValueSource(strings = Array("zk", "kraft"))
def testProduceWithTopicDescribe(quorum: String): Unit = {
createTopicWithBrokerPrincipal(topic)
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, DESCRIBE, ALLOW)), topicResource)
val producer = createProducer()
assertThrows(classOf[TopicAuthorizationException], () => sendRecords(producer, numRecords, tp))
}
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
@ValueSource(strings = Array("zk", "kraft"))
def testProduceWithTopicRead(quorum: String): Unit = {
createTopicWithBrokerPrincipal(topic)
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), topicResource)
val producer = createProducer()
assertThrows(classOf[TopicAuthorizationException], () => sendRecords(producer, numRecords, tp))
}
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)