Skip to content

Commit 50f145b

Browse files
committed
Porting from confluentinc#759
1 parent 5d5138a commit 50f145b

File tree

6 files changed

+252
-21
lines changed

6 files changed

+252
-21
lines changed

src/main/java/io/confluent/connect/elasticsearch/DataConverter.java

Lines changed: 17 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@
2020
import com.fasterxml.jackson.databind.ObjectMapper;
2121
import com.fasterxml.jackson.databind.node.ObjectNode;
2222
import io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.BehaviorOnNullValues;
23+
import io.confluent.connect.elasticsearch.util.ScriptParser;
2324
import org.apache.kafka.connect.data.ConnectSchema;
2425
import org.apache.kafka.connect.data.Date;
2526
import org.apache.kafka.connect.data.Decimal;
@@ -44,6 +45,7 @@
4445
import org.elasticsearch.action.update.UpdateRequest;
4546
import org.elasticsearch.common.xcontent.XContentType;
4647
import org.elasticsearch.index.VersionType;
48+
import org.elasticsearch.script.Script;
4749
import org.slf4j.Logger;
4850
import org.slf4j.LoggerFactory;
4951

@@ -186,6 +188,21 @@ public DocWriteRequest<?> convertRecord(SinkRecord record, String index) {
186188
new IndexRequest(index).id(id).source(payload, XContentType.JSON).opType(opType),
187189
record
188190
);
191+
case SCRIPTED_UPSERT:
192+
Script script = null;
193+
194+
try {
195+
script = ScriptParser.parseScript(config.getScript());
196+
} catch (JsonProcessingException jsonProcessingException) {
197+
throw new RuntimeException(jsonProcessingException);
198+
}
199+
200+
return new UpdateRequest(index, id)
201+
.doc(payload, XContentType.JSON)
202+
.upsert(payload, XContentType.JSON)
203+
.retryOnConflict(Math.min(config.maxInFlightRequests(), 5))
204+
.script(script)
205+
.scriptedUpsert(true);
189206
default:
190207
return null; // shouldn't happen
191208
}

src/main/java/io/confluent/connect/elasticsearch/ElasticsearchSinkConnectorConfig.java

Lines changed: 50 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,8 @@
2424
import java.util.Set;
2525
import java.util.stream.Collectors;
2626
import java.util.concurrent.TimeUnit;
27+
28+
import io.confluent.connect.elasticsearch.validator.ScriptValidator;
2729
import org.apache.kafka.common.config.AbstractConfig;
2830
import org.apache.kafka.common.config.ConfigDef;
2931
import org.apache.kafka.common.config.ConfigException;
@@ -277,6 +279,16 @@ public class ElasticsearchSinkConnectorConfig extends AbstractConfig {
277279
);
278280
private static final String WRITE_METHOD_DISPLAY = "Write Method";
279281
private static final String WRITE_METHOD_DEFAULT = WriteMethod.INSERT.name();
282+
283+
public static final String UPSERT_SCRIPT_CONFIG = "upsert.script";
284+
285+
private static final String UPSERT_SCRIPT_DOC = "Script used for"
286+
+ " upserting data to Elasticsearch. This script allows for"
287+
+ " customizable behavior upon upserting a document. Please refer to"
288+
+ " Elasticsearch scripted upsert documentation";
289+
290+
private static final String UPSERT_SCRIPT_DISPLAY = "Upsert Script";
291+
private static final String UPSERT_SCRIPT_DEFAULT = "";
280292
public static final String LOG_SENSITIVE_DATA_CONFIG = "log.sensitive.data";
281293
private static final String LOG_SENSITIVE_DATA_DISPLAY = "Log Sensitive data";
282294
private static final String LOG_SENSITIVE_DATA_DOC = "If true, logs sensitive data "
@@ -408,7 +420,8 @@ public enum SecurityProtocol {
408420

409421
public enum WriteMethod {
410422
INSERT,
411-
UPSERT
423+
UPSERT,
424+
SCRIPTED_UPSERT
412425
}
413426

414427
protected static ConfigDef baseConfigDef() {
@@ -622,8 +635,8 @@ private static void addConversionConfigs(ConfigDef configDef) {
622635
DATA_CONVERSION_GROUP,
623636
++order,
624637
Width.SHORT,
625-
IGNORE_KEY_DISPLAY
626-
).define(
638+
IGNORE_KEY_DISPLAY)
639+
.define(
627640
IGNORE_SCHEMA_CONFIG,
628641
Type.BOOLEAN,
629642
IGNORE_SCHEMA_DEFAULT,
@@ -632,8 +645,8 @@ private static void addConversionConfigs(ConfigDef configDef) {
632645
DATA_CONVERSION_GROUP,
633646
++order,
634647
Width.SHORT,
635-
IGNORE_SCHEMA_DISPLAY
636-
).define(
648+
IGNORE_SCHEMA_DISPLAY)
649+
.define(
637650
COMPACT_MAP_ENTRIES_CONFIG,
638651
Type.BOOLEAN,
639652
COMPACT_MAP_ENTRIES_DEFAULT,
@@ -642,8 +655,8 @@ private static void addConversionConfigs(ConfigDef configDef) {
642655
DATA_CONVERSION_GROUP,
643656
++order,
644657
Width.SHORT,
645-
COMPACT_MAP_ENTRIES_DISPLAY
646-
).define(
658+
COMPACT_MAP_ENTRIES_DISPLAY)
659+
.define(
647660
IGNORE_KEY_TOPICS_CONFIG,
648661
Type.LIST,
649662
IGNORE_KEY_TOPICS_DEFAULT,
@@ -652,8 +665,8 @@ private static void addConversionConfigs(ConfigDef configDef) {
652665
DATA_CONVERSION_GROUP,
653666
++order,
654667
Width.LONG,
655-
IGNORE_KEY_TOPICS_DISPLAY
656-
).define(
668+
IGNORE_KEY_TOPICS_DISPLAY)
669+
.define(
657670
IGNORE_SCHEMA_TOPICS_CONFIG,
658671
Type.LIST,
659672
IGNORE_SCHEMA_TOPICS_DEFAULT,
@@ -662,8 +675,8 @@ private static void addConversionConfigs(ConfigDef configDef) {
662675
DATA_CONVERSION_GROUP,
663676
++order,
664677
Width.LONG,
665-
IGNORE_SCHEMA_TOPICS_DISPLAY
666-
).define(
678+
IGNORE_SCHEMA_TOPICS_DISPLAY)
679+
.define(
667680
DROP_INVALID_MESSAGE_CONFIG,
668681
Type.BOOLEAN,
669682
DROP_INVALID_MESSAGE_DEFAULT,
@@ -672,8 +685,8 @@ private static void addConversionConfigs(ConfigDef configDef) {
672685
DATA_CONVERSION_GROUP,
673686
++order,
674687
Width.LONG,
675-
DROP_INVALID_MESSAGE_DISPLAY
676-
).define(
688+
DROP_INVALID_MESSAGE_DISPLAY)
689+
.define(
677690
BEHAVIOR_ON_NULL_VALUES_CONFIG,
678691
Type.STRING,
679692
BEHAVIOR_ON_NULL_VALUES_DEFAULT.name(),
@@ -684,8 +697,8 @@ private static void addConversionConfigs(ConfigDef configDef) {
684697
++order,
685698
Width.SHORT,
686699
BEHAVIOR_ON_NULL_VALUES_DISPLAY,
687-
new EnumRecommender<>(BehaviorOnNullValues.class)
688-
).define(
700+
new EnumRecommender<>(BehaviorOnNullValues.class))
701+
.define(
689702
BEHAVIOR_ON_MALFORMED_DOCS_CONFIG,
690703
Type.STRING,
691704
BEHAVIOR_ON_MALFORMED_DOCS_DEFAULT.name(),
@@ -696,8 +709,8 @@ private static void addConversionConfigs(ConfigDef configDef) {
696709
++order,
697710
Width.SHORT,
698711
BEHAVIOR_ON_MALFORMED_DOCS_DISPLAY,
699-
new EnumRecommender<>(BehaviorOnMalformedDoc.class)
700-
).define(
712+
new EnumRecommender<>(BehaviorOnMalformedDoc.class))
713+
.define(
701714
EXTERNAL_VERSION_HEADER_CONFIG,
702715
Type.STRING,
703716
EXTERNAL_VERSION_HEADER_DEFAULT,
@@ -706,8 +719,8 @@ private static void addConversionConfigs(ConfigDef configDef) {
706719
DATA_CONVERSION_GROUP,
707720
++order,
708721
Width.SHORT,
709-
EXTERNAL_VERSION_HEADER_DISPLAY
710-
).define(
722+
EXTERNAL_VERSION_HEADER_DISPLAY)
723+
.define(
711724
WRITE_METHOD_CONFIG,
712725
Type.STRING,
713726
WRITE_METHOD_DEFAULT,
@@ -718,8 +731,20 @@ private static void addConversionConfigs(ConfigDef configDef) {
718731
++order,
719732
Width.SHORT,
720733
WRITE_METHOD_DISPLAY,
721-
new EnumRecommender<>(WriteMethod.class)
722-
);
734+
new EnumRecommender<>(WriteMethod.class))
735+
.define(
736+
UPSERT_SCRIPT_CONFIG,
737+
Type.STRING,
738+
null,
739+
new ScriptValidator(),
740+
Importance.LOW,
741+
UPSERT_SCRIPT_DOC,
742+
DATA_CONVERSION_GROUP,
743+
++order,
744+
Width.SHORT,
745+
UPSERT_SCRIPT_DISPLAY,
746+
new ScriptValidator());
747+
;
723748
}
724749

725750
private static void addProxyConfigs(ConfigDef configDef) {
@@ -1078,6 +1103,10 @@ public WriteMethod writeMethod() {
10781103
return WriteMethod.valueOf(getString(WRITE_METHOD_CONFIG).toUpperCase());
10791104
}
10801105

1106+
public String getScript() {
1107+
return getString(UPSERT_SCRIPT_CONFIG);
1108+
}
1109+
10811110
private static class DataStreamDatasetValidator implements Validator {
10821111

10831112
@Override
Lines changed: 46 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,46 @@
1+
/*
2+
* Copyright 2018 Confluent Inc.
3+
*
4+
* Licensed under the Confluent Community License (the "License"); you may not use
5+
* this file except in compliance with the License. You may obtain a copy of the
6+
* License at
7+
*
8+
* http://www.confluent.io/confluent-community-license
9+
*
10+
* Unless required by applicable law or agreed to in writing, software
11+
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12+
* WARRANTIES OF ANY KIND, either express or implied. See the License for the
13+
* specific language governing permissions and limitations under the License.
14+
*/
15+
16+
package io.confluent.connect.elasticsearch.util;
17+
18+
import com.fasterxml.jackson.core.JsonProcessingException;
19+
import com.fasterxml.jackson.core.type.TypeReference;
20+
import com.fasterxml.jackson.databind.ObjectMapper;
21+
import org.elasticsearch.script.Script;
22+
23+
import java.util.Map;
24+
25+
public class ScriptParser {
26+
27+
public static Script parseScript(String scriptJson) throws JsonProcessingException {
28+
29+
Map<String, Object> map = ScriptParser.parseSchemaStringAsJson(scriptJson);
30+
31+
return Script.parse(map);
32+
}
33+
34+
private static Map<String, Object> parseSchemaStringAsJson(String scriptJson)
35+
throws JsonProcessingException {
36+
37+
ObjectMapper objectMapper = new ObjectMapper();
38+
39+
Map<String, Object> scriptConverted;
40+
41+
scriptConverted = objectMapper.readValue(
42+
scriptJson, new TypeReference<Map<String, Object>>(){});
43+
44+
return scriptConverted;
45+
}
46+
}
Lines changed: 74 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,74 @@
1+
/*
2+
* Copyright 2018 Confluent Inc.
3+
*
4+
* Licensed under the Confluent Community License (the "License"); you may not use
5+
* this file except in compliance with the License. You may obtain a copy of the
6+
* License at
7+
*
8+
* http://www.confluent.io/confluent-community-license
9+
*
10+
* Unless required by applicable law or agreed to in writing, software
11+
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12+
* WARRANTIES OF ANY KIND, either express or implied. See the License for the
13+
* specific language governing permissions and limitations under the License.
14+
*/
15+
16+
package io.confluent.connect.elasticsearch.validator;
17+
18+
import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.WRITE_METHOD_CONFIG;
19+
import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.WriteMethod.SCRIPTED_UPSERT;
20+
21+
import com.fasterxml.jackson.core.JsonProcessingException;
22+
import io.confluent.connect.elasticsearch.util.ScriptParser;
23+
import java.util.ArrayList;
24+
import java.util.List;
25+
import java.util.Map;
26+
import org.apache.kafka.common.config.ConfigDef;
27+
import org.apache.kafka.common.config.ConfigException;
28+
import org.elasticsearch.script.Script;
29+
30+
public class ScriptValidator implements ConfigDef.Validator, ConfigDef.Recommender {
31+
32+
@Override
33+
@SuppressWarnings("unchecked")
34+
public void ensureValid(String name, Object value) {
35+
36+
if (value == null) {
37+
return;
38+
}
39+
40+
String script = (String) value;
41+
42+
try {
43+
Script parsedScript = ScriptParser.parseScript(script);
44+
45+
if (parsedScript.getIdOrCode() == null) {
46+
throw new ConfigException(name, script, "The specified script is missing code");
47+
} else if (parsedScript.getLang() == null) {
48+
throw new ConfigException(name, script, "The specified script is missing lang");
49+
}
50+
51+
} catch (JsonProcessingException jsonProcessingException) {
52+
throw new ConfigException(
53+
name, script, "The specified script is not a valid Elasticsearch painless script");
54+
}
55+
}
56+
57+
@Override
58+
public String toString() {
59+
return "A valid script that is able to be parsed";
60+
}
61+
62+
@Override
63+
public List<Object> validValues(String name, Map<String, Object> parsedConfig) {
64+
if (!parsedConfig.get(WRITE_METHOD_CONFIG).equals(SCRIPTED_UPSERT)) {
65+
return new ArrayList<>();
66+
}
67+
return null;
68+
}
69+
70+
@Override
71+
public boolean visible(String name, Map<String, Object> parsedConfig) {
72+
return parsedConfig.get(WRITE_METHOD_CONFIG).equals(SCRIPTED_UPSERT.name());
73+
}
74+
}

src/test/java/io/confluent/connect/elasticsearch/DataConverterTest.java

Lines changed: 28 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@
2222
import org.apache.kafka.connect.sink.SinkRecord;
2323
import org.elasticsearch.action.delete.DeleteRequest;
2424
import org.elasticsearch.action.index.IndexRequest;
25+
import org.elasticsearch.action.update.UpdateRequest;
2526
import org.elasticsearch.index.VersionType;
2627
import org.junit.Before;
2728
import org.junit.Test;
@@ -39,8 +40,10 @@
3940
import static io.confluent.connect.elasticsearch.DataConverter.TIMESTAMP_FIELD;
4041
import static org.junit.Assert.assertEquals;
4142
import static org.junit.Assert.assertFalse;
43+
import static org.junit.Assert.assertNotNull;
4244
import static org.junit.Assert.assertNull;
4345
import static org.junit.Assert.assertThrows;
46+
import static org.junit.Assert.assertTrue;
4447
import static org.junit.Assert.fail;
4548

4649
public class DataConverterTest {
@@ -572,6 +575,31 @@ public void testDoNotAddExternalVersioningIfDataStream() {
572575
assertEquals(VersionType.INTERNAL, actualRecord.versionType());
573576
}
574577

578+
@Test
579+
public void upsertScript() {
580+
props.put(ElasticsearchSinkConnectorConfig.COMPACT_MAP_ENTRIES_CONFIG, "true");
581+
props.put(ElasticsearchSinkConnectorConfig.IGNORE_KEY_CONFIG, "false");
582+
props.put(ElasticsearchSinkConnectorConfig.IGNORE_SCHEMA_CONFIG, "false");
583+
props.put(ElasticsearchSinkConnectorConfig.WRITE_METHOD_CONFIG, ElasticsearchSinkConnectorConfig.WriteMethod.SCRIPTED_UPSERT.name());
584+
props.put(
585+
ElasticsearchSinkConnectorConfig.UPSERT_SCRIPT_CONFIG,
586+
"{\"lang\":\"painless\",\"source\":\"if ( ctx.op == 'create' ) ctx._source.counter = params.count} else {ctx._source.counter += params.count}\",\"params\":{\"count\":4}}");
587+
props.put(ElasticsearchSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, BehaviorOnNullValues.FAIL.name());
588+
converter = new DataConverter(new ElasticsearchSinkConnectorConfig(props));
589+
Schema preProcessedSchema = converter.preProcessSchema(schema);
590+
Struct struct = new Struct(preProcessedSchema).put("string", "myValue");
591+
SinkRecord sinkRecord = createSinkRecordWithValue(struct);
592+
593+
UpdateRequest actualRecord = (UpdateRequest) converter.convertRecord(sinkRecord, index);
594+
595+
assertNotNull(actualRecord.script());
596+
assertEquals("if ( ctx.op == 'create' ) ctx._source.counter = params.count} else {ctx._source.counter += params.count}", actualRecord.script().getIdOrCode());
597+
assertEquals("painless", actualRecord.script().getLang());
598+
assertEquals(4, actualRecord.script().getParams().getOrDefault("count", 0));
599+
assertTrue(actualRecord.scriptedUpsert());
600+
assertNotNull(actualRecord.doc());
601+
}
602+
575603
private void configureDataStream() {
576604
props.put(ElasticsearchSinkConnectorConfig.DATA_STREAM_TYPE_CONFIG, "logs");
577605
props.put(ElasticsearchSinkConnectorConfig.DATA_STREAM_DATASET_CONFIG, "dataset");

0 commit comments

Comments
 (0)