[GitHub] flink pull request #5040: [FLINK-8104][Table API] fixing ROW type value cons...
Github user asfgit closed the pull request at: https://github.com/apache/flink/pull/5040 ---
[GitHub] flink pull request #5040: [FLINK-8104][Table API] fixing ROW type value cons...
Github user walterddr commented on a diff in the pull request: https://github.com/apache/flink/pull/5040#discussion_r154181460 --- Diff: docs/dev/table/tableApi.md --- @@ -2993,6 +2993,30 @@ MAP.at(ANY) + + + + Row functions + Description + + + + + + + +{% highlight java %} --- End diff -- seems like it is also missing `Map function` scala doc as well. adding in ---
[GitHub] flink pull request #5040: [FLINK-8104][Table API] fixing ROW type value cons...
Github user walterddr commented on a diff in the pull request: https://github.com/apache/flink/pull/5040#discussion_r154179674 --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/RowTypeTestBase.scala --- @@ -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.expressions.utils + +import java.sql.Date + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.{ObjectArrayTypeInfo, RowTypeInfo} +import org.apache.flink.table.api.Types +import org.apache.flink.types.Row + +class RowTypeTestBase extends ExpressionTestBase { --- End diff -- Forgot to add `RowTypeValidationTest` in previous diff actually ---
[GitHub] flink pull request #5040: [FLINK-8104][Table API] fixing ROW type value cons...
Github user twalthr commented on a diff in the pull request: https://github.com/apache/flink/pull/5040#discussion_r153801798 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/expressionDsl.scala --- @@ -959,6 +959,19 @@ object array { } } +/** + * Creates an row of literals. --- End diff -- literals = expressions This should also be fixed for the map constructor. ---
[GitHub] flink pull request #5040: [FLINK-8104][Table API] fixing ROW type value cons...
Github user twalthr commented on a diff in the pull request: https://github.com/apache/flink/pull/5040#discussion_r153802956 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala --- @@ -1308,6 +1313,19 @@ abstract class CodeGenerator( } } + private[flink] def generateBoxedElementWithNullableSupport( + element: GeneratedExpression, + boxedTypeTerm: String) --- End diff -- This should be a parameter with TypeInformation type. ---
[GitHub] flink pull request #5040: [FLINK-8104][Table API] fixing ROW type value cons...
Github user twalthr commented on a diff in the pull request: https://github.com/apache/flink/pull/5040#discussion_r153803084 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala --- @@ -1556,6 +1574,21 @@ abstract class CodeGenerator( fieldTerms.toArray } + /** +* Add a reusable [[org.apache.flink.types.Row]] --- End diff -- Adds ---
[GitHub] flink pull request #5040: [FLINK-8104][Table API] fixing ROW type value cons...
Github user twalthr commented on a diff in the pull request: https://github.com/apache/flink/pull/5040#discussion_r153801541 --- Diff: docs/dev/table/tableApi.md --- @@ -2993,6 +2993,30 @@ MAP.at(ANY) + + + + Row functions + Description + + + + + + + +{% highlight java %} --- End diff -- Please also add documentation for the Scala. ---
[GitHub] flink pull request #5040: [FLINK-8104][Table API] fixing ROW type value cons...
Github user twalthr commented on a diff in the pull request: https://github.com/apache/flink/pull/5040#discussion_r153803851 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala --- @@ -849,6 +850,36 @@ object ScalarOperators { generateUnaryArithmeticOperator(operator, nullCheck, operand.resultType, operand) } + def generateRow( + codeGenerator: CodeGenerator, + resultType: TypeInformation[_], + elements: Seq[GeneratedExpression]) + : GeneratedExpression = { +val rowTerm = codeGenerator.addReusableRow(resultType.getArity, elements.size) --- End diff -- Maybe add a check here that validates `resultType.getArity == elements.size`. Otherwise throw a code generator exception. ---
[GitHub] flink pull request #5040: [FLINK-8104][Table API] fixing ROW type value cons...
Github user twalthr commented on a diff in the pull request: https://github.com/apache/flink/pull/5040#discussion_r153801369 --- Diff: docs/dev/table/tableApi.md --- @@ -2993,6 +2993,30 @@ MAP.at(ANY) + + + + Row functions + Description + + + + + + + +{% highlight java %} +row(ANY, [, ANY]*) +{% endhighlight %} + + +Creates a row from a list of values. row can be access via built-in functions (see Value access functions section) --- End diff -- `A row is a composite type that can row can be access via value access functions.` ---
[GitHub] flink pull request #5040: [FLINK-8104][Table API] fixing ROW type value cons...
Github user twalthr commented on a diff in the pull request: https://github.com/apache/flink/pull/5040#discussion_r153807597 --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala --- @@ -308,6 +308,27 @@ class CalcITCase( TestBaseUtils.compareResultAsText(results.asJava, expected) } + @Test + def testValueConstructor(): Unit = { +val env = ExecutionEnvironment.getExecutionEnvironment +val tEnv = TableEnvironment.getTableEnvironment(env, config) + +val sqlQuery = "SELECT (a, b, c), ARRAY[b, b] FROM MyTable" + +val ds = env.fromElements(( + "foo", + 12, + Timestamp.valueOf("1984-07-12 14:34:24"))) +tEnv.registerDataSet("MyTable", ds, 'a, 'b, 'c) + +val result = tEnv.sqlQuery(sqlQuery) + +val expected = "foo,12,1984-07-12 14:34:24.0,[12, 12]" +val results = result.toDataSet[Row].collect() --- End diff -- Can you test the real objects instead of the strings? To make sure that row has not been flattened by Calcite. ---
[GitHub] flink pull request #5040: [FLINK-8104][Table API] fixing ROW type value cons...
Github user twalthr commented on a diff in the pull request: https://github.com/apache/flink/pull/5040#discussion_r153803232 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala --- @@ -1556,6 +1574,21 @@ abstract class CodeGenerator( fieldTerms.toArray } + /** +* Add a reusable [[org.apache.flink.types.Row]] +* to the member area of the generated [[Function]]. +*/ + def addReusableRow(arity: Int, size: Int): String = { --- End diff -- Remove size. ---
[GitHub] flink pull request #5040: [FLINK-8104][Table API] fixing ROW type value cons...
Github user twalthr commented on a diff in the pull request: https://github.com/apache/flink/pull/5040#discussion_r153802731 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala --- @@ -1308,6 +1313,19 @@ abstract class CodeGenerator( } } + private[flink] def generateBoxedElementWithNullableSupport( --- End diff -- `generateNullableOutputBoxing` ---
[GitHub] flink pull request #5040: [FLINK-8104][Table API] fixing ROW type value cons...
Github user twalthr commented on a diff in the pull request: https://github.com/apache/flink/pull/5040#discussion_r153800721 --- Diff: docs/dev/table/sql.md --- @@ -2355,6 +2355,41 @@ map â[â key â]â + --- End diff -- I would put this under value constructor functions to stay in sync with Calcites documentation (http://calcite.apache.org/docs/reference.html#value-constructors). ---
[GitHub] flink pull request #5040: [FLINK-8104][Table API] fixing ROW type value cons...
Github user twalthr commented on a diff in the pull request: https://github.com/apache/flink/pull/5040#discussion_r153806967 --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/RowTypeTestBase.scala --- @@ -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.expressions.utils + +import java.sql.Date + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.{ObjectArrayTypeInfo, RowTypeInfo} +import org.apache.flink.table.api.Types +import org.apache.flink.types.Row + +class RowTypeTestBase extends ExpressionTestBase { --- End diff -- I think we don't need a `TestBase` class if there is only one class extending it. ---
[GitHub] flink pull request #5040: [FLINK-8104][Table API] fixing ROW type value cons...
Github user twalthr commented on a diff in the pull request: https://github.com/apache/flink/pull/5040#discussion_r152588446 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala --- @@ -835,6 +836,43 @@ object ScalarOperators { generateUnaryArithmeticOperator(operator, nullCheck, operand.resultType, operand) } + def generateRow( + codeGenerator: CodeGenerator, + resultType: TypeInformation[_], + elements: Seq[GeneratedExpression]) + : GeneratedExpression = { +val rowTerm = codeGenerator.addReusableRow(resultType.getArity, elements.size) + +val boxedElements: Seq[GeneratedExpression] = resultType match { + case ct: CompositeType[_] => +elements.zipWithIndex.map{ + case (e, idx) => +val boxedTypeTerm = boxedTypeTermForTypeInfo(ct.getTypeAt(idx)) --- End diff -- Can we put this into a private helper method? I think it is the same code for `generateArray`, `generateMap`, and `generateRow`. ---
[GitHub] flink pull request #5040: [FLINK-8104][Table API] fixing ROW type value cons...
Github user twalthr commented on a diff in the pull request: https://github.com/apache/flink/pull/5040#discussion_r152614003 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala --- @@ -417,6 +417,10 @@ object FlinkTypeFactory { val compositeRelDataType = relDataType.asInstanceOf[CompositeRelDataType] compositeRelDataType.compositeType +case ROW if relDataType.isInstanceOf[RelRecordType] => + val relRecordType = relDataType.asInstanceOf[RelRecordType] + new RowSchema(relRecordType).typeInfo + // ROW and CURSOR for UDTF case, whose type info will never be used, just a placeholder case ROW | CURSOR => new NothingTypeInfo --- End diff -- Maybe we can remove the `ROW` here if it has no impact on the existing tests. ---
[GitHub] flink pull request #5040: [FLINK-8104][Table API] fixing ROW type value cons...
Github user twalthr commented on a diff in the pull request: https://github.com/apache/flink/pull/5040#discussion_r152588168 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala --- @@ -835,6 +836,43 @@ object ScalarOperators { generateUnaryArithmeticOperator(operator, nullCheck, operand.resultType, operand) } + def generateRow( + codeGenerator: CodeGenerator, + resultType: TypeInformation[_], + elements: Seq[GeneratedExpression]) + : GeneratedExpression = { +val rowTerm = codeGenerator.addReusableRow(resultType.getArity, elements.size) + +val boxedElements: Seq[GeneratedExpression] = resultType match { + case ct: CompositeType[_] => --- End diff -- This should always be `RowTypeInfo`. I think we can simply cast it. ---
[GitHub] flink pull request #5040: [FLINK-8104][Table API] fixing ROW type value cons...
GitHub user walterddr opened a pull request: https://github.com/apache/flink/pull/5040 [FLINK-8104][Table API] fixing ROW type value constructor for SQL API. ## What is the purpose of the change Supports creation of a ROW based on fields or literals. such as: `ROW('hello', 12, true)` or simply `('hello', 12, true)`. ## Brief change log - Added in `generateRow` for Row type generation - Fix FlinkTypeFactory not able to recognize `RelRecordType` by resolving it through `RowSchema` class. ## Verifying this change This change added tests and can be verified as follows: - Added unit test for SQL expression and integration test for value constructor ## Does this pull request potentially affect one of the following parts: No ## Documentation - Does this pull request introduce a new feature? No You can merge this pull request into a Git repository by running: $ git pull https://github.com/walterddr/flink FLINK-8104 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/5040.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 #5040 commit 9909ad1b8abed74964b2fce1025ad3503b5abf98 Author: Rong RongDate: 2017-11-20T21:47:50Z fixing ROW type value constructor for SQL API. Supports creation of a ROW based on fields or literals ---