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

[GitHub] flink pull request #4078: [FLINK-6693] [table] Support DATE_FORMAT function ...

GitHub user haohui opened a pull request:

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

    [FLINK-6693] [table] Support DATE_FORMAT function in the Table / SQL API.

    

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

    $ git pull https://github.com/haohui/flink FLINK-6693

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

    https://github.com/apache/flink/pull/4078.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 #4078
    
----
commit 64f706adecec16b1592943bbd1780b3d222a220f
Author: Haohui Mai <wh...@apache.org>
Date:   2017-06-06T06:23:31Z

    [FLINK-6693] [table] Support DATE_FORMAT function in the Table / SQL API.

----


---
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 #4078: [FLINK-6693] [table] Support DATE_FORMAT function ...

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

    https://github.com/apache/flink/pull/4078#discussion_r126747178
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala ---
    @@ -2008,6 +2008,33 @@ class CodeGenerator(
       }
     
       /**
    +    * Adds a reusable [[java.util.Random]] to the member area of the generated [[Function]].
    +    *
    +    * The seed parameter must be a literal/constant expression.
    +    *
    +    * @return member variable term
    +    */
    +  def addReusableDateFormatter(format: GeneratedExpression): String = {
    +    val fieldTerm = newName("dateFormatter")
    +
    +    val field =
    +      s"""
    +         |transient org.joda.time.format.DateTimeFormatter $fieldTerm;
    +         |""".stripMargin
    +    reusableMemberStatements.add(field)
    +
    +    val fieldInit =
    +      s"""
    +         |${format.code}
    +         |$fieldTerm = org.apache.flink.table.runtime.functions.
    +         |DateTimeFunctions$$.MODULE$$.createDateTimeFormatter(${format.resultTerm});
    --- End diff --
    
    What is `$$.MODULE$$`?


---
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 #4078: [FLINK-6693] [table] Support DATE_FORMAT function ...

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

    https://github.com/apache/flink/pull/4078#discussion_r122255933
  
    --- Diff: flink-libraries/flink-table/pom.xml ---
    @@ -110,6 +110,12 @@ under the License.
                 <scope>compile</scope>
             </dependency>
     
    +		<dependency>
    +			<groupId>joda-time</groupId>
    +			<artifactId>joda-time</artifactId>
    +			<scope>provided</scope>
    --- End diff --
    
    Why is `joda-time` provided? Is it a transitive dependency?


---
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 #4078: [FLINK-6693] [table] Support DATE_FORMAT function ...

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

    https://github.com/apache/flink/pull/4078#discussion_r122255414
  
    --- Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/runtime/DateTimeFunctions.java ---
    @@ -0,0 +1,158 @@
    +/*
    + * 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;
    +
    +import org.joda.time.format.DateTimeFormatter;
    +import org.joda.time.format.DateTimeFormatterBuilder;
    +
    +/**
    + * Built-in scalar functions for date time related operations.
    + */
    +public class DateTimeFunctions {
    +	private static final int PIVOT_YEAR = 2020;
    +
    +	private static final ThreadLocalCache<String, DateTimeFormatter> DATETIME_FORMATTER_CACHE =
    +		new ThreadLocalCache<String, DateTimeFormatter>(100) {
    +			@Override
    +			protected DateTimeFormatter getNewInstance(String format) {
    +				return createDateTimeFormatter(format);
    +			}
    +		};
    +
    +	public static String dateFormat(long ts, String formatString) {
    +		DateTimeFormatter formatter = DATETIME_FORMATTER_CACHE.get(formatString);
    --- End diff --
    
    Would it make sense to add a shortcut during code-gen if the pattern is a string literal (this should be the most common case) to avoid the lookup?


---
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 #4078: [FLINK-6693] [table] Support DATE_FORMAT function ...

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

    https://github.com/apache/flink/pull/4078#discussion_r122240819
  
    --- Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/runtime/DateTimeFunctions.java ---
    @@ -0,0 +1,158 @@
    +/*
    + * 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;
    +
    +import org.joda.time.format.DateTimeFormatter;
    +import org.joda.time.format.DateTimeFormatterBuilder;
    +
    +/**
    + * Built-in scalar functions for date time related operations.
    + */
    +public class DateTimeFunctions {
    --- End diff --
    
    We try to keep the code base of the Table API consistently in Scala. There are a few classes in Java which are mostly copied from Calcite.
    Can you port this class?


---
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 #4078: [FLINK-6693] [table] Support DATE_FORMAT function ...

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

    https://github.com/apache/flink/pull/4078#discussion_r122254102
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/DateTimeFunctionTest.scala ---
    @@ -0,0 +1,49 @@
    +/*
    + * 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.expressions
    +
    +import java.sql.Timestamp
    +
    +import org.apache.flink.api.common.typeinfo.{TypeInformation, Types}
    +import org.apache.flink.api.java.typeutils.RowTypeInfo
    +import org.apache.flink.table.api.scala._
    +import org.apache.flink.table.expressions.utils.ExpressionTestBase
    +import org.apache.flink.types.Row
    +import org.junit.Test
    +
    +class DateTimeFunctionTest extends ExpressionTestBase {
    +
    +  @Test
    +  def testDateFormat(): Unit = {
    +    testAllApis(
    +      DateFormat('f0, "%Y"),
    --- End diff --
    
    I think it would be good to have a pattern that uses all features. 


---
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 #4078: [FLINK-6693] [table] Support DATE_FORMAT function ...

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

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


---
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 #4078: [FLINK-6693] [table] Support DATE_FORMAT function ...

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

    https://github.com/apache/flink/pull/4078#discussion_r126746604
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala ---
    @@ -2008,6 +2008,33 @@ class CodeGenerator(
       }
     
       /**
    +    * Adds a reusable [[java.util.Random]] to the member area of the generated [[Function]].
    --- End diff --
    
    `java.util.Random` -> `DataTimeFormatter`


---
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 #4078: [FLINK-6693] [table] Support DATE_FORMAT function in the ...

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

    https://github.com/apache/flink/pull/4078
  
    Thanks for the update @haohui!
    I'll tweak the docs a bit and merge this PR.
    



---
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 #4078: [FLINK-6693] [table] Support DATE_FORMAT function ...

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

    https://github.com/apache/flink/pull/4078#discussion_r123718414
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/time.scala ---
    @@ -375,3 +377,13 @@ case class TemporalOverlaps(
       }
     }
     
    +case class DateFormat(timestamp: Expression, format: Expression) extends Expression {
    +  override private[flink] def children = timestamp :: format :: Nil
    +
    +  override private[flink] def toRexNode(implicit relBuilder: RelBuilder) =
    +    relBuilder.call(DateTimeSqlFunction.DATE_FORMAT, timestamp.toRexNode, format.toRexNode)
    +
    +  override def toString: String = s"$timestamp.dateformat($format)"
    --- End diff --
    
    `dateFormat`


---
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 #4078: [FLINK-6693] [table] Support DATE_FORMAT function ...

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

    https://github.com/apache/flink/pull/4078#discussion_r126747228
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/BuiltInMethods.scala ---
    @@ -89,5 +89,4 @@ object BuiltInMethods {
       val CONCAT_WS =
         Types.lookupMethod(
           classOf[ScalarFunctions], "concat_ws", classOf[String], classOf[Array[String]])
    -
    --- End diff --
    
    remove unrelated change


---
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 #4078: [FLINK-6693] [table] Support DATE_FORMAT function ...

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

    https://github.com/apache/flink/pull/4078#discussion_r123717228
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/DateTimeFunctionTest.scala ---
    @@ -0,0 +1,49 @@
    +/*
    + * 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.expressions
    +
    +import java.sql.Timestamp
    +
    +import org.apache.flink.api.common.typeinfo.{TypeInformation, Types}
    +import org.apache.flink.api.java.typeutils.RowTypeInfo
    +import org.apache.flink.table.api.scala._
    +import org.apache.flink.table.expressions.utils.ExpressionTestBase
    +import org.apache.flink.types.Row
    +import org.junit.Test
    +
    +class DateTimeFunctionTest extends ExpressionTestBase {
    +
    +  @Test
    +  def testDateFormat(): Unit = {
    --- End diff --
    
    Can you add 2 or 3 more complicated tests?


---
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 #4078: [FLINK-6693] [table] Support DATE_FORMAT function ...

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

    https://github.com/apache/flink/pull/4078#discussion_r126747879
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/DateFormatCallGen.scala ---
    @@ -0,0 +1,44 @@
    +/*
    + * 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.codegen.calls
    +
    +import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
    +import org.apache.flink.table.codegen.{CodeGenerator, GeneratedExpression}
    +import org.apache.flink.table.codegen.calls.CallGenerator.generateCallIfArgsNotNull
    +
    +class DateFormatCallGen extends CallGenerator {
    +  override def generate(codeGenerator: CodeGenerator,
    +                        operands: Seq[GeneratedExpression])
    +  : GeneratedExpression = {
    +
    +    if (operands.last.literal) {
    +      val formatter = codeGenerator.addReusableDateFormatter(operands.last)
    +      generateCallIfArgsNotNull(codeGenerator.nullCheck, STRING_TYPE_INFO, operands) {
    +        terms =>s"$formatter.print(${terms.head})"
    --- End diff --
    
    +space between `=>` and `s`


---
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 #4078: [FLINK-6693] [table] Support DATE_FORMAT function ...

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

    https://github.com/apache/flink/pull/4078#discussion_r123718165
  
    --- Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/runtime/DateTimeFunctions.java ---
    @@ -0,0 +1,158 @@
    +/*
    + * 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;
    +
    +import org.joda.time.format.DateTimeFormatter;
    +import org.joda.time.format.DateTimeFormatterBuilder;
    +
    +/**
    + * Built-in scalar functions for date time related operations.
    + */
    +public class DateTimeFunctions {
    +	private static final int PIVOT_YEAR = 2020;
    +
    +	private static final ThreadLocalCache<String, DateTimeFormatter> DATETIME_FORMATTER_CACHE =
    +		new ThreadLocalCache<String, DateTimeFormatter>(100) {
    +			@Override
    +			protected DateTimeFormatter getNewInstance(String format) {
    +				return createDateTimeFormatter(format);
    +			}
    +		};
    +
    +	public static String dateFormat(long ts, String formatString) {
    +		DateTimeFormatter formatter = DATETIME_FORMATTER_CACHE.get(formatString);
    --- End diff --
    
    I recently added a `literal` flag to generated expressions. So we can check if the parameters of a constructor are constant (see `RandCallGen`). Would be great if you could also change `RandCallGen` to use the ThreadLocalCache if seed is not constant. 


---
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 #4078: [FLINK-6693] [table] Support DATE_FORMAT function ...

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

    https://github.com/apache/flink/pull/4078#discussion_r126870195
  
    --- Diff: flink-libraries/flink-table/pom.xml ---
    @@ -110,6 +110,11 @@ under the License.
                 <scope>compile</scope>
             </dependency>
     
    +		<dependency>
    +			<groupId>joda-time</groupId>
    +			<artifactId>joda-time</artifactId>
    --- End diff --
    
    There is a explicit version in the pom.xml at the root level.


---
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 #4078: [FLINK-6693] [table] Support DATE_FORMAT function ...

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

    https://github.com/apache/flink/pull/4078#discussion_r126750513
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala ---
    @@ -374,6 +375,7 @@ class BasicOperatorTable extends ReflectiveSqlOperatorTable {
         SqlStdOperatorTable.CURRENT_TIME,
         SqlStdOperatorTable.CURRENT_TIMESTAMP,
         SqlStdOperatorTable.CURRENT_DATE,
    +    DateTimeSqlFunction.DATE_FORMAT,
    --- End diff --
    
    The new function should also be added to the documentation (Table API + 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 #4078: [FLINK-6693] [table] Support DATE_FORMAT function ...

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

    https://github.com/apache/flink/pull/4078#discussion_r123718210
  
    --- Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/runtime/DateTimeFunctions.java ---
    @@ -0,0 +1,158 @@
    +/*
    + * 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;
    --- End diff --
    
    I would move this into a separate package. Maybe `org.apache.flink.table.runtime.functions`. I think there will be more functions of this kind soon.


---
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 #4078: [FLINK-6693] [table] Support DATE_FORMAT function ...

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

    https://github.com/apache/flink/pull/4078#discussion_r123718616
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/DateTimeSqlFunction.scala ---
    @@ -0,0 +1,34 @@
    +/*
    + * 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.functions
    --- End diff --
    
    I would move this to `org.apache.flink.table.functions.sql`. There will be more functions of this kind in the future.


---
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 #4078: [FLINK-6693] [table] Support DATE_FORMAT function ...

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

    https://github.com/apache/flink/pull/4078#discussion_r126746651
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala ---
    @@ -2008,6 +2008,33 @@ class CodeGenerator(
       }
     
       /**
    +    * Adds a reusable [[java.util.Random]] to the member area of the generated [[Function]].
    +    *
    +    * The seed parameter must be a literal/constant expression.
    --- End diff --
    
    The format parameter must be a literal/constant expression.


---
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 #4078: [FLINK-6693] [table] Support DATE_FORMAT function ...

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

    https://github.com/apache/flink/pull/4078#discussion_r126748981
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/DateTimeSqlFunction.scala ---
    @@ -0,0 +1,34 @@
    +/*
    + * 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.functions
    +
    +import org.apache.calcite.sql.`type`._
    +import org.apache.calcite.sql.{SqlFunction, SqlFunctionCategory, SqlKind}
    +
    +
    +object DateTimeSqlFunction {
    --- End diff --
    
    It seems that this class has been copied to `org.apache.flink.table.functions.sql` as suggested by @twalthr. I assume this version can be deleted, right?


---
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 #4078: [FLINK-6693] [table] Support DATE_FORMAT function in the ...

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

    https://github.com/apache/flink/pull/4078
  
    I also noticed that the function for the Scala Table API and the Table API documentation are missing. Will add those 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 #4078: [FLINK-6693] [table] Support DATE_FORMAT function ...

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

    https://github.com/apache/flink/pull/4078#discussion_r126746348
  
    --- Diff: flink-libraries/flink-table/pom.xml ---
    @@ -110,6 +110,11 @@ under the License.
                 <scope>compile</scope>
             </dependency>
     
    +		<dependency>
    +			<groupId>joda-time</groupId>
    +			<artifactId>joda-time</artifactId>
    --- End diff --
    
    Where is the `joda-time` version inferred from?


---
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 #4078: [FLINK-6693] [table] Support DATE_FORMAT function ...

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

    https://github.com/apache/flink/pull/4078#discussion_r126870280
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala ---
    @@ -2008,6 +2008,33 @@ class CodeGenerator(
       }
     
       /**
    +    * Adds a reusable [[java.util.Random]] to the member area of the generated [[Function]].
    +    *
    +    * The seed parameter must be a literal/constant expression.
    +    *
    +    * @return member variable term
    +    */
    +  def addReusableDateFormatter(format: GeneratedExpression): String = {
    +    val fieldTerm = newName("dateFormatter")
    +
    +    val field =
    +      s"""
    +         |transient org.joda.time.format.DateTimeFormatter $fieldTerm;
    +         |""".stripMargin
    +    reusableMemberStatements.add(field)
    +
    +    val fieldInit =
    +      s"""
    +         |${format.code}
    +         |$fieldTerm = org.apache.flink.table.runtime.functions.
    +         |DateTimeFunctions$$.MODULE$$.createDateTimeFormatter(${format.resultTerm});
    --- End diff --
    
    This is because the runtime ``DateTimeFunctions`` has been ported to Scala. This is the syntax to access the function.


---
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 #4078: [FLINK-6693] [table] Support DATE_FORMAT function in the ...

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

    https://github.com/apache/flink/pull/4078
  
    @fhueske @twalthr please take another look. Thanks!


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