Skip to content

Commit fa4a880

Browse files
committed
use QuestDB timestamp parser
So we don't allocate with every timestamp like psychopaths :)
1 parent 8355aff commit fa4a880

File tree

4 files changed

+49
-22
lines changed

4 files changed

+49
-22
lines changed

connector/src/main/java/io/questdb/kafka/QuestDBSinkConnectorConfig.java

Lines changed: 3 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -38,7 +38,7 @@ public final class QuestDBSinkConnectorConfig extends AbstractConfig {
3838
private static final String TIMESTAMP_STRING_FIELDS_DOC = "Comma separated list of string fields that should be parsed as timestamp.";
3939

4040
public static final String TIMESTAMP_UNITS_CONFIG = "timestamp.units";
41-
private static final String TIMESTAMP_UNITS_DOC = "Units of timestamp field. Possible values: auto, millis, micros, nanos";
41+
private static final String TIMESTAMP_UNITS_DOC = "Units of designated timestamp field. Possible values: auto, millis, micros, nanos";
4242

4343
public static final String INCLUDE_KEY_CONFIG = "include.key";
4444
private static final String INCLUDE_KEY_DOC = "Include key in the table";
@@ -64,8 +64,8 @@ public final class QuestDBSinkConnectorConfig extends AbstractConfig {
6464
public static final String MAX_RETRIES = "max.retries";
6565
private static final String MAX_RETRIES_DOC = "The maximum number of times to retry on errors before failing the task";
6666

67-
public static final String TIMESTAMP_FORMAT = "yyyy-MM-dd'T'HH:mm:ss.SSSSSS'Z'";
68-
private static final String TIMESTAMP_FORMAT_DOC = "Default timestamp format";
67+
public static final String TIMESTAMP_FORMAT = "timestamp.string.format";
68+
private static final String TIMESTAMP_FORMAT_DOC = "Default timestamp format. Used when parsing timestamp string fields";
6969

7070
private static final String DEFAULT_TIMESTAMP_FORMAT = "yyyy-MM-dd'T'HH:mm:ss.SSSSSS'Z'";
7171

@@ -200,11 +200,6 @@ public void ensureValid(String name, Object value) {
200200
if (!(value instanceof String)) {
201201
throw new ConfigException(name, value, "Timestamp format must be a string");
202202
}
203-
try {
204-
DateTimeFormatter.ofPattern((String) value);
205-
} catch (IllegalArgumentException e) {
206-
throw new ConfigException(name, value, "Timestamp format is not a valid DateTimeFormatter pattern. Error='" + e.getMessage() + "'");
207-
}
208203
}
209204
}
210205

connector/src/main/java/io/questdb/kafka/QuestDBSinkTask.java

Lines changed: 14 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -2,7 +2,10 @@
22

33
import io.questdb.client.Sender;
44
import io.questdb.cutlass.line.LineSenderException;
5+
import io.questdb.std.NumericException;
6+
import io.questdb.std.datetime.DateFormat;
57
import io.questdb.std.datetime.microtime.Timestamps;
8+
import io.questdb.std.datetime.millitime.DateFormatUtils;
69
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
710
import org.apache.kafka.common.TopicPartition;
811
import org.apache.kafka.connect.data.Date;
@@ -19,9 +22,6 @@
1922
import org.slf4j.Logger;
2023
import org.slf4j.LoggerFactory;
2124

22-
import java.time.format.DateTimeFormatter;
23-
import java.time.temporal.ChronoField;
24-
import java.time.temporal.TemporalAccessor;
2525
import java.util.Collection;
2626
import java.util.Collections;
2727
import java.util.HashSet;
@@ -45,7 +45,7 @@ public final class QuestDBSinkTask extends SinkTask {
4545
private Set<String> stringTimestampColumns;
4646
private int remainingRetries;
4747
private long batchesSinceLastError = 0;
48-
private DateTimeFormatter timestampParser;
48+
private DateFormat dataFormat;
4949

5050
@Override
5151
public String version() {
@@ -61,7 +61,7 @@ public void start(Map<String, String> map) {
6161
for (String symbolColumn : timestampStringFields.split(",")) {
6262
stringTimestampColumns.add(symbolColumn.trim());
6363
}
64-
this.timestampParser = DateTimeFormatter.ofPattern(config.getDefaultTimestampFormat());
64+
dataFormat = TimestampParserCompiler.compilePattern(config.getDefaultTimestampFormat());
6565
} else {
6666
stringTimestampColumns = Collections.emptySet();
6767
}
@@ -251,7 +251,7 @@ private long resolveDesignatedTimestampColumnValue(Object value, Schema schema)
251251
}
252252
if (value instanceof String) {
253253
log.debug("Timestamp column value is a string");
254-
return parseTimestamp((String) value, TimeUnit.NANOSECONDS);
254+
return parseToMicros((String) value) * 1000;
255255
}
256256
if (!(value instanceof Long)) {
257257
throw new ConnectException("Unsupported timestamp column type: " + value.getClass());
@@ -277,7 +277,7 @@ private void writePhysicalTypeWithoutSchema(String name, Object value, String fa
277277
if (value instanceof String) {
278278
String stringVal = (String) value;
279279
if (stringTimestampColumns.contains(actualName)) {
280-
long timestamp = parseTimestamp(stringVal, TimeUnit.MICROSECONDS);
280+
long timestamp = parseToMicros(stringVal);
281281
sender.timestampColumn(actualName, timestamp);
282282
} else {
283283
sender.stringColumn(actualName, stringVal);
@@ -310,11 +310,12 @@ private void writePhysicalTypeWithoutSchema(String name, Object value, String fa
310310
}
311311
}
312312

313-
private long parseTimestamp(String timestamp, TimeUnit unit) {
314-
TemporalAccessor accessor = timestampParser.parse(timestamp);
315-
long seconds = accessor.getLong(ChronoField.INSTANT_SECONDS);
316-
int nanos = accessor.get(ChronoField.NANO_OF_SECOND);
317-
return unit.convert(seconds, TimeUnit.SECONDS) + unit.convert(nanos, TimeUnit.NANOSECONDS);
313+
private long parseToMicros(String timestamp) {
314+
try {
315+
return dataFormat.parse(timestamp, DateFormatUtils.enLocale);
316+
} catch (NumericException e) {
317+
throw new ConnectException("Cannot parse timestamp: " + timestamp, e);
318+
}
318319
}
319320

320321
private static String sanitizeName(String name) {
@@ -349,7 +350,7 @@ private boolean tryWritePhysicalTypeFromSchema(String name, Schema schema, Objec
349350
case STRING:
350351
String s = (String) value;
351352
if (stringTimestampColumns.contains(primitiveTypesName)) {
352-
long timestamp = parseTimestamp(s, TimeUnit.MICROSECONDS);
353+
long timestamp = parseToMicros(s);
353354
sender.timestampColumn(sanitizedName, timestamp);
354355
} else {
355356
sender.stringColumn(sanitizedName, s);
Lines changed: 31 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,31 @@
1+
package io.questdb.kafka;
2+
3+
import io.questdb.std.CharSequenceObjHashMap;
4+
import io.questdb.std.datetime.DateFormat;
5+
import io.questdb.std.datetime.microtime.TimestampFormatCompiler;
6+
7+
class TimestampParserCompiler {
8+
private static TimestampFormatCompiler compiler;
9+
private static final Object MUTEX = new Object();
10+
// we assume that there will just a few patterns
11+
private static CharSequenceObjHashMap<DateFormat> cache;
12+
13+
public static DateFormat compilePattern(String timestampPattern) {
14+
synchronized (MUTEX) {
15+
if (compiler == null) {
16+
compiler = new TimestampFormatCompiler();
17+
}
18+
// DateFormat instances are thread-safe, so we can cache them and use for multiple workers
19+
if (cache == null) {
20+
cache = new CharSequenceObjHashMap<>();
21+
}
22+
DateFormat format = cache.get(timestampPattern);
23+
if (format != null) {
24+
return format;
25+
}
26+
format = compiler.compile(timestampPattern);
27+
cache.put(timestampPattern, format);
28+
return format;
29+
}
30+
}
31+
}

connector/src/test/java/io/questdb/kafka/QuestDBSinkConnectorEmbeddedTest.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -747,7 +747,7 @@ public void testParsingStringTimestamp() {
747747
props.put("value.converter.schemas.enable", "false");
748748
props.put(QuestDBSinkConnectorConfig.DESIGNATED_TIMESTAMP_COLUMN_NAME_CONFIG, "born");
749749
props.put(QuestDBSinkConnectorConfig.INCLUDE_KEY_CONFIG, "false");
750-
props.put(QuestDBSinkConnectorConfig.TIMESTAMP_FORMAT, "yyyy-MM-dd HH:mm:ss.SSSSSS z");
750+
props.put(QuestDBSinkConnectorConfig.TIMESTAMP_FORMAT, "yyyy-MM-dd HH:mm:ss.U+ z");
751751
props.put(QuestDBSinkConnectorConfig.TIMESTAMP_STRING_FIELDS, "born,death");
752752

753753
connect.configureConnector(CONNECTOR_NAME, props);

0 commit comments

Comments
 (0)