|
52 | 52 | public final class QuestDBSinkConnectorEmbeddedTest {
|
53 | 53 | private static int httpPort = -1;
|
54 | 54 | private static int ilpPort = -1;
|
55 |
| - private static final String OFFICIAL_QUESTDB_DOCKER = "questdb/questdb:8.1.1"; |
| 55 | + private static final String OFFICIAL_QUESTDB_DOCKER = "questdb/questdb:8.2.0"; |
56 | 56 | private static final boolean DUMP_QUESTDB_CONTAINER_LOGS = true;
|
57 | 57 |
|
58 | 58 | private EmbeddedConnectCluster connect;
|
@@ -248,6 +248,127 @@ public void testDeadLetterQueue_wrongJson(boolean useHttp) {
|
248 | 248 | Assertions.assertEquals("{\"not valid json}", new String(dqlRecord.value()));
|
249 | 249 | }
|
250 | 250 |
|
| 251 | + @Test |
| 252 | + public void testDeadLetterQueue_invalidTableName() { |
| 253 | + connect.kafka().createTopic(topicName, 1); |
| 254 | + Map<String, String> props = ConnectTestUtils.baseConnectorProps(questDBContainer, topicName, true); |
| 255 | + props.put("errors.deadletterqueue.topic.name", "dlq"); |
| 256 | + props.put("errors.deadletterqueue.topic.replication.factor", "1"); |
| 257 | + props.put("errors.tolerance", "all"); |
| 258 | + props.put("value.converter.schemas.enable", "false"); |
| 259 | + props.put(QuestDBSinkConnectorConfig.TABLE_CONFIG, "${key}"); |
| 260 | + connect.configureConnector(ConnectTestUtils.CONNECTOR_NAME, props); |
| 261 | + ConnectTestUtils.assertConnectorTaskRunningEventually(connect); |
| 262 | + |
| 263 | + // we send this with an invalid key - contains dots |
| 264 | + String badObjectString = "{\"firstname\":\"John\",\"lastname\":\"Doe\",\"age\":88}"; |
| 265 | + |
| 266 | + connect.kafka().produce(topicName, topicName, "{\"firstname\":\"John\",\"lastname\":\"Doe\",\"age\":42}"); |
| 267 | + connect.kafka().produce(topicName, "k,e,y", badObjectString); |
| 268 | + connect.kafka().produce(topicName, topicName, "{\"firstname\":\"Jane\",\"lastname\":\"Doe\",\"age\":41}"); |
| 269 | + |
| 270 | + QuestDBUtils.assertSqlEventually( "\"firstname\",\"lastname\",\"age\"\r\n" |
| 271 | + + "\"John\",\"Doe\",42\r\n" |
| 272 | + + "\"Jane\",\"Doe\",41\r\n", |
| 273 | + "select firstname,lastname,age from " + topicName, |
| 274 | + httpPort); |
| 275 | + |
| 276 | + ConsumerRecords<byte[], byte[]> fetchedRecords = connect.kafka().consume(1, 120_000, "dlq"); |
| 277 | + Assertions.assertEquals(1, fetchedRecords.count()); |
| 278 | + Iterator<ConsumerRecord<byte[], byte[]>> iterator = fetchedRecords.iterator(); |
| 279 | + Assertions.assertEquals(badObjectString, new String(iterator.next().value())); |
| 280 | + } |
| 281 | + |
| 282 | + @Test |
| 283 | + public void testDeadLetterQueue_invalidColumnName() { |
| 284 | + connect.kafka().createTopic(topicName, 1); |
| 285 | + Map<String, String> props = ConnectTestUtils.baseConnectorProps(questDBContainer, topicName, true); |
| 286 | + props.put("errors.deadletterqueue.topic.name", "dlq"); |
| 287 | + props.put("errors.deadletterqueue.topic.replication.factor", "1"); |
| 288 | + props.put("errors.tolerance", "all"); |
| 289 | + props.put("value.converter.schemas.enable", "false"); |
| 290 | + connect.configureConnector(ConnectTestUtils.CONNECTOR_NAME, props); |
| 291 | + ConnectTestUtils.assertConnectorTaskRunningEventually(connect); |
| 292 | + |
| 293 | + // invalid column - contains a star |
| 294 | + String badObjectString = "{\"firstname\":\"John\",\"lastname\":\"Doe\",\"a*g*e\":88}"; |
| 295 | + |
| 296 | + connect.kafka().produce(topicName, "key", "{\"firstname\":\"John\",\"lastname\":\"Doe\",\"age\":42}"); |
| 297 | + connect.kafka().produce(topicName, "key", badObjectString); |
| 298 | + connect.kafka().produce(topicName, "key", "{\"firstname\":\"Jane\",\"lastname\":\"Doe\",\"age\":41}"); |
| 299 | + |
| 300 | + QuestDBUtils.assertSqlEventually( "\"firstname\",\"lastname\",\"age\"\r\n" |
| 301 | + + "\"John\",\"Doe\",42\r\n" |
| 302 | + + "\"Jane\",\"Doe\",41\r\n", |
| 303 | + "select firstname,lastname,age from " + topicName, |
| 304 | + httpPort); |
| 305 | + |
| 306 | + ConsumerRecords<byte[], byte[]> fetchedRecords = connect.kafka().consume(1, 120_000, "dlq"); |
| 307 | + Assertions.assertEquals(1, fetchedRecords.count()); |
| 308 | + Iterator<ConsumerRecord<byte[], byte[]>> iterator = fetchedRecords.iterator(); |
| 309 | + Assertions.assertEquals(badObjectString, new String(iterator.next().value())); |
| 310 | + } |
| 311 | + |
| 312 | + @Test |
| 313 | + public void testDeadLetterQueue_unsupportedType() { |
| 314 | + connect.kafka().createTopic(topicName, 1); |
| 315 | + Map<String, String> props = ConnectTestUtils.baseConnectorProps(questDBContainer, topicName, true); |
| 316 | + props.put("errors.deadletterqueue.topic.name", "dlq"); |
| 317 | + props.put("errors.deadletterqueue.topic.replication.factor", "1"); |
| 318 | + props.put("errors.tolerance", "all"); |
| 319 | + props.put("value.converter.schemas.enable", "false"); |
| 320 | + connect.configureConnector(ConnectTestUtils.CONNECTOR_NAME, props); |
| 321 | + ConnectTestUtils.assertConnectorTaskRunningEventually(connect); |
| 322 | + |
| 323 | + // contains array - not supported |
| 324 | + String badObjectString = "{\"firstname\":\"John\",\"lastname\":\"Doe\",\"age\":[1, 2, 3]}"; |
| 325 | + |
| 326 | + connect.kafka().produce(topicName, "key", "{\"firstname\":\"John\",\"lastname\":\"Doe\",\"age\":42}"); |
| 327 | + connect.kafka().produce(topicName, "key", badObjectString); |
| 328 | + connect.kafka().produce(topicName, "key", "{\"firstname\":\"Jane\",\"lastname\":\"Doe\",\"age\":41}"); |
| 329 | + |
| 330 | + QuestDBUtils.assertSqlEventually( "\"firstname\",\"lastname\",\"age\"\r\n" |
| 331 | + + "\"John\",\"Doe\",42\r\n" |
| 332 | + + "\"Jane\",\"Doe\",41\r\n", |
| 333 | + "select firstname,lastname,age from " + topicName, |
| 334 | + httpPort); |
| 335 | + |
| 336 | + ConsumerRecords<byte[], byte[]> fetchedRecords = connect.kafka().consume(1, 120_000, "dlq"); |
| 337 | + Assertions.assertEquals(1, fetchedRecords.count()); |
| 338 | + Iterator<ConsumerRecord<byte[], byte[]>> iterator = fetchedRecords.iterator(); |
| 339 | + Assertions.assertEquals(badObjectString, new String(iterator.next().value())); |
| 340 | + } |
| 341 | + |
| 342 | + @Test |
| 343 | + public void testDeadLetterQueue_emptyTable() { |
| 344 | + connect.kafka().createTopic(topicName, 1); |
| 345 | + Map<String, String> props = ConnectTestUtils.baseConnectorProps(questDBContainer, topicName, true); |
| 346 | + props.put(QuestDBSinkConnectorConfig.TABLE_CONFIG, "${key}"); |
| 347 | + props.put(QuestDBSinkConnectorConfig.INCLUDE_KEY_CONFIG, "false"); |
| 348 | + props.put("value.converter.schemas.enable", "false"); |
| 349 | + props.put("errors.deadletterqueue.topic.name", "dlq"); |
| 350 | + props.put("errors.deadletterqueue.topic.replication.factor", "1"); |
| 351 | + props.put("errors.tolerance", "all"); |
| 352 | + connect.configureConnector(ConnectTestUtils.CONNECTOR_NAME, props); |
| 353 | + ConnectTestUtils.assertConnectorTaskRunningEventually(connect); |
| 354 | + |
| 355 | + connect.kafka().produce(topicName, "tab", "{\"firstname\":\"John\",\"lastname\":\"Doe\",\"age\":42}"); |
| 356 | + String emptyRecordValue = "{\"firstname\":\"empty\",\"lastname\":\"\",\"age\":-41}"; |
| 357 | + connect.kafka().produce(topicName, "", emptyRecordValue); |
| 358 | + connect.kafka().produce(topicName, "tab", "{\"firstname\":\"Jane\",\"lastname\":\"Doe\",\"age\":41}"); |
| 359 | + |
| 360 | + QuestDBUtils.assertSqlEventually( "\"firstname\",\"lastname\",\"age\"\r\n" |
| 361 | + + "\"John\",\"Doe\",42\r\n" |
| 362 | + + "\"Jane\",\"Doe\",41\r\n", |
| 363 | + "select firstname,lastname,age from tab", |
| 364 | + httpPort); |
| 365 | + |
| 366 | + ConsumerRecords<byte[], byte[]> fetchedRecords = connect.kafka().consume(1, 120_000, "dlq"); |
| 367 | + Assertions.assertEquals(1, fetchedRecords.count()); |
| 368 | + Iterator<ConsumerRecord<byte[], byte[]>> iterator = fetchedRecords.iterator(); |
| 369 | + Assertions.assertEquals(emptyRecordValue, new String(iterator.next().value())); |
| 370 | + } |
| 371 | + |
251 | 372 | @Test
|
252 | 373 | public void testDeadLetterQueue_badColumnType() {
|
253 | 374 | connect.kafka().createTopic(topicName, 1);
|
|
0 commit comments