8
8
9
9
package org .opensearch .plugin .kafka ;
10
10
11
- import com .carrotsearch .randomizedtesting .annotations .ThreadLeakFilters ;
12
-
13
- import org .apache .kafka .clients .producer .KafkaProducer ;
14
- import org .apache .kafka .clients .producer .Producer ;
15
- import org .apache .kafka .clients .producer .ProducerRecord ;
16
- import org .apache .kafka .common .serialization .StringSerializer ;
17
11
import org .opensearch .action .admin .cluster .node .info .NodeInfo ;
18
12
import org .opensearch .action .admin .cluster .node .info .NodesInfoRequest ;
19
13
import org .opensearch .action .admin .cluster .node .info .NodesInfoResponse ;
22
16
import org .opensearch .cluster .metadata .IndexMetadata ;
23
17
import org .opensearch .common .settings .Settings ;
24
18
import org .opensearch .index .query .RangeQueryBuilder ;
25
- import org .opensearch .plugins .Plugin ;
26
19
import org .opensearch .plugins .PluginInfo ;
27
20
import org .opensearch .test .OpenSearchIntegTestCase ;
28
21
import org .junit .Assert ;
29
22
30
- import java .util .Arrays ;
31
- import java .util .Collection ;
32
23
import java .util .List ;
33
- import java .util .Properties ;
34
24
import java .util .concurrent .TimeUnit ;
35
25
import java .util .function .Function ;
36
26
import java .util .stream .Collectors ;
37
27
import java .util .stream .Stream ;
38
28
39
- import org .testcontainers .containers .KafkaContainer ;
40
- import org .testcontainers .utility .DockerImageName ;
41
-
42
29
import static org .hamcrest .Matchers .is ;
43
30
import static org .awaitility .Awaitility .await ;
44
31
45
32
/**
46
33
* Integration test for Kafka ingestion
47
34
*/
48
- @ ThreadLeakFilters (filters = TestContainerWatchdogThreadLeakFilter .class )
49
- public class IngestFromKafkaIT extends OpenSearchIntegTestCase {
50
- static final String topicName = "test" ;
51
-
52
- private KafkaContainer kafka ;
53
-
54
- @ Override
55
- protected Collection <Class <? extends Plugin >> nodePlugins () {
56
- return Arrays .asList (KafkaPlugin .class );
57
- }
58
-
35
+ @ OpenSearchIntegTestCase .ClusterScope (scope = OpenSearchIntegTestCase .Scope .TEST , numDataNodes = 0 )
36
+ public class IngestFromKafkaIT extends KafkaIngestionBaseIT {
59
37
/**
60
38
* test ingestion-kafka-plugin is installed
61
39
*/
@@ -75,128 +53,86 @@ public void testPluginsAreInstalled() {
75
53
}
76
54
77
55
public void testKafkaIngestion () {
78
- try {
79
- setupKafka ();
80
- // create an index with ingestion source from kafka
81
- createIndex (
82
- "test" ,
83
- Settings .builder ()
84
- .put (IndexMetadata .SETTING_NUMBER_OF_SHARDS , 1 )
85
- .put (IndexMetadata .SETTING_NUMBER_OF_REPLICAS , 0 )
86
- .put ("ingestion_source.type" , "kafka" )
87
- .put ("ingestion_source.pointer.init.reset" , "earliest" )
88
- .put ("ingestion_source.param.topic" , "test" )
89
- .put ("ingestion_source.param.bootstrap_servers" , kafka .getBootstrapServers ())
90
- .build (),
91
- "{\" properties\" :{\" name\" :{\" type\" : \" text\" },\" age\" :{\" type\" : \" integer\" }}}}"
92
- );
56
+ produceData ("1" , "name1" , "24" );
57
+ produceData ("2" , "name2" , "20" );
58
+
59
+ createIndex (
60
+ "test" ,
61
+ Settings .builder ()
62
+ .put (IndexMetadata .SETTING_NUMBER_OF_SHARDS , 1 )
63
+ .put (IndexMetadata .SETTING_NUMBER_OF_REPLICAS , 0 )
64
+ .put ("ingestion_source.type" , "kafka" )
65
+ .put ("ingestion_source.pointer.init.reset" , "earliest" )
66
+ .put ("ingestion_source.param.topic" , "test" )
67
+ .put ("ingestion_source.param.bootstrap_servers" , kafka .getBootstrapServers ())
68
+ .put ("index.replication.type" , "SEGMENT" )
69
+ .build (),
70
+ "{\" properties\" :{\" name\" :{\" type\" : \" text\" },\" age\" :{\" type\" : \" integer\" }}}}"
71
+ );
93
72
94
- RangeQueryBuilder query = new RangeQueryBuilder ("age" ).gte (21 );
95
- await ().atMost (10 , TimeUnit .SECONDS ).untilAsserted (() -> {
96
- refresh ("test" );
97
- SearchResponse response = client ().prepareSearch ("test" ).setQuery (query ).get ();
98
- assertThat (response .getHits ().getTotalHits ().value (), is (1L ));
99
- });
100
- } finally {
101
- stopKafka ();
102
- }
73
+ RangeQueryBuilder query = new RangeQueryBuilder ("age" ).gte (21 );
74
+ await ().atMost (10 , TimeUnit .SECONDS ).untilAsserted (() -> {
75
+ refresh ("test" );
76
+ SearchResponse response = client ().prepareSearch ("test" ).setQuery (query ).get ();
77
+ assertThat (response .getHits ().getTotalHits ().value (), is (1L ));
78
+ });
103
79
}
104
80
105
81
public void testKafkaIngestion_RewindByTimeStamp () {
106
- try {
107
- setupKafka ();
108
- // create an index with ingestion source from kafka
109
- createIndex (
110
- "test_rewind_by_timestamp" ,
111
- Settings .builder ()
112
- .put (IndexMetadata .SETTING_NUMBER_OF_SHARDS , 1 )
113
- .put (IndexMetadata .SETTING_NUMBER_OF_REPLICAS , 0 )
114
- .put ("ingestion_source.type" , "kafka" )
115
- .put ("ingestion_source.pointer.init.reset" , "rewind_by_timestamp" )
116
- // 1739459500000 is the timestamp of the first message
117
- // 1739459800000 is the timestamp of the second message
118
- // by resetting to 1739459600000, only the second message will be ingested
119
- .put ("ingestion_source.pointer.init.reset.value" , "1739459600000" )
120
- .put ("ingestion_source.param.topic" , "test" )
121
- .put ("ingestion_source.param.bootstrap_servers" , kafka .getBootstrapServers ())
122
- .put ("ingestion_source.param.auto.offset.reset" , "latest" )
123
- .build (),
124
- "{\" properties\" :{\" name\" :{\" type\" : \" text\" },\" age\" :{\" type\" : \" integer\" }}}}"
125
- );
82
+ produceData ("1" , "name1" , "24" , 1739459500000L );
83
+ produceData ("2" , "name2" , "20" , 1739459800000L );
84
+
85
+ // create an index with ingestion source from kafka
86
+ createIndex (
87
+ "test_rewind_by_timestamp" ,
88
+ Settings .builder ()
89
+ .put (IndexMetadata .SETTING_NUMBER_OF_SHARDS , 1 )
90
+ .put (IndexMetadata .SETTING_NUMBER_OF_REPLICAS , 0 )
91
+ .put ("ingestion_source.type" , "kafka" )
92
+ .put ("ingestion_source.pointer.init.reset" , "rewind_by_timestamp" )
93
+ // 1739459500000 is the timestamp of the first message
94
+ // 1739459800000 is the timestamp of the second message
95
+ // by resetting to 1739459600000, only the second message will be ingested
96
+ .put ("ingestion_source.pointer.init.reset.value" , "1739459600000" )
97
+ .put ("ingestion_source.param.topic" , "test" )
98
+ .put ("ingestion_source.param.bootstrap_servers" , kafka .getBootstrapServers ())
99
+ .put ("ingestion_source.param.auto.offset.reset" , "latest" )
100
+ .build (),
101
+ "{\" properties\" :{\" name\" :{\" type\" : \" text\" },\" age\" :{\" type\" : \" integer\" }}}}"
102
+ );
126
103
127
- RangeQueryBuilder query = new RangeQueryBuilder ("age" ).gte (0 );
128
- await ().atMost (10 , TimeUnit .SECONDS ).untilAsserted (() -> {
129
- refresh ("test_rewind_by_timestamp" );
130
- SearchResponse response = client ().prepareSearch ("test_rewind_by_timestamp" ).setQuery (query ).get ();
131
- assertThat (response .getHits ().getTotalHits ().value (), is (1L ));
132
- });
133
- } finally {
134
- stopKafka ();
135
- }
104
+ RangeQueryBuilder query = new RangeQueryBuilder ("age" ).gte (0 );
105
+ await ().atMost (10 , TimeUnit .SECONDS ).untilAsserted (() -> {
106
+ refresh ("test_rewind_by_timestamp" );
107
+ SearchResponse response = client ().prepareSearch ("test_rewind_by_timestamp" ).setQuery (query ).get ();
108
+ assertThat (response .getHits ().getTotalHits ().value (), is (1L ));
109
+ });
136
110
}
137
111
138
112
public void testKafkaIngestion_RewindByOffset () {
139
- try {
140
- setupKafka ();
141
- // create an index with ingestion source from kafka
142
- createIndex (
143
- "test_rewind_by_offset" ,
144
- Settings .builder ()
145
- .put (IndexMetadata .SETTING_NUMBER_OF_SHARDS , 1 )
146
- .put (IndexMetadata .SETTING_NUMBER_OF_REPLICAS , 0 )
147
- .put ("ingestion_source.type" , "kafka" )
148
- .put ("ingestion_source.pointer.init.reset" , "rewind_by_offset" )
149
- .put ("ingestion_source.pointer.init.reset.value" , "1" )
150
- .put ("ingestion_source.param.topic" , "test" )
151
- .put ("ingestion_source.param.bootstrap_servers" , kafka .getBootstrapServers ())
152
- .put ("ingestion_source.param.auto.offset.reset" , "latest" )
153
- .build (),
154
- "{\" properties\" :{\" name\" :{\" type\" : \" text\" },\" age\" :{\" type\" : \" integer\" }}}}"
155
- );
156
-
157
- RangeQueryBuilder query = new RangeQueryBuilder ("age" ).gte (0 );
158
- await ().atMost (10 , TimeUnit .SECONDS ).untilAsserted (() -> {
159
- refresh ("test_rewind_by_offset" );
160
- SearchResponse response = client ().prepareSearch ("test_rewind_by_offset" ).setQuery (query ).get ();
161
- assertThat (response .getHits ().getTotalHits ().value (), is (1L ));
162
- });
163
- } finally {
164
- stopKafka ();
165
- }
166
- }
167
-
168
- private void setupKafka () {
169
- kafka = new KafkaContainer (DockerImageName .parse ("confluentinc/cp-kafka:6.2.1" ))
170
- // disable topic auto creation
171
- .withEnv ("KAFKA_AUTO_CREATE_TOPICS_ENABLE" , "false" );
172
- kafka .start ();
173
- prepareKafkaData ();
174
- }
175
-
176
- private void stopKafka () {
177
- if (kafka != null ) {
178
- kafka .stop ();
179
- }
180
- }
181
-
182
- private void prepareKafkaData () {
183
- String boostrapServers = kafka .getBootstrapServers ();
184
- KafkaUtils .createTopic (topicName , 1 , boostrapServers );
185
- Properties props = new Properties ();
186
- props .put ("bootstrap.servers" , kafka .getBootstrapServers ());
187
- Producer <String , String > producer = new KafkaProducer <>(props , new StringSerializer (), new StringSerializer ());
188
- producer .send (
189
- new ProducerRecord <>(topicName , null , 1739459500000L , "null" , "{\" _id\" :\" 1\" ,\" _source\" :{\" name\" :\" bob\" , \" age\" : 24}}" )
190
- );
191
- producer .send (
192
- new ProducerRecord <>(
193
- topicName ,
194
- null ,
195
- 1739459800000L ,
196
- "null" ,
197
- "{\" _id\" :\" 2\" , \" _op_type:\" :\" index\" ,\" _source\" :{\" name\" :\" alice\" , \" age\" : 20}}"
198
- )
113
+ produceData ("1" , "name1" , "24" );
114
+ produceData ("2" , "name2" , "20" );
115
+ // create an index with ingestion source from kafka
116
+ createIndex (
117
+ "test_rewind_by_offset" ,
118
+ Settings .builder ()
119
+ .put (IndexMetadata .SETTING_NUMBER_OF_SHARDS , 1 )
120
+ .put (IndexMetadata .SETTING_NUMBER_OF_REPLICAS , 0 )
121
+ .put ("ingestion_source.type" , "kafka" )
122
+ .put ("ingestion_source.pointer.init.reset" , "rewind_by_offset" )
123
+ .put ("ingestion_source.pointer.init.reset.value" , "1" )
124
+ .put ("ingestion_source.param.topic" , "test" )
125
+ .put ("ingestion_source.param.bootstrap_servers" , kafka .getBootstrapServers ())
126
+ .put ("ingestion_source.param.auto.offset.reset" , "latest" )
127
+ .build (),
128
+ "{\" properties\" :{\" name\" :{\" type\" : \" text\" },\" age\" :{\" type\" : \" integer\" }}}}"
199
129
);
200
- producer .close ();
130
+
131
+ RangeQueryBuilder query = new RangeQueryBuilder ("age" ).gte (0 );
132
+ await ().atMost (10 , TimeUnit .SECONDS ).untilAsserted (() -> {
133
+ refresh ("test_rewind_by_offset" );
134
+ SearchResponse response = client ().prepareSearch ("test_rewind_by_offset" ).setQuery (query ).get ();
135
+ assertThat (response .getHits ().getTotalHits ().value (), is (1L ));
136
+ });
201
137
}
202
138
}
0 commit comments