You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/07/14 14:04:53 UTC

[GitHub] [flink] twalthr opened a new pull request #12900: [FLINK-18537][table] Remove RowDataTypeInfo

twalthr opened a new pull request #12900:
URL: https://github.com/apache/flink/pull/12900


   ## What is the purpose of the change
   
   `TypeInformation` is a legacy class for the sole purpose of creating a `TypeSerializer`.
    Instances of `TypeInformation` are not required in the table ecosystem but sometimes enforced
    by interfaces of other modules (such as `org.apache.flink.api.dag.Transformation`). Therefore, we recently introduced `WrapperTypeInfo` which acts as an adapter whenever type information is required. Instances of `WrapperTypeInfo` should only be created for passing it to interfaces that require type information. The class should not be used as a replacement for a `LogicalType`. Information such as the arity of a row type, field types, field names, etc. should be derived from the `LogicalType` directly.
   
   This PR updates the code base to not use `RowDataTypeInfo` anymore. `RowDataTypeInfo` was a composite type information with legacy types as field types. This refactoring unblocks FLINK-15803 and reduces potential bugs from type conversion. 
   
   ## Brief change log
   
   See commit messages.
   
   ## Verifying this change
   
   This change is already covered by existing tests.
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): no
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: yes
     - The serializers: yes
     - The runtime per-record code paths (performance sensitive): no
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: no
     - The S3 file system connector: no
   
   ## Documentation
   
     - Does this pull request introduce a new feature? no
     - If yes, how is the feature documented? not applicable
   


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

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



[GitHub] [flink] flinkbot edited a comment on pull request #12900: [FLINK-18537][table] Remove RowDataTypeInfo

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12900:
URL: https://github.com/apache/flink/pull/12900#issuecomment-658213136


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ceb895be265bd445b9d36bfc52cf8018a508c333",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4508",
       "triggerID" : "ceb895be265bd445b9d36bfc52cf8018a508c333",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e61dfa9f39d348b4e1273ab7329367706c0f6651",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4602",
       "triggerID" : "e61dfa9f39d348b4e1273ab7329367706c0f6651",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e61dfa9f39d348b4e1273ab7329367706c0f6651 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4602) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

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



[GitHub] [flink] twalthr commented on a change in pull request #12900: [FLINK-18537][table] Remove RowDataTypeInfo

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #12900:
URL: https://github.com/apache/flink/pull/12900#discussion_r456313358



##########
File path: flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/data/util/DataFormatConverters.java
##########
@@ -249,7 +251,15 @@ public static DataFormatConverter getConverterForDataType(DataType originDataTyp
 						DataTypes.INT().bridgedTo(Integer.class));
 			case ROW:
 			case STRUCTURED_TYPE:
-				CompositeType compositeType = (CompositeType) fromDataTypeToTypeInfo(dataType);
+				TypeInformation<?> asTypeInfo = fromDataTypeToTypeInfo(dataType);
+				if (asTypeInfo instanceof WrapperTypeInfo && clazz == RowData.class) {
+					LogicalType realLogicalType = ((WrapperTypeInfo<?>) asTypeInfo).toLogicalType();
+					return new RowDataConverter(getFieldCount(realLogicalType));

Review comment:
       All cases that went through `TypeInfoDataTypeConverter#fromDataTypeToTypeInfo` e.g. the legacy sinks.




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

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



[GitHub] [flink] wuchong commented on a change in pull request #12900: [FLINK-18537][table] Remove RowDataTypeInfo

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #12900:
URL: https://github.com/apache/flink/pull/12900#discussion_r455525030



##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGeneratorContext.scala
##########
@@ -18,23 +18,23 @@
 
 package org.apache.flink.table.planner.codegen
 
+import java.util.TimeZone
+
+import org.apache.calcite.avatica.util.DateTimeUtils

Review comment:
       What's the preferred import order? AFAIK, the code style suggests to put flink imports first: https://flink.apache.org/contributing/code-style-and-quality-formatting.html#imports

##########
File path: flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/data/util/DataFormatConverters.java
##########
@@ -249,7 +251,15 @@ public static DataFormatConverter getConverterForDataType(DataType originDataTyp
 						DataTypes.INT().bridgedTo(Integer.class));
 			case ROW:
 			case STRUCTURED_TYPE:
-				CompositeType compositeType = (CompositeType) fromDataTypeToTypeInfo(dataType);
+				TypeInformation<?> asTypeInfo = fromDataTypeToTypeInfo(dataType);
+				if (asTypeInfo instanceof WrapperTypeInfo && clazz == RowData.class) {
+					LogicalType realLogicalType = ((WrapperTypeInfo<?>) asTypeInfo).toLogicalType();
+					return new RowDataConverter(getFieldCount(realLogicalType));

Review comment:
       In which case will we reach here?

##########
File path: flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/AsyncLookupJoinHarnessTest.java
##########
@@ -75,18 +74,19 @@
 	private static final int ASYNC_TIMEOUT_MS = 3000;
 
 	private final TypeSerializer<RowData> inSerializer = new RowDataSerializer(
-		new ExecutionConfig(),
 		new IntType(),
 		new VarCharType(VarCharType.MAX_LENGTH));
 
-	private final RowDataHarnessAssertor assertor = new RowDataHarnessAssertor(new TypeInformation[]{
-		Types.INT,
-		Types.STRING,
-		Types.INT,
-		Types.STRING
+	private final RowDataHarnessAssertor assertor = new RowDataHarnessAssertor(new LogicalType[]{
+		new IntType(),
+		new VarCharType(VarCharType.MAX_LENGTH),
+		new IntType(),
+		new VarCharType(VarCharType.MAX_LENGTH)

Review comment:
       Keep to use DataType here? 
   
   ```java
                   DataTypes.INT().getLogicalType(),
   		DataTypes.STRING().getLogicalType(),
   		DataTypes.INT().getLogicalType(),
   		DataTypes.STRING().getLogicalType()
   ```




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

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



[GitHub] [flink] flinkbot edited a comment on pull request #12900: [FLINK-18537][table] Remove RowDataTypeInfo

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12900:
URL: https://github.com/apache/flink/pull/12900#issuecomment-658213136


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ceb895be265bd445b9d36bfc52cf8018a508c333",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4508",
       "triggerID" : "ceb895be265bd445b9d36bfc52cf8018a508c333",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ceb895be265bd445b9d36bfc52cf8018a508c333 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4508) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

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



[GitHub] [flink] flinkbot edited a comment on pull request #12900: [FLINK-18537][table] Remove RowDataTypeInfo

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12900:
URL: https://github.com/apache/flink/pull/12900#issuecomment-658213136


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ceb895be265bd445b9d36bfc52cf8018a508c333",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4508",
       "triggerID" : "ceb895be265bd445b9d36bfc52cf8018a508c333",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e61dfa9f39d348b4e1273ab7329367706c0f6651",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4602",
       "triggerID" : "e61dfa9f39d348b4e1273ab7329367706c0f6651",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ceb895be265bd445b9d36bfc52cf8018a508c333 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4508) 
   * e61dfa9f39d348b4e1273ab7329367706c0f6651 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4602) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

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



[GitHub] [flink] flinkbot edited a comment on pull request #12900: [FLINK-18537][table] Remove RowDataTypeInfo

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12900:
URL: https://github.com/apache/flink/pull/12900#issuecomment-658213136


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ceb895be265bd445b9d36bfc52cf8018a508c333",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4508",
       "triggerID" : "ceb895be265bd445b9d36bfc52cf8018a508c333",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e61dfa9f39d348b4e1273ab7329367706c0f6651",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4602",
       "triggerID" : "e61dfa9f39d348b4e1273ab7329367706c0f6651",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ea9bb9484110719ab5dc19c00d4a25ab243d334",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4690",
       "triggerID" : "7ea9bb9484110719ab5dc19c00d4a25ab243d334",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e61dfa9f39d348b4e1273ab7329367706c0f6651 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4602) 
   * 7ea9bb9484110719ab5dc19c00d4a25ab243d334 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4690) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

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



[GitHub] [flink] flinkbot edited a comment on pull request #12900: [FLINK-18537][table] Remove RowDataTypeInfo

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12900:
URL: https://github.com/apache/flink/pull/12900#issuecomment-658213136


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ceb895be265bd445b9d36bfc52cf8018a508c333",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4508",
       "triggerID" : "ceb895be265bd445b9d36bfc52cf8018a508c333",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e61dfa9f39d348b4e1273ab7329367706c0f6651",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e61dfa9f39d348b4e1273ab7329367706c0f6651",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ceb895be265bd445b9d36bfc52cf8018a508c333 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4508) 
   * e61dfa9f39d348b4e1273ab7329367706c0f6651 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

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



[GitHub] [flink] wuchong commented on a change in pull request #12900: [FLINK-18537][table] Remove RowDataTypeInfo

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #12900:
URL: https://github.com/apache/flink/pull/12900#discussion_r455549636



##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/DataTypeQueryable.java
##########
@@ -0,0 +1,30 @@
+/*
+ * 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.types;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/**
+ * Indicates that a {@link DataType} can be retrieved from a class that implements this interfaces.
+ */
+@PublicEvolving
+public interface DataTypeQueryable {

Review comment:
       Is it possible to make the construct of `WrapperTypeInfo` to be package visible? We can move the utility `TypeInfoDataTypeConverter#createWrapperTypeInfo` into `WrapperTypeInfo`. And the remaining constructor invoking are the tests under the same package. 




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

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



[GitHub] [flink] dawidwys commented on a change in pull request #12900: [FLINK-18537][table] Remove RowDataTypeInfo

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #12900:
URL: https://github.com/apache/flink/pull/12900#discussion_r454415036



##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/DataTypeQueryable.java
##########
@@ -0,0 +1,30 @@
+/*
+ * 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.types;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/**
+ * Indicates that a {@link DataType} can be retrieved from a class that implements this interfaces.
+ */
+@PublicEvolving
+public interface DataTypeQueryable {

Review comment:
       Is this interface really necessary? Especially with `@PublicEvolving` annotation? How are users supposed to use it? If I understand it correctly you need it for internal operations. Moreover you need it because the `WrapperTypeInfo` is in `blink-runtime`, right?
   
   Can't we move the `WrapperTypeInfo` to the `table-common` instead?  The class itself has no runtime dependencies. Only the factory methods need some runtime classes.




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

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



[GitHub] [flink] twalthr commented on a change in pull request #12900: [FLINK-18537][table] Remove RowDataTypeInfo

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #12900:
URL: https://github.com/apache/flink/pull/12900#discussion_r456301095



##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGeneratorContext.scala
##########
@@ -18,23 +18,23 @@
 
 package org.apache.flink.table.planner.codegen
 
+import java.util.TimeZone
+
+import org.apache.calcite.avatica.util.DateTimeUtils

Review comment:
       Thanks for the reminder. I only set this for Java not for Scala. However, it difficult to enforce this without a checkstyle plugin in place.




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

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



[GitHub] [flink] twalthr commented on a change in pull request #12900: [FLINK-18537][table] Remove RowDataTypeInfo

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #12900:
URL: https://github.com/apache/flink/pull/12900#discussion_r454506847



##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/DataTypeQueryable.java
##########
@@ -0,0 +1,30 @@
+/*
+ * 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.types;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/**
+ * Indicates that a {@link DataType} can be retrieved from a class that implements this interfaces.
+ */
+@PublicEvolving
+public interface DataTypeQueryable {

Review comment:
       I thought about this as well and decided for keeping `WrapperTypeInfo` in the runtime package. The reason for this is to keep the serializers close to this utility class in the same package. `WrapperTypeInfo` is not needed in `table-common` because it is internal. Actually the constructor of `WrappenTypeInfo` should even be private but this is not possible due to other failing tests so far. We can mark `DataTypeQueryable` internal if that helps, but the past around `TypeInformation` has shown that you need such a simple interface over and over again. It will be more useful in the future once we introduce a "converter type serializer" where the return data type is not of internal data structure.




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

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



[GitHub] [flink] twalthr commented on pull request #12900: [FLINK-18537][table] Remove RowDataTypeInfo

Posted by GitBox <gi...@apache.org>.
twalthr commented on pull request #12900:
URL: https://github.com/apache/flink/pull/12900#issuecomment-658201596


   CC @wuchong @dawidwys
   Btw after all this refactoring I was thinking about renaming `WrapperTypeInfo` to `InternalTypeInfo`. This would better match to its purpose. What do you think?


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

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



[GitHub] [flink] flinkbot edited a comment on pull request #12900: [FLINK-18537][table] Remove RowDataTypeInfo

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12900:
URL: https://github.com/apache/flink/pull/12900#issuecomment-658213136


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ceb895be265bd445b9d36bfc52cf8018a508c333",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4508",
       "triggerID" : "ceb895be265bd445b9d36bfc52cf8018a508c333",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e61dfa9f39d348b4e1273ab7329367706c0f6651",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4602",
       "triggerID" : "e61dfa9f39d348b4e1273ab7329367706c0f6651",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ea9bb9484110719ab5dc19c00d4a25ab243d334",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7ea9bb9484110719ab5dc19c00d4a25ab243d334",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e61dfa9f39d348b4e1273ab7329367706c0f6651 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4602) 
   * 7ea9bb9484110719ab5dc19c00d4a25ab243d334 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

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



[GitHub] [flink] flinkbot commented on pull request #12900: [FLINK-18537][table] Remove RowDataTypeInfo

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #12900:
URL: https://github.com/apache/flink/pull/12900#issuecomment-658202311


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit ceb895be265bd445b9d36bfc52cf8018a508c333 (Tue Jul 14 14:09:33 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


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

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



[GitHub] [flink] flinkbot edited a comment on pull request #12900: [FLINK-18537][table] Remove RowDataTypeInfo

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12900:
URL: https://github.com/apache/flink/pull/12900#issuecomment-658213136


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ceb895be265bd445b9d36bfc52cf8018a508c333",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4508",
       "triggerID" : "ceb895be265bd445b9d36bfc52cf8018a508c333",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ceb895be265bd445b9d36bfc52cf8018a508c333 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4508) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

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



[GitHub] [flink] twalthr commented on pull request #12900: [FLINK-18537][table] Remove RowDataTypeInfo

Posted by GitBox <gi...@apache.org>.
twalthr commented on pull request #12900:
URL: https://github.com/apache/flink/pull/12900#issuecomment-660201270


   Thanks for the feedback @wuchong and @dawidwys. I hope I addressed all comments. I renamed `WrapperTypeInfo` to `InternalTypeInfo` and introduced `ExternalTypeInfo` already for the raw type and for the future bridging to type information of DataStream API.


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

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



[GitHub] [flink] flinkbot commented on pull request #12900: [FLINK-18537][table] Remove RowDataTypeInfo

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #12900:
URL: https://github.com/apache/flink/pull/12900#issuecomment-658213136


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ceb895be265bd445b9d36bfc52cf8018a508c333",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ceb895be265bd445b9d36bfc52cf8018a508c333",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ceb895be265bd445b9d36bfc52cf8018a508c333 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

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



[GitHub] [flink] twalthr closed pull request #12900: [FLINK-18537][table] Remove RowDataTypeInfo

Posted by GitBox <gi...@apache.org>.
twalthr closed pull request #12900:
URL: https://github.com/apache/flink/pull/12900


   


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

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



[GitHub] [flink] wuchong commented on a change in pull request #12900: [FLINK-18537][table] Remove RowDataTypeInfo

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #12900:
URL: https://github.com/apache/flink/pull/12900#discussion_r455548049



##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/DataTypeQueryable.java
##########
@@ -0,0 +1,30 @@
+/*
+ * 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.types;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/**
+ * Indicates that a {@link DataType} can be retrieved from a class that implements this interfaces.
+ */
+@PublicEvolving
+public interface DataTypeQueryable {

Review comment:
       I prefer to keep `WrapperTypeInfo` in `blink-runtime`, especially we have a public construct which will cause a lot of problems if the `logicalType` and `typeSerializer` are not mapped.




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

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