You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by sunjincheng121 <gi...@git.apache.org> on 2017/06/19 07:45:27 UTC

[GitHub] flink pull request #4138: [FLINK-6925][table]Add CONCAT/CONCAT_WS supported ...

GitHub user sunjincheng121 opened a pull request:

    https://github.com/apache/flink/pull/4138

    [FLINK-6925][table]Add CONCAT/CONCAT_WS supported in SQL

    In this PR. I have add CONCAT/CONCAT_WS supported in SQL.
    
    - [x] General
      - The pull request references the related JIRA issue ("[FLINK-6925][table]Add CONCAT/CONCAT_WS supported in SQL")
      - The pull request addresses only one issue
      - Each commit in the PR has a meaningful commit message (including the JIRA id)
    
    - [ ] Documentation
      - Documentation has been added for new functionality
      - Old documentation affected by the pull request has been updated
      - JavaDoc for public methods has been added
    
    - [x] Tests & Build
      - Functionality added by the pull request is covered by tests
      - `mvn clean verify` has been executed successfully locally or a Travis build has passed


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/sunjincheng121/flink FLINK-6925-PR

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/4138.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #4138
    
----
commit 17b9fbbf3e1b213c15707a49405f5e870d47aa5f
Author: sunjincheng121 <su...@gmail.com>
Date:   2017-06-16T09:59:44Z

    [FLINK-6925][table]Add CONCAT/CONCAT_WS supported in SQL

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4138: [FLINK-6925][table]Add CONCAT/CONCAT_WS supported ...

Posted by sunjincheng121 <gi...@git.apache.org>.
Github user sunjincheng121 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4138#discussion_r124007599
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/functions/ScalarFunctions.scala ---
    @@ -0,0 +1,79 @@
    +/*
    + * 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
    +
    +import scala.annotation.varargs
    +import java.math.{BigDecimal => JBigDecimal}
    +import java.lang.{StringBuffer => JStringBuffer}
    +
    +/**
    +  * All build-in scalar scalar functions.
    +  */
    +class ScalarFunctions {}
    +
    +object ScalarFunctions {
    +
    +  def power(a: Double, b: JBigDecimal): Double = {
    +    Math.pow(a, b.doubleValue())
    +  }
    +
    +  /**
    +    * Returns the string that results from concatenating the arguments.
    +    * Returns NULL if any argument is NULL.
    +    */
    +  @varargs
    +  def concat(args: String*): String = {
    +    val sb = new JStringBuffer
    +    var i = 0
    +    while (i < args.length) {
    +      if (args(i) == null) {
    +        return null
    +      }
    +      sb.append(args(i))
    +      i += 1
    +    }
    +    sb.toString
    +  }
    +
    +  /**
    +    * Returns the string that results from concatenating the arguments and separator.
    +    * Returns NULL If the separator is NULL.
    +    *
    +    * Note: CONCAT_WS() does not skip empty strings. However, it does skip any NULL values after
    +    * the separator argument.
    +    *
    +    * @param args The first element of argument is the separator for the rest of the arguments.
    +    */
    +  @varargs
    +  def concat_ws(args: String*): String = {
    +    val separator = args(0)
    +    if (null == separator) {
    +      return null
    +    }
    +
    +    val sb = new JStringBuffer
    +
    +    var i = 1
    +    val dataList = args.filter(null != _)
    --- End diff --
    
    Yes, good catch. :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4138: [FLINK-6925][table]Add CONCAT/CONCAT_WS supported ...

Posted by sunjincheng121 <gi...@git.apache.org>.
Github user sunjincheng121 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4138#discussion_r124588270
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/functions/ScalarFunctions.scala ---
    @@ -0,0 +1,87 @@
    +/*
    + * 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
    +
    +import scala.annotation.varargs
    +import java.math.{BigDecimal => JBigDecimal}
    +import java.lang.{StringBuffer => JStringBuffer}
    +
    +/**
    +  * Built-in scalar runtime functions.
    +  */
    +class ScalarFunctions {}
    +
    +object ScalarFunctions {
    +
    +  def power(a: Double, b: JBigDecimal): Double = {
    +    Math.pow(a, b.doubleValue())
    +  }
    +
    +  /**
    +    * Returns the string that results from concatenating the arguments.
    +    * Returns NULL if any argument is NULL.
    +    */
    +  @varargs
    +  def concat(args: String*): String = {
    +    val sb = new JStringBuffer
    --- End diff --
    
    Yes, we do not need `synchronized ` the `append` method. Good catch!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4138: [FLINK-6925][table]Add CONCAT/CONCAT_WS supported ...

Posted by sunjincheng121 <gi...@git.apache.org>.
Github user sunjincheng121 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4138#discussion_r124588196
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/functions/ScalarFunctions.scala ---
    @@ -0,0 +1,87 @@
    +/*
    + * 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
    +
    +import scala.annotation.varargs
    +import java.math.{BigDecimal => JBigDecimal}
    +import java.lang.{StringBuffer => JStringBuffer}
    +
    +/**
    +  * Built-in scalar runtime functions.
    +  */
    +class ScalarFunctions {}
    +
    +object ScalarFunctions {
    +
    +  def power(a: Double, b: JBigDecimal): Double = {
    +    Math.pow(a, b.doubleValue())
    +  }
    +
    +  /**
    +    * Returns the string that results from concatenating the arguments.
    +    * Returns NULL if any argument is NULL.
    +    */
    +  @varargs
    +  def concat(args: String*): String = {
    +    val sb = new JStringBuffer
    --- End diff --
    
    Yes, we do not need `supplemented ` the `append` method. Good catch!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4138: [FLINK-6925][table]Add CONCAT/CONCAT_WS supported ...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/flink/pull/4138


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4138: [FLINK-6925][table]Add CONCAT/CONCAT_WS supported in SQL

Posted by sunjincheng121 <gi...@git.apache.org>.
Github user sunjincheng121 commented on the issue:

    https://github.com/apache/flink/pull/4138
  
    Thanks @wuchong ! From my point of view +1 to merge.
    @twalthr do you have any other comments? :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4138: [FLINK-6925][table]Add CONCAT/CONCAT_WS supported ...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4138#discussion_r123944651
  
    --- Diff: docs/dev/table/sql.md ---
    @@ -1573,6 +1573,29 @@ INITCAP(string)
           </td>
         </tr>
     
    +    <tr>
    +      <td>
    +        {% highlight text %}
    +CONCAT(string1, string2,...)
    +{% endhighlight %}
    +      </td>
    +      <td>
    +        <p>Returns the string that results from concatenating the arguments. Returns NULL if any argument is NULL.</p>
    --- End diff --
    
    Can you add a little example? `E.g. CONCAT('a', 'b', 'c') returns 'XXX'.`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4138: [FLINK-6925][table]Add CONCAT/CONCAT_WS supported in SQL

Posted by sunjincheng121 <gi...@git.apache.org>.
Github user sunjincheng121 commented on the issue:

    https://github.com/apache/flink/pull/4138
  
    I have updated the PR. Welcome anyone to review.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4138: [FLINK-6925][table]Add CONCAT/CONCAT_WS supported ...

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4138#discussion_r124452756
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/functions/ScalarFunctions.scala ---
    @@ -0,0 +1,87 @@
    +/*
    + * 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
    +
    +import scala.annotation.varargs
    +import java.math.{BigDecimal => JBigDecimal}
    +import java.lang.{StringBuffer => JStringBuffer}
    +
    +/**
    +  * Built-in scalar runtime functions.
    +  */
    +class ScalarFunctions {}
    +
    +object ScalarFunctions {
    +
    +  def power(a: Double, b: JBigDecimal): Double = {
    +    Math.pow(a, b.doubleValue())
    +  }
    +
    +  /**
    +    * Returns the string that results from concatenating the arguments.
    +    * Returns NULL if any argument is NULL.
    +    */
    +  @varargs
    +  def concat(args: String*): String = {
    +    val sb = new JStringBuffer
    +    var i = 0
    +    while (i < args.length) {
    +      if (args(i) == null) {
    +        return null
    +      }
    +      sb.append(args(i))
    +      i += 1
    +    }
    +    sb.toString
    +  }
    +
    +  /**
    +    * Returns the string that results from concatenating the arguments and separator.
    +    * Returns NULL If the separator is NULL.
    +    *
    +    * Note: CONCAT_WS() does not skip empty strings. However, it does skip any NULL values after
    +    * the separator argument.
    +    *
    +    * @param args The first element of argument is the separator for the rest of the arguments.
    +    */
    +  @varargs
    +  def concat_ws(args: String*): String = {
    --- End diff --
    
    I would suggest to change the signature to `sep: String, strs: String*`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4138: [FLINK-6925][table]Add CONCAT/CONCAT_WS supported ...

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4138#discussion_r124456117
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/functions/ScalarFunctions.scala ---
    @@ -0,0 +1,87 @@
    +/*
    + * 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
    +
    +import scala.annotation.varargs
    +import java.math.{BigDecimal => JBigDecimal}
    +import java.lang.{StringBuffer => JStringBuffer}
    +
    +/**
    +  * Built-in scalar runtime functions.
    +  */
    +class ScalarFunctions {}
    +
    +object ScalarFunctions {
    +
    +  def power(a: Double, b: JBigDecimal): Double = {
    +    Math.pow(a, b.doubleValue())
    +  }
    +
    +  /**
    +    * Returns the string that results from concatenating the arguments.
    +    * Returns NULL if any argument is NULL.
    +    */
    +  @varargs
    +  def concat(args: String*): String = {
    +    val sb = new JStringBuffer
    +    var i = 0
    +    while (i < args.length) {
    +      if (args(i) == null) {
    +        return null
    +      }
    +      sb.append(args(i))
    +      i += 1
    +    }
    +    sb.toString
    +  }
    +
    +  /**
    +    * Returns the string that results from concatenating the arguments and separator.
    +    * Returns NULL If the separator is NULL.
    +    *
    +    * Note: CONCAT_WS() does not skip empty strings. However, it does skip any NULL values after
    +    * the separator argument.
    +    *
    +    * @param args The first element of argument is the separator for the rest of the arguments.
    +    */
    +  @varargs
    +  def concat_ws(args: String*): String = {
    +    val separator = args(0)
    +    if (null == separator) {
    +      return null
    +    }
    +
    +    val sb = new JStringBuffer
    --- End diff --
    
    Please use `StringBuilder` instead of `StringBuffer`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4138: [FLINK-6925][table]Add CONCAT/CONCAT_WS supported in SQL

Posted by sunjincheng121 <gi...@git.apache.org>.
Github user sunjincheng121 commented on the issue:

    https://github.com/apache/flink/pull/4138
  
    Hi @twalthr Thanks for your review. most of your comments are make sense to me. I have updated the PR. according your comments. 
    I hope hear your detail opinion about the `varargs` annotated. 
    
    Thanks,
    SunJincheng
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4138: [FLINK-6925][table]Add CONCAT/CONCAT_WS supported in SQL

Posted by sunjincheng121 <gi...@git.apache.org>.
Github user sunjincheng121 commented on the issue:

    https://github.com/apache/flink/pull/4138
  
    Need add SQL doc, And a `concat_ws`  bug need be fix. Updated the PR. later.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4138: [FLINK-6925][table]Add CONCAT/CONCAT_WS supported ...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4138#discussion_r123945022
  
    --- Diff: docs/dev/table/sql.md ---
    @@ -1573,6 +1573,29 @@ INITCAP(string)
           </td>
         </tr>
     
    +    <tr>
    +      <td>
    +        {% highlight text %}
    +CONCAT(string1, string2,...)
    +{% endhighlight %}
    +      </td>
    +      <td>
    +        <p>Returns the string that results from concatenating the arguments. Returns NULL if any argument is NULL.</p>
    +      </td>
    +    </tr>
    +
    +    <tr>
    +      <td>
    +        {% highlight text %}
    +CONCAT_WS(separator, string1, string2,...)
    +{% endhighlight %}
    +      </td>
    +      <td>
    +        <p>Returns the string that results from concatenating the arguments and separator. The separator is added between the strings to be concatenated. Returns NULL If the separator is NULL.</p>
    --- End diff --
    
    Replace `and` with `using a`. Can you add a little example here as well?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4138: [FLINK-6925][table]Add CONCAT/CONCAT_WS supported in SQL

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on the issue:

    https://github.com/apache/flink/pull/4138
  
    I will merge this today.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4138: [FLINK-6925][table]Add CONCAT/CONCAT_WS supported ...

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4138#discussion_r124456032
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/functions/ScalarFunctions.scala ---
    @@ -0,0 +1,87 @@
    +/*
    + * 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
    +
    +import scala.annotation.varargs
    +import java.math.{BigDecimal => JBigDecimal}
    +import java.lang.{StringBuffer => JStringBuffer}
    +
    +/**
    +  * Built-in scalar runtime functions.
    +  */
    +class ScalarFunctions {}
    +
    +object ScalarFunctions {
    +
    +  def power(a: Double, b: JBigDecimal): Double = {
    +    Math.pow(a, b.doubleValue())
    +  }
    +
    +  /**
    +    * Returns the string that results from concatenating the arguments.
    +    * Returns NULL if any argument is NULL.
    +    */
    +  @varargs
    +  def concat(args: String*): String = {
    +    val sb = new JStringBuffer
    --- End diff --
    
    Please use `StringBuilder` instead of `StringBuffer`.  `StringBuffer` is a thread-safe with poorer performance. 
    
    From StringBuffer JavaDoc:
    
    >As of release JDK 5, this class has been supplemented with an equivalent class designed for use by a single thread, StringBuilder. The StringBuilder class should generally be used in preference to this one, as it supports all of the same operations but it is faster, as it performs no synchronization.
    
    
    
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4138: [FLINK-6925][table]Add CONCAT/CONCAT_WS supported in SQL

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on the issue:

    https://github.com/apache/flink/pull/4138
  
    +1 to merge


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4138: [FLINK-6925][table]Add CONCAT/CONCAT_WS supported ...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4138#discussion_r123946426
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/functions/ScalarFunctions.scala ---
    @@ -0,0 +1,79 @@
    +/*
    + * 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
    +
    +import scala.annotation.varargs
    +import java.math.{BigDecimal => JBigDecimal}
    +import java.lang.{StringBuffer => JStringBuffer}
    +
    +/**
    +  * All build-in scalar scalar functions.
    +  */
    +class ScalarFunctions {}
    +
    +object ScalarFunctions {
    +
    +  def power(a: Double, b: JBigDecimal): Double = {
    +    Math.pow(a, b.doubleValue())
    +  }
    +
    +  /**
    +    * Returns the string that results from concatenating the arguments.
    +    * Returns NULL if any argument is NULL.
    +    */
    +  @varargs
    --- End diff --
    
    This is a internal method. We don't need varargs here, we can use `Array[String]`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4138: [FLINK-6925][table]Add CONCAT/CONCAT_WS supported ...

Posted by sunjincheng121 <gi...@git.apache.org>.
Github user sunjincheng121 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4138#discussion_r123974055
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/functions/ScalarFunctions.scala ---
    @@ -0,0 +1,79 @@
    +/*
    + * 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
    +
    +import scala.annotation.varargs
    +import java.math.{BigDecimal => JBigDecimal}
    +import java.lang.{StringBuffer => JStringBuffer}
    +
    +/**
    +  * All build-in scalar scalar functions.
    +  */
    +class ScalarFunctions {}
    +
    +object ScalarFunctions {
    +
    +  def power(a: Double, b: JBigDecimal): Double = {
    +    Math.pow(a, b.doubleValue())
    +  }
    +
    +  /**
    +    * Returns the string that results from concatenating the arguments.
    +    * Returns NULL if any argument is NULL.
    +    */
    +  @varargs
    --- End diff --
    
    If we using `Array[String]` we may get exception as follows:
    ```
    Caused by: org.codehaus.commons.compiler.CompileException: Line 44, Column 78: No applicable constructor/method found for actual parameters "java.lang.String, java.lang.String"; candidates are: "public static java.lang.String org.apache.flink.table.runtime.functions.ScalarFunctions.concat_ws(java.lang.String[])"
    	at org.codehaus.janino.UnitCompiler.compileError(UnitCompiler.java:11672)
    	at org.codehaus.janino.UnitCompiler.findMostSpecificIInvocable(UnitCompiler.java:8828)
    	
    ```
    Because the scalar function of  `concat/concat_ws` really a varargs. What do you think ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4138: [FLINK-6925][table]Add CONCAT/CONCAT_WS supported in SQL

Posted by sunjincheng121 <gi...@git.apache.org>.
Github user sunjincheng121 commented on the issue:

    https://github.com/apache/flink/pull/4138
  
    @shaoxuan-wang @wuchong 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4138: [FLINK-6925][table]Add CONCAT/CONCAT_WS supported in SQL

Posted by sunjincheng121 <gi...@git.apache.org>.
Github user sunjincheng121 commented on the issue:

    https://github.com/apache/flink/pull/4138
  
    @wuchong thanks very much for your reviewing. I have updated the PR. Please have look at it again.;)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4138: [FLINK-6925][table]Add CONCAT/CONCAT_WS supported ...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4138#discussion_r123946739
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/functions/ScalarFunctions.scala ---
    @@ -0,0 +1,79 @@
    +/*
    + * 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
    +
    +import scala.annotation.varargs
    +import java.math.{BigDecimal => JBigDecimal}
    +import java.lang.{StringBuffer => JStringBuffer}
    +
    +/**
    +  * All build-in scalar scalar functions.
    --- End diff --
    
    `Built-in scalar runtime functions.`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4138: [FLINK-6925][table]Add CONCAT/CONCAT_WS supported ...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4138#discussion_r123946049
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala ---
    @@ -1560,6 +1561,13 @@ class CodeGenerator(
             requireArray(array)
             generateArrayElement(this, array)
     
    +      case ScalarSqlFunctions.CONCAT | ScalarSqlFunctions.CONCAT_WS =>
    +        this.config.setNullCheck(false)
    --- End diff --
    
    We cannot modify the config here. Maybe it makes sense not to use a `CallGenerator` but add the logic to `ScalarOperators`. Other arrays functions such as `generateArrayCardinality`, `generateArrayElement` are there as well.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4138: [FLINK-6925][table]Add CONCAT/CONCAT_WS supported ...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4138#discussion_r123947651
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/functions/ScalarFunctions.scala ---
    @@ -0,0 +1,79 @@
    +/*
    + * 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
    +
    +import scala.annotation.varargs
    +import java.math.{BigDecimal => JBigDecimal}
    +import java.lang.{StringBuffer => JStringBuffer}
    +
    +/**
    +  * All build-in scalar scalar functions.
    +  */
    +class ScalarFunctions {}
    +
    +object ScalarFunctions {
    +
    +  def power(a: Double, b: JBigDecimal): Double = {
    +    Math.pow(a, b.doubleValue())
    +  }
    +
    +  /**
    +    * Returns the string that results from concatenating the arguments.
    +    * Returns NULL if any argument is NULL.
    +    */
    +  @varargs
    +  def concat(args: String*): String = {
    +    val sb = new JStringBuffer
    +    var i = 0
    +    while (i < args.length) {
    +      if (args(i) == null) {
    +        return null
    +      }
    +      sb.append(args(i))
    +      i += 1
    +    }
    +    sb.toString
    +  }
    +
    +  /**
    +    * Returns the string that results from concatenating the arguments and separator.
    +    * Returns NULL If the separator is NULL.
    +    *
    +    * Note: CONCAT_WS() does not skip empty strings. However, it does skip any NULL values after
    +    * the separator argument.
    +    *
    +    * @param args The first element of argument is the separator for the rest of the arguments.
    +    */
    +  @varargs
    +  def concat_ws(args: String*): String = {
    +    val separator = args(0)
    +    if (null == separator) {
    +      return null
    +    }
    +
    +    val sb = new JStringBuffer
    +
    +    var i = 1
    +    val dataList = args.filter(null != _)
    --- End diff --
    
    Why not doing the filtering within the while loop? You are iterating over the data twice. We should not use fancy Scala functions within runtime code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4138: [FLINK-6925][table]Add CONCAT/CONCAT_WS supported ...

Posted by wuchong <gi...@git.apache.org>.
Github user wuchong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4138#discussion_r123681972
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/scalarFunctions/ScalarFunctions.scala ---
    @@ -0,0 +1,79 @@
    +/*
    + * 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.scalarFunctions
    +
    +import scala.annotation.varargs
    +import java.math.{BigDecimal => JBigDecimal}
    +import java.lang.{StringBuffer => JStringBuffer}
    +
    +/**
    +  * All build-in scalar scalar functions.
    +  */
    +class ScalarFunctions {}
    +
    +object ScalarFunctions {
    +
    +  def power(a: Double, b: JBigDecimal): Double = {
    +    Math.pow(a, b.doubleValue())
    +  }
    +
    +  /**
    +    * Returns the string that results from concatenating the arguments.
    +    * Returns NULL if any argument is NULL.
    +    */
    +  @varargs
    +  def concat(args: String*): String = {
    +    val sb = new JStringBuffer
    +    var i = 0
    +    while (i < args.length) {
    +      if (args(i) == null) {
    +        return null
    +      }
    +      sb.append(args(i))
    +      i += 1
    +    }
    +    sb.toString
    +  }
    +
    +  /**
    +    * Returns the string that results from concatenating the arguments and separator.
    +    * Returns NULL If the separator is NULL.
    +    *
    +    * Note: CONCAT_WS() does not skip empty strings. However, it does skip any NULL values after
    +    * the separator argument.
    +    *
    +    * @param args The first element of argument is the separator for the rest of the arguments.
    +    */
    +  @varargs
    +  def concat_ws(args: String*): String = {
    --- End diff --
    
    I would like to declare the signature like this `(sep: String, str1: String, others: String*)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---