-
Notifications
You must be signed in to change notification settings - Fork 1k
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: new split_to_map udf #5563
Changes from 4 commits
532dd9c
cadab2b
62da1e8
6381324
2e24cc4
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change | ||||
---|---|---|---|---|---|---|
|
@@ -701,6 +701,23 @@ If the delimiter is found at the beginning or end | |||||
of the string, or there are contiguous delimiters, | ||||||
then an empty space is added to the array. | ||||||
|
||||||
### `SPLIT_TO_MAP` | ||||||
|
||||||
```sql | ||||||
SPLIT_TO_MAP(input, entryDelimiter, kvDelimiter) | ||||||
``` | ||||||
|
||||||
Splits a string into key-value pairs and creates a map from them. The | ||||||
'entryDelimiter' splits the string into key-value pairs which are then split by 'kvDelimiter'. If the same key is present multiple times in the input, the latest value for that key is returned. | ||||||
|
||||||
Returns NULL f the input text is NULL. | ||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. arghhh! thanks Jim :) |
||||||
Returns NULL if either of the delimiters is NULL or an empty string. | ||||||
|
||||||
Example: | ||||||
```sql | ||||||
SPLIT_TO_MAP('apple':='green'/'cherry':='red', '/', ':=') => { 'apple':'green', 'cherry':'red'} | ||||||
``` | ||||||
|
||||||
### `SUBSTRING` | ||||||
|
||||||
```sql | ||||||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,61 @@ | ||
/* | ||
* Copyright 2020 Confluent Inc. | ||
* | ||
* Licensed under the Confluent Community License (the "License"; you may not use this file except | ||
* in compliance with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.confluent.io/confluent-community-license | ||
* | ||
* Unless required by applicable law or agreed to in writing, software distributed under the License | ||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and limitations under the | ||
* License. | ||
*/ | ||
|
||
package io.confluent.ksql.function.udf.string; | ||
|
||
import com.google.common.base.Splitter; | ||
import io.confluent.ksql.function.udf.Udf; | ||
import io.confluent.ksql.function.udf.UdfDescription; | ||
import io.confluent.ksql.function.udf.UdfParameter; | ||
import java.util.Map; | ||
import java.util.stream.Collectors; | ||
import java.util.stream.StreamSupport; | ||
|
||
@UdfDescription( | ||
name = "split_to_map", | ||
description = "Splits a string into key-value pairs and creates a map from them. The " | ||
+ "'entryDelimiter' splits the string into key-value pairs which are then split by " | ||
+ "'kvDelimiter'. If the same key is present multiple times in the input, the latest " | ||
+ "value for that key is returned. Returns NULL f the input text or either of the " | ||
+ "delimiters is NULL.") | ||
public class SplitToMap { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We may also want to have There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. i'll add it to my udf backlog ;) - although i'm less concerned about that from a use-case perspective as you can always construct almost-arbitrary json output by using structs/maps/arrays if you need that for a downstream system. The primary motivator for this one is when you get, for example, some encoded message from a mainframe MQ system that needs to be parsed out this way |
||
@Udf | ||
public Map<String, String> splitToMap( | ||
@UdfParameter( | ||
description = "Separator string and values to join") final String input, | ||
@UdfParameter( | ||
description = "Separator string and values to join") final String entryDelimiter, | ||
@UdfParameter( | ||
description = "Separator string and values to join") final String kvDelimiter) { | ||
|
||
if (input == null || entryDelimiter == null || kvDelimiter == null) { | ||
return null; | ||
} | ||
|
||
if (entryDelimiter.isEmpty() || kvDelimiter.isEmpty() || entryDelimiter.equals(kvDelimiter)) { | ||
return null; | ||
} | ||
|
||
final Iterable<String> entries = Splitter.on(entryDelimiter).omitEmptyStrings().split(input); | ||
agavra marked this conversation as resolved.
Show resolved
Hide resolved
|
||
final Map<String, String> output = StreamSupport.stream(entries.spliterator(), false) | ||
.filter(e -> e.contains(kvDelimiter)) | ||
.map(kv -> Splitter.on(kvDelimiter).split(kv).iterator()) | ||
.collect(Collectors.toMap( | ||
kvIter -> kvIter.next(), | ||
kvIter -> kvIter.next(), | ||
(v1, v2) -> v2)); | ||
|
||
return output; | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,123 @@ | ||
/* | ||
* Copyright 2018 Confluent Inc. | ||
* | ||
* Licensed under the Confluent Community License (the "License"); you may not use | ||
* this file except in compliance with the License. You may obtain a copy of the | ||
* License at | ||
* | ||
* http://www.confluent.io/confluent-community-license | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT | ||
* WARRANTIES OF ANY KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations under the License. | ||
*/ | ||
package io.confluent.ksql.function.udf.string; | ||
|
||
import static org.hamcrest.MatcherAssert.assertThat; | ||
import static org.hamcrest.Matchers.equalTo; | ||
import static org.hamcrest.Matchers.hasEntry; | ||
import static org.hamcrest.Matchers.is; | ||
import static org.hamcrest.Matchers.nullValue; | ||
|
||
import java.util.Collections; | ||
import java.util.Map; | ||
import org.junit.Test; | ||
|
||
public class SplitToMapTest { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can you add a test with whitespace? what do we want the behavior to be when there is whitespace (e.g. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. good idea, test added! |
||
private final static SplitToMap udf = new SplitToMap(); | ||
|
||
@Test | ||
public void shouldSplitStringByGivenDelimiterChars() { | ||
Map<String, String> result = udf.splitToMap("foo=apple;bar=cherry", ";", "="); | ||
assertThat(result, hasEntry("foo", "apple")); | ||
assertThat(result, hasEntry("bar", "cherry")); | ||
assertThat(result.size(), equalTo(2)); | ||
} | ||
|
||
@Test | ||
public void shouldSplitStringGivenMultiCharDelimiters() { | ||
Map<String, String> result = udf.splitToMap("foo:=apple||bar:=cherry", "||", ":="); | ||
assertThat(result, hasEntry("foo", "apple")); | ||
assertThat(result, hasEntry("bar", "cherry")); | ||
assertThat(result.size(), equalTo(2)); | ||
} | ||
|
||
@Test | ||
public void shouldSplitStringWithOnlyOneEntry() { | ||
Map<String, String> result = udf.splitToMap("foo=apple", ";", "="); | ||
assertThat(result, hasEntry("foo", "apple")); | ||
assertThat(result.size(), equalTo(1)); | ||
} | ||
|
||
@Test | ||
public void shouldDropEmptyEntriesFromSplit() { | ||
Map<String, String> result = udf.splitToMap("/foo:=apple//bar:=cherry/", "/", ":="); | ||
assertThat(result, hasEntry("foo", "apple")); | ||
assertThat(result, hasEntry("bar", "cherry")); | ||
assertThat(result.size(), equalTo(2)); | ||
} | ||
|
||
@Test | ||
public void shouldDropEntriesWithoutKeyAndValue() { | ||
Map<String, String> result = udf.splitToMap("foo:=apple/cherry", "/", ":="); | ||
assertThat(result, hasEntry("foo", "apple")); | ||
assertThat(result.size(), equalTo(1)); | ||
} | ||
|
||
@Test | ||
public void shouldReturnEmptyForInputWithoutDelimiters() { | ||
Map<String, String> result = udf.splitToMap("cherry", "/", ":="); | ||
assertThat(result, is(Collections.EMPTY_MAP)); | ||
} | ||
|
||
@Test | ||
public void shouldReturnEmptyForEmptyInput() { | ||
Map<String, String> result = udf.splitToMap("", "/", ":="); | ||
assertThat(result, is(Collections.EMPTY_MAP)); | ||
} | ||
|
||
@Test | ||
public void shouldKeepLatestValueForDuplicateKey() { | ||
Map<String, String> result = udf.splitToMap("/foo:=apple/foo:=cherry/", "/", ":="); | ||
assertThat(result, hasEntry("foo", "cherry")); | ||
assertThat(result.size(), equalTo(1)); | ||
} | ||
|
||
@Test | ||
public void shouldReturnNullOnNullInputString() { | ||
Map<String, String> result = udf.splitToMap(null, "/", ":="); | ||
assertThat(result, is(nullValue())); | ||
} | ||
|
||
@Test | ||
public void shouldReturnNullOnSameDelimiterChars() { | ||
Map<String, String> result = udf.splitToMap("foo:=apple/bar:=cherry", "/", "/"); | ||
assertThat(result, is(nullValue())); | ||
} | ||
|
||
@Test | ||
public void shouldReturnNullOnEmptyEntryDelimiter() { | ||
Map<String, String> result = udf.splitToMap("foo:=apple/bar:=cherry", "", ":="); | ||
assertThat(result, is(nullValue())); | ||
} | ||
|
||
@Test | ||
public void shouldReturnNullOnNullEntryDelimiter() { | ||
Map<String, String> result = udf.splitToMap("foo:=apple/bar:=cherry", null, ":="); | ||
assertThat(result, is(nullValue())); | ||
} | ||
|
||
@Test | ||
public void shouldReturnNullOnEmptyValueDelimiter() { | ||
Map<String, String> result = udf.splitToMap("foo:=apple/bar:=cherry", "/", ""); | ||
assertThat(result, is(nullValue())); | ||
} | ||
|
||
@Test | ||
public void shouldReturnNullOnNullValueDelimiter() { | ||
Map<String, String> result = udf.splitToMap("foo:=apple/bar:=cherry", "/", null); | ||
assertThat(result, is(nullValue())); | ||
} | ||
|
||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,126 @@ | ||
{ | ||
"plan" : [ { | ||
"@type" : "ksqlPlanV1", | ||
"statementText" : "CREATE STREAM INPUT (ID STRING KEY, INPUT STRING, ENTRYDELIM STRING, KVDELIM STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", | ||
"ddlCommand" : { | ||
"@type" : "createStreamV1", | ||
"sourceName" : "INPUT", | ||
"schema" : "`ID` STRING KEY, `INPUT` STRING, `ENTRYDELIM` STRING, `KVDELIM` STRING", | ||
"topicName" : "test_topic", | ||
"formats" : { | ||
"keyFormat" : { | ||
"format" : "KAFKA" | ||
}, | ||
"valueFormat" : { | ||
"format" : "JSON" | ||
} | ||
} | ||
} | ||
}, { | ||
"@type" : "ksqlPlanV1", | ||
"statementText" : "CREATE STREAM OUTPUT AS SELECT\n INPUT.ID ID,\n SPLIT_TO_MAP(INPUT.INPUT, INPUT.ENTRYDELIM, INPUT.KVDELIM) RESULT\nFROM INPUT INPUT\nEMIT CHANGES", | ||
"ddlCommand" : { | ||
"@type" : "createStreamV1", | ||
"sourceName" : "OUTPUT", | ||
"schema" : "`ID` STRING KEY, `RESULT` MAP<STRING, STRING>", | ||
"topicName" : "OUTPUT", | ||
"formats" : { | ||
"keyFormat" : { | ||
"format" : "KAFKA" | ||
}, | ||
"valueFormat" : { | ||
"format" : "JSON" | ||
} | ||
} | ||
}, | ||
"queryPlan" : { | ||
"sources" : [ "INPUT" ], | ||
"sink" : "OUTPUT", | ||
"physicalPlan" : { | ||
"@type" : "streamSinkV1", | ||
"properties" : { | ||
"queryContext" : "OUTPUT" | ||
}, | ||
"source" : { | ||
"@type" : "streamSelectV1", | ||
"properties" : { | ||
"queryContext" : "Project" | ||
}, | ||
"source" : { | ||
"@type" : "streamSourceV1", | ||
"properties" : { | ||
"queryContext" : "KsqlTopic/Source" | ||
}, | ||
"topicName" : "test_topic", | ||
"formats" : { | ||
"keyFormat" : { | ||
"format" : "KAFKA" | ||
}, | ||
"valueFormat" : { | ||
"format" : "JSON" | ||
} | ||
}, | ||
"sourceSchema" : "`ID` STRING KEY, `INPUT` STRING, `ENTRYDELIM` STRING, `KVDELIM` STRING" | ||
}, | ||
"keyColumnNames" : [ "ID" ], | ||
"selectExpressions" : [ "SPLIT_TO_MAP(INPUT, ENTRYDELIM, KVDELIM) AS RESULT" ] | ||
}, | ||
"formats" : { | ||
"keyFormat" : { | ||
"format" : "KAFKA" | ||
}, | ||
"valueFormat" : { | ||
"format" : "JSON" | ||
} | ||
}, | ||
"topicName" : "OUTPUT" | ||
}, | ||
"queryId" : "CSAS_OUTPUT_0" | ||
} | ||
} ], | ||
"configs" : { | ||
"ksql.extension.dir" : "ext", | ||
"ksql.streams.cache.max.bytes.buffering" : "0", | ||
"ksql.security.extension.class" : null, | ||
"ksql.transient.prefix" : "transient_", | ||
"ksql.persistence.wrap.single.values" : "true", | ||
"ksql.authorization.cache.expiry.time.secs" : "30", | ||
"ksql.schema.registry.url" : "", | ||
"ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", | ||
"ksql.output.topic.name.prefix" : "", | ||
"ksql.streams.auto.offset.reset" : "earliest", | ||
"ksql.query.pull.enable.standby.reads" : "false", | ||
"ksql.connect.url" : "http://localhost:8083", | ||
"ksql.service.id" : "some.ksql.service.id", | ||
"ksql.internal.topic.min.insync.replicas" : "1", | ||
"ksql.streams.shutdown.timeout.ms" : "300000", | ||
"ksql.internal.topic.replicas" : "1", | ||
"ksql.insert.into.values.enabled" : "true", | ||
"ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", | ||
"ksql.query.pull.max.qps" : "2147483647", | ||
"ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", | ||
"ksql.access.validator.enable" : "auto", | ||
"ksql.streams.bootstrap.servers" : "localhost:0", | ||
"ksql.streams.commit.interval.ms" : "2000", | ||
"ksql.metric.reporters" : "", | ||
"ksql.query.pull.metrics.enabled" : "false", | ||
"ksql.streams.auto.commit.interval.ms" : "0", | ||
"ksql.metrics.extension" : null, | ||
"ksql.streams.topology.optimization" : "all", | ||
"ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", | ||
"ksql.streams.num.stream.threads" : "4", | ||
"ksql.timestamp.throw.on.invalid" : "false", | ||
"ksql.authorization.cache.max.entries" : "10000", | ||
"ksql.metrics.tags.custom" : "", | ||
"ksql.pull.queries.enable" : "true", | ||
"ksql.udfs.enabled" : "true", | ||
"ksql.udf.enable.security.manager" : "true", | ||
"ksql.connect.worker.config" : "", | ||
"ksql.sink.window.change.log.additional.retention" : "1000000", | ||
"ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", | ||
"ksql.udf.collect.metrics" : "false", | ||
"ksql.persistent.prefix" : "query_", | ||
"ksql.query.persistent.active.limit" : "2147483647", | ||
"ksql.error.classifier.regex" : "" | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.