55package com .linkedin .kafka .cruisecontrol .metricsreporter ;
66
77import com .linkedin .kafka .cruisecontrol .metricsreporter .exception .KafkaTopicDescriptionException ;
8+ import com .linkedin .kafka .cruisecontrol .metricsreporter .utils .CCContainerizedKraftCluster ;
89import com .linkedin .kafka .cruisecontrol .metricsreporter .utils .CCKafkaClientsIntegrationTestHarness ;
9- import com .linkedin .kafka .cruisecontrol .metricsreporter .utils .CCKafkaTestUtils ;
10+ import com .linkedin .kafka .cruisecontrol .metricsreporter .utils .KafkaServerConfigs ;
1011import org .apache .kafka .clients .CommonClientConfigs ;
1112import org .apache .kafka .clients .admin .AdminClient ;
1213import org .apache .kafka .clients .admin .CreateTopicsResult ;
1617import org .apache .kafka .clients .producer .Producer ;
1718import org .apache .kafka .clients .producer .ProducerConfig ;
1819import org .apache .kafka .clients .producer .ProducerRecord ;
19- import org .apache .kafka .coordinator .group .GroupCoordinatorConfig ;
20- import org .apache .kafka .network .SocketServerConfigs ;
21- import org .apache .kafka .server .config .ReplicationConfigs ;
22- import org .apache .kafka .server .config .ServerLogConfigs ;
2320import org .junit .After ;
2421import org .junit .Before ;
2522import org .junit .Test ;
3330public class CruiseControlMetricsReporterAutoCreateTopicTest extends CCKafkaClientsIntegrationTestHarness {
3431 protected static final String TOPIC = "CruiseControlMetricsReporterTest" ;
3532 protected static final String TEST_TOPIC = "TestTopic" ;
33+ private CCContainerizedKraftCluster _cluster ;
3634
3735 /**
3836 * Setup the unit test.
3937 */
4038 @ Before
4139 public void setUp () {
42- super .setUp ();
40+ Properties adminClientProps = new Properties ();
41+ setSecurityConfigs (adminClientProps , "admin" );
42+
43+ _cluster = new CCContainerizedKraftCluster (2 , buildBrokerConfigs (), adminClientProps );
44+ _cluster .start ();
45+ _bootstrapUrl = _cluster .getExternalBootstrapAddress ();
4346
4447 // creating the "TestTopic" explicitly because the topic auto-creation is disabled on the broker
4548 Properties adminProps = new Properties ();
@@ -73,19 +76,26 @@ public void setUp() {
7376 assertEquals (0 , producerFailed .get ());
7477 }
7578
79+ /**
80+ * Tear down the unit test.
81+ */
7682 @ After
7783 public void tearDown () {
78- super .tearDown ();
84+ if (_cluster != null ) {
85+ _cluster .close ();
86+ }
7987 }
8088
8189 @ Override
8290 public Properties overridingProps () {
8391 Properties props = new Properties ();
84- int port = CCKafkaTestUtils .findLocalPort ();
8592 props .setProperty (CommonClientConfigs .METRIC_REPORTER_CLASSES_CONFIG , CruiseControlMetricsReporter .class .getName ());
86- props .setProperty (SocketServerConfigs .LISTENERS_CONFIG , "PLAINTEXT://127.0.0.1:" + port );
8793 props .setProperty (CruiseControlMetricsReporterConfig .config (CommonClientConfigs .BOOTSTRAP_SERVERS_CONFIG ),
88- "127.0.0.1:" + port );
94+ "127.0.0.1:" + CCContainerizedKraftCluster .CONTAINER_INTERNAL_LISTENER_PORT );
95+ props .put ("listener.security.protocol.map" , String .join ("," ,
96+ CCContainerizedKraftCluster .CONTROLLER_LISTENER_NAME + ":PLAINTEXT" ,
97+ CCContainerizedKraftCluster .INTERNAL_LISTENER_NAME + ":PLAINTEXT" ,
98+ CCContainerizedKraftCluster .EXTERNAL_LISTENER_NAME + ":PLAINTEXT" ));
8999 props .setProperty (CruiseControlMetricsReporterConfig .CRUISE_CONTROL_METRICS_REPORTER_INTERVAL_MS_CONFIG , "100" );
90100 props .setProperty (CruiseControlMetricsReporterConfig .CRUISE_CONTROL_METRICS_TOPIC_CONFIG , TOPIC );
91101 // configure metrics topic auto-creation by the metrics reporter
@@ -95,11 +105,10 @@ public Properties overridingProps() {
95105 props .setProperty (CruiseControlMetricsReporterConfig .CRUISE_CONTROL_METRICS_TOPIC_NUM_PARTITIONS_CONFIG , "1" );
96106 props .setProperty (CruiseControlMetricsReporterConfig .CRUISE_CONTROL_METRICS_TOPIC_REPLICATION_FACTOR_CONFIG , "1" );
97107 // disable topic auto-creation to leave the metrics reporter to create the metrics topic
98- props .setProperty (ServerLogConfigs .AUTO_CREATE_TOPICS_ENABLE_CONFIG , "false" );
99- props .setProperty (ServerLogConfigs .LOG_FLUSH_INTERVAL_MESSAGES_CONFIG , "1" );
100- props .setProperty (GroupCoordinatorConfig .OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG , "1" );
101- props .setProperty (ReplicationConfigs .DEFAULT_REPLICATION_FACTOR_CONFIG , "2" );
102- props .setProperty (ServerLogConfigs .NUM_PARTITIONS_CONFIG , "2" );
108+ props .setProperty (KafkaServerConfigs .AUTO_CREATE_TOPICS_ENABLE_CONFIG , "false" );
109+ props .setProperty (KafkaServerConfigs .OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG , "1" );
110+ props .setProperty (KafkaServerConfigs .DEFAULT_REPLICATION_FACTOR_CONFIG , "2" );
111+ props .setProperty (KafkaServerConfigs .NUM_PARTITIONS_CONFIG , "2" );
103112 return props ;
104113 }
105114
0 commit comments