/
TestUtils.scala
executable file
·1593 lines (1440 loc) · 67 KB
/
TestUtils.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.utils
import java.io._
import java.nio._
import java.nio.channels._
import java.nio.charset.{Charset, StandardCharsets}
import java.nio.file.{Files, StandardOpenOption}
import java.security.cert.X509Certificate
import java.time.Duration
import java.util.Arrays
import java.util.Collections
import java.util.Properties
import java.util.concurrent.{Callable, ExecutionException, Executors, TimeUnit}
import javax.net.ssl.X509TrustManager
import kafka.api._
import kafka.cluster.{Broker, EndPoint}
import kafka.log._
import kafka.security.auth.{Acl, Authorizer, Resource}
import kafka.server._
import kafka.server.checkpoints.OffsetCheckpointFile
import Implicits._
import com.yammer.metrics.Metrics
import kafka.controller.LeaderIsrAndControllerEpoch
import kafka.zk._
import org.apache.kafka.clients.CommonClientConfigs
import org.apache.kafka.clients.admin.AlterConfigOp.OpType
import org.apache.kafka.clients.admin._
import org.apache.kafka.clients.consumer._
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord}
import org.apache.kafka.common.{KafkaFuture, TopicPartition}
import org.apache.kafka.common.config.ConfigResource
import org.apache.kafka.common.errors.RetriableException
import org.apache.kafka.common.header.Header
import org.apache.kafka.common.internals.Topic
import org.apache.kafka.common.network.{ListenerName, Mode}
import org.apache.kafka.common.record._
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySerializer, Deserializer, Serializer}
import org.apache.kafka.common.utils.Time
import org.apache.kafka.common.utils.Utils._
import org.apache.kafka.test.{TestSslUtils, TestUtils => JTestUtils}
import org.apache.zookeeper.KeeperException.SessionExpiredException
import org.apache.zookeeper.ZooDefs._
import org.apache.zookeeper.data.ACL
import org.junit.Assert._
import org.scalatest.Assertions.fail
import scala.collection.JavaConverters._
import scala.collection.{Map, Seq, mutable}
import scala.collection.mutable.{ArrayBuffer, ListBuffer}
import scala.concurrent.duration.FiniteDuration
import scala.concurrent.{Await, ExecutionContext, Future}
/**
* Utility functions to help with testing
*/
object TestUtils extends Logging {
val random = JTestUtils.RANDOM
/* 0 gives a random port; you can then retrieve the assigned port from the Socket object. */
val RandomPort = 0
/* Incorrect broker port which can used by kafka clients in tests. This port should not be used
by any other service and hence we use a reserved port. */
val IncorrectBrokerPort = 225
/** Port to use for unit tests that mock/don't require a real ZK server. */
val MockZkPort = 1
/** ZooKeeper connection string to use for unit tests that mock/don't require a real ZK server. */
val MockZkConnect = "127.0.0.1:" + MockZkPort
// CN in SSL certificates - this is used for endpoint validation when enabled
val SslCertificateCn = "localhost"
private val transactionStatusKey = "transactionStatus"
private val committedValue : Array[Byte] = "committed".getBytes(StandardCharsets.UTF_8)
private val abortedValue : Array[Byte] = "aborted".getBytes(StandardCharsets.UTF_8)
/**
* Create a temporary directory
*/
def tempDir(): File = JTestUtils.tempDirectory()
def tempTopic(): String = "testTopic" + random.nextInt(1000000)
/**
* Create a temporary relative directory
*/
def tempRelativeDir(parent: String): File = {
val parentFile = new File(parent)
parentFile.mkdirs()
JTestUtils.tempDirectory(parentFile.toPath, null)
}
/**
* Create a random log directory in the format <string>-<int> used for Kafka partition logs.
* It is the responsibility of the caller to set up a shutdown hook for deletion of the directory.
*/
def randomPartitionLogDir(parentDir: File): File = {
val attempts = 1000
val f = Iterator.continually(new File(parentDir, "kafka-" + random.nextInt(1000000)))
.take(attempts).find(_.mkdir())
.getOrElse(sys.error(s"Failed to create directory after $attempts attempts"))
f.deleteOnExit()
f
}
/**
* Create a temporary file
*/
def tempFile(): File = JTestUtils.tempFile()
/**
* Create a temporary file and return an open file channel for this file
*/
def tempChannel(): FileChannel =
FileChannel.open(tempFile().toPath, StandardOpenOption.READ, StandardOpenOption.WRITE)
/**
* Create a kafka server instance with appropriate test settings
* USING THIS IS A SIGN YOU ARE NOT WRITING A REAL UNIT TEST
*
* @param config The configuration of the server
*/
def createServer(config: KafkaConfig, time: Time = Time.SYSTEM): KafkaServer = {
val server = new KafkaServer(config, time)
server.startup()
server
}
def boundPort(server: KafkaServer, securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): Int =
server.boundPort(ListenerName.forSecurityProtocol(securityProtocol))
def createBroker(id: Int, host: String, port: Int, securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): Broker =
new Broker(id, host, port, ListenerName.forSecurityProtocol(securityProtocol), securityProtocol)
def createBrokerAndEpoch(id: Int, host: String, port: Int, securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT,
epoch: Long = 0): (Broker, Long) = {
(new Broker(id, host, port, ListenerName.forSecurityProtocol(securityProtocol), securityProtocol), epoch)
}
/**
* Create a test config for the provided parameters.
*
* Note that if `interBrokerSecurityProtocol` is defined, the listener for the `SecurityProtocol` will be enabled.
*/
def createBrokerConfigs(numConfigs: Int,
zkConnect: String,
enableControlledShutdown: Boolean = true,
enableDeleteTopic: Boolean = true,
interBrokerSecurityProtocol: Option[SecurityProtocol] = None,
trustStoreFile: Option[File] = None,
saslProperties: Option[Properties] = None,
enablePlaintext: Boolean = true,
enableSsl: Boolean = false,
enableSaslPlaintext: Boolean = false,
enableSaslSsl: Boolean = false,
rackInfo: Map[Int, String] = Map(),
logDirCount: Int = 1,
enableToken: Boolean = false,
numPartitions: Int = 1,
defaultReplicationFactor: Short = 1): Seq[Properties] = {
(0 until numConfigs).map { node =>
createBrokerConfig(node, zkConnect, enableControlledShutdown, enableDeleteTopic, RandomPort,
interBrokerSecurityProtocol, trustStoreFile, saslProperties, enablePlaintext = enablePlaintext, enableSsl = enableSsl,
enableSaslPlaintext = enableSaslPlaintext, enableSaslSsl = enableSaslSsl, rack = rackInfo.get(node), logDirCount = logDirCount, enableToken = enableToken,
numPartitions = numPartitions, defaultReplicationFactor = defaultReplicationFactor)
}
}
def getBrokerListStrFromServers(servers: Seq[KafkaServer], protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): String = {
servers.map { s =>
val listener = s.config.advertisedListeners.find(_.securityProtocol == protocol).getOrElse(
sys.error(s"Could not find listener with security protocol $protocol"))
formatAddress(listener.host, boundPort(s, protocol))
}.mkString(",")
}
def bootstrapServers(servers: Seq[KafkaServer], listenerName: ListenerName): String = {
servers.map { s =>
val listener = s.config.advertisedListeners.find(_.listenerName == listenerName).getOrElse(
sys.error(s"Could not find listener with name ${listenerName.value}"))
formatAddress(listener.host, s.boundPort(listenerName))
}.mkString(",")
}
/**
* Shutdown `servers` and delete their log directories.
*/
def shutdownServers(servers: Seq[KafkaServer]) {
import ExecutionContext.Implicits._
val future = Future.traverse(servers) { s =>
Future {
s.shutdown()
CoreUtils.delete(s.config.logDirs)
}
}
Await.result(future, FiniteDuration(5, TimeUnit.MINUTES))
}
/**
* Create a test config for the provided parameters.
*
* Note that if `interBrokerSecurityProtocol` is defined, the listener for the `SecurityProtocol` will be enabled.
*/
def createBrokerConfig(nodeId: Int,
zkConnect: String,
enableControlledShutdown: Boolean = true,
enableDeleteTopic: Boolean = true,
port: Int = RandomPort,
interBrokerSecurityProtocol: Option[SecurityProtocol] = None,
trustStoreFile: Option[File] = None,
saslProperties: Option[Properties] = None,
enablePlaintext: Boolean = true,
enableSaslPlaintext: Boolean = false,
saslPlaintextPort: Int = RandomPort,
enableSsl: Boolean = false,
sslPort: Int = RandomPort,
enableSaslSsl: Boolean = false,
saslSslPort: Int = RandomPort,
rack: Option[String] = None,
logDirCount: Int = 1,
enableToken: Boolean = false,
numPartitions: Int = 1,
defaultReplicationFactor: Short = 1): Properties = {
def shouldEnable(protocol: SecurityProtocol) = interBrokerSecurityProtocol.fold(false)(_ == protocol)
val protocolAndPorts = ArrayBuffer[(SecurityProtocol, Int)]()
if (enablePlaintext || shouldEnable(SecurityProtocol.PLAINTEXT))
protocolAndPorts += SecurityProtocol.PLAINTEXT -> port
if (enableSsl || shouldEnable(SecurityProtocol.SSL))
protocolAndPorts += SecurityProtocol.SSL -> sslPort
if (enableSaslPlaintext || shouldEnable(SecurityProtocol.SASL_PLAINTEXT))
protocolAndPorts += SecurityProtocol.SASL_PLAINTEXT -> saslPlaintextPort
if (enableSaslSsl || shouldEnable(SecurityProtocol.SASL_SSL))
protocolAndPorts += SecurityProtocol.SASL_SSL -> saslSslPort
val listeners = protocolAndPorts.map { case (protocol, port) =>
s"${protocol.name}://localhost:$port"
}.mkString(",")
val props = new Properties
if (nodeId >= 0) props.put(KafkaConfig.BrokerIdProp, nodeId.toString)
props.put(KafkaConfig.ListenersProp, listeners)
if (logDirCount > 1) {
val logDirs = (1 to logDirCount).toList.map(i =>
// We would like to allow user to specify both relative path and absolute path as log directory for backward-compatibility reason
// We can verify this by using a mixture of relative path and absolute path as log directories in the test
if (i % 2 == 0) TestUtils.tempDir().getAbsolutePath else TestUtils.tempRelativeDir("data")
).mkString(",")
props.put(KafkaConfig.LogDirsProp, logDirs)
} else {
props.put(KafkaConfig.LogDirProp, TestUtils.tempDir().getAbsolutePath)
}
props.put(KafkaConfig.ZkConnectProp, zkConnect)
props.put(KafkaConfig.ZkConnectionTimeoutMsProp, "10000")
props.put(KafkaConfig.ReplicaSocketTimeoutMsProp, "1500")
props.put(KafkaConfig.ControllerSocketTimeoutMsProp, "1500")
props.put(KafkaConfig.ControlledShutdownEnableProp, enableControlledShutdown.toString)
props.put(KafkaConfig.DeleteTopicEnableProp, enableDeleteTopic.toString)
props.put(KafkaConfig.LogDeleteDelayMsProp, "1000")
props.put(KafkaConfig.ControlledShutdownRetryBackoffMsProp, "100")
props.put(KafkaConfig.LogCleanerDedupeBufferSizeProp, "2097152")
props.put(KafkaConfig.LogMessageTimestampDifferenceMaxMsProp, Long.MaxValue.toString)
props.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
if (!props.containsKey(KafkaConfig.OffsetsTopicPartitionsProp))
props.put(KafkaConfig.OffsetsTopicPartitionsProp, "5")
if (!props.containsKey(KafkaConfig.GroupInitialRebalanceDelayMsProp))
props.put(KafkaConfig.GroupInitialRebalanceDelayMsProp, "0")
rack.foreach(props.put(KafkaConfig.RackProp, _))
if (protocolAndPorts.exists { case (protocol, _) => usesSslTransportLayer(protocol) })
props ++= sslConfigs(Mode.SERVER, false, trustStoreFile, s"server$nodeId")
if (protocolAndPorts.exists { case (protocol, _) => usesSaslAuthentication(protocol) })
props ++= JaasTestUtils.saslConfigs(saslProperties)
interBrokerSecurityProtocol.foreach { protocol =>
props.put(KafkaConfig.InterBrokerSecurityProtocolProp, protocol.name)
}
if (enableToken)
props.put(KafkaConfig.DelegationTokenMasterKeyProp, "masterkey")
props.put(KafkaConfig.NumPartitionsProp, numPartitions.toString)
props.put(KafkaConfig.DefaultReplicationFactorProp, defaultReplicationFactor.toString)
props
}
/**
* Create a topic in ZooKeeper.
* Wait until the leader is elected and the metadata is propagated to all brokers.
* Return the leader for each partition.
*/
def createTopic(zkClient: KafkaZkClient,
topic: String,
numPartitions: Int = 1,
replicationFactor: Int = 1,
servers: Seq[KafkaServer],
topicConfig: Properties = new Properties): scala.collection.immutable.Map[Int, Int] = {
val adminZkClient = new AdminZkClient(zkClient)
// create topic
waitUntilTrue( () => {
var hasSessionExpirationException = false
try {
adminZkClient.createTopic(topic, numPartitions, replicationFactor, topicConfig)
} catch {
case _: SessionExpiredException => hasSessionExpirationException = true
case e: Throwable => throw e // let other exceptions propagate
}
!hasSessionExpirationException},
s"Can't create topic $topic")
// wait until the update metadata request for new topic reaches all servers
(0 until numPartitions).map { i =>
TestUtils.waitUntilMetadataIsPropagated(servers, topic, i)
i -> TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, i)
}.toMap
}
/**
* Create a topic in ZooKeeper using a customized replica assignment.
* Wait until the leader is elected and the metadata is propagated to all brokers.
* Return the leader for each partition.
*/
def createTopic(zkClient: KafkaZkClient,
topic: String,
partitionReplicaAssignment: collection.Map[Int, Seq[Int]],
servers: Seq[KafkaServer]): scala.collection.immutable.Map[Int, Int] = {
createTopic(zkClient, topic, partitionReplicaAssignment, servers, new Properties())
}
/**
* Create a topic in ZooKeeper using a customized replica assignment.
* Wait until the leader is elected and the metadata is propagated to all brokers.
* Return the leader for each partition.
*/
def createTopic(zkClient: KafkaZkClient,
topic: String,
partitionReplicaAssignment: collection.Map[Int, Seq[Int]],
servers: Seq[KafkaServer],
topicConfig: Properties): scala.collection.immutable.Map[Int, Int] = {
val adminZkClient = new AdminZkClient(zkClient)
// create topic
waitUntilTrue( () => {
var hasSessionExpirationException = false
try {
adminZkClient.createTopicWithAssignment(topic, topicConfig, partitionReplicaAssignment)
} catch {
case _: SessionExpiredException => hasSessionExpirationException = true
case e: Throwable => throw e // let other exceptions propagate
}
!hasSessionExpirationException},
s"Can't create topic $topic")
// wait until the update metadata request for new topic reaches all servers
partitionReplicaAssignment.keySet.map { i =>
TestUtils.waitUntilMetadataIsPropagated(servers, topic, i)
i -> TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, i)
}.toMap
}
/**
* Create the consumer offsets/group metadata topic and wait until the leader is elected and metadata is propagated
* to all brokers.
*/
def createOffsetsTopic(zkClient: KafkaZkClient, servers: Seq[KafkaServer]): Unit = {
val server = servers.head
createTopic(zkClient, Topic.GROUP_METADATA_TOPIC_NAME,
server.config.getInt(KafkaConfig.OffsetsTopicPartitionsProp),
server.config.getShort(KafkaConfig.OffsetsTopicReplicationFactorProp).toInt,
servers,
server.groupCoordinator.offsetsTopicConfigs)
}
/**
* Wrap a single record log buffer.
*/
def singletonRecords(value: Array[Byte],
key: Array[Byte] = null,
codec: CompressionType = CompressionType.NONE,
timestamp: Long = RecordBatch.NO_TIMESTAMP,
magicValue: Byte = RecordBatch.CURRENT_MAGIC_VALUE): MemoryRecords = {
records(Seq(new SimpleRecord(timestamp, key, value)), magicValue = magicValue, codec = codec)
}
def recordsWithValues(magicValue: Byte,
codec: CompressionType,
values: Array[Byte]*): MemoryRecords = {
records(values.map(value => new SimpleRecord(value)), magicValue, codec)
}
def records(records: Iterable[SimpleRecord],
magicValue: Byte = RecordBatch.CURRENT_MAGIC_VALUE,
codec: CompressionType = CompressionType.NONE,
producerId: Long = RecordBatch.NO_PRODUCER_ID,
producerEpoch: Short = RecordBatch.NO_PRODUCER_EPOCH,
sequence: Int = RecordBatch.NO_SEQUENCE,
baseOffset: Long = 0L,
partitionLeaderEpoch: Int = RecordBatch.NO_PARTITION_LEADER_EPOCH): MemoryRecords = {
val buf = ByteBuffer.allocate(DefaultRecordBatch.sizeInBytes(records.asJava))
val builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, baseOffset,
System.currentTimeMillis, producerId, producerEpoch, sequence, false, partitionLeaderEpoch)
records.foreach(builder.append)
builder.build()
}
/**
* Generate an array of random bytes
*
* @param numBytes The size of the array
*/
def randomBytes(numBytes: Int): Array[Byte] = JTestUtils.randomBytes(numBytes)
/**
* Generate a random string of letters and digits of the given length
*
* @param len The length of the string
* @return The random string
*/
def randomString(len: Int): String = JTestUtils.randomString(len)
/**
* Check that the buffer content from buffer.position() to buffer.limit() is equal
*/
def checkEquals(b1: ByteBuffer, b2: ByteBuffer) {
assertEquals("Buffers should have equal length", b1.limit() - b1.position(), b2.limit() - b2.position())
for(i <- 0 until b1.limit() - b1.position())
assertEquals("byte " + i + " byte not equal.", b1.get(b1.position() + i), b2.get(b1.position() + i))
}
/**
* Throw an exception if the two iterators are of differing lengths or contain
* different messages on their Nth element
*/
def checkEquals[T](expected: Iterator[T], actual: Iterator[T]) {
var length = 0
while(expected.hasNext && actual.hasNext) {
length += 1
assertEquals(expected.next, actual.next)
}
// check if the expected iterator is longer
if (expected.hasNext) {
var length1 = length
while (expected.hasNext) {
expected.next
length1 += 1
}
assertFalse("Iterators have uneven length-- first has more: "+length1 + " > " + length, true)
}
// check if the actual iterator was longer
if (actual.hasNext) {
var length2 = length
while (actual.hasNext) {
actual.next
length2 += 1
}
assertFalse("Iterators have uneven length-- second has more: "+length2 + " > " + length, true)
}
}
/**
* Throw an exception if an iterable has different length than expected
*
*/
def checkLength[T](s1: Iterator[T], expectedLength:Int) {
var n = 0
while (s1.hasNext) {
n+=1
s1.next
}
assertEquals(expectedLength, n)
}
/**
* Throw an exception if the two iterators are of differing lengths or contain
* different messages on their Nth element
*/
def checkEquals[T](s1: java.util.Iterator[T], s2: java.util.Iterator[T]) {
while(s1.hasNext && s2.hasNext)
assertEquals(s1.next, s2.next)
assertFalse("Iterators have uneven length--first has more", s1.hasNext)
assertFalse("Iterators have uneven length--second has more", s2.hasNext)
}
def stackedIterator[T](s: Iterator[T]*): Iterator[T] = {
new Iterator[T] {
var cur: Iterator[T] = null
val topIterator = s.iterator
def hasNext: Boolean = {
while (true) {
if (cur == null) {
if (topIterator.hasNext)
cur = topIterator.next
else
return false
}
if (cur.hasNext)
return true
cur = null
}
// should never reach her
throw new RuntimeException("should not reach here")
}
def next() : T = cur.next
}
}
/**
* Create a hexadecimal string for the given bytes
*/
def hexString(bytes: Array[Byte]): String = hexString(ByteBuffer.wrap(bytes))
/**
* Create a hexadecimal string for the given bytes
*/
def hexString(buffer: ByteBuffer): String = {
val builder = new StringBuilder("0x")
for(i <- 0 until buffer.limit())
builder.append(String.format("%x", Integer.valueOf(buffer.get(buffer.position() + i))))
builder.toString
}
def securityConfigs(mode: Mode,
securityProtocol: SecurityProtocol,
trustStoreFile: Option[File],
certAlias: String,
certCn: String,
saslProperties: Option[Properties]): Properties = {
val props = new Properties
if (usesSslTransportLayer(securityProtocol))
props ++= sslConfigs(mode, securityProtocol == SecurityProtocol.SSL, trustStoreFile, certAlias, certCn)
if (usesSaslAuthentication(securityProtocol))
props ++= JaasTestUtils.saslConfigs(saslProperties)
props.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, securityProtocol.name)
props
}
def producerSecurityConfigs(securityProtocol: SecurityProtocol,
trustStoreFile: Option[File],
saslProperties: Option[Properties]): Properties =
securityConfigs(Mode.CLIENT, securityProtocol, trustStoreFile, "producer", SslCertificateCn, saslProperties)
/**
* Create a (new) producer with a few pre-configured properties.
*/
def createProducer[K, V](brokerList: String,
acks: Int = -1,
maxBlockMs: Long = 60 * 1000L,
bufferSize: Long = 1024L * 1024L,
retries: Int = Int.MaxValue,
deliveryTimeoutMs: Int = 30 * 1000,
lingerMs: Int = 0,
batchSize: Int = 16384,
compressionType: String = "none",
requestTimeoutMs: Int = 20 * 1000,
securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT,
trustStoreFile: Option[File] = None,
saslProperties: Option[Properties] = None,
keySerializer: Serializer[K] = new ByteArraySerializer,
valueSerializer: Serializer[V] = new ByteArraySerializer): KafkaProducer[K, V] = {
val producerProps = new Properties
producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList)
producerProps.put(ProducerConfig.ACKS_CONFIG, acks.toString)
producerProps.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, maxBlockMs.toString)
producerProps.put(ProducerConfig.BUFFER_MEMORY_CONFIG, bufferSize.toString)
producerProps.put(ProducerConfig.RETRIES_CONFIG, retries.toString)
producerProps.put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, deliveryTimeoutMs.toString)
producerProps.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, requestTimeoutMs.toString)
producerProps.put(ProducerConfig.LINGER_MS_CONFIG, lingerMs.toString)
producerProps.put(ProducerConfig.BATCH_SIZE_CONFIG, batchSize.toString)
producerProps.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, compressionType)
producerProps ++= producerSecurityConfigs(securityProtocol, trustStoreFile, saslProperties)
new KafkaProducer[K, V](producerProps, keySerializer, valueSerializer)
}
def usesSslTransportLayer(securityProtocol: SecurityProtocol): Boolean = securityProtocol match {
case SecurityProtocol.SSL | SecurityProtocol.SASL_SSL => true
case _ => false
}
def usesSaslAuthentication(securityProtocol: SecurityProtocol): Boolean = securityProtocol match {
case SecurityProtocol.SASL_PLAINTEXT | SecurityProtocol.SASL_SSL => true
case _ => false
}
def consumerSecurityConfigs(securityProtocol: SecurityProtocol, trustStoreFile: Option[File], saslProperties: Option[Properties]): Properties =
securityConfigs(Mode.CLIENT, securityProtocol, trustStoreFile, "consumer", SslCertificateCn, saslProperties)
def adminClientSecurityConfigs(securityProtocol: SecurityProtocol, trustStoreFile: Option[File], saslProperties: Option[Properties]): Properties =
securityConfigs(Mode.CLIENT, securityProtocol, trustStoreFile, "admin-client", SslCertificateCn, saslProperties)
/**
* Create a consumer with a few pre-configured properties.
*/
def createConsumer[K, V](brokerList: String,
groupId: String = "group",
autoOffsetReset: String = "earliest",
enableAutoCommit: Boolean = true,
readCommitted: Boolean = false,
maxPollRecords: Int = 500,
securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT,
trustStoreFile: Option[File] = None,
saslProperties: Option[Properties] = None,
keyDeserializer: Deserializer[K] = new ByteArrayDeserializer,
valueDeserializer: Deserializer[V] = new ByteArrayDeserializer): KafkaConsumer[K, V] = {
val consumerProps = new Properties
consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList)
consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, autoOffsetReset)
consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, groupId)
consumerProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, enableAutoCommit.toString)
consumerProps.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPollRecords.toString)
consumerProps.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, if (readCommitted) "read_committed" else "read_uncommitted")
consumerProps ++= consumerSecurityConfigs(securityProtocol, trustStoreFile, saslProperties)
new KafkaConsumer[K, V](consumerProps, keyDeserializer, valueDeserializer)
}
def createBrokersInZk(zkClient: KafkaZkClient, ids: Seq[Int]): Seq[Broker] =
createBrokersInZk(ids.map(kafka.admin.BrokerMetadata(_, None)), zkClient)
def createBrokersInZk(brokerMetadatas: Seq[kafka.admin.BrokerMetadata], zkClient: KafkaZkClient): Seq[Broker] = {
zkClient.makeSurePersistentPathExists(BrokerIdsZNode.path)
val brokers = brokerMetadatas.map { b =>
val protocol = SecurityProtocol.PLAINTEXT
val listenerName = ListenerName.forSecurityProtocol(protocol)
Broker(b.id, Seq(EndPoint("localhost", 6667, listenerName, protocol)), b.rack)
}
brokers.foreach(b => zkClient.registerBroker(BrokerInfo(Broker(b.id, b.endPoints, rack = b.rack),
ApiVersion.latestVersion, jmxPort = -1)))
brokers
}
def deleteBrokersInZk(zkClient: KafkaZkClient, ids: Seq[Int]): Seq[Broker] = {
val brokers = ids.map(createBroker(_, "localhost", 6667, SecurityProtocol.PLAINTEXT))
ids.foreach(b => zkClient.deletePath(BrokerIdsZNode.path + "/" + b))
brokers
}
def getMsgStrings(n: Int): Seq[String] = {
val buffer = new ListBuffer[String]
for (i <- 0 until n)
buffer += ("msg" + i)
buffer
}
def makeLeaderForPartition(zkClient: KafkaZkClient,
topic: String,
leaderPerPartitionMap: scala.collection.immutable.Map[Int, Int],
controllerEpoch: Int) {
val newLeaderIsrAndControllerEpochs = leaderPerPartitionMap.map { case (partition, leader) =>
val topicPartition = new TopicPartition(topic, partition)
val newLeaderAndIsr = zkClient.getTopicPartitionState(topicPartition)
.map(_.leaderAndIsr.newLeader(leader))
.getOrElse(LeaderAndIsr(leader, List(leader)))
topicPartition -> LeaderIsrAndControllerEpoch(newLeaderAndIsr, controllerEpoch)
}
zkClient.setTopicPartitionStatesRaw(newLeaderIsrAndControllerEpochs, ZkVersion.MatchAnyVersion)
}
/**
* If neither oldLeaderOpt nor newLeaderOpt is defined, wait until the leader of a partition is elected.
* If oldLeaderOpt is defined, it waits until the new leader is different from the old leader.
* If newLeaderOpt is defined, it waits until the new leader becomes the expected new leader.
*
* @return The new leader (note that negative values are used to indicate conditions like NoLeader and
* LeaderDuringDelete).
* @throws AssertionError if the expected condition is not true within the timeout.
*/
def waitUntilLeaderIsElectedOrChanged(zkClient: KafkaZkClient, topic: String, partition: Int, timeoutMs: Long = 30000L,
oldLeaderOpt: Option[Int] = None, newLeaderOpt: Option[Int] = None): Int = {
require(!(oldLeaderOpt.isDefined && newLeaderOpt.isDefined), "Can't define both the old and the new leader")
val startTime = System.currentTimeMillis()
val topicPartition = new TopicPartition(topic, partition)
trace(s"Waiting for leader to be elected or changed for partition $topicPartition, old leader is $oldLeaderOpt, " +
s"new leader is $newLeaderOpt")
var leader: Option[Int] = None
var electedOrChangedLeader: Option[Int] = None
while (electedOrChangedLeader.isEmpty && System.currentTimeMillis() < startTime + timeoutMs) {
// check if leader is elected
leader = zkClient.getLeaderForPartition(topicPartition)
leader match {
case Some(l) => (newLeaderOpt, oldLeaderOpt) match {
case (Some(newLeader), _) if newLeader == l =>
trace(s"Expected new leader $l is elected for partition $topicPartition")
electedOrChangedLeader = leader
case (_, Some(oldLeader)) if oldLeader != l =>
trace(s"Leader for partition $topicPartition is changed from $oldLeader to $l")
electedOrChangedLeader = leader
case (None, None) =>
trace(s"Leader $l is elected for partition $topicPartition")
electedOrChangedLeader = leader
case _ =>
trace(s"Current leader for partition $topicPartition is $l")
}
case None =>
trace(s"Leader for partition $topicPartition is not elected yet")
}
Thread.sleep(math.min(timeoutMs, 100L))
}
electedOrChangedLeader.getOrElse {
val errorMessage = (newLeaderOpt, oldLeaderOpt) match {
case (Some(newLeader), _) =>
s"Timing out after $timeoutMs ms since expected new leader $newLeader was not elected for partition $topicPartition, leader is $leader"
case (_, Some(oldLeader)) =>
s"Timing out after $timeoutMs ms since a new leader that is different from $oldLeader was not elected for partition $topicPartition, " +
s"leader is $leader"
case _ =>
s"Timing out after $timeoutMs ms since a leader was not elected for partition $topicPartition"
}
fail(errorMessage)
}
}
/**
* Execute the given block. If it throws an assert error, retry. Repeat
* until no error is thrown or the time limit elapses
*/
def retry(maxWaitMs: Long)(block: => Unit) {
var wait = 1L
val startTime = System.currentTimeMillis()
while(true) {
try {
block
return
} catch {
case e: AssertionError =>
val elapsed = System.currentTimeMillis - startTime
if (elapsed > maxWaitMs) {
throw e
} else {
info("Attempt failed, sleeping for " + wait + ", and then retrying.")
Thread.sleep(wait)
wait += math.min(wait, 1000)
}
}
}
}
def pollUntilTrue(consumer: Consumer[_, _],
action: () => Boolean,
msg: => String,
waitTimeMs: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): Unit = {
waitUntilTrue(() => {
consumer.poll(Duration.ofMillis(50))
action()
}, msg = msg, pause = 0L, waitTimeMs = waitTimeMs)
}
def pollRecordsUntilTrue[K, V](consumer: Consumer[K, V],
action: ConsumerRecords[K, V] => Boolean,
msg: => String,
waitTimeMs: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): Unit = {
waitUntilTrue(() => {
val records = consumer.poll(Duration.ofMillis(50))
action(records)
}, msg = msg, pause = 0L, waitTimeMs = waitTimeMs)
}
/**
* Wait for the presence of an optional value.
*
* @param func The function defining the optional value
* @param msg Error message in the case that the value never appears
* @param waitTimeMs Maximum time to wait
* @return The unwrapped value returned by the function
*/
def awaitValue[T](func: () => Option[T], msg: => String, waitTimeMs: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): T = {
var value: Option[T] = None
waitUntilTrue(() => {
value = func()
value.isDefined
}, msg, waitTimeMs)
value.get
}
/**
* Wait until the given condition is true or throw an exception if the given wait time elapses.
*
* @param condition condition to check
* @param msg error message
* @param waitTimeMs maximum time to wait and retest the condition before failing the test
* @param pause delay between condition checks
* @param maxRetries maximum number of retries to check the given condition if a retriable exception is thrown
*/
def waitUntilTrue(condition: () => Boolean, msg: => String,
waitTimeMs: Long = JTestUtils.DEFAULT_MAX_WAIT_MS, pause: Long = 100L, maxRetries: Int = 0): Unit = {
val startTime = System.currentTimeMillis()
var retry = 0
while (true) {
try {
if (condition())
return
if (System.currentTimeMillis() > startTime + waitTimeMs)
fail(msg)
Thread.sleep(waitTimeMs.min(pause))
}
catch {
case e: RetriableException if retry < maxRetries =>
debug("Retrying after error", e)
retry += 1
case e : Throwable => throw e
}
}
// should never hit here
throw new RuntimeException("unexpected error")
}
/**
* Invoke `compute` until `predicate` is true or `waitTime` elapses.
*
* Return the last `compute` result and a boolean indicating whether `predicate` succeeded for that value.
*
* This method is useful in cases where `waitUntilTrue` makes it awkward to provide good error messages.
*/
def computeUntilTrue[T](compute: => T, waitTime: Long = JTestUtils.DEFAULT_MAX_WAIT_MS, pause: Long = 100L)(
predicate: T => Boolean): (T, Boolean) = {
val startTime = System.currentTimeMillis()
while (true) {
val result = compute
if (predicate(result))
return result -> true
if (System.currentTimeMillis() > startTime + waitTime)
return result -> false
Thread.sleep(waitTime.min(pause))
}
// should never hit here
throw new RuntimeException("unexpected error")
}
def isLeaderLocalOnBroker(topic: String, partitionId: Int, server: KafkaServer): Boolean = {
server.replicaManager.nonOfflinePartition(new TopicPartition(topic, partitionId)).exists(_.leaderLogIfLocal.isDefined)
}
def findLeaderEpoch(brokerId: Int,
topicPartition: TopicPartition,
servers: Iterable[KafkaServer]): Int = {
val leaderServer = servers.find(_.config.brokerId == brokerId)
val leaderPartition = leaderServer.flatMap(_.replicaManager.nonOfflinePartition(topicPartition))
.getOrElse(fail(s"Failed to find expected replica on broker $brokerId"))
leaderPartition.getLeaderEpoch
}
def findFollowerId(topicPartition: TopicPartition,
servers: Iterable[KafkaServer]): Int = {
val followerOpt = servers.find { server =>
server.replicaManager.nonOfflinePartition(topicPartition) match {
case Some(partition) => !partition.leaderReplicaIdOpt.contains(server.config.brokerId)
case None => false
}
}
followerOpt
.map(_.config.brokerId)
.getOrElse(fail(s"Unable to locate follower for $topicPartition"))
}
/**
* Wait until all brokers know about each other.
*
* @param servers The Kafka broker servers.
* @param timeout The amount of time waiting on this condition before assert to fail
*/
def waitUntilBrokerMetadataIsPropagated(servers: Seq[KafkaServer],
timeout: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): Unit = {
val expectedBrokerIds = servers.map(_.config.brokerId).toSet
waitUntilTrue(() => servers.forall(server =>
expectedBrokerIds == server.dataPlaneRequestProcessor.metadataCache.getAliveBrokers.map(_.id).toSet
), "Timed out waiting for broker metadata to propagate to all servers", timeout)
}
/**
* Wait until a valid leader is propagated to the metadata cache in each broker.
* It assumes that the leader propagated to each broker is the same.
*
* @param servers The list of servers that the metadata should reach to
* @param topic The topic name
* @param partition The partition Id
* @param timeout The amount of time waiting on this condition before assert to fail
* @return The leader of the partition.
*/
def waitUntilMetadataIsPropagated(servers: Seq[KafkaServer], topic: String, partition: Int,
timeout: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): Int = {
var leader: Int = -1
waitUntilTrue(() =>
servers.foldLeft(true) {
(result, server) =>
val partitionStateOpt = server.dataPlaneRequestProcessor.metadataCache.getPartitionInfo(topic, partition)
partitionStateOpt match {
case None => false
case Some(partitionState) =>
leader = partitionState.basePartitionState.leader
result && Request.isValidBrokerId(leader)
}
},
"Partition [%s,%d] metadata not propagated after %d ms".format(topic, partition, timeout),
waitTimeMs = timeout)
leader
}
def waitUntilControllerElected(zkClient: KafkaZkClient, timeout: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): Int = {
val (controllerId, _) = TestUtils.computeUntilTrue(zkClient.getControllerId, waitTime = timeout)(_.isDefined)
controllerId.getOrElse(fail(s"Controller not elected after $timeout ms"))
}
def awaitLeaderChange(servers: Seq[KafkaServer],
tp: TopicPartition,
oldLeader: Int,
timeout: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): Int = {
def newLeaderExists: Option[Int] = {
servers.find { server =>
server.config.brokerId != oldLeader &&
server.replicaManager.nonOfflinePartition(tp).exists(_.leaderLogIfLocal.isDefined)
}.map(_.config.brokerId)
}
waitUntilTrue(() => newLeaderExists.isDefined,
s"Did not observe leader change for partition $tp after $timeout ms", waitTimeMs = timeout)
newLeaderExists.get
}
def waitUntilLeaderIsKnown(servers: Seq[KafkaServer],
tp: TopicPartition,
timeout: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): Int = {
def leaderIfExists: Option[Int] = {
servers.find { server =>
server.replicaManager.nonOfflinePartition(tp).exists(_.leaderLogIfLocal.isDefined)
}.map(_.config.brokerId)
}
waitUntilTrue(() => leaderIfExists.isDefined,
s"Partition $tp leaders not made yet after $timeout ms", waitTimeMs = timeout)
leaderIfExists.get
}
def writeNonsenseToFile(fileName: File, position: Long, size: Int) {
val file = new RandomAccessFile(fileName, "rw")
file.seek(position)
for (_ <- 0 until size)
file.writeByte(random.nextInt(255))
file.close()
}
def appendNonsenseToFile(file: File, size: Int) {
val outputStream = Files.newOutputStream(file.toPath(), StandardOpenOption.APPEND)
try {
for (_ <- 0 until size)
outputStream.write(random.nextInt(255))
} finally outputStream.close()
}
def checkForPhantomInSyncReplicas(zkClient: KafkaZkClient, topic: String, partitionToBeReassigned: Int, assignedReplicas: Seq[Int]) {
val inSyncReplicas = zkClient.getInSyncReplicasForPartition(new TopicPartition(topic, partitionToBeReassigned))
// in sync replicas should not have any replica that is not in the new assigned replicas
val phantomInSyncReplicas = inSyncReplicas.get.toSet -- assignedReplicas.toSet
assertTrue("All in sync replicas %s must be in the assigned replica list %s".format(inSyncReplicas, assignedReplicas),
phantomInSyncReplicas.isEmpty)
}
def ensureNoUnderReplicatedPartitions(zkClient: KafkaZkClient, topic: String, partitionToBeReassigned: Int, assignedReplicas: Seq[Int],
servers: Seq[KafkaServer]) {
val topicPartition = new TopicPartition(topic, partitionToBeReassigned)
waitUntilTrue(() => {
val inSyncReplicas = zkClient.getInSyncReplicasForPartition(topicPartition)
inSyncReplicas.get.size == assignedReplicas.size
},
"Reassigned partition [%s,%d] is under replicated".format(topic, partitionToBeReassigned))
var leader: Option[Int] = None
waitUntilTrue(() => {
leader = zkClient.getLeaderForPartition(topicPartition)
leader.isDefined