|
20 | 20 | import java.time.ZoneId; |
21 | 21 | import java.time.ZoneOffset; |
22 | 22 | import java.time.ZonedDateTime; |
| 23 | +import java.util.Locale; |
| 24 | +import java.util.Objects; |
23 | 25 |
|
| 26 | +import io.aiven.kafka.connect.common.config.extractors.DataExtractor; |
| 27 | +import io.aiven.kafka.connect.common.config.extractors.HeaderValueExtractor; |
| 28 | +import io.aiven.kafka.connect.common.config.extractors.SimpleValuePath; |
24 | 29 | import org.apache.kafka.connect.sink.SinkRecord; |
25 | 30 |
|
26 | 31 | public interface TimestampSource { |
27 | 32 |
|
28 | 33 | ZonedDateTime time(SinkRecord record); |
29 | 34 |
|
30 | | - @SuppressWarnings("PMD.ShortMethodName") |
31 | | - static TimestampSource of(final Type extractorType) { |
32 | | - return of(ZoneOffset.UTC, extractorType); |
33 | | - } |
34 | | - |
35 | | - @SuppressWarnings("PMD.ShortMethodName") |
36 | | - static TimestampSource of(final ZoneId zoneId, final Type extractorType) { |
37 | | - switch (extractorType) { |
38 | | - case WALLCLOCK : |
39 | | - return new WallclockTimestampSource(zoneId); |
40 | | - case EVENT : |
41 | | - return new EventTimestampSource(zoneId); |
42 | | - default : |
43 | | - throw new IllegalArgumentException( |
44 | | - String.format("Unsupported timestamp extractor type: %s", extractorType)); |
45 | | - } |
46 | | - } |
47 | 35 |
|
48 | 36 | Type type(); |
49 | 37 |
|
50 | 38 | enum Type { |
51 | 39 |
|
52 | | - WALLCLOCK, EVENT; |
| 40 | + WALLCLOCK, |
| 41 | + EVENT, |
| 42 | + HEADER, |
| 43 | + SIMPLE_DATA, |
| 44 | + CUSTOM |
| 45 | + |
| 46 | + } |
| 47 | + class Builder { |
| 48 | + private ZoneId zoneId = ZoneOffset.UTC; |
| 49 | + private Type type; |
| 50 | + private String additionalParameters; |
| 51 | + |
| 52 | + /** |
| 53 | + * set the zoneId to be used. If this method isnt called, the default is UTC |
| 54 | + * @return this |
| 55 | + * @throws NullPointerException if zoneId is null |
| 56 | + */ |
| 57 | + public Builder zoneId(final ZoneId zoneId) { |
| 58 | + Objects.requireNonNull(zoneId, "zoneId cannot be null"); |
| 59 | + this.zoneId = zoneId; |
| 60 | + return this; |
| 61 | + } |
53 | 62 |
|
54 | | - @SuppressWarnings("PMD.ShortMethodName") |
55 | | - public static Type of(final String name) { |
56 | | - for (final Type t : Type.values()) { |
57 | | - if (t.name().equalsIgnoreCase(name)) { |
58 | | - return t; |
59 | | - } |
| 63 | + /** |
| 64 | + * sets the type of the timestamp source and associated parameters (if needed) |
| 65 | + * The format of the configuration is <type>[:<data>] |
| 66 | + * i.e. the type name, optionally followed by data. |
| 67 | + * <br> |
| 68 | + * The data is type specific |
| 69 | + * <p> |
| 70 | + * For type WALLCLOCK or EVENT, no data is allowed |
| 71 | + * </p> |
| 72 | + * <p> |
| 73 | + * For type SIMPLE_DATA, data is required, and is a '.' separated series of |
| 74 | + * terms in the path |
| 75 | + * <br>If the '.' is something that should be included in the terms, and you |
| 76 | + * want to use a different separator, then you can specify a '.' as the first character, and the separator as the |
| 77 | + * second character, and then the path is the rest of the string |
| 78 | + * <br>For example "SIMPLE_DATA:a.b.c" would use into a path with |
| 79 | + * terms "a", "b", "c" |
| 80 | + * <br>For example "SIMPLE_DATA:.:a.b:c" would use a path with terms "a.b", "c" |
| 81 | + * </p> |
| 82 | + * For type HEADER, data is required, and is the name of the header to extract |
| 83 | + * <br>For example "HEADER:foo" would use to "foo" header (or null if its not available in the SinkRecord |
| 84 | + * </p> |
| 85 | + * </p> |
| 86 | + * For type CUSTOM, data is required, and is the name of the class to use, and any additional parameters for that custom time source. |
| 87 | + * The specified class must implement the TimestampSource interface and have a public constructor that takes a String and a ZoneId. Fort the meaning of the data, see the documentation of the custom class. |
| 88 | + * <br>For example "CUSTOM:my.custom.timesource:some more data" would be similar to calling new my.custom.timesource("some more data", zoneId) |
| 89 | + * </p> |
| 90 | + * |
| 91 | +
|
| 92 | + * @return this |
| 93 | + */ |
| 94 | + public Builder configuration(final String configuration) { |
| 95 | + final String[] parts = configuration.split(":", 2); |
| 96 | + final String typeName = parts[0]; |
| 97 | + try { |
| 98 | + this.type = Type.valueOf(typeName.toUpperCase(Locale.ENGLISH)); |
| 99 | + } catch (final IllegalArgumentException e) { |
| 100 | + throw new IllegalArgumentException("Unknown timestamp source: "+typeName); |
| 101 | + } |
| 102 | + |
| 103 | + this.additionalParameters = parts.length > 1 ? parts[1] : null; |
| 104 | + return this; |
| 105 | + } |
| 106 | + |
| 107 | + public TimestampSource build() { |
| 108 | + switch (type) { |
| 109 | + case WALLCLOCK: |
| 110 | + if (additionalParameters != null) { |
| 111 | + throw new IllegalArgumentException("Wallclock timestamp source does not support additionalParameters"); |
| 112 | + } |
| 113 | + return new WallclockTimestampSource(zoneId); |
| 114 | + case EVENT: |
| 115 | + if (additionalParameters != null) { |
| 116 | + throw new IllegalArgumentException("Event timestamp source does not support additionalParameters"); |
| 117 | + } |
| 118 | + return new EventTimestampSource(zoneId); |
| 119 | + case SIMPLE_DATA: |
| 120 | + if (additionalParameters == null) { |
| 121 | + throw new IllegalArgumentException("Data timestamp source requires additionalParameters"); |
| 122 | + } |
| 123 | + return new SimpleTimestampSource(zoneId, Type.SIMPLE_DATA, SimpleValuePath.parse(additionalParameters)); |
| 124 | + case HEADER: |
| 125 | + if (additionalParameters == null) { |
| 126 | + throw new IllegalArgumentException("Header timestamp source requires additionalParameters"); |
| 127 | + } |
| 128 | + return new SimpleTimestampSource(zoneId, Type.HEADER, new HeaderValueExtractor(additionalParameters)); |
| 129 | + case CUSTOM: |
| 130 | + if (additionalParameters == null) { |
| 131 | + throw new IllegalArgumentException("Header timestamp source requires additionalParameters"); |
| 132 | + } |
| 133 | + final String[] parts = additionalParameters.split(":", 2); |
| 134 | + final String className = parts[0]; |
| 135 | + final String params = parts.length > 1 ? parts[1] : null; |
| 136 | + try { |
| 137 | + final Class<?> clazz = Class.forName(className); |
| 138 | + return (TimestampSource) clazz.getConstructor(String.class, ZoneId.class).newInstance(params, zoneId); |
| 139 | + } catch (final Exception e) { |
| 140 | + throw new IllegalArgumentException("Failed to create custom timestamp source", e); |
| 141 | + } |
| 142 | + default: |
| 143 | + throw new IllegalArgumentException( |
| 144 | + String.format("Unsupported timestamp extractor type: %s", type)); |
60 | 145 | } |
61 | | - throw new IllegalArgumentException(String.format("Unknown timestamp source: %s", name)); |
62 | 146 | } |
63 | 147 |
|
64 | 148 | } |
65 | 149 |
|
66 | | - final class WallclockTimestampSource implements TimestampSource { |
67 | | - private final ZoneId zoneId; |
| 150 | + class SimpleTimestampSource implements TimestampSource { |
| 151 | + protected final ZoneId zoneId; |
| 152 | + private final Type type; |
| 153 | + private final DataExtractor dataExtractor; |
68 | 154 |
|
69 | | - protected WallclockTimestampSource(final ZoneId zoneId) { |
| 155 | + protected SimpleTimestampSource(final ZoneId zoneId, final Type type, DataExtractor dataExtractor) { |
70 | 156 | this.zoneId = zoneId; |
| 157 | + this.type = type; |
| 158 | + this.dataExtractor = dataExtractor; |
71 | 159 | } |
72 | 160 |
|
73 | 161 | @Override |
74 | | - public ZonedDateTime time(final SinkRecord record) { |
75 | | - return ZonedDateTime.now(zoneId); |
| 162 | + public Type type() { |
| 163 | + return type; |
76 | 164 | } |
77 | 165 |
|
78 | 166 | @Override |
79 | | - public Type type() { |
80 | | - return Type.WALLCLOCK; |
| 167 | + public ZonedDateTime time(SinkRecord record) { |
| 168 | + return fromRawTime(dataExtractor.extractDataFrom(record)); |
81 | 169 | } |
82 | | - } |
83 | 170 |
|
84 | | - final class EventTimestampSource implements TimestampSource { |
85 | | - private final ZoneId zoneId; |
| 171 | + protected ZonedDateTime fromRawTime(final Object rawValue) { |
| 172 | + if (rawValue == null) { |
| 173 | + return null; |
| 174 | + } else if (rawValue instanceof Long) { |
| 175 | + return withZone((Long) rawValue); |
| 176 | + } else if (rawValue instanceof ZonedDateTime) { |
| 177 | + return (ZonedDateTime) rawValue; |
| 178 | + } else if (rawValue instanceof Instant) { |
| 179 | + return withZone(((Instant) rawValue).toEpochMilli()); |
| 180 | + } |
| 181 | + return null; |
| 182 | + } |
86 | 183 |
|
87 | | - protected EventTimestampSource(final ZoneId zoneId) { |
88 | | - this.zoneId = zoneId; |
| 184 | + protected ZonedDateTime withZone(final long timestamp) { |
| 185 | + return ZonedDateTime.ofInstant(Instant.ofEpochMilli(timestamp), zoneId); |
| 186 | + } |
| 187 | + } |
| 188 | + |
| 189 | + final class WallclockTimestampSource extends SimpleTimestampSource { |
| 190 | + WallclockTimestampSource(final ZoneId zoneId) { |
| 191 | + super(zoneId, Type.WALLCLOCK, null); |
89 | 192 | } |
90 | 193 |
|
91 | 194 | @Override |
92 | 195 | public ZonedDateTime time(final SinkRecord record) { |
93 | | - return ZonedDateTime.ofInstant(Instant.ofEpochMilli(record.timestamp()), zoneId); |
| 196 | + return ZonedDateTime.now(zoneId); |
| 197 | + } |
| 198 | + } |
| 199 | + |
| 200 | + final class EventTimestampSource extends SimpleTimestampSource { |
| 201 | + EventTimestampSource(final ZoneId zoneId) { |
| 202 | + super(zoneId, Type.EVENT, null); |
94 | 203 | } |
95 | 204 |
|
96 | 205 | @Override |
97 | | - public Type type() { |
98 | | - return Type.EVENT; |
| 206 | + public ZonedDateTime time(final SinkRecord record) { |
| 207 | + return withZone(record.timestamp()); |
99 | 208 | } |
100 | 209 | } |
101 | 210 | } |
0 commit comments