Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-8255][DataSet API, DataStream API] key expressions on named row types do not work #5961

Conversation

snuyanzin
Copy link
Contributor

Thank you very much for contributing to Apache Flink - we are happy that you want to help us improve Flink. To help the community review your contribution in the best possible way, please go through the checklist below, which will get the contribution into a shape in which it can be best reviewed.

Please understand that we do not do this to make contributions to Flink a hassle. In order to uphold a high standard of quality for code contributions, while at the same time managing a large number of contributions, we need contributors to prepare the contributions well, and give reviewers enough contextual information for the review. Please also understand that contributions that do not follow this guide will take longer to review and thus typically be picked up with lower priority by the community.

Contribution Checklist

  • Make sure that the pull request corresponds to a JIRA issue. Exceptions are made for typos in JavaDoc or documentation files, which need no JIRA issue.

  • Name the pull request in the form "[FLINK-XXXX] [component] Title of the pull request", where FLINK-XXXX should be replaced by the actual issue number. Skip component if you are unsure about which is the best component.
    Typo fixes that have no associated JIRA issue should be named following this pattern: [hotfix] [docs] Fix typo in event time introduction or [hotfix] [javadocs] Expand JavaDoc for PuncuatedWatermarkGenerator.

  • Fill out the template below to describe the changes contributed by the pull request. That will give reviewers the context they need to do the review.

  • Make sure that the change passes the automated tests, i.e., mvn clean verify passes. You can set up Travis CI to do that following this guide.

  • Each pull request should address only one issue, not mix up code from multiple issues.

  • Each commit in the pull request has a meaningful commit message (including the JIRA id)

  • Once all items of the checklist are addressed, remove the above text and this checklist, leaving only the filled out template below.

(The sections below can be removed for hotfixes of typos)

What is the purpose of the change

Fix issues related to ClassCastExceptions from Flink-8255 + add more tests

Brief change log

  • Usage of casting to TupleTypeInfoBase rather than to TupleTypeInfo as RowTypeInfo is a child of TupleTypeInfoBase but in a different branch in compare with TupleTypeInfo
  • Add more tests which will fail with ClassCastException without changes from the previous item

Verifying this change

(Please pick either of the following options)
This change added tests and can be verified as follows:

  • Added tests that validates that ClassCastException related to Flink-8255 are not happen

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: (don't know)
  • The runtime per-record code paths (performance sensitive): (don't know)
  • Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, 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 / docs / JavaDocs / not documented)

@snuyanzin snuyanzin changed the title [Flink-8255][DataSet API, DataStream API] key expressions on named row types do not work [FLINK-8255][DataSet API, DataStream API] key expressions on named row types do not work May 7, 2018
Copy link
Contributor

@fhueske fhueske left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @snuyanzin, thanks for the PR.

I think we need to address this issue a bit differently.

  • For the DataSet API, I would not support the built-in aggregations for Row but just Tuple.
  • For the DataStream API, we would need dedicated FieldAccessors for Row. Also here, I'm not sure if it makes sense to add them. There were some discussions to deprecate the built-in aggregation functions. So we could also not allow Row for these operations and throw a meaningful not supported exception.

Best, Fabian

@@ -41,7 +41,7 @@
* is regarded in the reduce function. First index has highest priority and last index has
* least priority.
*/
public SelectByMinFunction(TupleTypeInfo<T> type, int... fields) {
public SelectByMinFunction(TupleTypeInfoBase<T> type, int... fields) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The ReduceFunction is still typed to T extends Tuple such that this will still fail at runtime. The same is true for all other built-in aggregation method like sum() and min() on DataSet and UnsortedGrouping.

This cannot be resolved without major changes. I don't think we should add these features, but rather throw meaningful error messages instead of ClassCastException.

Can you try to override the the isTupleType() method in RowTypeInfo and return false?
This would prevent Row from being used in contexts that are only supported for Tuple.

UnsortedGrouping groupDs = env.fromCollection(Collections.singleton(new Row(2)), rowTypeInfo).groupBy(0);

groupDs.maxBy(1);
groupDs.minBy(1);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The tests pass because the program is not executed.
You would have to call env.collect() to run the program and compare the returned result against the expected result. As I pointed out before, this will fail, because the operator will cast the Row objects to Tuple.

@@ -157,15 +156,15 @@ public T set(T record, F fieldValue) {

SimpleTupleFieldAccessor(int pos, TypeInformation<T> typeInfo) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

accessing fields in a Row will fail because Row does not extend Tuple. For a proper fix, we would need a RowFieldAccessor and use that one when we deal with a DataStream<Row>. We would then need to add the RowFieldAccessor to the FieldAccessorFactory.

@@ -197,7 +196,7 @@ public T set(T record, F fieldValue) {
checkNotNull(typeInfo, "typeInfo must not be null.");
checkNotNull(innerAccessor, "innerAccessor must not be null.");

int arity = ((TupleTypeInfo) typeInfo).getArity();
int arity = typeInfo.getArity();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same as for SimpleTupleFieldAccessor.

* should not fail e.g. like in FLINK-8255.
*/
@Test
public void testRowTypeInfo() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This test just validates that a FieldAccessor is created. At runtime it would fail with a ClassCastException.

@snuyanzin
Copy link
Contributor Author

Hello @fhueske
Thank you for your review

As you proposed I tried to use org.apache.flink.api.java.typeutils.RowTypeInfo#isTupleType

public boolean isTupleType() {
		return false;
	}

however after that these tests started to fail
org.apache.flink.table.api.batch.ExplainTest#testJoinWithoutExtended
org.apache.flink.table.api.batch.ExplainTest#testJoinWithExtended

like

testJoinWithoutExtended(org.apache.flink.table.api.batch.ExplainTest)  Time elapsed: 0.037 sec  <<< ERROR!

org.apache.flink.api.common.InvalidProgramException: Specifying keys via field positions is only valid for tuple data types. Type: Row(a: Integer, b: String)

	at org.apache.flink.api.common.operators.Keys$ExpressionKeys.<init>(Keys.java:232)

	at org.apache.flink.api.common.operators.Keys$ExpressionKeys.<init>(Keys.java:223)

	at org.apache.flink.api.java.operators.JoinOperator$JoinOperatorSets.where(JoinOperator.java:901)

	at org.apache.flink.table.plan.nodes.dataset.DataSetJoin.addInnerJoin(DataSetJoin.scala:243)

	at org.apache.flink.table.plan.nodes.dataset.DataSetJoin.translateToPlan(DataSetJoin.scala:170)

	at org.apache.flink.table.plan.nodes.dataset.DataSetCalc.translateToPlan(DataSetCalc.scala:91)

	at org.apache.flink.table.api.BatchTableEnvironment.translate(BatchTableEnvironment.scala:422)

	at org.apache.flink.table.api.BatchTableEnvironment.explain(BatchTableEnvironment.scala:249)

	at org.apache.flink.table.api.BatchTableEnvironment.explain(BatchTableEnvironment.scala:275)

	at org.apache.flink.table.api.batch.ExplainTest.testJoinWithoutExtended(ExplainTest.scala:72)

that is why I decided to use instanceof TupleTypeInfo check as anyway next line there is a cast to this type

could you please have a look at it and tell if it is acceptable or not?

@fhueske
Copy link
Contributor

fhueske commented May 16, 2018

Thanks for the update @snuyanzin. I'll try to have a look at the changes in the next days.
Best, Fabian

@snuyanzin snuyanzin closed this May 17, 2018
@snuyanzin snuyanzin deleted the FLINK-8255_Key_expressions_on_named_row_types_do_not_work branch May 17, 2018 07:42
@snuyanzin snuyanzin restored the FLINK-8255_Key_expressions_on_named_row_types_do_not_work branch May 17, 2018 07:46
@snuyanzin
Copy link
Contributor Author

sorry, was closed by mistake
reopened

@snuyanzin snuyanzin reopened this May 17, 2018
Squashing all the commits into one which includes
1) Test case from description + 2 more
2) Resolution of class cast by using of instanceof
3) Corrections based on comment [1]
[1] https://issues.apache.org/jira/browse/FLINK-8255?focusedCommentId=16466381&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16466381
@snuyanzin snuyanzin force-pushed the FLINK-8255_Key_expressions_on_named_row_types_do_not_work branch from 637e5de to d33921c Compare June 22, 2018 12:08
@snuyanzin
Copy link
Contributor Author

commits squashed into one

@aljoscha
Copy link
Contributor

aljoscha commented Jan 8, 2020

Thanks! I merged this.

@aljoscha aljoscha closed this Jan 8, 2020
@snuyanzin snuyanzin deleted the FLINK-8255_Key_expressions_on_named_row_types_do_not_work branch May 5, 2022 11:08
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
4 participants