[jira] [Commented] (FLINK-33200) ItemAt Expression validation fail in Table API due to type mismatch

2023-10-12 Thread Timo Walther (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-33200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17774387#comment-17774387
 ] 

Timo Walther commented on FLINK-33200:
--

We should not use `PlannerTypeInferenceUtilImpl` anymore. There are still 
roughly 20 functions left that use the old inference. Upgrading those functions 
should also fix this issue here. The new stack works purely on logical types 
and the class does not matter anymore. The old stack was based on 
TypeInformation where the class clearly mattered.

> ItemAt Expression validation fail in Table API due to type mismatch
> ---
>
> Key: FLINK-33200
> URL: https://issues.apache.org/jira/browse/FLINK-33200
> Project: Flink
>  Issue Type: Bug
>  Components: Table SQL / API
>Affects Versions: 1.18.0, 1.17.1
>Reporter: Zhenqiu Huang
>Priority: Minor
> Attachments: IssueDemo.java
>
>
> The table schema is defined as below:
> public static final DataType DATA_TYPE = DataTypes.ROW(
> DataTypes.FIELD("id", DataTypes.STRING()),
> DataTypes.FIELD("events", 
> DataTypes.ARRAY(DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING(
> );
> public static final Schema SCHEMA = 
> Schema.newBuilder().fromRowDataType(DATA_TYPE).build();
> inTable.select(Expressions.$("events").at(1).at("eventType").as("firstEventType")
> The validation fail as "eventType" is inferred as 
> BasicTypeInfo.STRING_TYPE_INFO, the table key internally is a 
> StringDataTypeInfo. The validation fail at 
> case mti: MapTypeInfo[_, _] =>
> if (key.resultType == mti.getKeyTypeInfo) {
>   ValidationSuccess
> } else {
>   ValidationFailure(
> s"Map entry access needs a valid key of type " +
>   s"'${mti.getKeyTypeInfo}', found '${key.resultType}'.")
> }



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (FLINK-33200) ItemAt Expression validation fail in Table API due to type mismatch

2023-10-11 Thread xuyang (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-33200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17774296#comment-17774296
 ] 

xuyang commented on FLINK-33200:


cc [~twalthr] 

> ItemAt Expression validation fail in Table API due to type mismatch
> ---
>
> Key: FLINK-33200
> URL: https://issues.apache.org/jira/browse/FLINK-33200
> Project: Flink
>  Issue Type: Bug
>  Components: Table SQL / API
>Affects Versions: 1.18.0, 1.17.1
>Reporter: Zhenqiu Huang
>Priority: Minor
> Attachments: IssueDemo.java
>
>
> The table schema is defined as below:
> public static final DataType DATA_TYPE = DataTypes.ROW(
> DataTypes.FIELD("id", DataTypes.STRING()),
> DataTypes.FIELD("events", 
> DataTypes.ARRAY(DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING(
> );
> public static final Schema SCHEMA = 
> Schema.newBuilder().fromRowDataType(DATA_TYPE).build();
> inTable.select(Expressions.$("events").at(1).at("eventType").as("firstEventType")
> The validation fail as "eventType" is inferred as 
> BasicTypeInfo.STRING_TYPE_INFO, the table key internally is a 
> StringDataTypeInfo. The validation fail at 
> case mti: MapTypeInfo[_, _] =>
> if (key.resultType == mti.getKeyTypeInfo) {
>   ValidationSuccess
> } else {
>   ValidationFailure(
> s"Map entry access needs a valid key of type " +
>   s"'${mti.getKeyTypeInfo}', found '${key.resultType}'.")
> }



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (FLINK-33200) ItemAt Expression validation fail in Table API due to type mismatch

2023-10-11 Thread xuyang (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-33200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17774295#comment-17774295
 ] 

xuyang commented on FLINK-33200:


Hi, [~andyglow] , maybe this flip[1] can be referred.

[1][https://cwiki.apache.org/confluence/display/FLINK/FLIP-37%3A+Rework+of+the+Table+API+Type+System]

I think it's ok for user to use StringData, and the root problem is the misuse 
of 'InternalTypeInfo' which is tagged as '@Internal'.

> ItemAt Expression validation fail in Table API due to type mismatch
> ---
>
> Key: FLINK-33200
> URL: https://issues.apache.org/jira/browse/FLINK-33200
> Project: Flink
>  Issue Type: Bug
>  Components: Table SQL / API
>Affects Versions: 1.18.0, 1.17.1
>Reporter: Zhenqiu Huang
>Priority: Minor
> Attachments: IssueDemo.java
>
>
> The table schema is defined as below:
> public static final DataType DATA_TYPE = DataTypes.ROW(
> DataTypes.FIELD("id", DataTypes.STRING()),
> DataTypes.FIELD("events", 
> DataTypes.ARRAY(DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING(
> );
> public static final Schema SCHEMA = 
> Schema.newBuilder().fromRowDataType(DATA_TYPE).build();
> inTable.select(Expressions.$("events").at(1).at("eventType").as("firstEventType")
> The validation fail as "eventType" is inferred as 
> BasicTypeInfo.STRING_TYPE_INFO, the table key internally is a 
> StringDataTypeInfo. The validation fail at 
> case mti: MapTypeInfo[_, _] =>
> if (key.resultType == mti.getKeyTypeInfo) {
>   ValidationSuccess
> } else {
>   ValidationFailure(
> s"Map entry access needs a valid key of type " +
>   s"'${mti.getKeyTypeInfo}', found '${key.resultType}'.")
> }



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (FLINK-33200) ItemAt Expression validation fail in Table API due to type mismatch

2023-10-11 Thread Andriy Onyshchuk (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-33200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17774153#comment-17774153
 ] 

Andriy Onyshchuk commented on FLINK-33200:
--

Hello, [~xuyangzhong]. Thanks for your analysis!

Can I get some guidance regarding Row vs RowData, String vs StringData, 
Timestamp/LocalDateTime vs TimestampData, T[] vs ArrayData, etc

I think I saw some comments saying that `*Data` struct types are the 
perspective API that is going to be used going forward. May be I misinterpret 
it. How would you define a guidline?

Thank you

> ItemAt Expression validation fail in Table API due to type mismatch
> ---
>
> Key: FLINK-33200
> URL: https://issues.apache.org/jira/browse/FLINK-33200
> Project: Flink
>  Issue Type: Bug
>  Components: Table SQL / API
>Affects Versions: 1.18.0, 1.17.1
>Reporter: Zhenqiu Huang
>Priority: Minor
> Attachments: IssueDemo.java
>
>
> The table schema is defined as below:
> public static final DataType DATA_TYPE = DataTypes.ROW(
> DataTypes.FIELD("id", DataTypes.STRING()),
> DataTypes.FIELD("events", 
> DataTypes.ARRAY(DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING(
> );
> public static final Schema SCHEMA = 
> Schema.newBuilder().fromRowDataType(DATA_TYPE).build();
> inTable.select(Expressions.$("events").at(1).at("eventType").as("firstEventType")
> The validation fail as "eventType" is inferred as 
> BasicTypeInfo.STRING_TYPE_INFO, the table key internally is a 
> StringDataTypeInfo. The validation fail at 
> case mti: MapTypeInfo[_, _] =>
> if (key.resultType == mti.getKeyTypeInfo) {
>   ValidationSuccess
> } else {
>   ValidationFailure(
> s"Map entry access needs a valid key of type " +
>   s"'${mti.getKeyTypeInfo}', found '${key.resultType}'.")
> }



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (FLINK-33200) ItemAt Expression validation fail in Table API due to type mismatch

2023-10-11 Thread xuyang (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-33200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17773962#comment-17773962
 ] 

xuyang commented on FLINK-33200:


Hi, [~ZhenqiuHuang] , can you provide the code caused this issue?

>From the example provided by [~andyglow] , I found that an internal 
>TypeInformation 'InternalTypeInfo' which should not be used actually is used. 
>After replacing 'InternalTypeInfo' with 'ExternalTypeInfo', issue 1 and 2 all 
>work fine. The new example can be found following. The main changes is:
 # use 'ExternalTypeInfo.of(DATA_TYPE)' instead of 
'InternalTypeInfo.of(DATA_TYPE.getLogicalType())'
 # remove 'ArrayData', 'GenericMapData' and 'StringData'
 # use 'Row' instead of 'RowData'

 
{code:java}
public class DemoTest {

public static class DataEnvelope {
public final String id;
public final List> events;

public DataEnvelope(String id, List> events) {
this.id = id;
this.events = events;
}

public Row toRowData() {
final Row row = new Row(2);
row.setField(0, id);
//row.setField(1, eventsMapData());
row.setField(1, events.toArray());

return row;
}

public static final DataType DATA_TYPE =
DataTypes.ROW(
DataTypes.FIELD("id", DataTypes.STRING()),
DataTypes.FIELD(
"events",
DataTypes.ARRAY(
DataTypes.MAP(DataTypes.STRING(), 
DataTypes.STRING();
public static final TypeInformation TYPE_INFO = 
ExternalTypeInfo.of(DATA_TYPE);

public static final Schema SCHEMA = 
Schema.newBuilder().fromRowDataType(DATA_TYPE).build();
}

public static final List TEST_DATA =
Lists.newArrayList(
new DataEnvelope(
"1", 
Lists.newArrayList(Maps.newHashMap("eventType", "enter"))),
new DataEnvelope(
"2",
Lists.newArrayList(
Maps.newHashMap("eventType", "dialog"),
Maps.newHashMap("eventType", "exit";

public static void main(String[] args) throws Exception {
final StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
final StreamTableEnvironment tenv = StreamTableEnvironment.create(env);
final List inRows =

TEST_DATA.stream().map(DataEnvelope::toRowData).collect(Collectors.toList());
final DataStreamSource inStream = env.fromCollection(inRows, 
DataEnvelope.TYPE_INFO);
final Table inTable = tenv.fromDataStream(inStream, 
DataEnvelope.SCHEMA);
// issue #1

inTable.select(Expressions.$("events").at(1).at("eventType").as("firstEventType"))
.execute()
.print();
// issue #2

inTable.select(Expressions.$("events").at(1).as("firstEvent")).execute().print();
env.close();
}
}
 {code}
 

 

> ItemAt Expression validation fail in Table API due to type mismatch
> ---
>
> Key: FLINK-33200
> URL: https://issues.apache.org/jira/browse/FLINK-33200
> Project: Flink
>  Issue Type: Bug
>  Components: Table SQL / API
>Affects Versions: 1.18.0, 1.17.1
>Reporter: Zhenqiu Huang
>Priority: Minor
> Attachments: IssueDemo.java
>
>
> The table schema is defined as below:
> public static final DataType DATA_TYPE = DataTypes.ROW(
> DataTypes.FIELD("id", DataTypes.STRING()),
> DataTypes.FIELD("events", 
> DataTypes.ARRAY(DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING(
> );
> public static final Schema SCHEMA = 
> Schema.newBuilder().fromRowDataType(DATA_TYPE).build();
> inTable.select(Expressions.$("events").at(1).at("eventType").as("firstEventType")
> The validation fail as "eventType" is inferred as 
> BasicTypeInfo.STRING_TYPE_INFO, the table key internally is a 
> StringDataTypeInfo. The validation fail at 
> case mti: MapTypeInfo[_, _] =>
> if (key.resultType == mti.getKeyTypeInfo) {
>   ValidationSuccess
> } else {
>   ValidationFailure(
> s"Map entry access needs a valid key of type " +
>   s"'${mti.getKeyTypeInfo}', found '${key.resultType}'.")
> }



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (FLINK-33200) ItemAt Expression validation fail in Table API due to type mismatch

2023-10-11 Thread xuyang (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-33200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17773908#comment-17773908
 ] 

xuyang commented on FLINK-33200:


Hi, [~andyglow] . I think the issue 1 and 2 are caused by different bugs. 

 

The issue 1 is caused by that the field 'events' refs from the table, and its 
actual type is StringData, converted from the origin type DataTypes.STRING (). 
But the key of the 'AT' named 'eventType' is DataType.STRING, not StringData. 
So the difference about these two types causes the failure about validating 
'AT'.

 

The issue 2 is caused by that if the planner can't infer the result type of the 
table expression (that means the result type is Polymorphic, depending on the 
type of args), the legacy rule is used to infer the result type. But the legacy 
rules wrongly create a LegacyTypeInformation for StringData. Not only 'AT', but 
also 'COLLECT' 
(inTable.select(Expressions.$("id").collect()).execute().print();) will cause 
this bug.

 

I'll try to fix all of them.

> ItemAt Expression validation fail in Table API due to type mismatch
> ---
>
> Key: FLINK-33200
> URL: https://issues.apache.org/jira/browse/FLINK-33200
> Project: Flink
>  Issue Type: Bug
>  Components: Table SQL / API
>Affects Versions: 1.18.0, 1.17.1
>Reporter: Zhenqiu Huang
>Priority: Minor
> Attachments: IssueDemo.java
>
>
> The table schema is defined as below:
> public static final DataType DATA_TYPE = DataTypes.ROW(
> DataTypes.FIELD("id", DataTypes.STRING()),
> DataTypes.FIELD("events", 
> DataTypes.ARRAY(DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING(
> );
> public static final Schema SCHEMA = 
> Schema.newBuilder().fromRowDataType(DATA_TYPE).build();
> inTable.select(Expressions.$("events").at(1).at("eventType").as("firstEventType")
> The validation fail as "eventType" is inferred as 
> BasicTypeInfo.STRING_TYPE_INFO, the table key internally is a 
> StringDataTypeInfo. The validation fail at 
> case mti: MapTypeInfo[_, _] =>
> if (key.resultType == mti.getKeyTypeInfo) {
>   ValidationSuccess
> } else {
>   ValidationFailure(
> s"Map entry access needs a valid key of type " +
>   s"'${mti.getKeyTypeInfo}', found '${key.resultType}'.")
> }



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (FLINK-33200) ItemAt Expression validation fail in Table API due to type mismatch

2023-10-10 Thread Andriy Onyshchuk (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-33200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17773794#comment-17773794
 ] 

Andriy Onyshchuk commented on FLINK-33200:
--

Seems like I have caught several more issues related to `AT` expression and 
type resolution.
 # accessing array of primitives doesn't work if at data level arrays 
represented as ArrayData. What I got is: `Unsupported conversion from data type 
'ARRAY' (conversion class: org.apache.flink.table.data.ArrayData) to type 
information. Only data types that originated from type information fully 
support a reverse conversion.`
 # accessing `Map` throws too. `map.at(0L)` gets failed with 
Incompatible types for sink column
{{Exception in thread "main" org.apache.flink.table.api.ValidationException: 
Column types of query result and sink for '*anonymous_collect$3*' do not 
match.}}
{{Cause: Incompatible types for sink column 'longData_at_0' at position 0.}}
{{Query schema: [longData_at_0: STRING]}}
{{Sink schema: [longData_at_0: RAW('org.apache.flink.table.data.StringData', 
?)]}}

All issues are reflected in `IssueDemo.java` (see attachments).

> ItemAt Expression validation fail in Table API due to type mismatch
> ---
>
> Key: FLINK-33200
> URL: https://issues.apache.org/jira/browse/FLINK-33200
> Project: Flink
>  Issue Type: Bug
>  Components: Table SQL / API
>Affects Versions: 1.18.0, 1.17.1
>Reporter: Zhenqiu Huang
>Priority: Minor
> Attachments: IssueDemo.java
>
>
> The table schema is defined as below:
> public static final DataType DATA_TYPE = DataTypes.ROW(
> DataTypes.FIELD("id", DataTypes.STRING()),
> DataTypes.FIELD("events", 
> DataTypes.ARRAY(DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING(
> );
> public static final Schema SCHEMA = 
> Schema.newBuilder().fromRowDataType(DATA_TYPE).build();
> inTable.select(Expressions.$("events").at(1).at("eventType").as("firstEventType")
> The validation fail as "eventType" is inferred as 
> BasicTypeInfo.STRING_TYPE_INFO, the table key internally is a 
> StringDataTypeInfo. The validation fail at 
> case mti: MapTypeInfo[_, _] =>
> if (key.resultType == mti.getKeyTypeInfo) {
>   ValidationSuccess
> } else {
>   ValidationFailure(
> s"Map entry access needs a valid key of type " +
>   s"'${mti.getKeyTypeInfo}', found '${key.resultType}'.")
> }



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (FLINK-33200) ItemAt Expression validation fail in Table API due to type mismatch

2023-10-09 Thread Andriy Onyshchuk (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-33200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17773466#comment-17773466
 ] 

Andriy Onyshchuk commented on FLINK-33200:
--

Hello, team. Here is a bit of the summary to this bug
h3. Given 
​
Working on projection over dataset that contains data represented as array of 
maps of
string to string `array[map[string, string]]` several issues was identified.
h3. Data Schema

{{DataTypes.ROW(}}
{{   DataTypes.FIELD("id", DataTypes.STRING()),}}
{{   DataTypes.FIELD("events", 
DataTypes.ARRAY(DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING(}}
{{{}){}}}​
h3. Issues
h4. Issue 1. Extraction of map values by index and key. 
 
{{inTable.select(}}
{{  Expressions.$("events").at(1).at("eventType").as("firstEventType")}}
{{).execute().print();}}
 
 
results in
 
{{Exception in thread "main" org.apache.flink.table.api.ValidationException: 
Map entry access needs a valid key of type 'StringData', found 'String'.}}{{ at 
org.apache.flink.table.planner.expressions.PlannerTypeInferenceUtilImpl.validateArguments(PlannerTypeInferenceUtilImpl.java:111)}}{{
 at 
org.apache.flink.table.planner.expressions.PlannerTypeInferenceUtilImpl.runTypeInference(PlannerTypeInferenceUtilImpl.java:69)}}{{
 at 
org.apache.flink.table.expressions.resolver.rules.ResolveCallByArgumentsRule$ResolvingCallVisitor.runLegacyTypeInference(ResolveCallByArgumentsRule.java:284)}}{{
 at 
org.apache.flink.table.expressions.resolver.rules.ResolveCallByArgumentsRule$ResolvingCallVisitor.lambda$visit$2(ResolveCallByArgumentsRule.java:164)}}{{
 at java.base/java.util.Optional.orElseGet(Optional.java:369)}}{{ at 
org.apache.flink.table.expressions.resolver.rules.ResolveCallByArgumentsRule$ResolvingCallVisitor.visit(ResolveCallByArgumentsRule.java:164)}}{{
 at 
org.apache.flink.table.expressions.resolver.rules.ResolveCallByArgumentsRule$ResolvingCallVisitor.visit(ResolveCallByArgumentsRule.java:98)}}{{
 at 
org.apache.flink.table.expressions.ApiExpressionVisitor.visit(ApiExpressionVisitor.java:37)}}{{
 at 
org.apache.flink.table.expressions.UnresolvedCallExpression.accept(UnresolvedCallExpression.java:97)}}{{
 at 
org.apache.flink.table.expressions.resolver.rules.ResolveCallByArgumentsRule$ResolvingCallVisitor.visit(ResolveCallByArgumentsRule.java:147)}}{{
 at 
org.apache.flink.table.expressions.resolver.rules.ResolveCallByArgumentsRule$ResolvingCallVisitor.visit(ResolveCallByArgumentsRule.java:98)}}{{
 at 
org.apache.flink.table.expressions.ApiExpressionVisitor.visit(ApiExpressionVisitor.java:37)}}{{
 at 
org.apache.flink.table.expressions.UnresolvedCallExpression.accept(UnresolvedCallExpression.java:97)}}{{
 at 
org.apache.flink.table.expressions.resolver.rules.ResolveCallByArgumentsRule.lambda$apply$0(ResolveCallByArgumentsRule.java:92)}}{{
 at 
java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:271)}}{{
 at 
java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1655)}}{{
 at 
java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:484)}}{{
 at 
java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474)}}{{
 at 
java.base/java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:913)}}{{
 at 
java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)}}{{
 at 
java.base/java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:578)}}{{
 at 
org.apache.flink.table.expressions.resolver.rules.ResolveCallByArgumentsRule.apply(ResolveCallByArgumentsRule.java:93)}}{{
 at 
org.apache.flink.table.expressions.resolver.ExpressionResolver.lambda$concatenateRules$2(ExpressionResolver.java:247)}}{{
 at 
java.base/java.util.function.Function.lambda$andThen$1(Function.java:88)}}{{ at 
org.apache.flink.table.expressions.resolver.ExpressionResolver.resolve(ExpressionResolver.java:210)}}{{
 at 
org.apache.flink.table.operations.utils.OperationTreeBuilder.projectInternal(OperationTreeBuilder.java:199)}}{{
 at 
org.apache.flink.table.operations.utils.OperationTreeBuilder.project(OperationTreeBuilder.java:174)}}{{
 at 
org.apache.flink.table.api.internal.TableImpl.select(TableImpl.java:142)}}{{ at 
IssueDemo.main(IssueDemo.java:73)}}
h4. Issue 2. Extraction of entire map by index.

{{inTable.select(}}
{{  Expressions.$("events").at(1).as("firstEvent")}}
{{).execute().print();}}
​
results in
 
{{{}Query schema: [firstEvent: MAP]{}}}{{{}Sink schema: 
[firstEvent: MAP]{}}}{{ at 
org.apache.flink.table.planner.connectors.DynamicSinkUtils.createSchemaMismatchException(DynamicSinkUtils.java:1005)}}{{
 at 
org.apache.flink.table.planner.connectors.DynamicSinkUtils.validateSchemaAndApplyImplicitCast(DynamicSinkUtils.java:349)}}{{
 at 
org.apache.flink.table.planner.connectors.DynamicSinkUtils.validateSchemaAndApplyImplicitCast(DynamicSinkUtils.java:307)}}{{
 at 

[jira] [Commented] (FLINK-33200) ItemAt Expression validation fail in Table API due to type mismatch

2023-10-09 Thread Zhenqiu Huang (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-33200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17773373#comment-17773373
 ] 

Zhenqiu Huang commented on FLINK-33200:
---

We tried both 1.17.1 and 1.18-SNAPSHOT

> ItemAt Expression validation fail in Table API due to type mismatch
> ---
>
> Key: FLINK-33200
> URL: https://issues.apache.org/jira/browse/FLINK-33200
> Project: Flink
>  Issue Type: Bug
>  Components: Table SQL / API
>Reporter: Zhenqiu Huang
>Priority: Minor
> Fix For: 1.8.4
>
>
> The table schema is defined as below:
> public static final DataType DATA_TYPE = DataTypes.ROW(
> DataTypes.FIELD("id", DataTypes.STRING()),
> DataTypes.FIELD("events", 
> DataTypes.ARRAY(DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING(
> );
> public static final Schema SCHEMA = 
> Schema.newBuilder().fromRowDataType(DATA_TYPE).build();
> inTable.select(Expressions.$("events").at(1).at("eventType").as("firstEventType")
> The validation fail as "eventType" is inferred as 
> BasicTypeInfo.STRING_TYPE_INFO, the table key internally is a 
> StringDataTypeInfo. The validation fail at 
> case mti: MapTypeInfo[_, _] =>
> if (key.resultType == mti.getKeyTypeInfo) {
>   ValidationSuccess
> } else {
>   ValidationFailure(
> s"Map entry access needs a valid key of type " +
>   s"'${mti.getKeyTypeInfo}', found '${key.resultType}'.")
> }



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (FLINK-33200) ItemAt Expression validation fail in Table API due to type mismatch

2023-10-09 Thread lincoln lee (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-33200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17773361#comment-17773361
 ] 

lincoln lee commented on FLINK-33200:
-

[~ZhenqiuHuang] What flink version are you using? 

> ItemAt Expression validation fail in Table API due to type mismatch
> ---
>
> Key: FLINK-33200
> URL: https://issues.apache.org/jira/browse/FLINK-33200
> Project: Flink
>  Issue Type: Bug
>  Components: Table SQL / API
>Reporter: Zhenqiu Huang
>Priority: Minor
> Fix For: 1.8.4
>
>
> The table schema is defined as below:
> public static final DataType DATA_TYPE = DataTypes.ROW(
> DataTypes.FIELD("id", DataTypes.STRING()),
> DataTypes.FIELD("events", 
> DataTypes.ARRAY(DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING(
> );
> public static final Schema SCHEMA = 
> Schema.newBuilder().fromRowDataType(DATA_TYPE).build();
> inTable.select(Expressions.$("events").at(1).at("eventType").as("firstEventType")
> The validation fail as "eventType" is inferred as 
> BasicTypeInfo.STRING_TYPE_INFO, the table key internally is a 
> StringDataTypeInfo. The validation fail at 
> case mti: MapTypeInfo[_, _] =>
> if (key.resultType == mti.getKeyTypeInfo) {
>   ValidationSuccess
> } else {
>   ValidationFailure(
> s"Map entry access needs a valid key of type " +
>   s"'${mti.getKeyTypeInfo}', found '${key.resultType}'.")
> }



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (FLINK-33200) ItemAt Expression validation fail in Table API due to type mismatch

2023-10-08 Thread Jingsong Lee (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-33200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17773075#comment-17773075
 ] 

Jingsong Lee commented on FLINK-33200:
--

CC [~lincoln.86xy] 

> ItemAt Expression validation fail in Table API due to type mismatch
> ---
>
> Key: FLINK-33200
> URL: https://issues.apache.org/jira/browse/FLINK-33200
> Project: Flink
>  Issue Type: Bug
>  Components: Table SQL / API
>Reporter: Zhenqiu Huang
>Priority: Minor
> Fix For: 1.8.4
>
>
> The table schema is defined as below:
> public static final DataType DATA_TYPE = DataTypes.ROW(
> DataTypes.FIELD("id", DataTypes.STRING()),
> DataTypes.FIELD("events", 
> DataTypes.ARRAY(DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING(
> );
> public static final Schema SCHEMA = 
> Schema.newBuilder().fromRowDataType(DATA_TYPE).build();
> inTable.select(Expressions.$("events").at(1).at("eventType").as("firstEventType")
> The validation fail as "eventType" is inferred as 
> BasicTypeInfo.STRING_TYPE_INFO, the table key internally is a 
> StringDataTypeInfo. The validation fail at 
> case mti: MapTypeInfo[_, _] =>
> if (key.resultType == mti.getKeyTypeInfo) {
>   ValidationSuccess
> } else {
>   ValidationFailure(
> s"Map entry access needs a valid key of type " +
>   s"'${mti.getKeyTypeInfo}', found '${key.resultType}'.")
> }



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (FLINK-33200) ItemAt Expression validation fail in Table API due to type mismatch

2023-10-06 Thread Zhenqiu Huang (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-33200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17772723#comment-17772723
 ] 

Zhenqiu Huang commented on FLINK-33200:
---

[~lzljs3620320]

As StringDataTypeInfo is the default key type in Flink table, shall we loose 
the validation to 
 
if (key.resultType == StringDataTypeInfo || key.resultType == 
BasicTypeInfo.STRING_TYPE_INFO)

> ItemAt Expression validation fail in Table API due to type mismatch
> ---
>
> Key: FLINK-33200
> URL: https://issues.apache.org/jira/browse/FLINK-33200
> Project: Flink
>  Issue Type: Bug
>  Components: Table SQL / API
>Reporter: Zhenqiu Huang
>Priority: Minor
> Fix For: 1.8.4
>
>
> The table schema is defined as below:
> public static final DataType DATA_TYPE = DataTypes.ROW(
> DataTypes.FIELD("id", DataTypes.STRING()),
> DataTypes.FIELD("events", 
> DataTypes.ARRAY(DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING(
> );
> public static final Schema SCHEMA = 
> Schema.newBuilder().fromRowDataType(DATA_TYPE).build();
> inTable.select(Expressions.$("events").at(1).at("eventType").as("firstEventType")
> The validation fail as "eventType" is inferred as 
> BasicTypeInfo.STRING_TYPE_INFO, the table key internally is a 
> StringDataTypeInfo. The validation fail at 
> case mti: MapTypeInfo[_, _] =>
> if (key.resultType == mti.getKeyTypeInfo) {
>   ValidationSuccess
> } else {
>   ValidationFailure(
> s"Map entry access needs a valid key of type " +
>   s"'${mti.getKeyTypeInfo}', found '${key.resultType}'.")
> }



--
This message was sent by Atlassian Jira
(v8.20.10#820010)