Re: [PR] [FLINK-32706][table] Add built-in SPLIT_STRING function [flink]

2024-05-07 Thread via GitHub


dawidwys commented on PR #24365:
URL: https://github.com/apache/flink/pull/24365#issuecomment-2097889690

   > Now we use splitByWholeSeparatorPreserveAllTokens, when delimiter is 
empty, it will return entire string.
   
   Actually it splits by whitespaces: 
https://github.com/apache/flink/blob/8e5220b288e49c99333a4bc8ef7e3d5d27193921/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/binary/BinaryStringDataUtil.java#L107
   
   > I think we can change the description,
   
   Let's first agree what is the desired behaviour.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-32706][table] Add built-in SPLIT_STRING function [flink]

2024-05-06 Thread via GitHub


hanyuzheng7 commented on code in PR #24365:
URL: https://github.com/apache/flink/pull/24365#discussion_r1591189575


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/SplitFunction.java:
##
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.table.runtime.functions.scalar;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.data.binary.BinaryStringDataUtil;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.SpecializedFunction;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+/** Implementation of {@link BuiltInFunctionDefinitions#SPLIT}. */
+@Internal
+public class SplitFunction extends BuiltInScalarFunction {
+public SplitFunction(SpecializedFunction.SpecializedContext context) {
+super(BuiltInFunctionDefinitions.SPLIT, context);
+}
+
+public @Nullable ArrayData eval(@Nullable StringData string, @Nullable 
StringData delimiter) {
+try {
+if (string == null || delimiter == null) {
+return null;
+}
+BinaryStringData[] binaryStringData =
+
BinaryStringDataUtil.splitByWholeSeparatorPreserveAllTokens(
+(BinaryStringData) string, (BinaryStringData) 
delimiter);
+Object[] objects = Arrays.stream(binaryStringData).toArray();
+List objectList = new ArrayList<>(Arrays.asList(objects));

Review Comment:
   Ok, directly pass `binaryStringData` to `GenericArrayData` now



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-32706][table] Add built-in SPLIT_STRING function [flink]

2024-05-06 Thread via GitHub


hanyuzheng7 commented on PR #24365:
URL: https://github.com/apache/flink/pull/24365#issuecomment-2096212517

   @dawidwys @MartijnVisser I think I should change the description,  If the 
delimiter is empty, we will return entire string.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-32706][table] Add built-in SPLIT_STRING function [flink]

2024-05-06 Thread via GitHub


dawidwys commented on code in PR #24365:
URL: https://github.com/apache/flink/pull/24365#discussion_r1591045316


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/SplitFunction.java:
##
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.table.runtime.functions.scalar;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.data.binary.BinaryStringDataUtil;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.SpecializedFunction;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+/** Implementation of {@link BuiltInFunctionDefinitions#SPLIT}. */
+@Internal
+public class SplitFunction extends BuiltInScalarFunction {
+public SplitFunction(SpecializedFunction.SpecializedContext context) {
+super(BuiltInFunctionDefinitions.SPLIT, context);
+}
+
+public @Nullable ArrayData eval(@Nullable StringData string, @Nullable 
StringData delimiter) {
+try {
+if (string == null || delimiter == null) {
+return null;
+}
+BinaryStringData[] binaryStringData =
+
BinaryStringDataUtil.splitByWholeSeparatorPreserveAllTokens(
+(BinaryStringData) string, (BinaryStringData) 
delimiter);
+Object[] objects = Arrays.stream(binaryStringData).toArray();
+List objectList = new ArrayList<>(Arrays.asList(objects));

Review Comment:
   This is entirely unnecessary conversion. You can just pass 
`binaryStringData` to `GenericArrayData` ctor directly.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-32706][table] Add built-in SPLIT_STRING function [flink]

2024-05-06 Thread via GitHub


dawidwys commented on PR #24365:
URL: https://github.com/apache/flink/pull/24365#issuecomment-2096053400

   > If the delimiter is empty, every character in the string is split
   
   Is that a requirement? As far as I can tell ksqlDB is the only engine that 
has that behaviour. Moreover after switching to 
`splitByWholeSeparatorPreserveAllTokens` you do not guarantee this behaviour 
anymore. 
   
   @MartijnVisser What do you think should be the behaviour for an empty 
delimiter?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-32706][table] Add built-in SPLIT_STRING function [flink]

2024-04-25 Thread via GitHub


hanyuzheng7 commented on code in PR #24365:
URL: https://github.com/apache/flink/pull/24365#discussion_r1580488725


##
flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java:
##
@@ -409,6 +409,18 @@ ANY, and(logical(LogicalTypeRoot.BOOLEAN), LITERAL)
 
"org.apache.flink.table.runtime.functions.scalar.ArrayMinFunction")
 .build();
 
+public static final BuiltInFunctionDefinition SPLIT =
+BuiltInFunctionDefinition.newBuilder()
+.name("SPLIT")
+.kind(SCALAR)
+.inputTypeStrategy(
+sequence(
+
logical(LogicalTypeFamily.CHARACTER_STRING),
+
logical(LogicalTypeFamily.CHARACTER_STRING)))
+
.outputTypeStrategy(forceNullable(explicit(DataTypes.ARRAY(STRING()

Review Comment:
   Ok, I use `nullableIfArgs(explicit(DataTypes.ARRAY(STRING(` now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-32706][table] Add built-in SPLIT_STRING function [flink]

2024-04-25 Thread via GitHub


hanyuzheng7 commented on code in PR #24365:
URL: https://github.com/apache/flink/pull/24365#discussion_r1580488012


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/SplitFunction.java:
##
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.table.runtime.functions.scalar;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.SpecializedFunction;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Implementation of {@link BuiltInFunctionDefinitions#SPLIT}. */
+@Internal
+public class SplitFunction extends BuiltInScalarFunction {
+public SplitFunction(SpecializedFunction.SpecializedContext context) {
+super(BuiltInFunctionDefinitions.SPLIT, context);
+}
+
+public @Nullable ArrayData eval(@Nullable StringData string, @Nullable 
StringData delimiter) {

Review Comment:
   Ok, I have already used `splitByWholeSeparatorPreserveAllTokens`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-32706][table] Add built-in SPLIT_STRING function [flink]

2024-04-25 Thread via GitHub


dawidwys commented on code in PR #24365:
URL: https://github.com/apache/flink/pull/24365#discussion_r1579590240


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/SplitFunction.java:
##
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.table.runtime.functions.scalar;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.SpecializedFunction;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Implementation of {@link BuiltInFunctionDefinitions#SPLIT}. */
+@Internal
+public class SplitFunction extends BuiltInScalarFunction {
+public SplitFunction(SpecializedFunction.SpecializedContext context) {
+super(BuiltInFunctionDefinitions.SPLIT, context);
+}
+
+public @Nullable ArrayData eval(@Nullable StringData string, @Nullable 
StringData delimiter) {

Review Comment:
   Could we reuse 
https://github.com/apache/flink/blob/8e5220b288e49c99333a4bc8ef7e3d5d27193921/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/binary/BinaryStringDataUtil.java#L96
 and or `String#split`?



##
flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java:
##
@@ -409,6 +409,18 @@ ANY, and(logical(LogicalTypeRoot.BOOLEAN), LITERAL)
 
"org.apache.flink.table.runtime.functions.scalar.ArrayMinFunction")
 .build();
 
+public static final BuiltInFunctionDefinition SPLIT =
+BuiltInFunctionDefinition.newBuilder()
+.name("SPLIT")
+.kind(SCALAR)
+.inputTypeStrategy(
+sequence(
+
logical(LogicalTypeFamily.CHARACTER_STRING),
+
logical(LogicalTypeFamily.CHARACTER_STRING)))
+
.outputTypeStrategy(forceNullable(explicit(DataTypes.ARRAY(STRING()

Review Comment:
   I think this `outputTypeStrategy` does not follow the description.
   
   The description says output can be `null` only if any of the input 
parameters are `null`. If both arguments are `NOT NULL` the result is also `NOT 
NULL`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-32706][table] Add built-in SPLIT_STRING function [flink]

2024-03-20 Thread via GitHub


hanyuzheng7 commented on code in PR #24365:
URL: https://github.com/apache/flink/pull/24365#discussion_r1532815010


##
flink-python/pyflink/table/expression.py:
##
@@ -1609,6 +1609,17 @@ def array_min(self) -> 'Expression':
 """
 return _unary_op("arrayMin")(self)
 
+def split(self, delimiter) -> 'Expression':

Review Comment:
   I just support Java one, so I think we don't need add python test(s) for 
`split`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-32706][table] Add built-in SPLIT_STRING function [flink]

2024-03-09 Thread via GitHub


jeyhunkarimov commented on code in PR #24365:
URL: https://github.com/apache/flink/pull/24365#discussion_r1518655288


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/SplitFunction.java:
##
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.table.runtime.functions.scalar;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.SpecializedFunction;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Implementation of {@link BuiltInFunctionDefinitions#SPLIT}. */
+@Internal
+public class SplitFunction extends BuiltInScalarFunction {
+public SplitFunction(SpecializedFunction.SpecializedContext context) {
+super(BuiltInFunctionDefinitions.SPLIT, context);
+}
+
+public @Nullable ArrayData eval(@Nullable StringData string, @Nullable 
StringData delimiter) {
+try {
+if (string == null || delimiter == null) {
+return null;
+}
+String string1 = string.toString();
+String delimiter1 = delimiter.toString();
+List resultList = new ArrayList<>();
+
+if (delimiter1.equals("")) {
+for (char c : string1.toCharArray()) {
+
resultList.add(BinaryStringData.fromString(String.valueOf(c)));
+}
+} else {
+int start = 0;
+int end = string1.indexOf(delimiter1);
+while (end != -1) {
+String substring = string1.substring(start, end);
+resultList.add(
+BinaryStringData.fromString(
+substring.isEmpty()
+? ""
+: substring)); // Added this check 
to handle consecutive
+// delimiters
+start = end + delimiter1.length();
+end = string1.indexOf(delimiter1, start);
+}
+String remaining = string1.substring(start);

Review Comment:
   What if the delimiter is found at the very end of the string, the remaining 
string will be empty, but you should still add an empty string to the 
resultList to indicate the presence of the delimiter at the end of the string. 
Or am I missing sth?



##
flink-python/pyflink/table/expression.py:
##
@@ -1609,6 +1609,17 @@ def array_min(self) -> 'Expression':
 """
 return _unary_op("arrayMin")(self)
 
+def split(self, delimiter) -> 'Expression':

Review Comment:
   Please add python test(s) for `split` as well



##
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CollectionFunctionsITCase.java:
##
@@ -1596,4 +1597,83 @@ private Stream arraySortTestCases() {
 },
 DataTypes.ARRAY(DataTypes.DATE(;
 }
+
+private Stream splitTestCases() {
+return Stream.of(
+TestSetSpec.forFunction(BuiltInFunctionDefinitions.SPLIT)
+.onFieldsWithData(

Review Comment:
   Please include tests also for
   - SPLIT("", "")
   - SPLIT("", ",")
   - SPLIT(",,", ",,")
   - - SPLIT(",,", ",,,")
   - SPLIT("s", "ss")



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-32706][table] Add built-in SPLIT_STRING function [flink]

2024-02-21 Thread via GitHub


mohitjain2504 commented on code in PR #24365:
URL: https://github.com/apache/flink/pull/24365#discussion_r1498640212


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/SplitFunction.java:
##
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.table.runtime.functions.scalar;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.SpecializedFunction;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Implementation of {@link BuiltInFunctionDefinitions#SPLIT}. */
+@Internal
+public class SplitFunction extends BuiltInScalarFunction {
+public SplitFunction(SpecializedFunction.SpecializedContext context) {
+super(BuiltInFunctionDefinitions.SPLIT, context);
+}
+
+public @Nullable ArrayData eval(@Nullable StringData string, @Nullable 
StringData delimiter) {
+try {
+if (string == null || delimiter == null) {
+return null;
+}
+String string1 = string.toString();
+String delimiter1 = delimiter.toString();
+List resultList = new ArrayList<>();
+
+if (delimiter1.equals("")) {
+for (char c : string1.toCharArray()) {
+
resultList.add(BinaryStringData.fromString(String.valueOf(c)));
+}
+} else {
+int start = 0;
+int end = string1.indexOf(delimiter1);
+while (end != -1) {
+String substring = string1.substring(start, end);
+resultList.add(
+BinaryStringData.fromString(
+substring.isEmpty()
+? ""
+: substring)); // Added this check 
to handle consecutive
+// delimiters
+start = end + delimiter1.length();
+end = string1.indexOf(delimiter1, start);
+}
+String remaining = string1.substring(start);
+resultList.add(
+BinaryStringData.fromString(
+remaining.isEmpty()
+? ""

Review Comment:
   Similar issue: isEmpty check is redundant



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-32706][table] Add built-in SPLIT_STRING function [flink]

2024-02-21 Thread via GitHub


mohitjain2504 commented on code in PR #24365:
URL: https://github.com/apache/flink/pull/24365#discussion_r1498639719


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/SplitFunction.java:
##
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.table.runtime.functions.scalar;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.SpecializedFunction;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Implementation of {@link BuiltInFunctionDefinitions#SPLIT}. */
+@Internal
+public class SplitFunction extends BuiltInScalarFunction {
+public SplitFunction(SpecializedFunction.SpecializedContext context) {
+super(BuiltInFunctionDefinitions.SPLIT, context);
+}
+
+public @Nullable ArrayData eval(@Nullable StringData string, @Nullable 
StringData delimiter) {
+try {
+if (string == null || delimiter == null) {
+return null;
+}
+String string1 = string.toString();

Review Comment:
   can change the same to str and delim instead of string1 and delimiter1



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-32706][table] Add built-in SPLIT_STRING function [flink]

2024-02-21 Thread via GitHub


mohitjain2504 commented on code in PR #24365:
URL: https://github.com/apache/flink/pull/24365#discussion_r1498639316


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/SplitFunction.java:
##
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.table.runtime.functions.scalar;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.SpecializedFunction;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Implementation of {@link BuiltInFunctionDefinitions#SPLIT}. */
+@Internal
+public class SplitFunction extends BuiltInScalarFunction {
+public SplitFunction(SpecializedFunction.SpecializedContext context) {
+super(BuiltInFunctionDefinitions.SPLIT, context);
+}
+
+public @Nullable ArrayData eval(@Nullable StringData string, @Nullable 
StringData delimiter) {
+try {
+if (string == null || delimiter == null) {
+return null;
+}
+String string1 = string.toString();
+String delimiter1 = delimiter.toString();
+List resultList = new ArrayList<>();
+
+if (delimiter1.equals("")) {
+for (char c : string1.toCharArray()) {
+
resultList.add(BinaryStringData.fromString(String.valueOf(c)));
+}
+} else {
+int start = 0;
+int end = string1.indexOf(delimiter1);
+while (end != -1) {
+String substring = string1.substring(start, end);
+resultList.add(
+BinaryStringData.fromString(
+substring.isEmpty()
+? ""

Review Comment:
   This check seems to be redundant. 
   substring.isEmpty() returns `true` in case substring=""; else return false. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-32706][table] Add built-in SPLIT_STRING function [flink]

2024-02-21 Thread via GitHub


mohitjain2504 commented on code in PR #24365:
URL: https://github.com/apache/flink/pull/24365#discussion_r1498635291


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/SplitFunction.java:
##
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.table.runtime.functions.scalar;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.SpecializedFunction;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Implementation of {@link BuiltInFunctionDefinitions#SPLIT}. */
+@Internal
+public class SplitFunction extends BuiltInScalarFunction {
+public SplitFunction(SpecializedFunction.SpecializedContext context) {
+super(BuiltInFunctionDefinitions.SPLIT, context);
+}
+
+public @Nullable ArrayData eval(@Nullable StringData string, @Nullable 
StringData delimiter) {
+try {
+if (string == null || delimiter == null) {
+return null;
+}
+String string1 = string.toString();
+String delimiter1 = delimiter.toString();
+List resultList = new ArrayList<>();
+
+if (delimiter1.equals("")) {

Review Comment:
   delimiter1.isEmpty() can be a cleaner approach here



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-32706][table] Add built-in SPLIT_STRING function [flink]

2024-02-21 Thread via GitHub


mohitjain2504 commented on code in PR #24365:
URL: https://github.com/apache/flink/pull/24365#discussion_r1498635291


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/SplitFunction.java:
##
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.table.runtime.functions.scalar;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.SpecializedFunction;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Implementation of {@link BuiltInFunctionDefinitions#SPLIT}. */
+@Internal
+public class SplitFunction extends BuiltInScalarFunction {
+public SplitFunction(SpecializedFunction.SpecializedContext context) {
+super(BuiltInFunctionDefinitions.SPLIT, context);
+}
+
+public @Nullable ArrayData eval(@Nullable StringData string, @Nullable 
StringData delimiter) {
+try {
+if (string == null || delimiter == null) {
+return null;
+}
+String string1 = string.toString();
+String delimiter1 = delimiter.toString();
+List resultList = new ArrayList<>();
+
+if (delimiter1.equals("")) {

Review Comment:
   delim.isEmpty() can be a cleaner approach here



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-32706][table] Add built-in SPLIT_STRING function [flink]

2024-02-21 Thread via GitHub


flinkbot commented on PR #24365:
URL: https://github.com/apache/flink/pull/24365#issuecomment-1958613193

   
   ## CI report:
   
   * 67fb0726819a76eb0121a51bd3f2b2ded14c45bb UNKNOWN
   
   
   Bot commands
 The @flinkbot bot supports the following commands:
   
- `@flinkbot run azure` re-run the last Azure build
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[PR] [FLINK-32706][table] Add built-in SPLIT_STRING function [flink]

2024-02-21 Thread via GitHub


hanyuzheng7 opened a new pull request, #24365:
URL: https://github.com/apache/flink/pull/24365

   ## What is the purpose of the change
   This is an implementation of SPLIT
   
   
   ## Brief change log
   Splits a string into an array of substrings based on a delimiter. If the 
delimiter is not found, then the original string is returned as the only 
element in the array. If the delimiter is empty, then all characters in the 
string are split. If either, string or delimiter, are NULL, then a NULL value 
is returned.
   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.
   
   - Syntax
   `SPLIT(string, delimiter)`
   
   - Arguments
   string: The string need to be split
   delimiter: Splits a string into an array of substrings based on a delimiter
   
   - Returns
   If the delimiter is not found, then the original string is returned as the 
only element in the array. If the delimiter is empty, then all characters in 
the string are split. If either, string or delimiter, are NULL, then a NULL 
value is returned.
   
   - Examples
   ```
   SELECT SPLIT('abcdefg', 'c');
   Result: ['ab', 'defg']
   ```
   
   - See also
   1. ksqlDB Split function
   ksqlDB provides a scalar function named SPLIT which splits a string into an 
array of substrings based on a delimiter.
   Syntax: SPLIT(string, delimiter)
   For example: SPLIT('a,b,c', ',') will return ['a', 'b', 'c'].
   
https://docs.ksqldb.io/en/0.8.1-ksqldb/developer-guide/ksqldb-reference/scalar-functions/#split
   
   2. Apache Hive Split function
   Hive offers a function named split which splits a string around a specified 
delimiter and returns an array of strings.
   Syntax: array split(string str, string pat)
   For example: split('a,b,c', ',') will return ["a", "b", "c"].
   https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF
   
   3. Spark SQL Split function
   Spark SQL also offers a function named split, similar to the one in Hive.
   Syntax: split(str, pattern[, limit])
   Here, limit is an optional parameter to specify the maximum length of the 
returned array.
   For example: split('oneAtwoBthreeC', '[ABC]', 2) will return ["one", 
"twoBthreeC"].
   https://spark.apache.org/docs/latest/api/sql/index.html#split
   
   4. Presto Split function
   Presto offers a function named split which splits a string around a regular 
expression and returns an array of strings.
   Syntax: array split(string str, string regex)
   For example: split('a.b.c', '\.') will return ["a", "b", "c"].
   https://prestodb.io/docs/current/functions/string.html
   
   ## Verifying this change
   This change added tests in CollectionFunctionsITCase.
   
   ## Does this pull request potentially affect one of the following parts:  
 - Dependencies (does it add or upgrade a dependency): (yes / no)
 - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: (yes / no)
 - The serializers: (yes / no / don't know)
 - The runtime per-record code paths (performance sensitive): (yes / no / 
don't know)
 - Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (yes / no / don't know)
 - The S3 file system connector: (yes / no / don't know)
   
   ## Documentation
   
 - Does this pull request introduce a new feature? (yes / no)
 - If yes, how is the feature documented? (not applicable / docs / JavaDocs 
/ not documented)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] [FLINK-32706][table] Add built-in SPLIT_STRING function [flink]

2024-02-21 Thread via GitHub


hanyuzheng7 closed pull request #23108: [FLINK-32706][table] Add built-in 
SPLIT_STRING function
URL: https://github.com/apache/flink/pull/23108


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org