forked from apache/kafka
-
Notifications
You must be signed in to change notification settings - Fork 0
/
KafkaZkClient.scala
2357 lines (2136 loc) · 106 KB
/
KafkaZkClient.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.zk
import java.util.Properties
import com.yammer.metrics.core.MetricName
import kafka.api.LeaderAndIsr
import kafka.cluster.Broker
import kafka.controller.{KafkaController, LeaderIsrAndControllerEpoch, ReplicaAssignment}
import kafka.metrics.KafkaMetricsGroup
import kafka.security.authorizer.AclAuthorizer.{NoAcls, VersionedAcls}
import kafka.security.authorizer.AclEntry
import kafka.server.{ConfigType, KafkaConfig}
import kafka.utils.Logging
import kafka.zk.TopicZNode.TopicIdReplicaAssignment
import kafka.zookeeper._
import org.apache.kafka.common.errors.ControllerMovedException
import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourceType}
import org.apache.kafka.common.security.JaasUtils
import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation}
import org.apache.kafka.common.utils.{Time, Utils}
import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid}
import org.apache.kafka.metadata.migration.ZkMigrationLeadershipState
import org.apache.kafka.storage.internals.log.LogConfig
import org.apache.zookeeper.KeeperException.{Code, NodeExistsException}
import org.apache.zookeeper.OpResult.{CheckResult, CreateResult, ErrorResult, SetDataResult}
import org.apache.zookeeper.client.ZKClientConfig
import org.apache.zookeeper.common.ZKConfig
import org.apache.zookeeper.data.{ACL, Stat}
import org.apache.zookeeper.{CreateMode, KeeperException, OpResult, ZooKeeper}
import java.lang.{Long => JLong}
import scala.collection.{Map, Seq, mutable}
sealed trait KRaftRegistrationResult
case class FailedRegistrationResult() extends KRaftRegistrationResult
case class SuccessfulRegistrationResult(zkControllerEpoch: Int, controllerEpochZkVersion: Int) extends KRaftRegistrationResult
/**
* Provides higher level Kafka-specific operations on top of the pipelined [[kafka.zookeeper.ZooKeeperClient]].
*
* Implementation note: this class includes methods for various components (Controller, Configs, Old Consumer, etc.)
* and returns instances of classes from the calling packages in some cases. This is not ideal, but it made it
* easier to migrate away from `ZkUtils` (since removed). We should revisit this. We should also consider whether a
* monolithic [[kafka.zk.ZkData]] is the way to go.
*/
class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boolean, time: Time) extends AutoCloseable with
Logging with KafkaMetricsGroup {
override def metricName(name: String, metricTags: scala.collection.Map[String, String]): MetricName = {
explicitMetricName("kafka.server", "ZooKeeperClientMetrics", name, metricTags)
}
private val latencyMetric = newHistogram("ZooKeeperRequestLatencyMs")
import KafkaZkClient._
// Only for testing
private[kafka] def currentZooKeeper: ZooKeeper = zooKeeperClient.currentZooKeeper
// This variable holds the Zookeeper session id at the moment a Broker gets registered in Zookeeper and the subsequent
// updates of the session id. It is possible that the session id changes over the time for 'Session expired'.
// This code is part of the work around done in the KAFKA-7165, once ZOOKEEPER-2985 is complete, this code must
// be deleted.
private var currentZooKeeperSessionId: Long = -1
/**
* Create a sequential persistent path. That is, the znode will not be automatically deleted upon client's disconnect
* and a monotonically increasing number will be appended to its name.
*
* @param path the path to create (with the monotonically increasing number appended)
* @param data the znode data
* @return the created path (including the appended monotonically increasing number)
*/
private[kafka] def createSequentialPersistentPath(path: String, data: Array[Byte]): String = {
val createRequest = CreateRequest(path, data, defaultAcls(path), CreateMode.PERSISTENT_SEQUENTIAL)
val createResponse = retryRequestUntilConnected(createRequest)
createResponse.maybeThrow()
createResponse.name
}
/**
* Registers the broker in zookeeper and return the broker epoch.
* @param brokerInfo payload of the broker znode
* @return broker epoch (znode create transaction id)
*/
def registerBroker(brokerInfo: BrokerInfo): Long = {
val path = brokerInfo.path
val stat = checkedEphemeralCreate(path, brokerInfo.toJsonBytes)
info(s"Registered broker ${brokerInfo.broker.id} at path $path with addresses: " +
s"${brokerInfo.broker.endPoints.map(_.connectionString).mkString(",")}, czxid (broker epoch): ${stat.getCzxid}")
stat.getCzxid
}
/**
* Registers a given broker in zookeeper as the controller and increments controller epoch.
* @param controllerId the id of the broker that is to be registered as the controller.
* @return the (updated controller epoch, epoch zkVersion) tuple
* @throws ControllerMovedException if fail to create /controller or fail to increment controller epoch.
*/
def registerControllerAndIncrementControllerEpoch(controllerId: Int): (Int, Int) = {
val timestamp = time.milliseconds()
// Read /controller_epoch to get the current controller epoch and zkVersion,
// create /controller_epoch with initial value if not exists
val (curEpoch, curEpochZkVersion) = getControllerEpoch
.map(e => (e._1, e._2.getVersion))
.getOrElse(maybeCreateControllerEpochZNode())
// Create /controller and update /controller_epoch atomically
val newControllerEpoch = curEpoch + 1
val expectedControllerEpochZkVersion = curEpochZkVersion
debug(s"Try to create ${ControllerZNode.path} and increment controller epoch to $newControllerEpoch with expected controller epoch zkVersion $expectedControllerEpochZkVersion")
def checkControllerAndEpoch(): (Int, Int) = {
val curControllerId = getControllerId.getOrElse(throw new ControllerMovedException(
s"The ephemeral node at ${ControllerZNode.path} went away while checking whether the controller election succeeds. " +
s"Aborting controller startup procedure"))
if (controllerId == curControllerId) {
val (epoch, stat) = getControllerEpoch.getOrElse(
throw new IllegalStateException(s"${ControllerEpochZNode.path} existed before but goes away while trying to read it"))
// If the epoch is the same as newControllerEpoch, it is safe to infer that the returned epoch zkVersion
// is associated with the current broker during controller election because we already knew that the zk
// transaction succeeds based on the controller znode verification. Other rounds of controller
// election will result in larger epoch number written in zk.
if (epoch == newControllerEpoch)
return (newControllerEpoch, stat.getVersion)
}
throw new ControllerMovedException("Controller moved to another broker. Aborting controller startup procedure")
}
def tryCreateControllerZNodeAndIncrementEpoch(): (Int, Int) = {
val response = retryRequestUntilConnected(
MultiRequest(Seq(
CreateOp(ControllerZNode.path, ControllerZNode.encode(controllerId, timestamp), defaultAcls(ControllerZNode.path), CreateMode.EPHEMERAL),
SetDataOp(ControllerEpochZNode.path, ControllerEpochZNode.encode(newControllerEpoch), expectedControllerEpochZkVersion)))
)
response.resultCode match {
case Code.NODEEXISTS | Code.BADVERSION => checkControllerAndEpoch()
case Code.OK =>
val setDataResult = response.zkOpResults(1).rawOpResult.asInstanceOf[SetDataResult]
(newControllerEpoch, setDataResult.getStat.getVersion)
case code => throw KeeperException.create(code)
}
}
tryCreateControllerZNodeAndIncrementEpoch()
}
/**
* Registers a given KRaft controller in zookeeper as the active controller. Unlike the ZK equivalent of this method,
* this creates /controller as a persistent znode. This prevents ZK brokers from attempting to claim the controller
* leadership during a KRaft leadership failover.
*
* This method is called at the beginning of a KRaft migration and during subsequent KRaft leadership changes during
* the migration.
*
* To ensure that the KRaft controller epoch exceeds the current ZK controller epoch, this registration algorithm
* uses a conditional update on the /controller and /controller_epoch znodes.
*
* If a new controller is registered concurrently with this registration, one of the two will fail the CAS
* operation on /controller_epoch. For KRaft, we have an extra guard against the registered KRaft epoch going
* backwards. If a KRaft controller had previously registered, an additional CAS operation is done on the /controller
* ZNode to ensure that the KRaft epoch being registered is newer.
*
* @param kraftControllerId ID of the KRaft controller node
* @param kraftControllerEpoch Epoch of the KRaft controller node
* @return A result object containing the written ZK controller epoch and version, or nothing.
*/
def tryRegisterKRaftControllerAsActiveController(kraftControllerId: Int, kraftControllerEpoch: Int): KRaftRegistrationResult = {
val timestamp = time.milliseconds()
val curEpochOpt: Option[(Int, Int)] = getControllerEpoch.map(e => (e._1, e._2.getVersion))
val controllerOpt = getControllerRegistration
// If we have a KRaft epoch registered in /controller, and it is not _older_ than the requested epoch, throw an error.
controllerOpt.flatMap(_.kraftEpoch).foreach { kraftEpochInZk =>
if (kraftEpochInZk >= kraftControllerEpoch) {
throw new ControllerMovedException(s"Cannot register KRaft controller $kraftControllerId with epoch $kraftControllerEpoch " +
s"as the current controller register in ZK has the same or newer epoch $kraftEpochInZk.")
}
}
curEpochOpt match {
case None =>
throw new IllegalStateException(s"Cannot register KRaft controller $kraftControllerId as the active controller " +
s"since there is no ZK controller epoch present.")
case Some((curEpoch: Int, curEpochZk: Int)) =>
val newControllerEpoch = curEpoch + 1
val response = controllerOpt match {
case Some(controller) =>
info(s"KRaft controller $kraftControllerId overwriting ${ControllerZNode.path} to become the active " +
s"controller with ZK epoch $newControllerEpoch. The previous controller was ${controller.broker}.")
retryRequestUntilConnected(
MultiRequest(Seq(
SetDataOp(ControllerEpochZNode.path, ControllerEpochZNode.encode(newControllerEpoch), curEpochZk),
DeleteOp(ControllerZNode.path, controller.zkVersion),
CreateOp(ControllerZNode.path, ControllerZNode.encode(kraftControllerId, timestamp, kraftControllerEpoch),
defaultAcls(ControllerZNode.path), CreateMode.PERSISTENT)))
)
case None =>
info(s"KRaft controller $kraftControllerId creating ${ControllerZNode.path} to become the active " +
s"controller with ZK epoch $newControllerEpoch. There was no active controller.")
retryRequestUntilConnected(
MultiRequest(Seq(
SetDataOp(ControllerEpochZNode.path, ControllerEpochZNode.encode(newControllerEpoch), curEpochZk),
CreateOp(ControllerZNode.path, ControllerZNode.encode(kraftControllerId, timestamp, kraftControllerEpoch),
defaultAcls(ControllerZNode.path), CreateMode.PERSISTENT)))
)
}
val failureSuffix = s"while trying to register KRaft controller $kraftControllerId with ZK epoch " +
s"$newControllerEpoch. KRaft controller was not registered."
response.resultCode match {
case Code.OK =>
info(s"Successfully registered KRaft controller $kraftControllerId with ZK epoch $newControllerEpoch")
// First op is always SetData on /controller_epoch
val setDataResult = response.zkOpResults(0).rawOpResult.asInstanceOf[SetDataResult]
SuccessfulRegistrationResult(newControllerEpoch, setDataResult.getStat.getVersion)
case Code.BADVERSION =>
info(s"The ZK controller epoch changed $failureSuffix")
FailedRegistrationResult()
case Code.NONODE =>
info(s"The ephemeral node at ${ControllerZNode.path} went away $failureSuffix")
FailedRegistrationResult()
case Code.NODEEXISTS =>
info(s"The ephemeral node at ${ControllerZNode.path} was created by another controller $failureSuffix")
FailedRegistrationResult()
case code =>
error(s"ZooKeeper had an error $failureSuffix")
throw KeeperException.create(code)
}
}
}
private def maybeCreateControllerEpochZNode(): (Int, Int) = {
createControllerEpochRaw(KafkaController.InitialControllerEpoch).resultCode match {
case Code.OK =>
info(s"Successfully created ${ControllerEpochZNode.path} with initial epoch ${KafkaController.InitialControllerEpoch}")
(KafkaController.InitialControllerEpoch, KafkaController.InitialControllerEpochZkVersion)
case Code.NODEEXISTS =>
val (epoch, stat) = getControllerEpoch.getOrElse(throw new IllegalStateException(s"${ControllerEpochZNode.path} existed before but goes away while trying to read it"))
(epoch, stat.getVersion)
case code =>
throw KeeperException.create(code)
}
}
def updateBrokerInfo(brokerInfo: BrokerInfo): Unit = {
val brokerIdPath = brokerInfo.path
val setDataRequest = SetDataRequest(brokerIdPath, brokerInfo.toJsonBytes, ZkVersion.MatchAnyVersion)
val response = retryRequestUntilConnected(setDataRequest)
response.maybeThrow()
info("Updated broker %d at path %s with addresses: %s".format(brokerInfo.broker.id, brokerIdPath, brokerInfo.broker.endPoints))
}
/**
* Gets topic partition states for the given partitions.
* @param partitions the partitions for which we want ot get states.
* @return sequence of GetDataResponses whose contexts are the partitions they are associated with.
*/
def getTopicPartitionStatesRaw(partitions: Seq[TopicPartition]): Seq[GetDataResponse] = {
val getDataRequests = partitions.map { partition =>
GetDataRequest(TopicPartitionStateZNode.path(partition), ctx = Some(partition))
}
retryRequestsUntilConnected(getDataRequests)
}
/**
* Sets topic partition states for the given partitions.
* @param leaderIsrAndControllerEpochs the partition states of each partition whose state we wish to set.
* @param expectedControllerEpochZkVersion expected controller epoch zkVersion.
* @return sequence of SetDataResponse whose contexts are the partitions they are associated with.
*/
def setTopicPartitionStatesRaw(leaderIsrAndControllerEpochs: Map[TopicPartition, LeaderIsrAndControllerEpoch], expectedControllerEpochZkVersion: Int): Seq[SetDataResponse] = {
val setDataRequests = leaderIsrAndControllerEpochs.map { case (partition, leaderIsrAndControllerEpoch) =>
val path = TopicPartitionStateZNode.path(partition)
val data = TopicPartitionStateZNode.encode(leaderIsrAndControllerEpoch)
SetDataRequest(path, data, leaderIsrAndControllerEpoch.leaderAndIsr.partitionEpoch, Some(partition))
}
retryRequestsUntilConnected(setDataRequests.toSeq, expectedControllerEpochZkVersion)
}
/**
* Creates topic partition state znodes for the given partitions.
* @param leaderIsrAndControllerEpochs the partition states of each partition whose state we wish to set.
* @param expectedControllerEpochZkVersion expected controller epoch zkVersion.
* @return sequence of CreateResponse whose contexts are the partitions they are associated with.
*/
def createTopicPartitionStatesRaw(leaderIsrAndControllerEpochs: Map[TopicPartition, LeaderIsrAndControllerEpoch], expectedControllerEpochZkVersion: Int): Seq[CreateResponse] = {
createTopicPartitions(leaderIsrAndControllerEpochs.keys.map(_.topic).toSeq.distinct, expectedControllerEpochZkVersion)
createTopicPartition(leaderIsrAndControllerEpochs.keys.toSeq, expectedControllerEpochZkVersion)
val createRequests = leaderIsrAndControllerEpochs.map { case (partition, leaderIsrAndControllerEpoch) =>
val path = TopicPartitionStateZNode.path(partition)
val data = TopicPartitionStateZNode.encode(leaderIsrAndControllerEpoch)
CreateRequest(path, data, defaultAcls(path), CreateMode.PERSISTENT, Some(partition))
}
retryRequestsUntilConnected(createRequests.toSeq, expectedControllerEpochZkVersion)
}
/**
* Sets the controller epoch conditioned on the given epochZkVersion.
* @param epoch the epoch to set
* @param epochZkVersion the expected version number of the epoch znode.
* @return SetDataResponse
*/
def setControllerEpochRaw(epoch: Int, epochZkVersion: Int): SetDataResponse = {
val setDataRequest = SetDataRequest(ControllerEpochZNode.path, ControllerEpochZNode.encode(epoch), epochZkVersion)
retryRequestUntilConnected(setDataRequest)
}
/**
* Creates the controller epoch znode.
* @param epoch the epoch to set
* @return CreateResponse
*/
def createControllerEpochRaw(epoch: Int): CreateResponse = {
val createRequest = CreateRequest(ControllerEpochZNode.path, ControllerEpochZNode.encode(epoch),
defaultAcls(ControllerEpochZNode.path), CreateMode.PERSISTENT)
retryRequestUntilConnected(createRequest)
}
/**
* Update the partition states of multiple partitions in zookeeper.
* @param leaderAndIsrs The partition states to update.
* @param controllerEpoch The current controller epoch.
* @param expectedControllerEpochZkVersion expected controller epoch zkVersion.
* @return UpdateLeaderAndIsrResult instance containing per partition results.
*/
def updateLeaderAndIsr(
leaderAndIsrs: Map[TopicPartition, LeaderAndIsr],
controllerEpoch: Int,
expectedControllerEpochZkVersion: Int
): UpdateLeaderAndIsrResult = {
val leaderIsrAndControllerEpochs = leaderAndIsrs.map { case (partition, leaderAndIsr) =>
partition -> LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch)
}
val setDataResponses = try {
setTopicPartitionStatesRaw(leaderIsrAndControllerEpochs, expectedControllerEpochZkVersion)
} catch {
case e: ControllerMovedException => throw e
case e: Exception =>
return UpdateLeaderAndIsrResult(leaderAndIsrs.keys.iterator.map(_ -> Left(e)).toMap, Seq.empty)
}
val updatesToRetry = mutable.Buffer.empty[TopicPartition]
val finished = setDataResponses.iterator.flatMap { setDataResponse =>
val partition = setDataResponse.ctx.get.asInstanceOf[TopicPartition]
setDataResponse.resultCode match {
case Code.OK =>
val updatedLeaderAndIsr = leaderAndIsrs(partition).withPartitionEpoch(setDataResponse.stat.getVersion)
Some(partition -> Right(updatedLeaderAndIsr))
case Code.BADVERSION =>
// Update the buffer for partitions to retry
updatesToRetry += partition
None
case _ =>
Some(partition -> Left(setDataResponse.resultException.get))
}
}.toMap
UpdateLeaderAndIsrResult(finished, updatesToRetry)
}
/**
* Get log configs that merge local configs with topic-level configs in zookeeper.
* @param topics The topics to get log configs for.
* @param config The local configs.
* @return A tuple of two values:
* 1. The successfully gathered log configs
* 2. Exceptions corresponding to failed log config lookups.
*/
def getLogConfigs(
topics: Set[String],
config: java.util.Map[String, AnyRef]
): (Map[String, LogConfig], Map[String, Exception]) = {
val logConfigs = mutable.Map.empty[String, LogConfig]
val failed = mutable.Map.empty[String, Exception]
val configResponses = try {
getTopicConfigs(topics)
} catch {
case e: Exception =>
topics.foreach(topic => failed.put(topic, e))
return (logConfigs.toMap, failed.toMap)
}
configResponses.foreach { configResponse =>
val topic = configResponse.ctx.get.asInstanceOf[String]
configResponse.resultCode match {
case Code.OK =>
val overrides = ConfigEntityZNode.decode(configResponse.data)
val logConfig = LogConfig.fromProps(config, overrides)
logConfigs.put(topic, logConfig)
case Code.NONODE =>
val logConfig = LogConfig.fromProps(config, new Properties)
logConfigs.put(topic, logConfig)
case _ => failed.put(topic, configResponse.resultException.get)
}
}
(logConfigs.toMap, failed.toMap)
}
/**
* Get entity configs for a given entity name
* @param rootEntityType entity type
* @param sanitizedEntityName entity name
* @return The successfully gathered log configs
*/
def getEntityConfigs(rootEntityType: String, sanitizedEntityName: String): Properties = {
val getDataRequest = GetDataRequest(ConfigEntityZNode.path(rootEntityType, sanitizedEntityName))
val getDataResponse = retryRequestUntilConnected(getDataRequest)
getDataResponse.resultCode match {
case Code.OK =>
ConfigEntityZNode.decode(getDataResponse.data)
case Code.NONODE => new Properties()
case _ => throw getDataResponse.resultException.get
}
}
def getEntitiesConfigs(rootEntityType: String, sanitizedEntityNames: Set[String]): Map[String, Properties] = {
val getDataRequests: Seq[GetDataRequest] = sanitizedEntityNames.map { entityName =>
GetDataRequest(ConfigEntityZNode.path(rootEntityType, entityName), Some(entityName))
}.toSeq
val getDataResponses = retryRequestsUntilConnected(getDataRequests)
getDataResponses.map { response =>
val entityName = response.ctx.get.asInstanceOf[String]
response.resultCode match {
case Code.OK =>
entityName -> ConfigEntityZNode.decode(response.data)
case Code.NONODE =>
entityName -> new Properties()
case _ => throw response.resultException.get
}
}.toMap
}
/**
* Sets or creates the entity znode path with the given configs depending
* on whether it already exists or not.
*
* If this is method is called concurrently, the last writer wins. In cases where we update configs and then
* partition assignment (i.e. create topic), it's possible for one thread to set this and the other to set the
* partition assignment. As such, the recommendation is to never call create topic for the same topic with different
* configs/partition assignment concurrently.
*
* @param rootEntityType entity type
* @param sanitizedEntityName entity name
* @throws KeeperException if there is an error while setting or creating the znode
*/
def setOrCreateEntityConfigs(rootEntityType: String, sanitizedEntityName: String, config: Properties) = {
def set(configData: Array[Byte]): SetDataResponse = {
val setDataRequest = SetDataRequest(ConfigEntityZNode.path(rootEntityType, sanitizedEntityName),
configData, ZkVersion.MatchAnyVersion)
retryRequestUntilConnected(setDataRequest)
}
def createOrSet(configData: Array[Byte]): Unit = {
val path = ConfigEntityZNode.path(rootEntityType, sanitizedEntityName)
try createRecursive(path, configData)
catch {
case _: NodeExistsException => set(configData).maybeThrow()
}
}
val configData = ConfigEntityZNode.encode(config)
val setDataResponse = set(configData)
setDataResponse.resultCode match {
case Code.NONODE => createOrSet(configData)
case _ => setDataResponse.maybeThrow()
}
}
/**
* Returns all the entities for a given entityType
* @param entityType entity type
* @return List of all entity names
*/
def getAllEntitiesWithConfig(entityType: String): Seq[String] = {
getChildren(ConfigEntityTypeZNode.path(entityType))
}
/**
* Creates config change notification
* @param sanitizedEntityPath sanitizedEntityPath path to write
* @throws KeeperException if there is an error while setting or creating the znode
*/
def createConfigChangeNotification(sanitizedEntityPath: String): Unit = {
makeSurePersistentPathExists(ConfigEntityChangeNotificationZNode.path)
val path = ConfigEntityChangeNotificationSequenceZNode.createPath
val createRequest = CreateRequest(path, ConfigEntityChangeNotificationSequenceZNode.encode(sanitizedEntityPath), defaultAcls(path), CreateMode.PERSISTENT_SEQUENTIAL)
val createResponse = retryRequestUntilConnected(createRequest)
createResponse.maybeThrow()
}
/**
* Gets all brokers in the cluster.
* @return sequence of brokers in the cluster.
*/
def getAllBrokersInCluster: Seq[Broker] = {
val brokerIds = getSortedBrokerList
val getDataRequests = brokerIds.map(brokerId => GetDataRequest(BrokerIdZNode.path(brokerId), ctx = Some(brokerId)))
val getDataResponses = retryRequestsUntilConnected(getDataRequests)
getDataResponses.flatMap { getDataResponse =>
val brokerId = getDataResponse.ctx.get.asInstanceOf[Int]
getDataResponse.resultCode match {
case Code.OK =>
Option(BrokerIdZNode.decode(brokerId, getDataResponse.data).broker)
case Code.NONODE => None
case _ => throw getDataResponse.resultException.get
}
}
}
/**
* Gets all brokers with broker epoch in the cluster.
* @return map of broker to epoch in the cluster.
*/
def getAllBrokerAndEpochsInCluster: Map[Broker, Long] = {
val brokerIds = getSortedBrokerList
val getDataRequests = brokerIds.map(brokerId => GetDataRequest(BrokerIdZNode.path(brokerId), ctx = Some(brokerId)))
val getDataResponses = retryRequestsUntilConnected(getDataRequests)
getDataResponses.flatMap { getDataResponse =>
val brokerId = getDataResponse.ctx.get.asInstanceOf[Int]
getDataResponse.resultCode match {
case Code.OK =>
Some((BrokerIdZNode.decode(brokerId, getDataResponse.data).broker, getDataResponse.stat.getCzxid))
case Code.NONODE => None
case _ => throw getDataResponse.resultException.get
}
}.toMap
}
/**
* Get a broker from ZK
* @return an optional Broker
*/
def getBroker(brokerId: Int): Option[Broker] = {
val getDataRequest = GetDataRequest(BrokerIdZNode.path(brokerId))
val getDataResponse = retryRequestUntilConnected(getDataRequest)
getDataResponse.resultCode match {
case Code.OK =>
Option(BrokerIdZNode.decode(brokerId, getDataResponse.data).broker)
case Code.NONODE => None
case _ => throw getDataResponse.resultException.get
}
}
/**
* Gets the list of sorted broker Ids
*/
def getSortedBrokerList: Seq[Int] = getChildren(BrokerIdsZNode.path).map(_.toInt).sorted
/**
* Gets all topics in the cluster.
* @param registerWatch indicates if a watch must be registered or not
* @return sequence of topics in the cluster.
*/
def getAllTopicsInCluster(registerWatch: Boolean = false): Set[String] = {
val getChildrenResponse = retryRequestUntilConnected(
GetChildrenRequest(TopicsZNode.path, registerWatch))
getChildrenResponse.resultCode match {
case Code.OK => getChildrenResponse.children.toSet
case Code.NONODE => Set.empty
case _ => throw getChildrenResponse.resultException.get
}
}
/**
* Checks the topic existence
* @param topicName
* @return true if topic exists else false
*/
def topicExists(topicName: String): Boolean = {
pathExists(TopicZNode.path(topicName))
}
/**
* Adds a topic ID to existing topic and replica assignments
* @param topicIdReplicaAssignments the TopicIDReplicaAssignments to add a topic ID to
* @return the updated TopicIdReplicaAssigments including the newly created topic IDs
*/
def setTopicIds(topicIdReplicaAssignments: collection.Set[TopicIdReplicaAssignment],
expectedControllerEpochZkVersion: Int): Set[TopicIdReplicaAssignment] = {
val updatedAssignments = topicIdReplicaAssignments.map {
case TopicIdReplicaAssignment(topic, None, assignments) =>
TopicIdReplicaAssignment(topic, Some(Uuid.randomUuid()), assignments)
case TopicIdReplicaAssignment(topic, Some(_), _) =>
throw new IllegalArgumentException("TopicIdReplicaAssignment for " + topic + " already contains a topic ID.")
}.toSet
val setDataRequests = updatedAssignments.map { case TopicIdReplicaAssignment(topic, topicIdOpt, assignments) =>
SetDataRequest(TopicZNode.path(topic), TopicZNode.encode(topicIdOpt, assignments), ZkVersion.MatchAnyVersion)
}.toSeq
retryRequestsUntilConnected(setDataRequests, expectedControllerEpochZkVersion)
updatedAssignments
}
/**
* Sets the topic znode with the given assignment.
* @param topic the topic whose assignment is being set.
* @param topicId unique topic ID for the topic if the version supports it
* @param assignment the partition to replica mapping to set for the given topic
* @param expectedControllerEpochZkVersion expected controller epoch zkVersion.
* @return SetDataResponse
*/
def setTopicAssignmentRaw(topic: String,
topicId: Option[Uuid],
assignment: collection.Map[TopicPartition, ReplicaAssignment],
expectedControllerEpochZkVersion: Int): SetDataResponse = {
val setDataRequest = SetDataRequest(TopicZNode.path(topic), TopicZNode.encode(topicId, assignment), ZkVersion.MatchAnyVersion)
retryRequestUntilConnected(setDataRequest, expectedControllerEpochZkVersion)
}
/**
* Sets the topic znode with the given assignment.
* @param topic the topic whose assignment is being set.
* @param topicId unique topic ID for the topic if the version supports it
* @param assignment the partition to replica mapping to set for the given topic
* @param expectedControllerEpochZkVersion expected controller epoch zkVersion.
* @throws KeeperException if there is an error while setting assignment
*/
def setTopicAssignment(topic: String,
topicId: Option[Uuid],
assignment: Map[TopicPartition, ReplicaAssignment],
expectedControllerEpochZkVersion: Int = ZkVersion.MatchAnyVersion) = {
val setDataResponse = setTopicAssignmentRaw(topic, topicId, assignment, expectedControllerEpochZkVersion)
setDataResponse.maybeThrow()
}
/**
* Create the topic znode with the given assignment.
* @param topic the topic whose assignment is being set.
* @param topicId unique topic ID for the topic if the version supports it
* @param assignment the partition to replica mapping to set for the given topic
* @throws KeeperException if there is an error while creating assignment
*/
def createTopicAssignment(topic: String, topicId: Option[Uuid], assignment: Map[TopicPartition, Seq[Int]]): Unit = {
val persistedAssignments = assignment.map { case (k, v) => k -> ReplicaAssignment(v) }
createRecursive(TopicZNode.path(topic), TopicZNode.encode(topicId, persistedAssignments))
}
/**
* Gets the log dir event notifications as strings. These strings are the znode names and not the absolute znode path.
* @return sequence of znode names and not the absolute znode path.
*/
def getAllLogDirEventNotifications: Seq[String] = {
val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(LogDirEventNotificationZNode.path, registerWatch = true))
getChildrenResponse.resultCode match {
case Code.OK => getChildrenResponse.children.map(LogDirEventNotificationSequenceZNode.sequenceNumber)
case Code.NONODE => Seq.empty
case _ => throw getChildrenResponse.resultException.get
}
}
/**
* Reads each of the log dir event notifications associated with the given sequence numbers and extracts the broker ids.
* @param sequenceNumbers the sequence numbers associated with the log dir event notifications.
* @return broker ids associated with the given log dir event notifications.
*/
def getBrokerIdsFromLogDirEvents(sequenceNumbers: Seq[String]): Seq[Int] = {
val getDataRequests = sequenceNumbers.map { sequenceNumber =>
GetDataRequest(LogDirEventNotificationSequenceZNode.path(sequenceNumber))
}
val getDataResponses = retryRequestsUntilConnected(getDataRequests)
getDataResponses.flatMap { getDataResponse =>
getDataResponse.resultCode match {
case Code.OK => LogDirEventNotificationSequenceZNode.decode(getDataResponse.data)
case Code.NONODE => None
case _ => throw getDataResponse.resultException.get
}
}
}
/**
* Deletes all log dir event notifications.
* @param expectedControllerEpochZkVersion expected controller epoch zkVersion.
*/
def deleteLogDirEventNotifications(expectedControllerEpochZkVersion: Int): Unit = {
val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(LogDirEventNotificationZNode.path, registerWatch = true))
if (getChildrenResponse.resultCode == Code.OK) {
deleteLogDirEventNotifications(getChildrenResponse.children.map(LogDirEventNotificationSequenceZNode.sequenceNumber), expectedControllerEpochZkVersion)
} else if (getChildrenResponse.resultCode != Code.NONODE) {
getChildrenResponse.maybeThrow()
}
}
/**
* Deletes the log dir event notifications associated with the given sequence numbers.
* @param sequenceNumbers the sequence numbers associated with the log dir event notifications to be deleted.
* @param expectedControllerEpochZkVersion expected controller epoch zkVersion.
*/
def deleteLogDirEventNotifications(sequenceNumbers: Seq[String], expectedControllerEpochZkVersion: Int): Unit = {
val deleteRequests = sequenceNumbers.map { sequenceNumber =>
DeleteRequest(LogDirEventNotificationSequenceZNode.path(sequenceNumber), ZkVersion.MatchAnyVersion)
}
retryRequestsUntilConnected(deleteRequests, expectedControllerEpochZkVersion)
}
/**
* Gets the topic IDs for the given topics.
* @param topics the topics we wish to retrieve the Topic IDs for
* @return the Topic IDs
*/
def getTopicIdsForTopics(topics: Set[String]): Map[String, Uuid] = {
val getDataRequests = topics.map(topic => GetDataRequest(TopicZNode.path(topic), ctx = Some(topic)))
val getDataResponses = retryRequestsUntilConnected(getDataRequests.toSeq)
getDataResponses.map { getDataResponse =>
val topic = getDataResponse.ctx.get.asInstanceOf[String]
getDataResponse.resultCode match {
case Code.OK => Some(TopicZNode.decode(topic, getDataResponse.data))
case Code.NONODE => None
case _ => throw getDataResponse.resultException.get
}
}.filter(_.flatMap(_.topicId).isDefined)
.map(_.get)
.map(topicIdAssignment => (topicIdAssignment.topic, topicIdAssignment.topicId.get))
.toMap
}
/**
* Gets the replica assignments for the given topics.
* This function does not return information about which replicas are being added or removed from the assignment.
* @param topics the topics whose partitions we wish to get the assignments for.
* @return the replica assignment for each partition from the given topics.
*/
def getReplicaAssignmentForTopics(topics: Set[String]): Map[TopicPartition, Seq[Int]] = {
getFullReplicaAssignmentForTopics(topics).map { case (k, v) => k -> v.replicas }
}
/**
* Gets the TopicID and replica assignments for the given topics.
* @param topics the topics whose partitions we wish to get the assignments for.
* @return the TopicIdReplicaAssignment for each partition for the given topics.
*/
def getReplicaAssignmentAndTopicIdForTopics(topics: Set[String]): Set[TopicIdReplicaAssignment] = {
val getDataRequests = topics.map(topic => GetDataRequest(TopicZNode.path(topic), ctx = Some(topic)))
val getDataResponses = retryRequestsUntilConnected(getDataRequests.toSeq)
getDataResponses.map { getDataResponse =>
val topic = getDataResponse.ctx.get.asInstanceOf[String]
getDataResponse.resultCode match {
case Code.OK => TopicZNode.decode(topic, getDataResponse.data)
case Code.NONODE => TopicIdReplicaAssignment(topic, None, Map.empty[TopicPartition, ReplicaAssignment])
case _ => throw getDataResponse.resultException.get
}
}.toSet
}
/**
* Gets the replica assignments for the given topics.
* @param topics the topics whose partitions we wish to get the assignments for.
* @return the full replica assignment for each partition from the given topics.
*/
def getFullReplicaAssignmentForTopics(topics: Set[String]): Map[TopicPartition, ReplicaAssignment] = {
val getDataRequests = topics.map(topic => GetDataRequest(TopicZNode.path(topic), ctx = Some(topic)))
val getDataResponses = retryRequestsUntilConnected(getDataRequests.toSeq)
getDataResponses.flatMap { getDataResponse =>
val topic = getDataResponse.ctx.get.asInstanceOf[String]
getDataResponse.resultCode match {
case Code.OK => TopicZNode.decode(topic, getDataResponse.data).assignment
case Code.NONODE => Map.empty[TopicPartition, ReplicaAssignment]
case _ => throw getDataResponse.resultException.get
}
}.toMap
}
/**
* Gets partition the assignments for the given topics.
* @param topics the topics whose partitions we wish to get the assignments for.
* @return the partition assignment for each partition from the given topics.
*/
def getPartitionAssignmentForTopics(topics: Set[String]): Map[String, Map[Int, ReplicaAssignment]] = {
val getDataRequests = topics.map(topic => GetDataRequest(TopicZNode.path(topic), ctx = Some(topic)))
val getDataResponses = retryRequestsUntilConnected(getDataRequests.toSeq)
getDataResponses.flatMap { getDataResponse =>
val topic = getDataResponse.ctx.get.asInstanceOf[String]
getDataResponse.resultCode match {
case Code.OK =>
val partitionMap = TopicZNode.decode(topic, getDataResponse.data).assignment.map { case (k, v) => (k.partition, v) }
Map(topic -> partitionMap)
case Code.NONODE => Map.empty[String, Map[Int, ReplicaAssignment]]
case _ => throw getDataResponse.resultException.get
}
}.toMap
}
/**
* Gets the partition numbers for the given topics
* @param topics the topics whose partitions we wish to get.
* @return the partition array for each topic from the given topics.
*/
def getPartitionsForTopics(topics: Set[String]): Map[String, Seq[Int]] = {
getPartitionAssignmentForTopics(topics).map { topicAndPartitionMap =>
val topic = topicAndPartitionMap._1
val partitionMap = topicAndPartitionMap._2
topic -> partitionMap.keys.toSeq.sorted
}
}
/**
* Gets the partition count for a given topic
* @param topic The topic to get partition count for.
* @return optional integer that is Some if the topic exists and None otherwise.
*/
def getTopicPartitionCount(topic: String): Option[Int] = {
val topicData = getReplicaAssignmentForTopics(Set(topic))
if (topicData.nonEmpty)
Some(topicData.size)
else
None
}
/**
* Gets the assigned replicas for a specific topic and partition
* @param topicPartition TopicAndPartition to get assigned replicas for .
* @return List of assigned replicas
*/
def getReplicasForPartition(topicPartition: TopicPartition): Seq[Int] = {
val topicData = getReplicaAssignmentForTopics(Set(topicPartition.topic))
topicData.getOrElse(topicPartition, Seq.empty)
}
/**
* Gets all partitions in the cluster
* @return all partitions in the cluster
*/
def getAllPartitions: Set[TopicPartition] = {
val topics = getChildren(TopicsZNode.path)
if (topics == null) Set.empty
else {
topics.flatMap { topic =>
// The partitions path may not exist if the topic is in the process of being deleted
getChildren(TopicPartitionsZNode.path(topic)).map(_.toInt).map(new TopicPartition(topic, _))
}.toSet
}
}
/**
* Gets the data and version at the given zk path
* @param path zk node path
* @return A tuple of 2 elements, where first element is zk node data as an array of bytes
* and second element is zk node version.
* returns (None, ZkVersion.UnknownVersion) if node doesn't exist and throws exception for any error
*/
def getDataAndVersion(path: String): (Option[Array[Byte]], Int) = {
val (data, stat) = getDataAndStat(path)
stat match {
case ZkStat.NoStat => (data, ZkVersion.UnknownVersion)
case _ => (data, stat.getVersion)
}
}
/**
* Gets the data and Stat at the given zk path
* @param path zk node path
* @return A tuple of 2 elements, where first element is zk node data as an array of bytes
* and second element is zk node stats.
* returns (None, ZkStat.NoStat) if node doesn't exists and throws exception for any error
*/
def getDataAndStat(path: String): (Option[Array[Byte]], Stat) = {
val getDataRequest = GetDataRequest(path)
val getDataResponse = retryRequestUntilConnected(getDataRequest)
getDataResponse.resultCode match {
case Code.OK => (Option(getDataResponse.data), getDataResponse.stat)
case Code.NONODE => (None, ZkStat.NoStat)
case _ => throw getDataResponse.resultException.get
}
}
/**
* Gets all the child nodes at a given zk node path
* @param path
* @return list of child node names
*/
def getChildren(path : String): Seq[String] = {
val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(path, registerWatch = true))
getChildrenResponse.resultCode match {
case Code.OK => getChildrenResponse.children
case Code.NONODE => Seq.empty
case _ => throw getChildrenResponse.resultException.get
}
}
/**
* Conditional update the persistent path data, return (true, newVersion) if it succeeds, otherwise (the path doesn't
* exist, the current version is not the expected version, etc.) return (false, ZkVersion.UnknownVersion)
*
* When there is a ConnectionLossException during the conditional update, ZookeeperClient will retry the update and may fail
* since the previous update may have succeeded (but the stored zkVersion no longer matches the expected one).
* In this case, we will run the optionalChecker to further check if the previous write did indeed succeeded.
*/
def conditionalUpdatePath(path: String, data: Array[Byte], expectVersion: Int,
optionalChecker: Option[(KafkaZkClient, String, Array[Byte]) => (Boolean,Int)] = None): (Boolean, Int) = {
val setDataRequest = SetDataRequest(path, data, expectVersion)
val setDataResponse = retryRequestUntilConnected(setDataRequest)
setDataResponse.resultCode match {
case Code.OK =>
debug("Conditional update of path %s with value %s and expected version %d succeeded, returning the new version: %d"
.format(path, Utils.utf8(data), expectVersion, setDataResponse.stat.getVersion))
(true, setDataResponse.stat.getVersion)
case Code.BADVERSION =>
optionalChecker match {
case Some(checker) => checker(this, path, data)
case _ =>
debug("Checker method is not passed skipping zkData match")
debug("Conditional update of path %s with data %s and expected version %d failed due to %s"
.format(path, Utils.utf8(data), expectVersion, setDataResponse.resultException.get.getMessage))
(false, ZkVersion.UnknownVersion)
}
case Code.NONODE =>
debug("Conditional update of path %s with data %s and expected version %d failed due to %s".format(path,
Utils.utf8(data), expectVersion, setDataResponse.resultException.get.getMessage))
(false, ZkVersion.UnknownVersion)
case _ =>
debug("Conditional update of path %s with data %s and expected version %d failed due to %s".format(path,
Utils.utf8(data), expectVersion, setDataResponse.resultException.get.getMessage))
throw setDataResponse.resultException.get
}
}
/**
* Creates the delete topic znode.
* @param topicName topic name
* @throws KeeperException if there is an error while setting or creating the znode
*/
def createDeleteTopicPath(topicName: String): Unit = {
createRecursive(DeleteTopicsTopicZNode.path(topicName))
}
/**
* Checks if topic is marked for deletion
* @param topic
* @return true if topic is marked for deletion, else false
*/
def isTopicMarkedForDeletion(topic: String): Boolean = {
pathExists(DeleteTopicsTopicZNode.path(topic))
}
/**
* Get all topics marked for deletion.
* @return sequence of topics marked for deletion.
*/
def getTopicDeletions: Seq[String] = {
val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(DeleteTopicsZNode.path, registerWatch = true))
getChildrenResponse.resultCode match {
case Code.OK => getChildrenResponse.children
case Code.NONODE => Seq.empty
case _ => throw getChildrenResponse.resultException.get
}
}
/**
* Remove the given topics from the topics marked for deletion.
* @param topics the topics to remove.
* @param expectedControllerEpochZkVersion expected controller epoch zkVersion.
*/
def deleteTopicDeletions(topics: Seq[String], expectedControllerEpochZkVersion: Int): Unit = {
val deleteRequests = topics.map(topic => DeleteRequest(DeleteTopicsTopicZNode.path(topic), ZkVersion.MatchAnyVersion))
retryRequestsUntilConnected(deleteRequests, expectedControllerEpochZkVersion)
}
/**
* Returns all reassignments.
* @return the reassignments for each partition.
*/
def getPartitionReassignment: collection.Map[TopicPartition, Seq[Int]] = {
val getDataRequest = GetDataRequest(ReassignPartitionsZNode.path)
val getDataResponse = retryRequestUntilConnected(getDataRequest)
getDataResponse.resultCode match {
case Code.OK =>
ReassignPartitionsZNode.decode(getDataResponse.data) match {
case Left(e) =>
logger.warn(s"Ignoring partition reassignment due to invalid json: ${e.getMessage}", e)
Map.empty[TopicPartition, Seq[Int]]
case Right(assignments) => assignments