-
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 SIGN, REPLACE and INITCAP (#3189)
* feat: add sign function * feat: add INITCAP and REPLACE udfs * refactor: update docs, return int instead of double * docs: reword a sentence
- Loading branch information
Zara Lim
authored
Aug 14, 2019
1 parent
51312cc
commit ab67684
Showing
10 changed files
with
385 additions
and
0 deletions.
There are no files selected for viewing
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
65 changes: 65 additions & 0 deletions
65
ksql-engine/src/main/java/io/confluent/ksql/function/udf/math/Sign.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,65 @@ | ||
/* | ||
* 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.math; | ||
|
||
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; | ||
|
||
@SuppressWarnings("WeakerAccess") // Invoked via reflection | ||
@UdfDescription( | ||
name = "sign", | ||
author = KsqlConstants.CONFLUENT_AUTHOR, | ||
description = "The sign of a value." | ||
) | ||
public class Sign { | ||
@Udf(description = "Returns the sign of an INT value, denoted by 1, 0 or -1.") | ||
public Integer sign( | ||
@UdfParameter( | ||
value = "value", | ||
description = "The value to get the sign of." | ||
) final Integer value | ||
) { | ||
return value == null | ||
? null | ||
: Integer.signum(value); | ||
} | ||
|
||
@Udf(description = "Returns the sign of an BIGINT value, denoted by 1, 0 or -1.") | ||
public Integer sign( | ||
@UdfParameter( | ||
value = "value", | ||
description = "The value to get the sign of." | ||
) final Long value | ||
) { | ||
return value == null | ||
? null | ||
: Long.signum(value); | ||
} | ||
|
||
@Udf(description = "Returns the sign of an DOUBLE value, denoted by 1, 0 or -1.") | ||
public Integer sign( | ||
@UdfParameter( | ||
value = "value", | ||
description = "The value to get the sign of." | ||
) final Double value | ||
) { | ||
return value == null | ||
? null | ||
: (int) Math.signum(value); | ||
} | ||
} |
52 changes: 52 additions & 0 deletions
52
ksql-engine/src/main/java/io/confluent/ksql/function/udf/string/InitCap.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,52 @@ | ||
/* | ||
* Copyright 2019 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; | ||
|
||
@SuppressWarnings("unused") // Invoked via reflection. | ||
@UdfDescription(name = "initcap", | ||
author = KsqlConstants.CONFLUENT_AUTHOR, | ||
description = "Capitalizes the first letter of each word in a string and the rest lowercased." | ||
+ " Words are delimited by whitespace.") | ||
public class InitCap { | ||
@Udf(description = "Returns the string with the the first letter" | ||
+ " of each word capitalized and the rest lowercased") | ||
public String initcap( | ||
@UdfParameter( | ||
description = "The source string." | ||
+ " If null, then function returns null.") final String str) { | ||
if (str == null) { | ||
return null; | ||
} | ||
|
||
final Pattern pattern = Pattern.compile("[^\\s]+\\s*"); | ||
final Matcher matcher = pattern.matcher(str.toLowerCase()); | ||
String initCapped = ""; | ||
while (matcher.find()) { | ||
final String part = matcher.group(); | ||
initCapped = initCapped.concat(part.substring(0, 1).toUpperCase() + part.substring(1)); | ||
} | ||
|
||
return initCapped; | ||
} | ||
} |
44 changes: 44 additions & 0 deletions
44
ksql-engine/src/main/java/io/confluent/ksql/function/udf/string/Replace.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,44 @@ | ||
/* | ||
* Copyright 2019 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; | ||
|
||
@SuppressWarnings("unused") // Invoked via reflection. | ||
@UdfDescription(name = "replace", | ||
author = KsqlConstants.CONFLUENT_AUTHOR, | ||
description = "Replaces all occurances of a substring in a string with a new substring.") | ||
public class Replace { | ||
@Udf(description = "Returns a new string with all occurences of oldStr in str with newStr") | ||
public String replace( | ||
@UdfParameter( | ||
description = "The source string. If null, then function returns null.") final String str, | ||
@UdfParameter( | ||
description = "The substring to replace." | ||
+ " If null, then function returns null.") final String oldStr, | ||
@UdfParameter( | ||
description = "The string to replace the old substrings with." | ||
+ " If null, then function returns null.") final String newStr) { | ||
if (str == null || oldStr == null || newStr == null) { | ||
return null; | ||
} | ||
|
||
return str.replace(oldStr, newStr); | ||
} | ||
} |
59 changes: 59 additions & 0 deletions
59
ksql-engine/src/test/java/io/confluent/ksql/function/udf/math/SignTest.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,59 @@ | ||
/* | ||
* Copyright 2019 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.math; | ||
|
||
import org.junit.Before; | ||
import org.junit.Test; | ||
|
||
import static org.hamcrest.Matchers.is; | ||
import static org.hamcrest.Matchers.nullValue; | ||
import static org.junit.Assert.*; | ||
|
||
public class SignTest { | ||
private Sign udf; | ||
|
||
@Before | ||
public void setUp() { | ||
udf = new Sign(); | ||
} | ||
|
||
@Test | ||
public void shouldHandleNull() { | ||
assertThat(udf.sign((Integer)null), is(nullValue())); | ||
assertThat(udf.sign((Long)null), is(nullValue())); | ||
assertThat(udf.sign((Double)null), is(nullValue())); | ||
} | ||
|
||
@Test | ||
public void shouldHandleNegative() { | ||
assertThat(udf.sign(-10.5), is(-1)); | ||
assertThat(udf.sign(-10), is(-1)); | ||
assertThat(udf.sign(-1L), is(-1)); | ||
} | ||
|
||
@Test | ||
public void shouldHandleZero() { | ||
assertThat(udf.sign(0.0), is(0)); | ||
assertThat(udf.sign(0), is(0)); | ||
} | ||
|
||
@Test | ||
public void shouldHandlePositive() { | ||
assertThat(udf.sign(1), is(1)); | ||
assertThat(udf.sign(1L), is(1)); | ||
assertThat(udf.sign(1.5), is(1)); | ||
} | ||
} |
30 changes: 30 additions & 0 deletions
30
ksql-engine/src/test/java/io/confluent/ksql/function/udf/string/InitCapTest.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,30 @@ | ||
package io.confluent.ksql.function.udf.string; | ||
|
||
import org.junit.Before; | ||
import org.junit.Test; | ||
|
||
import static org.hamcrest.Matchers.is; | ||
import static org.hamcrest.Matchers.isEmptyOrNullString; | ||
import static org.junit.Assert.*; | ||
|
||
public class InitCapTest { | ||
private InitCap udf; | ||
|
||
@Before | ||
public void setUp() { | ||
udf = new InitCap(); | ||
} | ||
|
||
@Test | ||
public void shouldHandleNull() { | ||
assertThat(udf.initcap(null), isEmptyOrNullString()); | ||
} | ||
|
||
@Test | ||
public void shouldInitCap() { | ||
assertThat(udf.initcap("worD"), is("Word")); | ||
assertThat(udf.initcap("a"), is("A")); | ||
assertThat(udf.initcap("the Quick br0wn fOx"), is("The Quick Br0wn Fox")); | ||
assertThat(udf.initcap("spacing should be preserved"), is("Spacing Should Be Preserved")); | ||
} | ||
} |
47 changes: 47 additions & 0 deletions
47
ksql-engine/src/test/java/io/confluent/ksql/function/udf/string/ReplaceTest.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,47 @@ | ||
/* | ||
* Copyright 2019 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 org.junit.Before; | ||
import org.junit.Test; | ||
|
||
import static org.hamcrest.Matchers.is; | ||
import static org.hamcrest.Matchers.isEmptyOrNullString; | ||
import static org.junit.Assert.*; | ||
|
||
public class ReplaceTest { | ||
private Replace udf; | ||
|
||
@Before | ||
public void setUp() { | ||
udf = new Replace(); | ||
} | ||
|
||
@Test | ||
public void shouldHandleNull() { | ||
assertThat(udf.replace(null, "foo", "bar"), isEmptyOrNullString()); | ||
assertThat(udf.replace("foo", null, "bar"), isEmptyOrNullString()); | ||
assertThat(udf.replace("foo", "bar", null), isEmptyOrNullString()); | ||
} | ||
|
||
@Test | ||
public void shouldReplace() { | ||
assertThat(udf.replace("foobar", "foo", "bar"), is("barbar")); | ||
assertThat(udf.replace("foobar", "fooo", "bar"), is("foobar")); | ||
assertThat(udf.replace("foobar", "o", ""), is("fbar")); | ||
assertThat(udf.replace("abc", "", "n"), is("nanbncn")); | ||
} | ||
} |
28 changes: 28 additions & 0 deletions
28
ksql-functional-tests/src/test/resources/query-validation-tests/initcap.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,28 @@ | ||
{ | ||
"comments": [ | ||
"Tests covering the use of the INITCAP function." | ||
], | ||
"tests": [ | ||
{ | ||
"name": "do initcap", | ||
"format": ["JSON"], | ||
"properties": { | ||
"ksql.functions.substring.legacy.args": false | ||
}, | ||
"statements": [ | ||
"CREATE STREAM TEST (source VARCHAR) WITH (kafka_topic='test_topic', value_format='JSON');", | ||
"CREATE STREAM OUTPUT AS SELECT INITCAP(source) AS INITCAP FROM TEST;" | ||
], | ||
"inputs": [ | ||
{"topic": "test_topic", "value": {"source": "some_string"}}, | ||
{"topic": "test_topic", "value": {"source": null}}, | ||
{"topic": "test_topic", "value": {"source": "the Quick br0wn fOx"}} | ||
], | ||
"outputs": [ | ||
{"topic": "OUTPUT", "value": {"INITCAP":"Some_string"}}, | ||
{"topic": "OUTPUT", "value": {"INITCAP":null}}, | ||
{"topic": "OUTPUT", "value": {"INITCAP":"The Quick Br0wn Fox"}} | ||
] | ||
} | ||
] | ||
} |
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
Oops, something went wrong.