[GitHub] flink pull request #5961: [FLINK-8255][DataSet API, DataStream API] key expr...
GitHub user snuyanzin reopened a pull request: https://github.com/apache/flink/pull/5961 [FLINK-8255][DataSet API, DataStream API] key expressions on named row types do not work *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](https://issues.apache.org/jira/projects/FLINK/issues). Exceptions are made for typos in JavaDoc or documentation files, which need no JIRA issue. - Name the pull request in the form "[FLINK-] [component] Title of the pull request", where *FLINK-* 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](http://flink.apache.org/contribute-code.html#best-practices). - 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) You can merge this pull request into a Git repository by running: $ git pull https://github.com/snuyanzin/flink FLINK-8255_Key_expressions_on_named_row_types_do_not_work Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/5961.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 #5961 commit 50eb4ead75ddc22e0eee2bf5a2b9d12c37dcaeb4 Author: snuyanzin Date: 2018-04-29T18:37:05Z FLINK-8255 Key expressions on named row types do not work Test case from description + 2 more Resolution of class cast by using of TupleTypeInfoBase rather than TupleTypeInfo commit 96d569025e21172059bb5b34cf93f3a60b5f0a0e Author: snuyanzin Date: 2018-04-29T18:47:52Z FLINK-8255 Key expressions on named row types do not work Test case from description + 2 more Resolution of class cast by using of TupleTypeInfoBase rather than TupleTypeInfo
[GitHub] flink pull request #5961: [FLINK-8255][DataSet API, DataStream API] key expr...
Github user snuyanzin closed the pull request at: https://github.com/apache/flink/pull/5961 ---
[GitHub] flink pull request #5961: [FLINK-8255][DataSet API, DataStream API] key expr...
Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/5961#discussion_r186528901 --- Diff: flink-streaming-java/src/test/java/org/apache/flink/streaming/util/typeutils/FieldAccessorTest.java --- @@ -368,4 +369,23 @@ public void testIllegalBasicType2() { FieldAccessor f = FieldAccessorFactory.getAccessor(tpeInfo, "foo", null); } + + /** +* Validates that no ClassCastException happens +* should not fail e.g. like in FLINK-8255. +*/ + @Test + public void testRowTypeInfo() { --- End diff -- This test just validates that a `FieldAccessor` is created. At runtime it would fail with a `ClassCastException`. ---
[GitHub] flink pull request #5961: [FLINK-8255][DataSet API, DataStream API] key expr...
Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/5961#discussion_r186484649 --- Diff: flink-java/src/test/java/org/apache/flink/api/java/operator/MaxByOperatorTest.java --- @@ -230,4 +235,43 @@ public String toString() { } } + /** +* Validates that no ClassCastException happens +* should not fail e.g. like in FLINK-8255. +*/ + @Test + public void testMaxMinByRowTypeInfoKeyFieldsDataset() { + + final ExecutionEnvironment env = ExecutionEnvironment + .getExecutionEnvironment(); + TypeInformation[] types = new TypeInformation[] {Types.INT, Types.INT}; + + String[] fieldNames = new String[]{"id", "value"}; + RowTypeInfo rowTypeInfo = new RowTypeInfo(types, fieldNames); + DataSet tupleDs = env + .fromCollection(Collections.singleton(new Row(2)), rowTypeInfo); + + tupleDs.maxBy(0); + tupleDs.minBy(0); + } + +/** + * Validates that no ClassCastException happens +* should not fail e.g. like in FLINK-8255. +*/ + @Test + public void testMaxMinByRowTypeInfoKeyFieldsForUnsortedGrouping() { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + TypeInformation[] types = new TypeInformation[]{Types.INT, Types.INT}; + + String[] fieldNames = new String[]{"id", "value"}; + RowTypeInfo rowTypeInfo = new RowTypeInfo(types, fieldNames); + + UnsortedGrouping groupDs = env.fromCollection(Collections.singleton(new Row(2)), rowTypeInfo).groupBy(0); + + groupDs.maxBy(1); + groupDs.minBy(1); --- End diff -- 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`. ---
[GitHub] flink pull request #5961: [FLINK-8255][DataSet API, DataStream API] key expr...
Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/5961#discussion_r186477977 --- Diff: flink-java/src/main/java/org/apache/flink/api/java/functions/SelectByMinFunction.java --- @@ -41,7 +41,7 @@ * is regarded in the reduce function. First index has highest priority and last index has * least priority. */ - public SelectByMinFunction(TupleTypeInfo type, int... fields) { + public SelectByMinFunction(TupleTypeInfoBase type, int... fields) { --- End diff -- 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`. ---
[GitHub] flink pull request #5961: [FLINK-8255][DataSet API, DataStream API] key expr...
Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/5961#discussion_r186527205 --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/util/typeutils/FieldAccessor.java --- @@ -157,15 +156,15 @@ public T set(T record, F fieldValue) { SimpleTupleFieldAccessor(int pos, TypeInformation typeInfo) { --- End diff -- 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`. We would then need to add the `RowFieldAccessor` to the `FieldAccessorFactory`. ---
[GitHub] flink pull request #5961: [FLINK-8255][DataSet API, DataStream API] key expr...
Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/5961#discussion_r186527277 --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/util/typeutils/FieldAccessor.java --- @@ -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(); --- End diff -- Same as for `SimpleTupleFieldAccessor`. ---
[GitHub] flink pull request #5961: [Flink-8255][DataSet API, DataStream API] key expr...
GitHub user snuyanzin opened a pull request: https://github.com/apache/flink/pull/5961 [Flink-8255][DataSet API, DataStream API] key expressions on named row types do not work *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](https://issues.apache.org/jira/projects/FLINK/issues). Exceptions are made for typos in JavaDoc or documentation files, which need no JIRA issue. - Name the pull request in the form "[FLINK-] [component] Title of the pull request", where *FLINK-* 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](http://flink.apache.org/contribute-code.html#best-practices). - 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) You can merge this pull request into a Git repository by running: $ git pull https://github.com/snuyanzin/flink FLINK-8255_Key_expressions_on_named_row_types_do_not_work Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/5961.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 #5961 commit 50eb4ead75ddc22e0eee2bf5a2b9d12c37dcaeb4 Author: snuyanzin Date: 2018-04-29T18:37:05Z FLINK-8255 Key expressions on named row types do not work Test case from description + 2 more Resolution of class cast by using of TupleTypeInfoBase rather than TupleTypeInfo commit 96d569025e21172059bb5b34cf93f3a60b5f0a0e Author: snuyanzin Date: 2018-04-29T18:47:52Z FLINK-8255 Key expressions on named row types do not work Test case from description + 2 more Resolution of class cast by using of TupleTypeInfoBase rather than TupleTypeInfo co