Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feat: use auto flush configuration from client config string #22

Merged
merged 5 commits into from
May 24, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
132 changes: 93 additions & 39 deletions connector/src/main/java/io/questdb/kafka/ClientConfUtils.java
Original file line number Diff line number Diff line change
Expand Up @@ -2,65 +2,119 @@

import io.questdb.client.impl.ConfStringParser;
import io.questdb.std.Chars;
import io.questdb.std.Misc;
import io.questdb.std.Numbers;
import io.questdb.std.NumericException;
import io.questdb.std.str.StringSink;
import org.apache.kafka.common.config.ConfigException;

import java.util.concurrent.TimeUnit;

final class ClientConfUtils {
private ClientConfUtils() {
}

static boolean patchConfStr(String confStr, StringSink sink) {
int pos = ConfStringParser.of(confStr, sink);

static boolean patchConfStr(String confStr, StringSink sink, FlushConfig flushConfig) {
flushConfig.reset();

sink.clear();
StringSink tmpSink = Misc.getThreadLocalSink();
int pos = ConfStringParser.of(confStr, tmpSink);
if (pos < 0) {
sink.clear();
sink.put(confStr);
return false;
}

boolean isHttpTransport = Chars.equals(sink, "http") || Chars.equals(sink, "https");
boolean intervalFlushSetExplicitly = false;
boolean flushesDisabled = false;
boolean parseError = false;
boolean hasAtLeastOneParam = false;
boolean isHttpTransport = Chars.equals(tmpSink, "http") || Chars.equals(tmpSink, "https");
if (!isHttpTransport) {
sink.put(confStr);
// no patching for TCP transport
return false;
}
sink.put(tmpSink).put("::");

// disable interval based flushes
// unless they are explicitly set or auto_flush is entirely off
// why? the connector has its own mechanism to flush data in a timely manner
boolean hasAtLeastOneParam = false;
while (ConfStringParser.hasNext(confStr, pos)) {
hasAtLeastOneParam = true;
pos = ConfStringParser.nextKey(confStr, pos, sink);
pos = ConfStringParser.nextKey(confStr, pos, tmpSink);
if (pos < 0) {
parseError = true;
break;
sink.clear();
sink.put(confStr);
return true;
}
if (Chars.equals(sink, "auto_flush_interval")) {
intervalFlushSetExplicitly = true;
pos = ConfStringParser.value(confStr, pos, sink);
} else if (Chars.equals(sink, "auto_flush")) {
pos = ConfStringParser.value(confStr, pos, sink);
flushesDisabled = Chars.equals(sink, "off");
if (Chars.equals(tmpSink, "auto_flush_interval")) {
pos = ConfStringParser.value(confStr, pos, tmpSink);
if (pos < 0) {
sink.clear();
sink.put(confStr);
// invalid config, let the real client parser to fail
return true;
}
if (Chars.equals(tmpSink, "off")) {
throw new ConfigException("QuestDB Kafka connector cannot have auto_flush_interval disabled");
}
try {
flushConfig.autoFlushNanos = TimeUnit.MILLISECONDS.toNanos(Numbers.parseLong(tmpSink));
} catch (NumericException e) {
throw new ConfigException("Invalid auto_flush_interval value [auto_flush_interval=" + tmpSink + ']');
}
} else if (Chars.equals(tmpSink, "auto_flush_rows")) {
pos = ConfStringParser.value(confStr, pos, tmpSink);
if (pos < 0) {
sink.clear();
sink.put(confStr);
return true;
}
if (Chars.equals(tmpSink, "off")) {
throw new ConfigException("QuestDB Kafka connector cannot have auto_flush_rows disabled");
} else {
try {
flushConfig.autoFlushRows = Numbers.parseInt(tmpSink);
} catch (NumericException e) {
throw new ConfigException("Invalid auto_flush_rows value [auto_flush_rows=" + tmpSink + ']');
}
}
} else if (Chars.equals(tmpSink, "auto_flush")) {
pos = ConfStringParser.value(confStr, pos, tmpSink);
if (pos < 0) {
sink.clear();
sink.put(confStr);
return true;
}
if (Chars.equals(tmpSink, "off")) {
throw new ConfigException("QuestDB Kafka connector cannot have auto_flush disabled");
} else if (!Chars.equals(tmpSink, "on")) {
throw new ConfigException("Unknown auto_flush value [auto_flush=" + tmpSink + ']');
}
} else {
pos = ConfStringParser.value(confStr, pos, sink); // skip other values
}
if (pos < 0) {
parseError = true;
break;
// copy other params
sink.put(tmpSink).put('=');
pos = ConfStringParser.value(confStr, pos, tmpSink);
if (pos < 0) {
sink.clear();
sink.put(confStr);
return true;
}
for (int i = 0; i < tmpSink.length(); i++) {
char ch = tmpSink.charAt(i);
sink.put(ch);
// re-escape semicolon
if (ch == ';') {
sink.put(';');
}
}
sink.put(';');
}
}
sink.clear();
sink.put(confStr);
if (!parseError // we don't want to mess with the config if there was a parse error
&& isHttpTransport // we only want to patch http transport
&& !flushesDisabled // if auto-flush is disabled we don't need to do anything
&& !intervalFlushSetExplicitly // if auto_flush_interval is set explicitly we don't want to override it
&& hasAtLeastOneParam // no parameter is also an error since at least address should be set. we let client throw exception in this case
) {
// if everything is ok, we set auto_flush_interval to max value
// this will effectively disable interval based flushes
// and the connector will flush data only when it is told to do so by Connector
// or if a row count limit is reached
sink.put("auto_flush_interval=").put(Integer.MAX_VALUE).put(';');
if (!hasAtLeastOneParam) {
// this is invalid, let the real client parser to fail
sink.clear();
sink.put(confStr);
return true;
}
sink.put("auto_flush=off;");

return isHttpTransport;
return true;
}
}
13 changes: 13 additions & 0 deletions connector/src/main/java/io/questdb/kafka/FlushConfig.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,13 @@
package io.questdb.kafka;

import java.util.concurrent.TimeUnit;

final class FlushConfig {
int autoFlushRows;
long autoFlushNanos;

void reset() {
autoFlushRows = 75_000;
autoFlushNanos = TimeUnit.SECONDS.toNanos(1);
}
}
64 changes: 44 additions & 20 deletions connector/src/main/java/io/questdb/kafka/QuestDBSinkTask.java
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,9 @@ public final class QuestDBSinkTask extends SinkTask {
private boolean kafkaTimestampsEnabled;
private boolean httpTransport;
private int allowedLag;
private long nextFlushNanos;
private int pendingRows;
private final FlushConfig flushConfig = new FlushConfig();

@Override
public String version() {
Expand Down Expand Up @@ -79,6 +82,7 @@ public void start(Map<String, String> map) {
this.kafkaTimestampsEnabled = config.isDesignatedTimestampKafkaNative();
this.timestampUnits = config.getTimestampUnitsOrNull();
this.allowedLag = config.getAllowedLag();
this.nextFlushNanos = System.nanoTime() + flushConfig.autoFlushNanos;
}

private Sender createRawSender() {
Expand All @@ -91,10 +95,13 @@ private Sender createRawSender() {
if (confStr != null && !confStr.isEmpty()) {
log.debug("Using client configuration string");
StringSink sink = new StringSink();
httpTransport = ClientConfUtils.patchConfStr(confStr, sink);
httpTransport = ClientConfUtils.patchConfStr(confStr, sink, flushConfig);
if (!httpTransport) {
log.info("Using TCP transport, consider using HTTP transport for improved fault tolerance and error handling");
}
return Sender.fromConfig(sink);
}
log.debug("Using legacy client configuration");
log.warn("Configuration options 'host', 'tsl', 'token' and 'username' are deprecated and will be removed in the future. Use 'client.conf.string' instead. See: https://questdb.io/docs/third-party-tools/kafka/questdb-kafka/#configuration-options");
Sender.LineSenderBuilder builder = Sender.builder(Sender.Transport.TCP).address(config.getHost());
if (config.isTls()) {
builder.enableTls();
Expand Down Expand Up @@ -132,11 +139,7 @@ public void put(Collection<SinkRecord> collection) {
// We do not want locally buffered row to be stuck in the buffer for too long. It increases
// latency between the time the record is produced and the time it is visible in QuestDB.
// If the local buffer is empty then flushing is a cheap no-op.
try {
sender.flush();
} catch (LineSenderException | HttpClientException e) {
onSenderException(e);
}
flushAndResetCounters();
} else {
log.debug("Received empty collection, nothing to do");
}
Expand All @@ -156,7 +159,27 @@ public void put(Collection<SinkRecord> collection) {
handleSingleRecord(record);
}

if (!httpTransport) {
if (httpTransport) {
if (pendingRows >= flushConfig.autoFlushRows) {
log.debug("Flushing data to QuestDB due to auto_flush_rows limit [pending-rows={}, max-pending-rows={}]",
pendingRows, flushConfig.autoFlushRows);
flushAndResetCounters();
} else {
long remainingNanos = nextFlushNanos - System.nanoTime();
long remainingMs = TimeUnit.NANOSECONDS.toMillis(remainingNanos);
if (remainingMs <= 0) {
log.debug("Flushing data to QuestDB due to auto_flush_interval timeout");
flushAndResetCounters();
} else if (allowedLag == 0) {
log.debug("Flushing data to QuestDB due to zero allowed lag");
flushAndResetCounters();
} else {
log.debug("Flushing data to QuestDB in {} ms", remainingMs);
long maxWaitTime = Math.min(remainingMs, allowedLag);
context.timeout(maxWaitTime);
}
}
} else {
log.debug("Sending {} records", collection.size());
sender.flush();
log.debug("Successfully sent {} records", collection.size());
Expand All @@ -177,18 +200,24 @@ public void put(Collection<SinkRecord> collection) {
} catch (LineSenderException | HttpClientException e) {
onSenderException(e);
}
}

if (httpTransport) {
// we successfully added some rows to the local buffer.
// let's set a timeout so Kafka Connect will call us again in time even if there are
// no new records to send. this gives us a chance to flush the buffer.
context.timeout(allowedLag);
private void flushAndResetCounters() {
log.debug("Flushing data to QuestDB");
try {
sender.flush();
nextFlushNanos = System.nanoTime() + flushConfig.autoFlushNanos;
pendingRows = 0;
} catch (LineSenderException | HttpClientException e) {
onSenderException(e);
}
}

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);
}

Expand Down Expand Up @@ -239,6 +268,7 @@ private void handleSingleRecord(SinkRecord record) {
timestampColumnValue = Long.MIN_VALUE;
}
}
pendingRows++;
}

private void handleStruct(String parentName, Struct value, Schema schema) {
Expand Down Expand Up @@ -482,13 +512,7 @@ public Map<TopicPartition, OffsetAndMetadata> preCommit(Map<TopicPartition, Offs
@Override
public void flush(Map<TopicPartition, OffsetAndMetadata> map) {
if (httpTransport) {
try {
log.debug("Flushing data to QuestDB");
sender.flush();
} catch (LineSenderException | HttpClientException e) {
onSenderException(e);
throw new ConnectException("Failed to flush data to QuestDB", e);
}
flushAndResetCounters();
}
// TCP transport flushes after each batch so no need to flush here
}
Expand Down
Loading
Loading