-
Notifications
You must be signed in to change notification settings - Fork 1k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
feat: add REGEXP_EXTRACT UDF (#4728)
* feat add REGEXP_EXTRACT UDF * plans
- Loading branch information
1 parent
7bf5c12
commit a25f0fb
Showing
9 changed files
with
646 additions
and
0 deletions.
There are no files selected for viewing
58 changes: 58 additions & 0 deletions
58
ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/string/RegexpExtract.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,58 @@ | ||
/* | ||
* 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 io.confluent.ksql.function.udf.Udf; | ||
import io.confluent.ksql.function.udf.UdfDescription; | ||
import io.confluent.ksql.function.udf.UdfParameter; | ||
import io.confluent.ksql.util.KsqlConstants; | ||
import java.util.regex.Matcher; | ||
import java.util.regex.Pattern; | ||
|
||
@UdfDescription( | ||
name = "regexp_extract", | ||
author = KsqlConstants.CONFLUENT_AUTHOR, | ||
description = "extract the first subtring matched by a regex pattern") | ||
public class RegexpExtract { | ||
@Udf(description = "Returns first substring of the input that matches the given regex pattern") | ||
public String regexpExtract( | ||
@UdfParameter(description = "The regex pattern") final String pattern, | ||
@UdfParameter(description = "The input string to apply regex on") final String input | ||
) { | ||
return regexpExtract(pattern, input, 0); | ||
} | ||
|
||
@Udf(description = "Returns the first substring of the " | ||
+ "input that matches the regex pattern and the capturing group number specified") | ||
public String regexpExtract( | ||
@UdfParameter(description = "The regex pattern") final String pattern, | ||
@UdfParameter(description = "The input string to apply regex on") final String input, | ||
@UdfParameter(description = "The capturing group number") final Integer group | ||
) { | ||
|
||
if (pattern == null || input == null || group == null) { | ||
return null; | ||
} | ||
|
||
final Pattern p = Pattern.compile(pattern); | ||
final Matcher m = p.matcher(input); | ||
|
||
if (group > m.groupCount()) { | ||
return null; | ||
} | ||
return m.find() ? m.group(group) : null; | ||
} | ||
} |
67 changes: 67 additions & 0 deletions
67
ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/string/RegexpExtractTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,67 @@ | ||
/* | ||
* 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 static org.hamcrest.Matchers.is; | ||
import static org.hamcrest.Matchers.nullValue; | ||
import static org.junit.Assert.assertEquals; | ||
import static org.junit.Assert.assertNull; | ||
import static org.junit.Assert.assertThat; | ||
|
||
import org.junit.Before; | ||
import org.junit.Test; | ||
|
||
public class RegexpExtractTest { | ||
|
||
private RegexpExtract udf; | ||
|
||
@Before | ||
public void setUp() { | ||
udf = new RegexpExtract(); | ||
} | ||
|
||
@Test | ||
public void shouldReturnNullOnNullValue() { | ||
assertNull(udf.regexpExtract(null, null)); | ||
assertNull(udf.regexpExtract(null, null, null)); | ||
assertNull(udf.regexpExtract(null, "", 1)); | ||
assertNull(udf.regexpExtract("some string", null, 1)); | ||
assertNull(udf.regexpExtract("some string", "", null)); | ||
} | ||
|
||
@Test | ||
public void shouldReturnSubstringWhenMatched() { | ||
assertEquals(udf.regexpExtract("e.*", "test string"), "est string"); | ||
assertEquals(udf.regexpExtract(".", "test string"), "t"); | ||
assertEquals(udf.regexpExtract("[AEIOU].{4}", "usEr nAme 1"), "Er nA"); | ||
} | ||
|
||
@Test | ||
public void shouldReturnNullWhenNoMatch() { | ||
assertNull(udf.regexpExtract("tst", "test string")); | ||
} | ||
|
||
@Test | ||
public void shouldReturnSubstringCapturedByGroupNumber() { | ||
assertEquals(udf.regexpExtract("(.*) (.*)", "test string", 1), "test"); | ||
assertEquals(udf.regexpExtract("(.*) (.*)", "test string", 2), "string"); | ||
} | ||
|
||
@Test | ||
public void shouldReturnNullIfGivenGroupNumberGreaterThanAvailableGroupNumbers() { | ||
assertNull(udf.regexpExtract("e", "test string", 3), null); | ||
} | ||
} |
146 changes: 146 additions & 0 deletions
146
.../test/resources/historical_plans/regex_-_extract_with_group/6.0.0_1584139441429/plan.json
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,146 @@ | ||
{ | ||
"plan" : [ { | ||
"@type" : "ksqlPlanV1", | ||
"statementText" : "CREATE STREAM TEST (INPUT_STRING STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", | ||
"ddlCommand" : { | ||
"@type" : "createStreamV1", | ||
"sourceName" : "TEST", | ||
"schema" : "`ROWKEY` STRING KEY, `INPUT_STRING` STRING", | ||
"keyField" : null, | ||
"timestampColumn" : null, | ||
"topicName" : "test_topic", | ||
"formats" : { | ||
"keyFormat" : { | ||
"format" : "KAFKA", | ||
"properties" : { } | ||
}, | ||
"valueFormat" : { | ||
"format" : "JSON", | ||
"properties" : { } | ||
}, | ||
"options" : [ ] | ||
}, | ||
"windowInfo" : null | ||
}, | ||
"queryPlan" : null | ||
}, { | ||
"@type" : "ksqlPlanV1", | ||
"statementText" : "CREATE STREAM OUTPUT AS SELECT REGEXP_EXTRACT('(.*) (.*)', TEST.INPUT_STRING, 2) EXTRACTED\nFROM TEST TEST\nEMIT CHANGES", | ||
"ddlCommand" : { | ||
"@type" : "createStreamV1", | ||
"sourceName" : "OUTPUT", | ||
"schema" : "`ROWKEY` STRING KEY, `EXTRACTED` STRING", | ||
"keyField" : null, | ||
"timestampColumn" : null, | ||
"topicName" : "OUTPUT", | ||
"formats" : { | ||
"keyFormat" : { | ||
"format" : "KAFKA", | ||
"properties" : { } | ||
}, | ||
"valueFormat" : { | ||
"format" : "JSON", | ||
"properties" : { } | ||
}, | ||
"options" : [ ] | ||
}, | ||
"windowInfo" : null | ||
}, | ||
"queryPlan" : { | ||
"sources" : [ "TEST" ], | ||
"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", | ||
"properties" : { } | ||
}, | ||
"valueFormat" : { | ||
"format" : "JSON", | ||
"properties" : { } | ||
}, | ||
"options" : [ ] | ||
}, | ||
"timestampColumn" : null, | ||
"sourceSchema" : "`ROWKEY` STRING KEY, `INPUT_STRING` STRING" | ||
}, | ||
"selectExpressions" : [ "REGEXP_EXTRACT('(.*) (.*)', INPUT_STRING, 2) AS EXTRACTED" ] | ||
}, | ||
"formats" : { | ||
"keyFormat" : { | ||
"format" : "KAFKA", | ||
"properties" : { } | ||
}, | ||
"valueFormat" : { | ||
"format" : "JSON", | ||
"properties" : { } | ||
}, | ||
"options" : [ ] | ||
}, | ||
"topicName" : "OUTPUT", | ||
"timestampColumn" : null | ||
}, | ||
"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.streams.state.dir" : "/var/folders/3g/w0lmttbn1xn5_15g9jlrb6440000gp/T/confluent1047019602767828263", | ||
"ksql.internal.topic.replicas" : "1", | ||
"ksql.insert.into.values.enabled" : "true", | ||
"ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", | ||
"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.streams.auto.commit.interval.ms" : "0", | ||
"ksql.metrics.extension" : null, | ||
"ksql.streams.topology.optimization" : "all", | ||
"ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", | ||
"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.any.key.name.enabled" : "false", | ||
"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" | ||
} | ||
} |
82 changes: 82 additions & 0 deletions
82
.../test/resources/historical_plans/regex_-_extract_with_group/6.0.0_1584139441429/spec.json
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,82 @@ | ||
{ | ||
"version" : "6.0.0", | ||
"timestamp" : 1584139441429, | ||
"schemas" : { | ||
"CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT<INPUT_STRING VARCHAR> NOT NULL", | ||
"CSAS_OUTPUT_0.OUTPUT" : "STRUCT<EXTRACTED VARCHAR> NOT NULL" | ||
}, | ||
"inputs" : [ { | ||
"topic" : "test_topic", | ||
"key" : "", | ||
"value" : { | ||
"input_string" : "steven zhang" | ||
} | ||
}, { | ||
"topic" : "test_topic", | ||
"key" : "", | ||
"value" : { | ||
"input_string" : "andy coates" | ||
} | ||
}, { | ||
"topic" : "test_topic", | ||
"key" : "", | ||
"value" : { | ||
"input_string" : "victoria xia" | ||
} | ||
}, { | ||
"topic" : "test_topic", | ||
"key" : "", | ||
"value" : { | ||
"input_string" : "apurva mehta" | ||
} | ||
}, { | ||
"topic" : "test_topic", | ||
"key" : "", | ||
"value" : { | ||
"input_string" : "agavra" | ||
} | ||
}, { | ||
"topic" : "test_topic", | ||
"key" : "", | ||
"value" : { | ||
"input_string" : null | ||
} | ||
} ], | ||
"outputs" : [ { | ||
"topic" : "OUTPUT", | ||
"key" : "", | ||
"value" : { | ||
"EXTRACTED" : "zhang" | ||
} | ||
}, { | ||
"topic" : "OUTPUT", | ||
"key" : "", | ||
"value" : { | ||
"EXTRACTED" : "coates" | ||
} | ||
}, { | ||
"topic" : "OUTPUT", | ||
"key" : "", | ||
"value" : { | ||
"EXTRACTED" : "xia" | ||
} | ||
}, { | ||
"topic" : "OUTPUT", | ||
"key" : "", | ||
"value" : { | ||
"EXTRACTED" : "mehta" | ||
} | ||
}, { | ||
"topic" : "OUTPUT", | ||
"key" : "", | ||
"value" : { | ||
"EXTRACTED" : null | ||
} | ||
}, { | ||
"topic" : "OUTPUT", | ||
"key" : "", | ||
"value" : { | ||
"EXTRACTED" : null | ||
} | ||
} ] | ||
} |
13 changes: 13 additions & 0 deletions
13
...c/test/resources/historical_plans/regex_-_extract_with_group/6.0.0_1584139441429/topology
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,13 @@ | ||
Topologies: | ||
Sub-topology: 0 | ||
Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) | ||
--> KSTREAM-TRANSFORMVALUES-0000000001 | ||
Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) | ||
--> Project | ||
<-- KSTREAM-SOURCE-0000000000 | ||
Processor: Project (stores: []) | ||
--> KSTREAM-SINK-0000000003 | ||
<-- KSTREAM-TRANSFORMVALUES-0000000001 | ||
Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) | ||
<-- Project | ||
|
Oops, something went wrong.