diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 0e0a1d9..be45c34 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -18,4 +18,9 @@ jobs: distribution: 'temurin' cache: maven - name: Build with Maven - run: mvn -B package --file pom.xml \ No newline at end of file + run: mvn -B package --file pom.xml + - name: Archive connector artifact + uses: actions/upload-artifact@v4 + with: + name: connector-snapshot + path: connector/target/kafka-questdb-connector-*-bin.zip diff --git a/connector/src/main/java/io/questdb/kafka/QuestDBSinkTask.java b/connector/src/main/java/io/questdb/kafka/QuestDBSinkTask.java index cdd3ea8..4a803fd 100644 --- a/connector/src/main/java/io/questdb/kafka/QuestDBSinkTask.java +++ b/connector/src/main/java/io/questdb/kafka/QuestDBSinkTask.java @@ -4,6 +4,7 @@ import io.questdb.cutlass.http.client.HttpClientException; import io.questdb.cutlass.line.LineSenderException; import io.questdb.std.NumericException; +import io.questdb.std.ObjList; import io.questdb.std.datetime.DateFormat; import io.questdb.std.datetime.microtime.Timestamps; import io.questdb.std.datetime.millitime.DateFormatUtils; @@ -15,6 +16,7 @@ import org.apache.kafka.connect.data.*; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.RetriableException; +import org.apache.kafka.connect.sink.ErrantRecordReporter; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTask; import org.slf4j.Logger; @@ -48,6 +50,8 @@ public final class QuestDBSinkTask extends SinkTask { private long nextFlushNanos; private int pendingRows; private final FlushConfig flushConfig = new FlushConfig(); + private final ObjList inflightSinkRecords = new ObjList<>(); + private ErrantRecordReporter reporter; @Override public String version() { @@ -86,6 +90,12 @@ public void start(Map map) { this.allowedLag = config.getAllowedLag(); this.nextFlushNanos = System.nanoTime() + flushConfig.autoFlushNanos; this.recordToTable = Templating.newTableTableFn(config.getTable()); + try { + reporter = context.errantRecordReporter(); + } catch (NoSuchMethodError | NoClassDefFoundError e) { + // Kafka older than 2.6 + reporter = null; + } } private Sender createRawSender() { @@ -159,6 +169,9 @@ public void put(Collection collection) { sender = createSender(); } for (SinkRecord record : collection) { + if (httpTransport) { + inflightSinkRecords.add(record); + } handleSingleRecord(record); } @@ -208,22 +221,27 @@ public void put(Collection collection) { private void flushAndResetCounters() { log.debug("Flushing data to QuestDB"); try { - sender.flush(); + if (sender != null) { + sender.flush(); + } nextFlushNanos = System.nanoTime() + flushConfig.autoFlushNanos; pendingRows = 0; } catch (LineSenderException | HttpClientException e) { onSenderException(e); + } finally { + inflightSinkRecords.clear(); } } private void onSenderException(Exception e) { if (httpTransport) { - closeSenderSilently(); - nextFlushNanos = System.nanoTime() + flushConfig.autoFlushNanos; - pendingRows = 0; - throw new ConnectException("Failed to send data to QuestDB", e); + onHttpSenderException(e); + } else { + onTcpSenderException(e); } + } + private void onTcpSenderException(Exception e) { batchesSinceLastError = 0; if (--remainingRetries > 0) { closeSenderSilently(); @@ -235,6 +253,36 @@ private void onSenderException(Exception e) { } } + private void onHttpSenderException(Exception e) { + closeSenderSilently(); + if ( + (reporter != null && e.getMessage() != null) // hack to detect data parsing errors + && (e.getMessage().contains("error in line") || e.getMessage().contains("failed to parse line protocol")) + ) { + // ok, we have a parsing error, let's try to send records one by one to find the problematic record + // and we will report it to the error handler. the rest of the records will make it to QuestDB + sender = createSender(); + for (int i = 0; i < inflightSinkRecords.size(); i++) { + SinkRecord sinkRecord = inflightSinkRecords.get(i); + try { + handleSingleRecord(sinkRecord); + sender.flush(); + } catch (Exception ex) { + context.errantRecordReporter().report(sinkRecord, ex); + closeSenderSilently(); + sender = createSender(); + } + } + nextFlushNanos = System.nanoTime() + flushConfig.autoFlushNanos; + pendingRows = 0; + } else { + // ok, this is not a parsing error, let's just close the sender and rethrow the exception + nextFlushNanos = System.nanoTime() + flushConfig.autoFlushNanos; + pendingRows = 0; + throw new ConnectException("Failed to send data to QuestDB", e); + } + } + private void closeSenderSilently() { if (sender != null) { try { diff --git a/connector/src/test/java/io/questdb/kafka/QuestDBSinkConnectorEmbeddedTest.java b/connector/src/test/java/io/questdb/kafka/QuestDBSinkConnectorEmbeddedTest.java index 84b39e8..1d416dd 100644 --- a/connector/src/test/java/io/questdb/kafka/QuestDBSinkConnectorEmbeddedTest.java +++ b/connector/src/test/java/io/questdb/kafka/QuestDBSinkConnectorEmbeddedTest.java @@ -52,7 +52,7 @@ public final class QuestDBSinkConnectorEmbeddedTest { private static int httpPort = -1; private static int ilpPort = -1; - private static final String OFFICIAL_QUESTDB_DOCKER = "questdb/questdb:7.4.0"; + private static final String OFFICIAL_QUESTDB_DOCKER = "questdb/questdb:8.1.1"; private static final boolean DUMP_QUESTDB_CONTAINER_LOGS = true; private EmbeddedConnectCluster connect; @@ -248,6 +248,81 @@ public void testDeadLetterQueue_wrongJson(boolean useHttp) { Assertions.assertEquals("{\"not valid json}", new String(dqlRecord.value())); } + @Test + public void testDeadLetterQueue_badColumnType() { + connect.kafka().createTopic(topicName, 1); + Map props = ConnectTestUtils.baseConnectorProps(questDBContainer, topicName, true); + props.put("value.converter.schemas.enable", "false"); + props.put("errors.deadletterqueue.topic.name", "dlq"); + props.put("errors.deadletterqueue.topic.replication.factor", "1"); + props.put("errors.tolerance", "all"); + connect.configureConnector(ConnectTestUtils.CONNECTOR_NAME, props); + ConnectTestUtils.assertConnectorTaskRunningEventually(connect); + + QuestDBUtils.assertSql( + "{\"ddl\":\"OK\"}", + "create table " + topicName + " (firstname string, lastname string, age int, id uuid, ts timestamp) timestamp(ts) partition by day wal", + httpPort, + QuestDBUtils.Endpoint.EXEC); + + String goodRecordA = "{\"firstname\":\"John\",\"lastname\":\"Doe\",\"age\":42,\"id\":\"ad956a45-a55b-441e-b80d-023a2bf5d041\"}"; + String goodRecordB = "{\"firstname\":\"John\",\"lastname\":\"Doe\",\"age\":42,\"id\":\"ad956a45-a55b-441e-b80d-023a2bf5d042\"}"; + String goodRecordC = "{\"firstname\":\"John\",\"lastname\":\"Doe\",\"age\":42,\"id\":\"ad956a45-a55b-441e-b80d-023a2bf5d043\"}"; + String badRecordA = "{\"firstname\":\"John\",\"lastname\":\"Doe\",\"age\":42,\"id\":\"Invalid UUID\"}"; + String badRecordB = "{\"firstname\":\"John\",\"lastname\":\"Doe\",\"age\":\"not a number\",\"id\":\"ad956a45-a55b-441e-b80d-023a2bf5d041\"}"; + + // interleave good and bad records + connect.kafka().produce(topicName, "key", goodRecordA); + connect.kafka().produce(topicName, "key", badRecordA); + connect.kafka().produce(topicName, "key", goodRecordB); + connect.kafka().produce(topicName, "key", badRecordB); + connect.kafka().produce(topicName, "key", goodRecordC); + + ConsumerRecords fetchedRecords = connect.kafka().consume(2, 120_000, "dlq"); + Assertions.assertEquals(2, fetchedRecords.count()); + Iterator> iterator = fetchedRecords.iterator(); + Assertions.assertEquals(badRecordA, new String(iterator.next().value())); + Assertions.assertEquals(badRecordB, new String(iterator.next().value())); + + QuestDBUtils.assertSqlEventually("\"firstname\",\"lastname\",\"age\",\"id\"\r\n" + + "\"John\",\"Doe\",42,ad956a45-a55b-441e-b80d-023a2bf5d041\r\n" + + "\"John\",\"Doe\",42,ad956a45-a55b-441e-b80d-023a2bf5d042\r\n" + + "\"John\",\"Doe\",42,ad956a45-a55b-441e-b80d-023a2bf5d043\r\n", + "select firstname,lastname,age, id from " + topicName, + httpPort); + + } + + @Test + public void testbadColumnType_noDLQ() { + connect.kafka().createTopic(topicName, 1); + Map props = ConnectTestUtils.baseConnectorProps(questDBContainer, topicName, true); + props.put("value.converter.schemas.enable", "false"); + connect.configureConnector(ConnectTestUtils.CONNECTOR_NAME, props); + ConnectTestUtils.assertConnectorTaskRunningEventually(connect); + + QuestDBUtils.assertSql( + "{\"ddl\":\"OK\"}", + "create table " + topicName + " (firstname string, lastname string, age int, id uuid, ts timestamp) timestamp(ts) partition by day wal", + httpPort, + QuestDBUtils.Endpoint.EXEC); + + String goodRecordA = "{\"firstname\":\"John\",\"lastname\":\"Doe\",\"age\":42,\"id\":\"ad956a45-a55b-441e-b80d-023a2bf5d041\"}"; + String goodRecordB = "{\"firstname\":\"John\",\"lastname\":\"Doe\",\"age\":42,\"id\":\"ad956a45-a55b-441e-b80d-023a2bf5d042\"}"; + String goodRecordC = "{\"firstname\":\"John\",\"lastname\":\"Doe\",\"age\":42,\"id\":\"ad956a45-a55b-441e-b80d-023a2bf5d043\"}"; + String badRecordA = "{\"firstname\":\"John\",\"lastname\":\"Doe\",\"age\":42,\"id\":\"Invalid UUID\"}"; + String badRecordB = "{\"firstname\":\"John\",\"lastname\":\"Doe\",\"age\":\"not a number\",\"id\":\"ad956a45-a55b-441e-b80d-023a2bf5d041\"}"; + + // interleave good and bad records + connect.kafka().produce(topicName, "key", goodRecordA); + connect.kafka().produce(topicName, "key", badRecordA); + connect.kafka().produce(topicName, "key", goodRecordB); + connect.kafka().produce(topicName, "key", badRecordB); + connect.kafka().produce(topicName, "key", goodRecordC); + + ConnectTestUtils.assertConnectorTaskStateEventually(connect, AbstractStatus.State.FAILED); + } + @ParameterizedTest @ValueSource(booleans = {true, false}) public void testSymbol(boolean useHttp) {