-
Notifications
You must be signed in to change notification settings - Fork 652
/
adminapi.go
3463 lines (3007 loc) · 115 KB
/
adminapi.go
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
/**
* Copyright 2018 Confluent Inc.
*
* Licensed 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
import (
"context"
"fmt"
"strings"
"sync/atomic"
"time"
"unsafe"
)
/*
#include "select_rdkafka.h"
#include <stdlib.h>
static const rd_kafka_group_result_t *
group_result_by_idx (const rd_kafka_group_result_t **groups, size_t cnt, size_t idx) {
if (idx >= cnt)
return NULL;
return groups[idx];
}
static const rd_kafka_topic_result_t *
topic_result_by_idx (const rd_kafka_topic_result_t **topics, size_t cnt, size_t idx) {
if (idx >= cnt)
return NULL;
return topics[idx];
}
static const rd_kafka_ConfigResource_t *
ConfigResource_by_idx (const rd_kafka_ConfigResource_t **res, size_t cnt, size_t idx) {
if (idx >= cnt)
return NULL;
return res[idx];
}
static const rd_kafka_ConfigEntry_t *
ConfigEntry_by_idx (const rd_kafka_ConfigEntry_t **entries, size_t cnt, size_t idx) {
if (idx >= cnt)
return NULL;
return entries[idx];
}
static const rd_kafka_acl_result_t *
acl_result_by_idx (const rd_kafka_acl_result_t **acl_results, size_t cnt, size_t idx) {
if (idx >= cnt)
return NULL;
return acl_results[idx];
}
static const rd_kafka_DeleteAcls_result_response_t *
DeleteAcls_result_response_by_idx (const rd_kafka_DeleteAcls_result_response_t **delete_acls_result_responses, size_t cnt, size_t idx) {
if (idx >= cnt)
return NULL;
return delete_acls_result_responses[idx];
}
static const rd_kafka_AclBinding_t *
AclBinding_by_idx (const rd_kafka_AclBinding_t **acl_bindings, size_t cnt, size_t idx) {
if (idx >= cnt)
return NULL;
return acl_bindings[idx];
}
static const rd_kafka_ConsumerGroupListing_t *
ConsumerGroupListing_by_idx(const rd_kafka_ConsumerGroupListing_t **result_groups, size_t cnt, size_t idx) {
if (idx >= cnt)
return NULL;
return result_groups[idx];
}
static const rd_kafka_ConsumerGroupDescription_t *
ConsumerGroupDescription_by_idx(const rd_kafka_ConsumerGroupDescription_t **result_groups, size_t cnt, size_t idx) {
if (idx >= cnt)
return NULL;
return result_groups[idx];
}
static const rd_kafka_TopicDescription_t *
TopicDescription_by_idx(const rd_kafka_TopicDescription_t **result_topics, size_t cnt, size_t idx) {
if (idx >= cnt)
return NULL;
return result_topics[idx];
}
static const rd_kafka_TopicPartitionInfo_t *
TopicPartitionInfo_by_idx(const rd_kafka_TopicPartitionInfo_t **partitions, size_t cnt, size_t idx) {
if (idx >= cnt)
return NULL;
return partitions[idx];
}
static const rd_kafka_AclOperation_t AclOperation_by_idx(const rd_kafka_AclOperation_t *acl_operations, size_t cnt, size_t idx) {
if (idx >= cnt)
return RD_KAFKA_ACL_OPERATION_UNKNOWN;
return acl_operations[idx];
}
static const rd_kafka_Node_t *Node_by_idx(const rd_kafka_Node_t **nodes, size_t cnt, size_t idx) {
if (idx >= cnt)
return NULL;
return nodes[idx];
}
static const rd_kafka_UserScramCredentialsDescription_t *
DescribeUserScramCredentials_result_description_by_idx(const rd_kafka_UserScramCredentialsDescription_t **descriptions, size_t cnt, size_t idx) {
if (idx >= cnt)
return NULL;
return descriptions[idx];
}
static const rd_kafka_AlterUserScramCredentials_result_response_t*
AlterUserScramCredentials_result_response_by_idx(const rd_kafka_AlterUserScramCredentials_result_response_t **responses, size_t cnt, size_t idx) {
if (idx >= cnt)
return NULL;
return responses[idx];
}
static const rd_kafka_ListOffsetsResultInfo_t *
ListOffsetsResultInfo_by_idx(const rd_kafka_ListOffsetsResultInfo_t **result_infos, size_t cnt, size_t idx) {
if (idx >= cnt)
return NULL;
return result_infos[idx];
}
static const rd_kafka_error_t *
error_by_idx(const rd_kafka_error_t **errors, size_t cnt, size_t idx) {
if (idx >= cnt)
return NULL;
return errors[idx];
}
*/
import "C"
// AdminClient is derived from an existing Producer or Consumer
type AdminClient struct {
handle *handle
isDerived bool // Derived from existing client handle
isClosed uint32 // to check if Admin Client is closed or not.
}
// IsClosed returns boolean representing if client is closed or not
func (a *AdminClient) IsClosed() bool {
return atomic.LoadUint32(&a.isClosed) == 1
}
func (a *AdminClient) verifyClient() error {
if a.IsClosed() {
return getOperationNotAllowedErrorForClosedClient()
}
return nil
}
func durationToMilliseconds(t time.Duration) int {
if t > 0 {
return (int)(t.Seconds() * 1000.0)
}
return (int)(t)
}
// TopicResult provides per-topic operation result (error) information.
type TopicResult struct {
// Topic name
Topic string
// Error, if any, of result. Check with `Error.Code() != ErrNoError`.
Error Error
}
// String returns a human-readable representation of a TopicResult.
func (t TopicResult) String() string {
if t.Error.code == 0 {
return t.Topic
}
return fmt.Sprintf("%s (%s)", t.Topic, t.Error.str)
}
// ConsumerGroupResult provides per-group operation result (error) information.
type ConsumerGroupResult struct {
// Group name
Group string
// Error, if any, of result. Check with `Error.Code() != ErrNoError`.
Error Error
}
// String returns a human-readable representation of a ConsumerGroupResult.
func (g ConsumerGroupResult) String() string {
if g.Error.code == ErrNoError {
return g.Group
}
return fmt.Sprintf("%s (%s)", g.Group, g.Error.str)
}
// ConsumerGroupState represents a consumer group state
type ConsumerGroupState int
const (
// ConsumerGroupStateUnknown - Unknown ConsumerGroupState
ConsumerGroupStateUnknown = ConsumerGroupState(C.RD_KAFKA_CONSUMER_GROUP_STATE_UNKNOWN)
// ConsumerGroupStatePreparingRebalance - preparing rebalance
ConsumerGroupStatePreparingRebalance = ConsumerGroupState(C.RD_KAFKA_CONSUMER_GROUP_STATE_PREPARING_REBALANCE)
// ConsumerGroupStateCompletingRebalance - completing rebalance
ConsumerGroupStateCompletingRebalance = ConsumerGroupState(C.RD_KAFKA_CONSUMER_GROUP_STATE_COMPLETING_REBALANCE)
// ConsumerGroupStateStable - stable
ConsumerGroupStateStable = ConsumerGroupState(C.RD_KAFKA_CONSUMER_GROUP_STATE_STABLE)
// ConsumerGroupStateDead - dead group
ConsumerGroupStateDead = ConsumerGroupState(C.RD_KAFKA_CONSUMER_GROUP_STATE_DEAD)
// ConsumerGroupStateEmpty - empty group
ConsumerGroupStateEmpty = ConsumerGroupState(C.RD_KAFKA_CONSUMER_GROUP_STATE_EMPTY)
)
// String returns the human-readable representation of a consumer_group_state
func (t ConsumerGroupState) String() string {
return C.GoString(C.rd_kafka_consumer_group_state_name(
C.rd_kafka_consumer_group_state_t(t)))
}
// ConsumerGroupStateFromString translates a consumer group state name/string to
// a ConsumerGroupStateFromString value.
func ConsumerGroupStateFromString(stateString string) (ConsumerGroupState, error) {
cStr := C.CString(stateString)
defer C.free(unsafe.Pointer(cStr))
state := ConsumerGroupState(C.rd_kafka_consumer_group_state_code(cStr))
return state, nil
}
// ConsumerGroupListing represents the result of ListConsumerGroups for a single
// group.
type ConsumerGroupListing struct {
// Group id.
GroupID string
// Is a simple consumer group.
IsSimpleConsumerGroup bool
// Group state.
State ConsumerGroupState
}
// ListConsumerGroupsResult represents ListConsumerGroups results and errors.
type ListConsumerGroupsResult struct {
// List of valid ConsumerGroupListings.
Valid []ConsumerGroupListing
// List of errors.
Errors []error
}
// MemberAssignment represents the assignment of a consumer group member.
type MemberAssignment struct {
// Partitions assigned to current member.
TopicPartitions []TopicPartition
}
// MemberDescription represents the description of a consumer group member.
type MemberDescription struct {
// Client id.
ClientID string
// Group instance id.
GroupInstanceID string
// Consumer id.
ConsumerID string
// Group member host.
Host string
// Member assignment.
Assignment MemberAssignment
}
// ConsumerGroupDescription represents the result of DescribeConsumerGroups for
// a single group.
type ConsumerGroupDescription struct {
// Group id.
GroupID string
// Error, if any, of result. Check with `Error.Code() != ErrNoError`.
Error Error
// Is a simple consumer group.
IsSimpleConsumerGroup bool
// Partition assignor identifier.
PartitionAssignor string
// Consumer group state.
State ConsumerGroupState
// Consumer group coordinator (has ID == -1 if not known).
Coordinator Node
// Members list.
Members []MemberDescription
// Operations allowed for the group (nil if not available or not requested)
AuthorizedOperations []ACLOperation
}
// DescribeConsumerGroupsResult represents the result of a
// DescribeConsumerGroups call.
type DescribeConsumerGroupsResult struct {
// Slice of ConsumerGroupDescription.
ConsumerGroupDescriptions []ConsumerGroupDescription
}
// TopicCollection represents a collection of topics.
type TopicCollection struct {
// Slice of topic names.
topicNames []string
}
// NewTopicCollectionOfTopicNames creates a new TopicCollection based on a list
// of topic names.
func NewTopicCollectionOfTopicNames(names []string) TopicCollection {
return TopicCollection{
topicNames: names,
}
}
// TopicPartitionInfo represents a specific partition's information inside a
// TopicDescription.
type TopicPartitionInfo struct {
// Partition id.
Partition int
// Leader broker.
Leader *Node
// Replicas of the partition.
Replicas []Node
// In-Sync-Replicas of the partition.
Isr []Node
}
// TopicDescription represents the result of DescribeTopics for
// a single topic.
type TopicDescription struct {
// Topic name.
Name string
// Topic Id
TopicID UUID
// Error, if any, of the result. Check with `Error.Code() != ErrNoError`.
Error Error
// Is the topic internal to Kafka?
IsInternal bool
// Partitions' information list.
Partitions []TopicPartitionInfo
// Operations allowed for the topic (nil if not available or not requested).
AuthorizedOperations []ACLOperation
}
// DescribeTopicsResult represents the result of a
// DescribeTopics call.
type DescribeTopicsResult struct {
// Slice of TopicDescription.
TopicDescriptions []TopicDescription
}
// DescribeClusterResult represents the result of DescribeCluster.
type DescribeClusterResult struct {
// Cluster id for the cluster (always available if broker version >= 0.10.1.0, otherwise nil).
ClusterID *string
// Current controller broker for the cluster (nil if there is none).
Controller *Node
// List of brokers in the cluster.
Nodes []Node
// Operations allowed for the cluster (nil if not available or not requested).
AuthorizedOperations []ACLOperation
}
// DeleteConsumerGroupsResult represents the result of a DeleteConsumerGroups
// call.
type DeleteConsumerGroupsResult struct {
// Slice of ConsumerGroupResult.
ConsumerGroupResults []ConsumerGroupResult
}
// ListConsumerGroupOffsetsResult represents the result of a
// ListConsumerGroupOffsets operation.
type ListConsumerGroupOffsetsResult struct {
// A slice of ConsumerGroupTopicPartitions, each element represents a group's
// TopicPartitions and Offsets.
ConsumerGroupsTopicPartitions []ConsumerGroupTopicPartitions
}
// AlterConsumerGroupOffsetsResult represents the result of a
// AlterConsumerGroupOffsets operation.
type AlterConsumerGroupOffsetsResult struct {
// A slice of ConsumerGroupTopicPartitions, each element represents a group's
// TopicPartitions and Offsets.
ConsumerGroupsTopicPartitions []ConsumerGroupTopicPartitions
}
// TopicSpecification holds parameters for creating a new topic.
// TopicSpecification is analogous to NewTopic in the Java Topic Admin API.
type TopicSpecification struct {
// Topic name to create.
Topic string
// Number of partitions in topic.
NumPartitions int
// Default replication factor for the topic's partitions, or zero
// if an explicit ReplicaAssignment is set.
ReplicationFactor int
// (Optional) Explicit replica assignment. The outer array is
// indexed by the partition number, while the inner per-partition array
// contains the replica broker ids. The first broker in each
// broker id list will be the preferred replica.
ReplicaAssignment [][]int32
// Topic configuration.
Config map[string]string
}
// PartitionsSpecification holds parameters for creating additional partitions for a topic.
// PartitionsSpecification is analogous to NewPartitions in the Java Topic Admin API.
type PartitionsSpecification struct {
// Topic to create more partitions for.
Topic string
// New partition count for topic, must be higher than current partition count.
IncreaseTo int
// (Optional) Explicit replica assignment. The outer array is
// indexed by the new partition index (i.e., 0 for the first added
// partition), while the inner per-partition array
// contains the replica broker ids. The first broker in each
// broker id list will be the preferred replica.
ReplicaAssignment [][]int32
}
// ResourceType represents an Apache Kafka resource type
type ResourceType int
const (
// ResourceUnknown - Unknown
ResourceUnknown = ResourceType(C.RD_KAFKA_RESOURCE_UNKNOWN)
// ResourceAny - match any resource type (DescribeConfigs)
ResourceAny = ResourceType(C.RD_KAFKA_RESOURCE_ANY)
// ResourceTopic - Topic
ResourceTopic = ResourceType(C.RD_KAFKA_RESOURCE_TOPIC)
// ResourceGroup - Group
ResourceGroup = ResourceType(C.RD_KAFKA_RESOURCE_GROUP)
// ResourceBroker - Broker
ResourceBroker = ResourceType(C.RD_KAFKA_RESOURCE_BROKER)
)
// String returns the human-readable representation of a ResourceType
func (t ResourceType) String() string {
return C.GoString(C.rd_kafka_ResourceType_name(C.rd_kafka_ResourceType_t(t)))
}
// ResourceTypeFromString translates a resource type name/string to
// a ResourceType value.
func ResourceTypeFromString(typeString string) (ResourceType, error) {
switch strings.ToUpper(typeString) {
case "ANY":
return ResourceAny, nil
case "TOPIC":
return ResourceTopic, nil
case "GROUP":
return ResourceGroup, nil
case "BROKER":
return ResourceBroker, nil
default:
return ResourceUnknown, NewError(ErrInvalidArg, "Unknown resource type", false)
}
}
// ConfigSource represents an Apache Kafka config source
type ConfigSource int
const (
// ConfigSourceUnknown is the default value
ConfigSourceUnknown = ConfigSource(C.RD_KAFKA_CONFIG_SOURCE_UNKNOWN_CONFIG)
// ConfigSourceDynamicTopic is dynamic topic config that is configured for a specific topic
ConfigSourceDynamicTopic = ConfigSource(C.RD_KAFKA_CONFIG_SOURCE_DYNAMIC_TOPIC_CONFIG)
// ConfigSourceDynamicBroker is dynamic broker config that is configured for a specific broker
ConfigSourceDynamicBroker = ConfigSource(C.RD_KAFKA_CONFIG_SOURCE_DYNAMIC_BROKER_CONFIG)
// ConfigSourceDynamicDefaultBroker is dynamic broker config that is configured as default for all brokers in the cluster
ConfigSourceDynamicDefaultBroker = ConfigSource(C.RD_KAFKA_CONFIG_SOURCE_DYNAMIC_DEFAULT_BROKER_CONFIG)
// ConfigSourceStaticBroker is static broker config provided as broker properties at startup (e.g. from server.properties file)
ConfigSourceStaticBroker = ConfigSource(C.RD_KAFKA_CONFIG_SOURCE_STATIC_BROKER_CONFIG)
// ConfigSourceDefault is built-in default configuration for configs that have a default value
ConfigSourceDefault = ConfigSource(C.RD_KAFKA_CONFIG_SOURCE_DEFAULT_CONFIG)
)
// String returns the human-readable representation of a ConfigSource type
func (t ConfigSource) String() string {
return C.GoString(C.rd_kafka_ConfigSource_name(C.rd_kafka_ConfigSource_t(t)))
}
// ConfigResource holds parameters for altering an Apache Kafka configuration resource
type ConfigResource struct {
// Type of resource to set.
Type ResourceType
// Name of resource to set.
Name string
// Config entries to set.
// Configuration updates are atomic, any configuration property not provided
// here will be reverted (by the broker) to its default value.
// Use DescribeConfigs to retrieve the list of current configuration entry values.
Config []ConfigEntry
}
// String returns a human-readable representation of a ConfigResource
func (c ConfigResource) String() string {
return fmt.Sprintf("Resource(%s, %s)", c.Type, c.Name)
}
// AlterOperation specifies the operation to perform on the ConfigEntry.
// Currently only AlterOperationSet.
type AlterOperation int
const (
// AlterOperationSet sets/overwrites the configuration setting.
AlterOperationSet = iota
)
// String returns the human-readable representation of an AlterOperation
func (o AlterOperation) String() string {
switch o {
case AlterOperationSet:
return "Set"
default:
return fmt.Sprintf("Unknown%d?", int(o))
}
}
// AlterConfigOpType specifies the operation to perform
// on the ConfigEntry for IncrementalAlterConfig
type AlterConfigOpType int
const (
// AlterConfigOpTypeSet sets/overwrites the configuration
// setting.
AlterConfigOpTypeSet = AlterConfigOpType(C.RD_KAFKA_ALTER_CONFIG_OP_TYPE_SET)
// AlterConfigOpTypeDelete sets the configuration setting
// to default or NULL.
AlterConfigOpTypeDelete = AlterConfigOpType(C.RD_KAFKA_ALTER_CONFIG_OP_TYPE_DELETE)
// AlterConfigOpTypeAppend appends the value to existing
// configuration settings.
AlterConfigOpTypeAppend = AlterConfigOpType(C.RD_KAFKA_ALTER_CONFIG_OP_TYPE_APPEND)
// AlterConfigOpTypeSubtract subtracts the value from
// existing configuration settings.
AlterConfigOpTypeSubtract = AlterConfigOpType(C.RD_KAFKA_ALTER_CONFIG_OP_TYPE_SUBTRACT)
)
// String returns the human-readable representation of an AlterOperation
func (o AlterConfigOpType) String() string {
switch o {
case AlterConfigOpTypeSet:
return "Set"
case AlterConfigOpTypeDelete:
return "Delete"
case AlterConfigOpTypeAppend:
return "Append"
case AlterConfigOpTypeSubtract:
return "Subtract"
default:
return fmt.Sprintf("Unknown %d", int(o))
}
}
// ConfigEntry holds parameters for altering a resource's configuration.
type ConfigEntry struct {
// Name of configuration entry, e.g., topic configuration property name.
Name string
// Value of configuration entry.
Value string
// Deprecated: Operation to perform on the entry.
Operation AlterOperation
// Operation to perform on the entry incrementally.
IncrementalOperation AlterConfigOpType
}
// StringMapToConfigEntries creates a new map of ConfigEntry objects from the
// provided string map. The AlterOperation is set on each created entry.
func StringMapToConfigEntries(stringMap map[string]string, operation AlterOperation) []ConfigEntry {
var ceList []ConfigEntry
for k, v := range stringMap {
ceList = append(ceList, ConfigEntry{Name: k, Value: v, Operation: operation})
}
return ceList
}
// StringMapToIncrementalConfigEntries creates a new map of ConfigEntry objects from the
// provided string map an operation map. The AlterConfigOpType is set on each created entry.
func StringMapToIncrementalConfigEntries(stringMap map[string]string,
operationMap map[string]AlterConfigOpType) []ConfigEntry {
var ceList []ConfigEntry
for k, v := range stringMap {
ceList = append(ceList, ConfigEntry{Name: k, Value: v, IncrementalOperation: operationMap[k]})
}
return ceList
}
// String returns a human-readable representation of a ConfigEntry.
func (c ConfigEntry) String() string {
return fmt.Sprintf("%v %s=\"%s\"", c.Operation, c.Name, c.Value)
}
// ConfigEntryResult contains the result of a single configuration entry from a
// DescribeConfigs request.
type ConfigEntryResult struct {
// Name of configuration entry, e.g., topic configuration property name.
Name string
// Value of configuration entry.
Value string
// Source indicates the configuration source.
Source ConfigSource
// IsReadOnly indicates whether the configuration entry can be altered.
IsReadOnly bool
// IsDefault indicates whether the value is at its default.
IsDefault bool
// IsSensitive indicates whether the configuration entry contains sensitive information, in which case the value will be unset.
IsSensitive bool
// IsSynonym indicates whether the configuration entry is a synonym for another configuration property.
IsSynonym bool
// Synonyms contains a map of configuration entries that are synonyms to this configuration entry.
Synonyms map[string]ConfigEntryResult
}
// String returns a human-readable representation of a ConfigEntryResult.
func (c ConfigEntryResult) String() string {
return fmt.Sprintf("%s=\"%s\"", c.Name, c.Value)
}
// setFromC sets up a ConfigEntryResult from a C ConfigEntry
func configEntryResultFromC(cEntry *C.rd_kafka_ConfigEntry_t) (entry ConfigEntryResult) {
entry.Name = C.GoString(C.rd_kafka_ConfigEntry_name(cEntry))
cValue := C.rd_kafka_ConfigEntry_value(cEntry)
if cValue != nil {
entry.Value = C.GoString(cValue)
}
entry.Source = ConfigSource(C.rd_kafka_ConfigEntry_source(cEntry))
entry.IsReadOnly = cint2bool(C.rd_kafka_ConfigEntry_is_read_only(cEntry))
entry.IsDefault = cint2bool(C.rd_kafka_ConfigEntry_is_default(cEntry))
entry.IsSensitive = cint2bool(C.rd_kafka_ConfigEntry_is_sensitive(cEntry))
entry.IsSynonym = cint2bool(C.rd_kafka_ConfigEntry_is_synonym(cEntry))
var cSynCnt C.size_t
cSyns := C.rd_kafka_ConfigEntry_synonyms(cEntry, &cSynCnt)
if cSynCnt > 0 {
entry.Synonyms = make(map[string]ConfigEntryResult)
}
for si := 0; si < int(cSynCnt); si++ {
cSyn := C.ConfigEntry_by_idx(cSyns, cSynCnt, C.size_t(si))
Syn := configEntryResultFromC(cSyn)
entry.Synonyms[Syn.Name] = Syn
}
return entry
}
// ConfigResourceResult provides the result for a resource from a AlterConfigs or
// DescribeConfigs request.
type ConfigResourceResult struct {
// Type of returned result resource.
Type ResourceType
// Name of returned result resource.
Name string
// Error, if any, of returned result resource.
Error Error
// Config entries, if any, of returned result resource.
Config map[string]ConfigEntryResult
}
// String returns a human-readable representation of a ConfigResourceResult.
func (c ConfigResourceResult) String() string {
if c.Error.Code() != 0 {
return fmt.Sprintf("ResourceResult(%s, %s, \"%v\")", c.Type, c.Name, c.Error)
}
return fmt.Sprintf("ResourceResult(%s, %s, %d config(s))", c.Type, c.Name, len(c.Config))
}
// ResourcePatternType enumerates the different types of Kafka resource patterns.
type ResourcePatternType int
const (
// ResourcePatternTypeUnknown is a resource pattern type not known or not set.
ResourcePatternTypeUnknown = ResourcePatternType(C.RD_KAFKA_RESOURCE_PATTERN_UNKNOWN)
// ResourcePatternTypeAny matches any resource, used for lookups.
ResourcePatternTypeAny = ResourcePatternType(C.RD_KAFKA_RESOURCE_PATTERN_ANY)
// ResourcePatternTypeMatch will perform pattern matching
ResourcePatternTypeMatch = ResourcePatternType(C.RD_KAFKA_RESOURCE_PATTERN_MATCH)
// ResourcePatternTypeLiteral matches a literal resource name
ResourcePatternTypeLiteral = ResourcePatternType(C.RD_KAFKA_RESOURCE_PATTERN_LITERAL)
// ResourcePatternTypePrefixed matches a prefixed resource name
ResourcePatternTypePrefixed = ResourcePatternType(C.RD_KAFKA_RESOURCE_PATTERN_PREFIXED)
)
// String returns the human-readable representation of a ResourcePatternType
func (t ResourcePatternType) String() string {
return C.GoString(C.rd_kafka_ResourcePatternType_name(C.rd_kafka_ResourcePatternType_t(t)))
}
// ResourcePatternTypeFromString translates a resource pattern type name to
// a ResourcePatternType value.
func ResourcePatternTypeFromString(patternTypeString string) (ResourcePatternType, error) {
switch strings.ToUpper(patternTypeString) {
case "ANY":
return ResourcePatternTypeAny, nil
case "MATCH":
return ResourcePatternTypeMatch, nil
case "LITERAL":
return ResourcePatternTypeLiteral, nil
case "PREFIXED":
return ResourcePatternTypePrefixed, nil
default:
return ResourcePatternTypeUnknown, NewError(ErrInvalidArg, "Unknown resource pattern type", false)
}
}
// ACLOperation enumerates the different types of ACL operation.
type ACLOperation int
const (
// ACLOperationUnknown represents an unknown or unset operation
ACLOperationUnknown = ACLOperation(C.RD_KAFKA_ACL_OPERATION_UNKNOWN)
// ACLOperationAny in a filter, matches any ACLOperation
ACLOperationAny = ACLOperation(C.RD_KAFKA_ACL_OPERATION_ANY)
// ACLOperationAll represents all the operations
ACLOperationAll = ACLOperation(C.RD_KAFKA_ACL_OPERATION_ALL)
// ACLOperationRead a read operation
ACLOperationRead = ACLOperation(C.RD_KAFKA_ACL_OPERATION_READ)
// ACLOperationWrite represents a write operation
ACLOperationWrite = ACLOperation(C.RD_KAFKA_ACL_OPERATION_WRITE)
// ACLOperationCreate represents a create operation
ACLOperationCreate = ACLOperation(C.RD_KAFKA_ACL_OPERATION_CREATE)
// ACLOperationDelete represents a delete operation
ACLOperationDelete = ACLOperation(C.RD_KAFKA_ACL_OPERATION_DELETE)
// ACLOperationAlter represents an alter operation
ACLOperationAlter = ACLOperation(C.RD_KAFKA_ACL_OPERATION_ALTER)
// ACLOperationDescribe represents a describe operation
ACLOperationDescribe = ACLOperation(C.RD_KAFKA_ACL_OPERATION_DESCRIBE)
// ACLOperationClusterAction represents a cluster action operation
ACLOperationClusterAction = ACLOperation(C.RD_KAFKA_ACL_OPERATION_CLUSTER_ACTION)
// ACLOperationDescribeConfigs represents a describe configs operation
ACLOperationDescribeConfigs = ACLOperation(C.RD_KAFKA_ACL_OPERATION_DESCRIBE_CONFIGS)
// ACLOperationAlterConfigs represents an alter configs operation
ACLOperationAlterConfigs = ACLOperation(C.RD_KAFKA_ACL_OPERATION_ALTER_CONFIGS)
// ACLOperationIdempotentWrite represents an idempotent write operation
ACLOperationIdempotentWrite = ACLOperation(C.RD_KAFKA_ACL_OPERATION_IDEMPOTENT_WRITE)
)
// String returns the human-readable representation of an ACLOperation
func (o ACLOperation) String() string {
return C.GoString(C.rd_kafka_AclOperation_name(C.rd_kafka_AclOperation_t(o)))
}
// ACLOperationFromString translates a ACL operation name to
// a ACLOperation value.
func ACLOperationFromString(aclOperationString string) (ACLOperation, error) {
switch strings.ToUpper(aclOperationString) {
case "ANY":
return ACLOperationAny, nil
case "ALL":
return ACLOperationAll, nil
case "READ":
return ACLOperationRead, nil
case "WRITE":
return ACLOperationWrite, nil
case "CREATE":
return ACLOperationCreate, nil
case "DELETE":
return ACLOperationDelete, nil
case "ALTER":
return ACLOperationAlter, nil
case "DESCRIBE":
return ACLOperationDescribe, nil
case "CLUSTER_ACTION":
return ACLOperationClusterAction, nil
case "DESCRIBE_CONFIGS":
return ACLOperationDescribeConfigs, nil
case "ALTER_CONFIGS":
return ACLOperationAlterConfigs, nil
case "IDEMPOTENT_WRITE":
return ACLOperationIdempotentWrite, nil
default:
return ACLOperationUnknown, NewError(ErrInvalidArg, "Unknown ACL operation", false)
}
}
// ACLPermissionType enumerates the different types of ACL permission types.
type ACLPermissionType int
const (
// ACLPermissionTypeUnknown represents an unknown ACLPermissionType
ACLPermissionTypeUnknown = ACLPermissionType(C.RD_KAFKA_ACL_PERMISSION_TYPE_UNKNOWN)
// ACLPermissionTypeAny in a filter, matches any ACLPermissionType
ACLPermissionTypeAny = ACLPermissionType(C.RD_KAFKA_ACL_PERMISSION_TYPE_ANY)
// ACLPermissionTypeDeny disallows access
ACLPermissionTypeDeny = ACLPermissionType(C.RD_KAFKA_ACL_PERMISSION_TYPE_DENY)
// ACLPermissionTypeAllow grants access
ACLPermissionTypeAllow = ACLPermissionType(C.RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW)
)
// String returns the human-readable representation of an ACLPermissionType
func (o ACLPermissionType) String() string {
return C.GoString(C.rd_kafka_AclPermissionType_name(C.rd_kafka_AclPermissionType_t(o)))
}
// ACLPermissionTypeFromString translates a ACL permission type name to
// a ACLPermissionType value.
func ACLPermissionTypeFromString(aclPermissionTypeString string) (ACLPermissionType, error) {
switch strings.ToUpper(aclPermissionTypeString) {
case "ANY":
return ACLPermissionTypeAny, nil
case "DENY":
return ACLPermissionTypeDeny, nil
case "ALLOW":
return ACLPermissionTypeAllow, nil
default:
return ACLPermissionTypeUnknown, NewError(ErrInvalidArg, "Unknown ACL permission type", false)
}
}
// ACLBinding specifies the operation and permission type for a specific principal
// over one or more resources of the same type. Used by `AdminClient.CreateACLs`,
// returned by `AdminClient.DescribeACLs` and `AdminClient.DeleteACLs`.
type ACLBinding struct {
Type ResourceType // The resource type.
// The resource name, which depends on the resource type.
// For ResourceBroker the resource name is the broker id.
Name string
ResourcePatternType ResourcePatternType // The resource pattern, relative to the name.
Principal string // The principal this ACLBinding refers to.
Host string // The host that the call is allowed to come from.
Operation ACLOperation // The operation/s specified by this binding.
PermissionType ACLPermissionType // The permission type for the specified operation.
}
// ACLBindingFilter specifies a filter used to return a list of ACL bindings matching some or all of its attributes.
// Used by `AdminClient.DescribeACLs` and `AdminClient.DeleteACLs`.
type ACLBindingFilter = ACLBinding
// ACLBindings is a slice of ACLBinding that also implements
// the sort interface
type ACLBindings []ACLBinding
// ACLBindingFilters is a slice of ACLBindingFilter that also implements
// the sort interface
type ACLBindingFilters []ACLBindingFilter
func (a ACLBindings) Len() int {
return len(a)
}
func (a ACLBindings) Less(i, j int) bool {
if a[i].Type != a[j].Type {
return a[i].Type < a[j].Type
}
if a[i].Name != a[j].Name {
return a[i].Name < a[j].Name
}
if a[i].ResourcePatternType != a[j].ResourcePatternType {
return a[i].ResourcePatternType < a[j].ResourcePatternType
}
if a[i].Principal != a[j].Principal {
return a[i].Principal < a[j].Principal
}
if a[i].Host != a[j].Host {
return a[i].Host < a[j].Host
}
if a[i].Operation != a[j].Operation {
return a[i].Operation < a[j].Operation
}
if a[i].PermissionType != a[j].PermissionType {
return a[i].PermissionType < a[j].PermissionType
}
return true
}
func (a ACLBindings) Swap(i, j int) {
a[i], a[j] = a[j], a[i]
}
// CreateACLResult provides create ACL error information.
type CreateACLResult struct {
// Error, if any, of result. Check with `Error.Code() != ErrNoError`.
Error Error
}
// DescribeACLsResult provides describe ACLs result or error information.
type DescribeACLsResult struct {
// Slice of ACL bindings matching the provided filter
ACLBindings ACLBindings
// Error, if any, of result. Check with `Error.Code() != ErrNoError`.
Error Error
}
// DeleteACLsResult provides delete ACLs result or error information.
type DeleteACLsResult = DescribeACLsResult
// ScramMechanism enumerates SASL/SCRAM mechanisms.
// Used by `AdminClient.AlterUserScramCredentials`
// and `AdminClient.DescribeUserScramCredentials`.
type ScramMechanism int
const (
// ScramMechanismUnknown - Unknown SASL/SCRAM mechanism
ScramMechanismUnknown = ScramMechanism(C.RD_KAFKA_SCRAM_MECHANISM_UNKNOWN)
// ScramMechanismSHA256 - SCRAM-SHA-256 mechanism
ScramMechanismSHA256 = ScramMechanism(C.RD_KAFKA_SCRAM_MECHANISM_SHA_256)
// ScramMechanismSHA512 - SCRAM-SHA-512 mechanism
ScramMechanismSHA512 = ScramMechanism(C.RD_KAFKA_SCRAM_MECHANISM_SHA_512)
)
// String returns the human-readable representation of an ScramMechanism
func (o ScramMechanism) String() string {
switch o {
case ScramMechanismSHA256:
return "SCRAM-SHA-256"
case ScramMechanismSHA512:
return "SCRAM-SHA-512"
default:
return "UNKNOWN"
}
}
// ScramMechanismFromString translates a Scram Mechanism name to
// a ScramMechanism value.
func ScramMechanismFromString(mechanism string) (ScramMechanism, error) {
switch strings.ToUpper(mechanism) {
case "SCRAM-SHA-256":
return ScramMechanismSHA256, nil
case "SCRAM-SHA-512":
return ScramMechanismSHA512, nil
default:
return ScramMechanismUnknown,
NewError(ErrInvalidArg, "Unknown SCRAM mechanism", false)
}
}
// ScramCredentialInfo contains Mechanism and Iterations for a
// SASL/SCRAM credential associated with a user.
type ScramCredentialInfo struct {
// Iterations - positive number of iterations used when creating the credential
Iterations int
// Mechanism - SASL/SCRAM mechanism
Mechanism ScramMechanism
}
// UserScramCredentialsDescription represent all SASL/SCRAM credentials
// associated with a user that can be retrieved, or an error indicating
// why credentials could not be retrieved.
type UserScramCredentialsDescription struct {
// User - the user name.
User string
// ScramCredentialInfos - SASL/SCRAM credential representations for the user.
ScramCredentialInfos []ScramCredentialInfo
// Error - error corresponding to this user description.
Error Error
}
// UserScramCredentialDeletion is a request to delete
// a SASL/SCRAM credential for a user.
type UserScramCredentialDeletion struct {
// User - user name
User string
// Mechanism - SASL/SCRAM mechanism.
Mechanism ScramMechanism
}
// UserScramCredentialUpsertion is a request to update/insert
// a SASL/SCRAM credential for a user.
type UserScramCredentialUpsertion struct {
// User - user name
User string
// ScramCredentialInfo - the mechanism and iterations.
ScramCredentialInfo ScramCredentialInfo
// Password - password to HMAC before storage.
Password []byte
// Salt - salt to use. Will be generated randomly if nil. (optional)
Salt []byte
}
// DescribeUserScramCredentialsResult represents the result of a
// DescribeUserScramCredentials call.
type DescribeUserScramCredentialsResult struct {
// Descriptions - Map from user name
// to UserScramCredentialsDescription
Descriptions map[string]UserScramCredentialsDescription
}
// AlterUserScramCredentialsResult represents the result of a
// AlterUserScramCredentials call.
type AlterUserScramCredentialsResult struct {
// Errors - Map from user name
// to an Error, with ErrNoError code on success.
Errors map[string]Error
}
// OffsetSpec specifies desired offsets while using ListOffsets.
type OffsetSpec int64