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

2024-05-22 Thread via GitHub


dawidwys merged PR #24365:
URL: https://github.com/apache/flink/pull/24365


-- 
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-21 Thread via GitHub


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


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/SplitFunction.java:
##
@@ -0,0 +1,64 @@
+/*
+ * 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.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;
+}
+if (delimiter.toString().isEmpty()) {
+List res = new ArrayList<>();
+for (int i = 0; i < string.toString().length(); ++i) {

Review Comment:
   Updated it. Thank you!



-- 
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-21 Thread via GitHub


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


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/SplitFunction.java:
##
@@ -0,0 +1,64 @@
+/*
+ * 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.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;
+}
+if (delimiter.toString().isEmpty()) {
+List res = new ArrayList<>();
+for (int i = 0; i < string.toString().length(); ++i) {

Review Comment:
   I missed that `String#chars` is java 9+ which we cannot use yet.
   
   Let's do a proper `charAt()` implementation then. The `toCharArray` approach 
has the downside that it allocates the `char[]` array first before iterating.
   
   Let's do:
   ```
   public @Nullable ArrayData eval(@Nullable StringData stringData, @Nullable 
StringData delimiter) {
   
   
   String str = stringData.toString();
   for (int i = 0; i < str.length(); i++){
   char c = s.charAt(i);
   res.add(StringData.fromString(String.valueOf(str.charAt(i;
   }
   ```



-- 
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-21 Thread via GitHub


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


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/SplitFunction.java:
##
@@ -0,0 +1,64 @@
+/*
+ * 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.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;
+}
+if (delimiter.toString().isEmpty()) {
+List res = new ArrayList<>();

Review Comment:
   after using `toCharArray()`, we just call `string.toString()` in for loop.



##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/SplitFunction.java:
##
@@ -0,0 +1,64 @@
+/*
+ * 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.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;
+}
+if (delimiter.toString().isEmpty()) {
+List res = new ArrayList<>();

Review Comment:
   after using `toCharArray()`, we just call `string.toString()` one time in 
for loop.



-- 
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: issue

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

2024-05-21 Thread via GitHub


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


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/SplitFunction.java:
##
@@ -0,0 +1,64 @@
+/*
+ * 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.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;
+}
+if (delimiter.toString().isEmpty()) {
+List res = new ArrayList<>();
+for (int i = 0; i < string.toString().length(); ++i) {

Review Comment:
   use char[] 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-21 Thread via GitHub


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


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/SplitFunction.java:
##
@@ -0,0 +1,64 @@
+/*
+ * 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.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;
+}
+if (delimiter.toString().isEmpty()) {
+List res = new ArrayList<>();
+for (int i = 0; i < string.toString().length(); ++i) {

Review Comment:
   Agreed! 
   



-- 
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-21 Thread via GitHub


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


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/SplitFunction.java:
##
@@ -0,0 +1,64 @@
+/*
+ * 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.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;
+}
+if (delimiter.toString().isEmpty()) {
+List res = new ArrayList<>();
+for (int i = 0; i < string.toString().length(); ++i) {
+
res.add(StringData.fromString(String.valueOf(string.toString().charAt(i;
+}
+return new GenericArrayData(res.toArray());
+}
+BinaryStringData[] binaryStringData =

Review Comment:
   For the readeability I'd keep it the way it is.



-- 
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-21 Thread via GitHub


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


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/SplitFunction.java:
##
@@ -0,0 +1,64 @@
+/*
+ * 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.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;
+}
+if (delimiter.toString().isEmpty()) {
+List res = new ArrayList<>();
+for (int i = 0; i < string.toString().length(); ++i) {

Review Comment:
   Even better if we use `String#chars`, we will skip the underlying array copy.



-- 
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-20 Thread via GitHub


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


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/SplitFunction.java:
##
@@ -0,0 +1,64 @@
+/*
+ * 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.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;
+}
+if (delimiter.toString().isEmpty()) {
+List res = new ArrayList<>();
+for (int i = 0; i < string.toString().length(); ++i) {

Review Comment:
   can convert the for loop into foreach loop for better readability
   ```
   String str = string.toString();
   for (char c : str.toCharArray()) {
   res.add(StringData.fromString(String.valueOf(c)));
   }
   ```



-- 
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-20 Thread via GitHub


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


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/SplitFunction.java:
##
@@ -0,0 +1,64 @@
+/*
+ * 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.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;
+}
+if (delimiter.toString().isEmpty()) {
+List res = new ArrayList<>();
+for (int i = 0; i < string.toString().length(); ++i) {
+
res.add(StringData.fromString(String.valueOf(string.toString().charAt(i;
+}
+return new GenericArrayData(res.toArray());
+}
+BinaryStringData[] binaryStringData =

Review Comment:
   Just a thought: Do we need a try-catch block over the whole function or just 
capturing the below statement will be enough? If we don't expect any exception 
in the above lines, can keep the statements out of try-catch block. 



-- 
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-20 Thread via GitHub


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


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/SplitFunction.java:
##
@@ -0,0 +1,64 @@
+/*
+ * 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.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;
+}
+if (delimiter.toString().isEmpty()) {
+List res = new ArrayList<>();

Review Comment:
   It can be initialised directly to str.length()
   ```
   String str = string.toString();
   List res = new ArrayList<>(str.length());
   ```
   This will also help us reduce the redundant calls to `string.toString()`
   



-- 
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-20 Thread via GitHub


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


##
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/SplitFunction.java:
##
@@ -0,0 +1,64 @@
+/*
+ * 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.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;
+}
+if (delimiter.toString().isEmpty()) {
+List res = new ArrayList<>();
+for (int i = 0; i < string.toString().length(); ++i) {
+
res.add(StringData.fromString(String.valueOf(string.toString().charAt(i;
+}
+return new GenericArrayData(res.toArray());

Review Comment:
   handle empty delimiter at first.



-- 
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-20 Thread via GitHub


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

   > > When the delimiter is empty, do we need to handle this situation 
specially?/
   > > Yes, we do. Do you mind updating the PR?
   
   I have already updated the PR.
   
   


-- 
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-20 Thread via GitHub


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

   > When the delimiter is empty, do we need to handle this situation 
specially?/
   Yes, we do. Do you mind updating the PR?


-- 
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-16 Thread via GitHub


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

   @dawidwys So We can use splitByWholeSeparatorPreserveAllTokens when the 
delimiter is not empty. When the delimiter is empty, do we need to handle this 
situation specially?


-- 
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-16 Thread via GitHub


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

   > To confirm, you suggest to support an empty delimiter and in that case 
split all characters
   
   Yes


-- 
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-16 Thread via GitHub


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


##
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CollectionFunctionsITCase.java:
##
@@ -1721,4 +1722,83 @@ private Stream arrayExceptTestCases() {
 "Invalid input arguments. Expected signatures 
are:\n"
 + "ARRAY_EXCEPT(, )"));
 }
+
+private Stream splitTestCases() {
+return Stream.of(
+TestSetSpec.forFunction(BuiltInFunctionDefinitions.SPLIT)
+.onFieldsWithData(
+"123,123,23",
+null,
+",123,123",
+",123,123,",
+123,
+"12345",
+",123,,,123,")
+.andDataTypes(
+DataTypes.STRING().notNull(),
+DataTypes.STRING(),
+DataTypes.STRING().notNull(),
+DataTypes.STRING().notNull(),
+DataTypes.INT().notNull(),
+DataTypes.STRING().notNull(),
+DataTypes.STRING().notNull())
+.testResult(
+$("f0").split(","),
+"SPLIT(f0, ',')",
+new String[] {"123", "123", "23"},
+DataTypes.ARRAY(DataTypes.STRING()).notNull())
+.testResult(
+$("f0").split(null),
+"SPLIT(f0, NULL)",
+null,
+DataTypes.ARRAY(DataTypes.STRING()))
+.testResult(
+$("f0").split(""),
+"SPLIT(f0, '')",
+new String[] {"123,123,23"},

Review Comment:
   return input 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-16 Thread via GitHub


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

   > @MartijnVisser To confirm, you suggest to support an empty delimiter and 
in that case split all characters, meaning:
   > 
   > ```
   > SPLIT('abcde', '') = ['a', 'b', 'c', 'd', 'e']
   > SPLIT('It is', '')=['I', 't', ' ', 'i', 's']
   > ```
   
   
   
   > @MartijnVisser To confirm, you suggest to support an empty delimiter and 
in that case split all characters, meaning:
   > 
   > ```
   > SPLIT('abcde', '') = ['a', 'b', 'c', 'd', 'e']
   > SPLIT('It is', '')=['I', 't', ' ', 'i', 's']
   > ```
   
   
   
   > @MartijnVisser To confirm, you suggest to support an empty delimiter and 
in that case split all characters, meaning:
   > 
   > ```
   > SPLIT('abcde', '') = ['a', 'b', 'c', 'd', 'e']
   > SPLIT('It is', '')=['I', 't', ' ', 'i', 's']
   > ```
   
   Now if the delimiter is empty, we will return input string, here is a test 
case.
   https://github.com/apache/flink/pull/24365/files#r1603638142
   
   
   
   
   
   
   
   
   


-- 
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-16 Thread via GitHub


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


##
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CollectionFunctionsITCase.java:
##
@@ -1721,4 +1722,83 @@ private Stream arrayExceptTestCases() {
 "Invalid input arguments. Expected signatures 
are:\n"
 + "ARRAY_EXCEPT(, )"));
 }
+
+private Stream splitTestCases() {
+return Stream.of(
+TestSetSpec.forFunction(BuiltInFunctionDefinitions.SPLIT)
+.onFieldsWithData(
+"123,123,23",
+null,
+",123,123",
+",123,123,",
+123,
+"12345",
+",123,,,123,")
+.andDataTypes(
+DataTypes.STRING().notNull(),
+DataTypes.STRING(),
+DataTypes.STRING().notNull(),
+DataTypes.STRING().notNull(),
+DataTypes.INT().notNull(),
+DataTypes.STRING().notNull(),
+DataTypes.STRING().notNull())
+.testResult(
+$("f0").split(","),
+"SPLIT(f0, ',')",
+new String[] {"123", "123", "23"},
+DataTypes.ARRAY(DataTypes.STRING()).notNull())
+.testResult(
+$("f0").split(null),
+"SPLIT(f0, NULL)",
+null,
+DataTypes.ARRAY(DataTypes.STRING()))
+.testResult(
+$("f0").split(""),
+"SPLIT(f0, '')",
+new String[] {"123,123,23"},

Review Comment:
   return input 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-16 Thread via GitHub


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

   @MartijnVisser To confirm, you suggest to support an empty delimiter and in 
that case split all characters, meaning:
   ```
   SPLIT('abcde', '') = ['a', 'b', 'c', 'd', 'e']
   SPLIT('It is', '')=['I', 't', ' ', 'i', 's']
   ```


-- 
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-16 Thread via GitHub


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

   > @MartijnVisser What do you think should be the behaviour for an empty 
delimiter?
   
   I've taken a tour along the various databases:
   * Postgres doesn't have a SPLIT function, but leverages 
`regexp_split_to_table` or `regexp_split_to_array`. When an empty delimiter is 
provided, it throws an error. 
https://www.postgresql.org/docs/current/functions-string.html 
   * MySQL doesn't have a SPLIT function either, but has `SUBSTRING_INDEX`. 
   * MSSQL Server has `STRING_SPLIT` 
https://learn.microsoft.com/en-us/sql/t-sql/functions/string-split-transact-sql?view=sql-server-ver16
 which also doesn't accept empty delimiters
   * Spark has a `SPLIT` function and accepts empty delimiters, having the same 
behavior as ksqlDB 
https://spark.apache.org/docs/latest/api/python/reference/pyspark.sql/api/pyspark.sql.functions.split.html
   * Presto has a `SPLIT `function and accepts empty delimiters, same behavior 
as ksqlDB https://prestodb.io/docs/current/functions/string.html
   * Clickhouse actually has a wide variety with `splitByChar`, 
`splitByString`, `splitByRegexp` etc 
https://clickhouse.com/docs/en/sql-reference/functions/splitting-merging-functions.
 `splitByString` appears to have the same behavior as Spark and Presto
   
   All in all, I could find 3 implementations of `SPLIT` in ksqlDB, Spark, and 
Presto. All 3 accept empty delimiters. I would think then we should follow the 
same behavior, since all others have different function names for 
comparable/similar features


-- 
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-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



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