Details

    • Type: Bug
    • Status: Closed
    • Priority: Trivial
    • Resolution: Fixed
    • Affects Version/s: 1.2.0
    • Fix Version/s: 1.3.0
    • Labels:
      None

      Description

      When we use ExecutionEnvironment#fromElements(X... data) it takes first element of data to define a type. If first Row in collection has wrong number of fields (there are nulls) TypeExtractor returns not RowTypeInfo, but GenericType<Row>

        Issue Links

          Activity

          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user tonycox opened a pull request:

          https://github.com/apache/flink/pull/3127

          FLINK-5481 Add type extraction from collection

          Thanks for contributing to Apache Flink. Before you open your pull request, please take the following check list into consideration.
          If your changes take all of the items into account, feel free to open your pull request. For more information and/or questions please refer to the [How To Contribute guide](http://flink.apache.org/how-to-contribute.html).
          In addition to going through the list, please provide a meaningful description of your changes.

          • [ ] General
          • The pull request references the related JIRA issue ("[FLINK-XXX] Jira title text")
          • The pull request addresses only one issue
          • Each commit in the PR has a meaningful commit message (including the JIRA id)
          • [ ] Documentation
          • Documentation has been added for new functionality
          • Old documentation affected by the pull request has been updated
          • JavaDoc for public methods has been added
          • [ ] Tests & Build
          • Functionality added by the pull request is covered by tests
          • `mvn clean verify` has been executed successfully locally or a Travis build has passed

          You can merge this pull request into a Git repository by running:

          $ git pull https://github.com/tonycox/flink fromRowCollection

          Alternatively you can review and apply these changes as the patch at:

          https://github.com/apache/flink/pull/3127.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 #3127


          commit 9331999e2ff3b497f4414342e0602b0ea682c450
          Author: tonycox <anton_solovev@epam.com>
          Date: 2017-01-13T17:08:47Z

          FLINK-5481 Add type extraction from collection


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user tonycox opened a pull request: https://github.com/apache/flink/pull/3127 FLINK-5481 Add type extraction from collection Thanks for contributing to Apache Flink. Before you open your pull request, please take the following check list into consideration. If your changes take all of the items into account, feel free to open your pull request. For more information and/or questions please refer to the [How To Contribute guide] ( http://flink.apache.org/how-to-contribute.html ). In addition to going through the list, please provide a meaningful description of your changes. [ ] General The pull request references the related JIRA issue (" [FLINK-XXX] Jira title text") The pull request addresses only one issue Each commit in the PR has a meaningful commit message (including the JIRA id) [ ] Documentation Documentation has been added for new functionality Old documentation affected by the pull request has been updated JavaDoc for public methods has been added [ ] Tests & Build Functionality added by the pull request is covered by tests `mvn clean verify` has been executed successfully locally or a Travis build has passed You can merge this pull request into a Git repository by running: $ git pull https://github.com/tonycox/flink fromRowCollection Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/3127.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 #3127 commit 9331999e2ff3b497f4414342e0602b0ea682c450 Author: tonycox <anton_solovev@epam.com> Date: 2017-01-13T17:08:47Z FLINK-5481 Add type extraction from collection
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user zentol commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3127#discussion_r96414774

          — Diff: flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java —
          @@ -1916,6 +1918,46 @@ else if (value instanceof Row) {
          }
          }

          + public static <X> TypeInformation<X> getForCollection(Iterable<X> value)

          { + return new TypeExtractor().privateGetForIterable(value); + }

          +
          + public static <X> TypeInformation<X> getForCollection(X[] value)

          { + return new TypeExtractor().privateGetForIterable(Arrays.asList(value)); + }

          +
          + @SuppressWarnings(

          { "unchecked", "rawtypes" }

          )
          + private <X> TypeInformation<X> privateGetForIterable(Iterable<X> value) {
          + checkNotNull(value);
          +
          + Iterator<X> it = value.iterator();
          + X v = checkNotNull(it.next());
          +
          + if (v instanceof Row) {
          — End diff –

          So this only works if there is one `Row` for which every field is not null? Why not instead search the type of every column separately, and then create the `RowTypeInfo` manually? Then it would only fail if one field is null for every `Row`.

          Show
          githubbot ASF GitHub Bot added a comment - Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/3127#discussion_r96414774 — Diff: flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java — @@ -1916,6 +1918,46 @@ else if (value instanceof Row) { } } + public static <X> TypeInformation<X> getForCollection(Iterable<X> value) { + return new TypeExtractor().privateGetForIterable(value); + } + + public static <X> TypeInformation<X> getForCollection(X[] value) { + return new TypeExtractor().privateGetForIterable(Arrays.asList(value)); + } + + @SuppressWarnings( { "unchecked", "rawtypes" } ) + private <X> TypeInformation<X> privateGetForIterable(Iterable<X> value) { + checkNotNull(value); + + Iterator<X> it = value.iterator(); + X v = checkNotNull(it.next()); + + if (v instanceof Row) { — End diff – So this only works if there is one `Row` for which every field is not null? Why not instead search the type of every column separately, and then create the `RowTypeInfo` manually? Then it would only fail if one field is null for every `Row`.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tonycox commented on the issue:

          https://github.com/apache/flink/pull/3127

          @zentol I rewrite as you pointed

          Show
          githubbot ASF GitHub Bot added a comment - Github user tonycox commented on the issue: https://github.com/apache/flink/pull/3127 @zentol I rewrite as you pointed
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user twalthr commented on the issue:

          https://github.com/apache/flink/pull/3127

          I will shepherd this PR.

          Show
          githubbot ASF GitHub Bot added a comment - Github user twalthr commented on the issue: https://github.com/apache/flink/pull/3127 I will shepherd this PR.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user twalthr commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3127#discussion_r97355576

          — Diff: flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java —
          @@ -1916,6 +1918,50 @@ else if (value instanceof Row) {
          }
          }

          + public static <X> TypeInformation<X> getForCollection(Iterable<X> value)

          { + return new TypeExtractor().privateGetForIterable(value); + }

          +
          + public static <X> TypeInformation<X> getForCollection(X[] value)

          { + return new TypeExtractor().privateGetForIterable(Arrays.asList(value)); + }

          +
          + @SuppressWarnings(

          { "unchecked", "rawtypes" }

          )
          + private <X> TypeInformation<X> privateGetForIterable(Iterable<X> value) {
          + checkNotNull(value);
          +
          + Iterator<X> it = value.iterator();
          + X v = checkNotNull(it.next());
          +
          + if (v instanceof Row) {
          + int arity =((Row) v).getArity();
          + Iterable<Row> rows = (Iterable<Row>) value;
          + List<TypeInformation<?>> typesInRow = new ArrayList<>(arity);
          + TypeInformation<?>[] temp = new TypeInformation<?>[arity];
          + Collections.addAll(typesInRow, temp);
          +
          + for (Row r: rows) {
          — End diff –

          Missing space.

          Show
          githubbot ASF GitHub Bot added a comment - Github user twalthr commented on a diff in the pull request: https://github.com/apache/flink/pull/3127#discussion_r97355576 — Diff: flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java — @@ -1916,6 +1918,50 @@ else if (value instanceof Row) { } } + public static <X> TypeInformation<X> getForCollection(Iterable<X> value) { + return new TypeExtractor().privateGetForIterable(value); + } + + public static <X> TypeInformation<X> getForCollection(X[] value) { + return new TypeExtractor().privateGetForIterable(Arrays.asList(value)); + } + + @SuppressWarnings( { "unchecked", "rawtypes" } ) + private <X> TypeInformation<X> privateGetForIterable(Iterable<X> value) { + checkNotNull(value); + + Iterator<X> it = value.iterator(); + X v = checkNotNull(it.next()); + + if (v instanceof Row) { + int arity =((Row) v).getArity(); + Iterable<Row> rows = (Iterable<Row>) value; + List<TypeInformation<?>> typesInRow = new ArrayList<>(arity); + TypeInformation<?>[] temp = new TypeInformation<?> [arity] ; + Collections.addAll(typesInRow, temp); + + for (Row r: rows) { — End diff – Missing space.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user twalthr commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3127#discussion_r97355537

          — Diff: flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java —
          @@ -1916,6 +1918,50 @@ else if (value instanceof Row) {
          }
          }

          + public static <X> TypeInformation<X> getForCollection(Iterable<X> value)

          { + return new TypeExtractor().privateGetForIterable(value); + }

          +
          + public static <X> TypeInformation<X> getForCollection(X[] value)

          { + return new TypeExtractor().privateGetForIterable(Arrays.asList(value)); + }

          +
          + @SuppressWarnings(

          { "unchecked", "rawtypes" }

          )
          + private <X> TypeInformation<X> privateGetForIterable(Iterable<X> value) {
          + checkNotNull(value);
          +
          + Iterator<X> it = value.iterator();
          + X v = checkNotNull(it.next());
          +
          + if (v instanceof Row) {
          + int arity =((Row) v).getArity();
          — End diff –

          Missing space.

          Show
          githubbot ASF GitHub Bot added a comment - Github user twalthr commented on a diff in the pull request: https://github.com/apache/flink/pull/3127#discussion_r97355537 — Diff: flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java — @@ -1916,6 +1918,50 @@ else if (value instanceof Row) { } } + public static <X> TypeInformation<X> getForCollection(Iterable<X> value) { + return new TypeExtractor().privateGetForIterable(value); + } + + public static <X> TypeInformation<X> getForCollection(X[] value) { + return new TypeExtractor().privateGetForIterable(Arrays.asList(value)); + } + + @SuppressWarnings( { "unchecked", "rawtypes" } ) + private <X> TypeInformation<X> privateGetForIterable(Iterable<X> value) { + checkNotNull(value); + + Iterator<X> it = value.iterator(); + X v = checkNotNull(it.next()); + + if (v instanceof Row) { + int arity =((Row) v).getArity(); — End diff – Missing space.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user twalthr commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3127#discussion_r97359437

          — Diff: flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java —
          @@ -1916,6 +1918,50 @@ else if (value instanceof Row) {
          }
          }

          + public static <X> TypeInformation<X> getForCollection(Iterable<X> value)

          { + return new TypeExtractor().privateGetForIterable(value); + }

          +
          + public static <X> TypeInformation<X> getForCollection(X[] value)

          { + return new TypeExtractor().privateGetForIterable(Arrays.asList(value)); + }

          +
          + @SuppressWarnings(

          { "unchecked", "rawtypes" }

          )
          + private <X> TypeInformation<X> privateGetForIterable(Iterable<X> value) {
          — End diff –

          Would be great if you could add some inline comments in this method, just to quickly see what your code is doing.

          Show
          githubbot ASF GitHub Bot added a comment - Github user twalthr commented on a diff in the pull request: https://github.com/apache/flink/pull/3127#discussion_r97359437 — Diff: flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java — @@ -1916,6 +1918,50 @@ else if (value instanceof Row) { } } + public static <X> TypeInformation<X> getForCollection(Iterable<X> value) { + return new TypeExtractor().privateGetForIterable(value); + } + + public static <X> TypeInformation<X> getForCollection(X[] value) { + return new TypeExtractor().privateGetForIterable(Arrays.asList(value)); + } + + @SuppressWarnings( { "unchecked", "rawtypes" } ) + private <X> TypeInformation<X> privateGetForIterable(Iterable<X> value) { — End diff – Would be great if you could add some inline comments in this method, just to quickly see what your code is doing.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user twalthr commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3127#discussion_r97353282

          — Diff: flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java —
          @@ -1916,6 +1918,50 @@ else if (value instanceof Row) {
          }
          }

          + public static <X> TypeInformation<X> getForCollection(Iterable<X> value)

          { + return new TypeExtractor().privateGetForIterable(value); + }

          +
          + public static <X> TypeInformation<X> getForCollection(X[] value) {
          — End diff –

          Add Javadoc.

          Show
          githubbot ASF GitHub Bot added a comment - Github user twalthr commented on a diff in the pull request: https://github.com/apache/flink/pull/3127#discussion_r97353282 — Diff: flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java — @@ -1916,6 +1918,50 @@ else if (value instanceof Row) { } } + public static <X> TypeInformation<X> getForCollection(Iterable<X> value) { + return new TypeExtractor().privateGetForIterable(value); + } + + public static <X> TypeInformation<X> getForCollection(X[] value) { — End diff – Add Javadoc.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user twalthr commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3127#discussion_r97354898

          — Diff: flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java —
          @@ -1916,6 +1918,50 @@ else if (value instanceof Row) {
          }
          }

          + public static <X> TypeInformation<X> getForCollection(Iterable<X> value)

          { + return new TypeExtractor().privateGetForIterable(value); + }

          +
          + public static <X> TypeInformation<X> getForCollection(X[] value)

          { + return new TypeExtractor().privateGetForIterable(Arrays.asList(value)); + }

          +
          + @SuppressWarnings(

          { "unchecked", "rawtypes" }

          )
          + private <X> TypeInformation<X> privateGetForIterable(Iterable<X> value) {
          + checkNotNull(value);
          +
          + Iterator<X> it = value.iterator();
          + X v = checkNotNull(it.next());
          — End diff –

          Calling `next` without checking can result in `NoSuchElementException`. We should check that first and throw a helpful exception.

          Show
          githubbot ASF GitHub Bot added a comment - Github user twalthr commented on a diff in the pull request: https://github.com/apache/flink/pull/3127#discussion_r97354898 — Diff: flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java — @@ -1916,6 +1918,50 @@ else if (value instanceof Row) { } } + public static <X> TypeInformation<X> getForCollection(Iterable<X> value) { + return new TypeExtractor().privateGetForIterable(value); + } + + public static <X> TypeInformation<X> getForCollection(X[] value) { + return new TypeExtractor().privateGetForIterable(Arrays.asList(value)); + } + + @SuppressWarnings( { "unchecked", "rawtypes" } ) + private <X> TypeInformation<X> privateGetForIterable(Iterable<X> value) { + checkNotNull(value); + + Iterator<X> it = value.iterator(); + X v = checkNotNull(it.next()); — End diff – Calling `next` without checking can result in `NoSuchElementException`. We should check that first and throw a helpful exception.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user twalthr commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3127#discussion_r97353232

          — Diff: flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java —
          @@ -1916,6 +1918,50 @@ else if (value instanceof Row) {
          }
          }

          + public static <X> TypeInformation<X> getForCollection(Iterable<X> value) {
          — End diff –

          Add Javadoc.

          Show
          githubbot ASF GitHub Bot added a comment - Github user twalthr commented on a diff in the pull request: https://github.com/apache/flink/pull/3127#discussion_r97353232 — Diff: flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java — @@ -1916,6 +1918,50 @@ else if (value instanceof Row) { } } + public static <X> TypeInformation<X> getForCollection(Iterable<X> value) { — End diff – Add Javadoc.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StephanEwen commented on the issue:

          https://github.com/apache/flink/pull/3127

          I am a bit skeptical about the special case handling or Row in the type exatractor there.
          Why specifically support row and not other types? To me, Row does not seem like a most common type to be put into the collection inputs that it needs such special case handling.

          Also, this does not detect if the different rows have conflicting types in the columns (such as `row1[x] = int, row2[x] = String).

          What do you think about simply improving how to generate a row type info (similar as we did with `TypeHint` for other types)? Then `StreamExecutionEnvironment.fromCollection(rows, type)` should work quite well.

          Show
          githubbot ASF GitHub Bot added a comment - Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/3127 I am a bit skeptical about the special case handling or Row in the type exatractor there. Why specifically support row and not other types? To me, Row does not seem like a most common type to be put into the collection inputs that it needs such special case handling. Also, this does not detect if the different rows have conflicting types in the columns (such as `row1 [x] = int, row2 [x] = String). What do you think about simply improving how to generate a row type info (similar as we did with `TypeHint` for other types)? Then `StreamExecutionEnvironment.fromCollection(rows, type)` should work quite well.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user twalthr commented on the issue:

          https://github.com/apache/flink/pull/3127

          I would also be fine with `StreamExecutionEnvironment.fromCollection(rows, type)`. What do you think @tonycox?

          Show
          githubbot ASF GitHub Bot added a comment - Github user twalthr commented on the issue: https://github.com/apache/flink/pull/3127 I would also be fine with `StreamExecutionEnvironment.fromCollection(rows, type)`. What do you think @tonycox?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tonycox commented on the issue:

          https://github.com/apache/flink/pull/3127

          @StephanEwen I agree with types conflicting and that it's not common type for collection.
          So how should generating a row type info look like?

          Show
          githubbot ASF GitHub Bot added a comment - Github user tonycox commented on the issue: https://github.com/apache/flink/pull/3127 @StephanEwen I agree with types conflicting and that it's not common type for collection. So how should generating a row type info look like?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user twalthr commented on the issue:

          https://github.com/apache/flink/pull/3127

          We can simply do it like this:

          ```
          StreamExecutionEnvironment.fromCollection(
          rows,
          new RowTypeInfo(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO))
          ```

          Show
          githubbot ASF GitHub Bot added a comment - Github user twalthr commented on the issue: https://github.com/apache/flink/pull/3127 We can simply do it like this: ``` StreamExecutionEnvironment.fromCollection( rows, new RowTypeInfo(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO)) ```
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StephanEwen commented on the issue:

          https://github.com/apache/flink/pull/3127

          You could think of adding a utility method on the RowTypeInfo:
          ```java
          @SafeVarAgrs
          RowTypeInfo.from(Class<?>... types)
          ```
          and use it like
          ```java
          StreamExecutionEnvironment.fromCollection(
          rows,
          RowTypeInfo.from(String.class, int.class));
          ```

          Show
          githubbot ASF GitHub Bot added a comment - Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/3127 You could think of adding a utility method on the RowTypeInfo: ```java @SafeVarAgrs RowTypeInfo.from(Class<?>... types) ``` and use it like ```java StreamExecutionEnvironment.fromCollection( rows, RowTypeInfo.from(String.class, int.class)); ```
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tonycox commented on the issue:

          https://github.com/apache/flink/pull/3127

          I think all it's just a syntax sugar
          ```scala
          val data = List(
          Row.of(Row.of("data_1", "dob"), Row.of("info_4", "dub")),
          Row.of(Row.of("data_1", "dob"), Row.of("info_4", "dub")),
          Row.of(Row.of("data_1", "dob"), Row.of("info_4", "dub")))

          val info= RowTypeInfo.from(
          RowTypeInfo.from(classOf[String], classOf[String]),
          RowTypeInfo.from(classOf[String], classOf[String])
          )

          execEnv.fromCollection(data, info)
          ```
          instead of
          ```scala
          val info = new RowTypeInfo(
          Array[TypeInformation[_]](
          new RowTypeInfo(
          Array[TypeInformation[_]](
          BasicTypeInfo.STRING_TYPE_INFO,
          BasicTypeInfo.STRING_TYPE_INFO),
          Array("name", "age")),
          new RowTypeInfo(
          Array[TypeInformation[_]](
          BasicTypeInfo.STRING_TYPE_INFO,
          BasicTypeInfo.STRING_TYPE_INFO),
          Array("more_info", "and_so_on"))),
          Array("person", "additional")
          )

          execEnv.fromCollection(data, info)
          ```

          Show
          githubbot ASF GitHub Bot added a comment - Github user tonycox commented on the issue: https://github.com/apache/flink/pull/3127 I think all it's just a syntax sugar ```scala val data = List( Row.of(Row.of("data_1", "dob"), Row.of("info_4", "dub")), Row.of(Row.of("data_1", "dob"), Row.of("info_4", "dub")), Row.of(Row.of("data_1", "dob"), Row.of("info_4", "dub"))) val info= RowTypeInfo.from( RowTypeInfo.from(classOf [String] , classOf [String] ), RowTypeInfo.from(classOf [String] , classOf [String] ) ) execEnv.fromCollection(data, info) ``` instead of ```scala val info = new RowTypeInfo( Array[TypeInformation [_] ]( new RowTypeInfo( Array[TypeInformation [_] ]( BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO), Array("name", "age")), new RowTypeInfo( Array[TypeInformation [_] ]( BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO), Array("more_info", "and_so_on"))), Array("person", "additional") ) execEnv.fromCollection(data, info) ```
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StephanEwen commented on the issue:

          https://github.com/apache/flink/pull/3127

          Hmmm... what would you suggest to do instead?

          Show
          githubbot ASF GitHub Bot added a comment - Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/3127 Hmmm... what would you suggest to do instead?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tonycox commented on the issue:

          https://github.com/apache/flink/pull/3127

          I suggest add a little bit more sugar
          ```java
          TypeInformation<?> infoNew = RowTypeInfo.from(
          new String[]

          {"1c", "2c"},
          RowTypeInfo.from(
          new String[]{"1.1", "1.2c"},
          String.class, String.class),
          RowTypeInfo.from(
          new String[]{"2.1c", "2.2c"},
          String.class, String.class));

          instead of

          TypeInformation<?> infoOld = new RowTypeInfo(
          new TypeInformation<?>[]{
          new RowTypeInfo(
          new TypeInformation<?>[]{ BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO},
          new String[]{"1.1", "1.2c"}),
          new RowTypeInfo(
          new TypeInformation<?>[]{ BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO},
          new String[]{"2.1c", "2.2c"}
          )},
          new String[]{"1c", "2c"}

          );
          ```
          ```scala
          val infoNew = RowTypeInfo.from(
          Array("person", "additional"),
          RowTypeInfo.from(
          Array("name", "age"),
          classOf[String], classOf[String]),
          RowTypeInfo.from(
          Array("name", "age"),
          classOf[String], classOf[String])
          )

          instead of

          val infoOld = new RowTypeInfo(
          Array[TypeInformation[_]](
          new RowTypeInfo(
          Array[TypeInformation[_]](
          BasicTypeInfo.STRING_TYPE_INFO,
          BasicTypeInfo.STRING_TYPE_INFO),
          Array("name", "age")),
          new RowTypeInfo(
          Array[TypeInformation[_]](
          BasicTypeInfo.STRING_TYPE_INFO,
          BasicTypeInfo.STRING_TYPE_INFO),
          Array("more_info", "and_so_on"))),
          Array("person", "additional")
          )
          ```

          Show
          githubbot ASF GitHub Bot added a comment - Github user tonycox commented on the issue: https://github.com/apache/flink/pull/3127 I suggest add a little bit more sugar ```java TypeInformation<?> infoNew = RowTypeInfo.from( new String[] {"1c", "2c"}, RowTypeInfo.from( new String[]{"1.1", "1.2c"}, String.class, String.class), RowTypeInfo.from( new String[]{"2.1c", "2.2c"}, String.class, String.class)); instead of TypeInformation<?> infoOld = new RowTypeInfo( new TypeInformation<?>[]{ new RowTypeInfo( new TypeInformation<?>[]{ BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO}, new String[]{"1.1", "1.2c"}), new RowTypeInfo( new TypeInformation<?>[]{ BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO}, new String[]{"2.1c", "2.2c"} )}, new String[]{"1c", "2c"} ); ``` ```scala val infoNew = RowTypeInfo.from( Array("person", "additional"), RowTypeInfo.from( Array("name", "age"), classOf [String] , classOf [String] ), RowTypeInfo.from( Array("name", "age"), classOf [String] , classOf [String] ) ) instead of val infoOld = new RowTypeInfo( Array[TypeInformation [_] ]( new RowTypeInfo( Array[TypeInformation [_] ]( BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO), Array("name", "age")), new RowTypeInfo( Array[TypeInformation [_] ]( BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO), Array("more_info", "and_so_on"))), Array("person", "additional") ) ```
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user twalthr commented on the issue:

          https://github.com/apache/flink/pull/3127

          In the Table API we implemented `org.apache.flink.table.api.Types` to improve the user experience. We could also think about moving this to core. I would be helpful to have a class that enumerates all supported types. It could also support methods like `Types.ROW(Types.INT(), Types.DOUBLE())`. If you then import all members statically, you end up with `ROW(INT(), DOUBLE())`.

          Show
          githubbot ASF GitHub Bot added a comment - Github user twalthr commented on the issue: https://github.com/apache/flink/pull/3127 In the Table API we implemented `org.apache.flink.table.api.Types` to improve the user experience. We could also think about moving this to core. I would be helpful to have a class that enumerates all supported types. It could also support methods like `Types.ROW(Types.INT(), Types.DOUBLE())`. If you then import all members statically, you end up with `ROW(INT(), DOUBLE())`.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tonycox commented on the issue:

          https://github.com/apache/flink/pull/3127

          @twalthr I think it's good way.
          Should I move `TimeIntervalTypeInfo` to core as well or not?

          Show
          githubbot ASF GitHub Bot added a comment - Github user tonycox commented on the issue: https://github.com/apache/flink/pull/3127 @twalthr I think it's good way. Should I move `TimeIntervalTypeInfo` to core as well or not?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tonycox commented on the issue:

          https://github.com/apache/flink/pull/3127

          ```scala
          import Types._
          ROW(Array("person", "additional"),
          ROW(Array("name", "age"), STRING, STRING),
          ROW(Array("more_info", "and_so_on"), STRING, STRING))
          ```

          Show
          githubbot ASF GitHub Bot added a comment - Github user tonycox commented on the issue: https://github.com/apache/flink/pull/3127 ```scala import Types._ ROW(Array("person", "additional"), ROW(Array("name", "age"), STRING, STRING), ROW(Array("more_info", "and_so_on"), STRING, STRING)) ```
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tonycox commented on the issue:

          https://github.com/apache/flink/pull/3127

          I think I should rename this issuea to "Simplify Row creation".
          What do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user tonycox commented on the issue: https://github.com/apache/flink/pull/3127 I think I should rename this issuea to "Simplify Row creation". What do you think?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StephanEwen commented on the issue:

          https://github.com/apache/flink/pull/3127

          I think this goes into a good direction.
          I am wondering, though, if we need types like "time interval" in `flink-core`. I would prefer to keep then in the Table API scope.

          There can be a class `Types` in `flink-core` and one in `flink-table` - the table one extends the core one. That way you can have common types in core, extended types in table, and still get access to all by importing the `flink-table:Types` class.

          Show
          githubbot ASF GitHub Bot added a comment - Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/3127 I think this goes into a good direction. I am wondering, though, if we need types like "time interval" in `flink-core`. I would prefer to keep then in the Table API scope. There can be a class `Types` in `flink-core` and one in `flink-table` - the table one extends the core one. That way you can have common types in core, extended types in table, and still get access to all by importing the `flink-table:Types` class.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tonycox commented on the issue:

          https://github.com/apache/flink/pull/3127

          @StephanEwen @twalthr What do you think about this PR ?

          Show
          githubbot ASF GitHub Bot added a comment - Github user tonycox commented on the issue: https://github.com/apache/flink/pull/3127 @StephanEwen @twalthr What do you think about this PR ?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StephanEwen commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3127#discussion_r104641152

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala —
          @@ -17,29 +17,34 @@
          */
          package org.apache.flink.table.api

          -import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, SqlTimeTypeInfo}

          +import org.apache.flink.api.common.typeinfo.TypeInformation
          import org.apache.flink.table.typeutils.TimeIntervalTypeInfo
          +import org.apache.flink.api.java.typeutils.

          {Types => JTypes}

          /**

          • This class enumerates all supported types of the Table API.
            */
            -object Types {
            +object Types extends JTypes {
          • val STRING = BasicTypeInfo.STRING_TYPE_INFO
          • val BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO
            + val STRING = JTypes.STRING
            + val BOOLEAN = JTypes.BOOLEAN
          • val BYTE = BasicTypeInfo.BYTE_TYPE_INFO
          • val SHORT = BasicTypeInfo.SHORT_TYPE_INFO
          • val INT = BasicTypeInfo.INT_TYPE_INFO
          • val LONG = BasicTypeInfo.LONG_TYPE_INFO
          • val FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO
          • val DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO
          • val DECIMAL = BasicTypeInfo.BIG_DEC_TYPE_INFO
            + val BYTE = JTypes.BYTE
            + val SHORT = JTypes.SHORT
            + val INT = JTypes.INT
            + val LONG = JTypes.LONG
            + val FLOAT = JTypes.FLOAT
            + val DOUBLE = JTypes.DOUBLE
            + val DECIMAL = JTypes.DECIMAL
          • val DATE = SqlTimeTypeInfo.DATE
          • val TIME = SqlTimeTypeInfo.TIME
          • val TIMESTAMP = SqlTimeTypeInfo.TIMESTAMP
            + val DATE = JTypes.DATE
            + val TIME = JTypes.TIME
            + val TIMESTAMP = JTypes.TIMESTAMP
            val INTERVAL_MONTHS = TimeIntervalTypeInfo.INTERVAL_MONTHS
            val INTERVAL_MILLIS = TimeIntervalTypeInfo.INTERVAL_MILLIS

          + def ROW(types: TypeInformation[_]) = JTypes.ROW(types: _)
          +
          + def ROW(fieldNames: Array[String], types: TypeInformation[_]*) =
          — End diff –

          Maybe this would be nicer as `def ROW(fields: (String, TypeInformation[_])*)` so that one could write:
          ```scala
          val rowType = ROW( ("name", STRING), ("favorite_color", STRING), ("favorite_number", DOUBLE))
          ```

          Show
          githubbot ASF GitHub Bot added a comment - Github user StephanEwen commented on a diff in the pull request: https://github.com/apache/flink/pull/3127#discussion_r104641152 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala — @@ -17,29 +17,34 @@ */ package org.apache.flink.table.api -import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, SqlTimeTypeInfo} +import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.typeutils.TimeIntervalTypeInfo +import org.apache.flink.api.java.typeutils. {Types => JTypes} /** This class enumerates all supported types of the Table API. */ -object Types { +object Types extends JTypes { val STRING = BasicTypeInfo.STRING_TYPE_INFO val BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO + val STRING = JTypes.STRING + val BOOLEAN = JTypes.BOOLEAN val BYTE = BasicTypeInfo.BYTE_TYPE_INFO val SHORT = BasicTypeInfo.SHORT_TYPE_INFO val INT = BasicTypeInfo.INT_TYPE_INFO val LONG = BasicTypeInfo.LONG_TYPE_INFO val FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO val DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO val DECIMAL = BasicTypeInfo.BIG_DEC_TYPE_INFO + val BYTE = JTypes.BYTE + val SHORT = JTypes.SHORT + val INT = JTypes.INT + val LONG = JTypes.LONG + val FLOAT = JTypes.FLOAT + val DOUBLE = JTypes.DOUBLE + val DECIMAL = JTypes.DECIMAL val DATE = SqlTimeTypeInfo.DATE val TIME = SqlTimeTypeInfo.TIME val TIMESTAMP = SqlTimeTypeInfo.TIMESTAMP + val DATE = JTypes.DATE + val TIME = JTypes.TIME + val TIMESTAMP = JTypes.TIMESTAMP val INTERVAL_MONTHS = TimeIntervalTypeInfo.INTERVAL_MONTHS val INTERVAL_MILLIS = TimeIntervalTypeInfo.INTERVAL_MILLIS + def ROW(types: TypeInformation [_] ) = JTypes.ROW(types: _ ) + + def ROW(fieldNames: Array [String] , types: TypeInformation [_] *) = — End diff – Maybe this would be nicer as `def ROW(fields: (String, TypeInformation [_] )*)` so that one could write: ```scala val rowType = ROW( ("name", STRING), ("favorite_color", STRING), ("favorite_number", DOUBLE)) ```
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StephanEwen commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3127#discussion_r104641264

          — Diff: flink-core/src/main/java/org/apache/flink/api/java/typeutils/Types.java —
          @@ -0,0 +1,53 @@
          +/*
          + * 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.api.java.typeutils;
          +
          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
          +import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo;
          +import org.apache.flink.api.common.typeinfo.TypeInformation;
          +
          +import java.math.BigDecimal;
          +import java.sql.Date;
          +import java.sql.Time;
          +import java.sql.Timestamp;
          +
          +public class Types {
          +
          + public static final BasicTypeInfo<String> STRING = BasicTypeInfo.STRING_TYPE_INFO;
          + public static final BasicTypeInfo<Boolean> BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO;
          + public static final BasicTypeInfo<Byte> BYTE = BasicTypeInfo.BYTE_TYPE_INFO;
          + public static final BasicTypeInfo<Short> SHORT = BasicTypeInfo.SHORT_TYPE_INFO;
          + public static final BasicTypeInfo<Integer> INT = BasicTypeInfo.INT_TYPE_INFO;
          + public static final BasicTypeInfo<Long> LONG = BasicTypeInfo.LONG_TYPE_INFO;
          + public static final BasicTypeInfo<Float> FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO;
          + public static final BasicTypeInfo<Double> DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO;
          + public static final BasicTypeInfo<BigDecimal> DECIMAL = BasicTypeInfo.BIG_DEC_TYPE_INFO;
          +
          + public static final SqlTimeTypeInfo<Date> DATE = SqlTimeTypeInfo.DATE;
          — End diff –

          How about calling this `SQL_DATE`?

          Show
          githubbot ASF GitHub Bot added a comment - Github user StephanEwen commented on a diff in the pull request: https://github.com/apache/flink/pull/3127#discussion_r104641264 — Diff: flink-core/src/main/java/org/apache/flink/api/java/typeutils/Types.java — @@ -0,0 +1,53 @@ +/* + * 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.api.java.typeutils; + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; + +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; + +public class Types { + + public static final BasicTypeInfo<String> STRING = BasicTypeInfo.STRING_TYPE_INFO; + public static final BasicTypeInfo<Boolean> BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO; + public static final BasicTypeInfo<Byte> BYTE = BasicTypeInfo.BYTE_TYPE_INFO; + public static final BasicTypeInfo<Short> SHORT = BasicTypeInfo.SHORT_TYPE_INFO; + public static final BasicTypeInfo<Integer> INT = BasicTypeInfo.INT_TYPE_INFO; + public static final BasicTypeInfo<Long> LONG = BasicTypeInfo.LONG_TYPE_INFO; + public static final BasicTypeInfo<Float> FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO; + public static final BasicTypeInfo<Double> DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO; + public static final BasicTypeInfo<BigDecimal> DECIMAL = BasicTypeInfo.BIG_DEC_TYPE_INFO; + + public static final SqlTimeTypeInfo<Date> DATE = SqlTimeTypeInfo.DATE; — End diff – How about calling this `SQL_DATE`?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tonycox commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3127#discussion_r105972293

          — Diff: flink-core/src/main/java/org/apache/flink/api/java/typeutils/Types.java —
          @@ -0,0 +1,53 @@
          +/*
          + * 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.api.java.typeutils;
          +
          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
          +import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo;
          +import org.apache.flink.api.common.typeinfo.TypeInformation;
          +
          +import java.math.BigDecimal;
          +import java.sql.Date;
          +import java.sql.Time;
          +import java.sql.Timestamp;
          +
          +public class Types {
          +
          + public static final BasicTypeInfo<String> STRING = BasicTypeInfo.STRING_TYPE_INFO;
          + public static final BasicTypeInfo<Boolean> BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO;
          + public static final BasicTypeInfo<Byte> BYTE = BasicTypeInfo.BYTE_TYPE_INFO;
          + public static final BasicTypeInfo<Short> SHORT = BasicTypeInfo.SHORT_TYPE_INFO;
          + public static final BasicTypeInfo<Integer> INT = BasicTypeInfo.INT_TYPE_INFO;
          + public static final BasicTypeInfo<Long> LONG = BasicTypeInfo.LONG_TYPE_INFO;
          + public static final BasicTypeInfo<Float> FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO;
          + public static final BasicTypeInfo<Double> DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO;
          + public static final BasicTypeInfo<BigDecimal> DECIMAL = BasicTypeInfo.BIG_DEC_TYPE_INFO;
          +
          + public static final SqlTimeTypeInfo<Date> DATE = SqlTimeTypeInfo.DATE;
          — End diff –

          Done

          Show
          githubbot ASF GitHub Bot added a comment - Github user tonycox commented on a diff in the pull request: https://github.com/apache/flink/pull/3127#discussion_r105972293 — Diff: flink-core/src/main/java/org/apache/flink/api/java/typeutils/Types.java — @@ -0,0 +1,53 @@ +/* + * 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.api.java.typeutils; + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; + +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; + +public class Types { + + public static final BasicTypeInfo<String> STRING = BasicTypeInfo.STRING_TYPE_INFO; + public static final BasicTypeInfo<Boolean> BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO; + public static final BasicTypeInfo<Byte> BYTE = BasicTypeInfo.BYTE_TYPE_INFO; + public static final BasicTypeInfo<Short> SHORT = BasicTypeInfo.SHORT_TYPE_INFO; + public static final BasicTypeInfo<Integer> INT = BasicTypeInfo.INT_TYPE_INFO; + public static final BasicTypeInfo<Long> LONG = BasicTypeInfo.LONG_TYPE_INFO; + public static final BasicTypeInfo<Float> FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO; + public static final BasicTypeInfo<Double> DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO; + public static final BasicTypeInfo<BigDecimal> DECIMAL = BasicTypeInfo.BIG_DEC_TYPE_INFO; + + public static final SqlTimeTypeInfo<Date> DATE = SqlTimeTypeInfo.DATE; — End diff – Done
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tonycox commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3127#discussion_r105972317

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala —
          @@ -17,29 +17,34 @@
          */
          package org.apache.flink.table.api

          -import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, SqlTimeTypeInfo}

          +import org.apache.flink.api.common.typeinfo.TypeInformation
          import org.apache.flink.table.typeutils.TimeIntervalTypeInfo
          +import org.apache.flink.api.java.typeutils.

          {Types => JTypes}

          /**

          • This class enumerates all supported types of the Table API.
            */
            -object Types {
            +object Types extends JTypes {
          • val STRING = BasicTypeInfo.STRING_TYPE_INFO
          • val BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO
            + val STRING = JTypes.STRING
            + val BOOLEAN = JTypes.BOOLEAN
          • val BYTE = BasicTypeInfo.BYTE_TYPE_INFO
          • val SHORT = BasicTypeInfo.SHORT_TYPE_INFO
          • val INT = BasicTypeInfo.INT_TYPE_INFO
          • val LONG = BasicTypeInfo.LONG_TYPE_INFO
          • val FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO
          • val DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO
          • val DECIMAL = BasicTypeInfo.BIG_DEC_TYPE_INFO
            + val BYTE = JTypes.BYTE
            + val SHORT = JTypes.SHORT
            + val INT = JTypes.INT
            + val LONG = JTypes.LONG
            + val FLOAT = JTypes.FLOAT
            + val DOUBLE = JTypes.DOUBLE
            + val DECIMAL = JTypes.DECIMAL
          • val DATE = SqlTimeTypeInfo.DATE
          • val TIME = SqlTimeTypeInfo.TIME
          • val TIMESTAMP = SqlTimeTypeInfo.TIMESTAMP
            + val DATE = JTypes.DATE
            + val TIME = JTypes.TIME
            + val TIMESTAMP = JTypes.TIMESTAMP
            val INTERVAL_MONTHS = TimeIntervalTypeInfo.INTERVAL_MONTHS
            val INTERVAL_MILLIS = TimeIntervalTypeInfo.INTERVAL_MILLIS

          + def ROW(types: TypeInformation[_]) = JTypes.ROW(types: _)
          +
          + def ROW(fieldNames: Array[String], types: TypeInformation[_]*) =
          — End diff –

          Done

          Show
          githubbot ASF GitHub Bot added a comment - Github user tonycox commented on a diff in the pull request: https://github.com/apache/flink/pull/3127#discussion_r105972317 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala — @@ -17,29 +17,34 @@ */ package org.apache.flink.table.api -import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, SqlTimeTypeInfo} +import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.typeutils.TimeIntervalTypeInfo +import org.apache.flink.api.java.typeutils. {Types => JTypes} /** This class enumerates all supported types of the Table API. */ -object Types { +object Types extends JTypes { val STRING = BasicTypeInfo.STRING_TYPE_INFO val BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO + val STRING = JTypes.STRING + val BOOLEAN = JTypes.BOOLEAN val BYTE = BasicTypeInfo.BYTE_TYPE_INFO val SHORT = BasicTypeInfo.SHORT_TYPE_INFO val INT = BasicTypeInfo.INT_TYPE_INFO val LONG = BasicTypeInfo.LONG_TYPE_INFO val FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO val DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO val DECIMAL = BasicTypeInfo.BIG_DEC_TYPE_INFO + val BYTE = JTypes.BYTE + val SHORT = JTypes.SHORT + val INT = JTypes.INT + val LONG = JTypes.LONG + val FLOAT = JTypes.FLOAT + val DOUBLE = JTypes.DOUBLE + val DECIMAL = JTypes.DECIMAL val DATE = SqlTimeTypeInfo.DATE val TIME = SqlTimeTypeInfo.TIME val TIMESTAMP = SqlTimeTypeInfo.TIMESTAMP + val DATE = JTypes.DATE + val TIME = JTypes.TIME + val TIMESTAMP = JTypes.TIMESTAMP val INTERVAL_MONTHS = TimeIntervalTypeInfo.INTERVAL_MONTHS val INTERVAL_MILLIS = TimeIntervalTypeInfo.INTERVAL_MILLIS + def ROW(types: TypeInformation [_] ) = JTypes.ROW(types: _ ) + + def ROW(fieldNames: Array [String] , types: TypeInformation [_] *) = — End diff – Done
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tonycox commented on the issue:

          https://github.com/apache/flink/pull/3127

          Hi @StephanEwen
          I added javadoc

          Show
          githubbot ASF GitHub Bot added a comment - Github user tonycox commented on the issue: https://github.com/apache/flink/pull/3127 Hi @StephanEwen I added javadoc
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StephanEwen commented on the issue:

          https://github.com/apache/flink/pull/3127

          Looks good to me, +1

          @twalthr @fhueske Any concerns about merging this?

          Show
          githubbot ASF GitHub Bot added a comment - Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/3127 Looks good to me, +1 @twalthr @fhueske Any concerns about merging this?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StephanEwen commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3127#discussion_r106644720

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala —
          @@ -17,29 +17,51 @@
          */
          package org.apache.flink.table.api

          -import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, SqlTimeTypeInfo}

          +import org.apache.flink.api.common.typeinfo.TypeInformation
          import org.apache.flink.table.typeutils.TimeIntervalTypeInfo
          +import org.apache.flink.api.java.typeutils.

          {Types => JTypes}

          /**

          • This class enumerates all supported types of the Table API.
            */
            -object Types {
            +object Types extends JTypes {
          • val STRING = BasicTypeInfo.STRING_TYPE_INFO
          • val BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO
            + val STRING = JTypes.STRING
            + val BOOLEAN = JTypes.BOOLEAN
          • val BYTE = BasicTypeInfo.BYTE_TYPE_INFO
          • val SHORT = BasicTypeInfo.SHORT_TYPE_INFO
          • val INT = BasicTypeInfo.INT_TYPE_INFO
          • val LONG = BasicTypeInfo.LONG_TYPE_INFO
          • val FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO
          • val DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO
          • val DECIMAL = BasicTypeInfo.BIG_DEC_TYPE_INFO
            + val BYTE = JTypes.BYTE
            + val SHORT = JTypes.SHORT
            + val INT = JTypes.INT
            + val LONG = JTypes.LONG
            + val FLOAT = JTypes.FLOAT
            + val DOUBLE = JTypes.DOUBLE
            + val DECIMAL = JTypes.DECIMAL
          • val DATE = SqlTimeTypeInfo.DATE
          • val TIME = SqlTimeTypeInfo.TIME
          • val TIMESTAMP = SqlTimeTypeInfo.TIMESTAMP
            + val SQL_DATE = JTypes.SQL_DATE
            + val SQL_TIME = JTypes.SQL_TIME
            + val SQL_TIMESTAMP = JTypes.SQL_TIMESTAMP
            val INTERVAL_MONTHS = TimeIntervalTypeInfo.INTERVAL_MONTHS
            val INTERVAL_MILLIS = TimeIntervalTypeInfo.INTERVAL_MILLIS

          + /**
          + * Generates RowTypeInfo with default names (f1, f2 ..).
          + * same as ``new RowTypeInfo(types)``
          + *
          + * @param types of Row fields. e.g. ROW(Types.STRING, Types.INT)
          + */
          + def ROW[T](types: TypeInformation[_]*)(implicit m: Manifest[T]) = {
          — End diff –

          Out of curiosity: Why do you need the manifest? I think you don't need it as you don't reference `m` anywhere...

          Also, I think the common way of doing would be:
          ```scala
          def ROW[T: Manifest](types: TypeInformation[_]*) = { ...
          ```

          Show
          githubbot ASF GitHub Bot added a comment - Github user StephanEwen commented on a diff in the pull request: https://github.com/apache/flink/pull/3127#discussion_r106644720 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala — @@ -17,29 +17,51 @@ */ package org.apache.flink.table.api -import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, SqlTimeTypeInfo} +import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.typeutils.TimeIntervalTypeInfo +import org.apache.flink.api.java.typeutils. {Types => JTypes} /** This class enumerates all supported types of the Table API. */ -object Types { +object Types extends JTypes { val STRING = BasicTypeInfo.STRING_TYPE_INFO val BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO + val STRING = JTypes.STRING + val BOOLEAN = JTypes.BOOLEAN val BYTE = BasicTypeInfo.BYTE_TYPE_INFO val SHORT = BasicTypeInfo.SHORT_TYPE_INFO val INT = BasicTypeInfo.INT_TYPE_INFO val LONG = BasicTypeInfo.LONG_TYPE_INFO val FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO val DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO val DECIMAL = BasicTypeInfo.BIG_DEC_TYPE_INFO + val BYTE = JTypes.BYTE + val SHORT = JTypes.SHORT + val INT = JTypes.INT + val LONG = JTypes.LONG + val FLOAT = JTypes.FLOAT + val DOUBLE = JTypes.DOUBLE + val DECIMAL = JTypes.DECIMAL val DATE = SqlTimeTypeInfo.DATE val TIME = SqlTimeTypeInfo.TIME val TIMESTAMP = SqlTimeTypeInfo.TIMESTAMP + val SQL_DATE = JTypes.SQL_DATE + val SQL_TIME = JTypes.SQL_TIME + val SQL_TIMESTAMP = JTypes.SQL_TIMESTAMP val INTERVAL_MONTHS = TimeIntervalTypeInfo.INTERVAL_MONTHS val INTERVAL_MILLIS = TimeIntervalTypeInfo.INTERVAL_MILLIS + /** + * Generates RowTypeInfo with default names (f1, f2 ..). + * same as ``new RowTypeInfo(types)`` + * + * @param types of Row fields. e.g. ROW(Types.STRING, Types.INT) + */ + def ROW [T] (types: TypeInformation [_] *)(implicit m: Manifest [T] ) = { — End diff – Out of curiosity: Why do you need the manifest? I think you don't need it as you don't reference `m` anywhere... Also, I think the common way of doing would be: ```scala def ROW [T: Manifest] (types: TypeInformation [_] *) = { ... ```
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StephanEwen commented on the issue:

          https://github.com/apache/flink/pull/3127

          Found one more small concern (inline comment)

          Show
          githubbot ASF GitHub Bot added a comment - Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/3127 Found one more small concern (inline comment)
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tonycox commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3127#discussion_r106647040

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala —
          @@ -17,29 +17,51 @@
          */
          package org.apache.flink.table.api

          -import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, SqlTimeTypeInfo}

          +import org.apache.flink.api.common.typeinfo.TypeInformation
          import org.apache.flink.table.typeutils.TimeIntervalTypeInfo
          +import org.apache.flink.api.java.typeutils.

          {Types => JTypes}

          /**

          • This class enumerates all supported types of the Table API.
            */
            -object Types {
            +object Types extends JTypes {
          • val STRING = BasicTypeInfo.STRING_TYPE_INFO
          • val BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO
            + val STRING = JTypes.STRING
            + val BOOLEAN = JTypes.BOOLEAN
          • val BYTE = BasicTypeInfo.BYTE_TYPE_INFO
          • val SHORT = BasicTypeInfo.SHORT_TYPE_INFO
          • val INT = BasicTypeInfo.INT_TYPE_INFO
          • val LONG = BasicTypeInfo.LONG_TYPE_INFO
          • val FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO
          • val DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO
          • val DECIMAL = BasicTypeInfo.BIG_DEC_TYPE_INFO
            + val BYTE = JTypes.BYTE
            + val SHORT = JTypes.SHORT
            + val INT = JTypes.INT
            + val LONG = JTypes.LONG
            + val FLOAT = JTypes.FLOAT
            + val DOUBLE = JTypes.DOUBLE
            + val DECIMAL = JTypes.DECIMAL
          • val DATE = SqlTimeTypeInfo.DATE
          • val TIME = SqlTimeTypeInfo.TIME
          • val TIMESTAMP = SqlTimeTypeInfo.TIMESTAMP
            + val SQL_DATE = JTypes.SQL_DATE
            + val SQL_TIME = JTypes.SQL_TIME
            + val SQL_TIMESTAMP = JTypes.SQL_TIMESTAMP
            val INTERVAL_MONTHS = TimeIntervalTypeInfo.INTERVAL_MONTHS
            val INTERVAL_MILLIS = TimeIntervalTypeInfo.INTERVAL_MILLIS

          + /**
          + * Generates RowTypeInfo with default names (f1, f2 ..).
          + * same as ``new RowTypeInfo(types)``
          + *
          + * @param types of Row fields. e.g. ROW(Types.STRING, Types.INT)
          + */
          + def ROW[T](types: TypeInformation[_]*)(implicit m: Manifest[T]) = {
          — End diff –

          You are right. Done

          Show
          githubbot ASF GitHub Bot added a comment - Github user tonycox commented on a diff in the pull request: https://github.com/apache/flink/pull/3127#discussion_r106647040 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala — @@ -17,29 +17,51 @@ */ package org.apache.flink.table.api -import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, SqlTimeTypeInfo} +import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.typeutils.TimeIntervalTypeInfo +import org.apache.flink.api.java.typeutils. {Types => JTypes} /** This class enumerates all supported types of the Table API. */ -object Types { +object Types extends JTypes { val STRING = BasicTypeInfo.STRING_TYPE_INFO val BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO + val STRING = JTypes.STRING + val BOOLEAN = JTypes.BOOLEAN val BYTE = BasicTypeInfo.BYTE_TYPE_INFO val SHORT = BasicTypeInfo.SHORT_TYPE_INFO val INT = BasicTypeInfo.INT_TYPE_INFO val LONG = BasicTypeInfo.LONG_TYPE_INFO val FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO val DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO val DECIMAL = BasicTypeInfo.BIG_DEC_TYPE_INFO + val BYTE = JTypes.BYTE + val SHORT = JTypes.SHORT + val INT = JTypes.INT + val LONG = JTypes.LONG + val FLOAT = JTypes.FLOAT + val DOUBLE = JTypes.DOUBLE + val DECIMAL = JTypes.DECIMAL val DATE = SqlTimeTypeInfo.DATE val TIME = SqlTimeTypeInfo.TIME val TIMESTAMP = SqlTimeTypeInfo.TIMESTAMP + val SQL_DATE = JTypes.SQL_DATE + val SQL_TIME = JTypes.SQL_TIME + val SQL_TIMESTAMP = JTypes.SQL_TIMESTAMP val INTERVAL_MONTHS = TimeIntervalTypeInfo.INTERVAL_MONTHS val INTERVAL_MILLIS = TimeIntervalTypeInfo.INTERVAL_MILLIS + /** + * Generates RowTypeInfo with default names (f1, f2 ..). + * same as ``new RowTypeInfo(types)`` + * + * @param types of Row fields. e.g. ROW(Types.STRING, Types.INT) + */ + def ROW [T] (types: TypeInformation [_] *)(implicit m: Manifest [T] ) = { — End diff – You are right. Done
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StephanEwen commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3127#discussion_r106664092

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala —
          @@ -17,29 +17,51 @@
          */
          package org.apache.flink.table.api

          -import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, SqlTimeTypeInfo}

          +import org.apache.flink.api.common.typeinfo.TypeInformation
          import org.apache.flink.table.typeutils.TimeIntervalTypeInfo
          +import org.apache.flink.api.java.typeutils.

          {Types => JTypes}

          /**

          • This class enumerates all supported types of the Table API.
            */
            -object Types {
            +object Types extends JTypes {
          • val STRING = BasicTypeInfo.STRING_TYPE_INFO
          • val BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO
            + val STRING = JTypes.STRING
            + val BOOLEAN = JTypes.BOOLEAN
          • val BYTE = BasicTypeInfo.BYTE_TYPE_INFO
          • val SHORT = BasicTypeInfo.SHORT_TYPE_INFO
          • val INT = BasicTypeInfo.INT_TYPE_INFO
          • val LONG = BasicTypeInfo.LONG_TYPE_INFO
          • val FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO
          • val DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO
          • val DECIMAL = BasicTypeInfo.BIG_DEC_TYPE_INFO
            + val BYTE = JTypes.BYTE
            + val SHORT = JTypes.SHORT
            + val INT = JTypes.INT
            + val LONG = JTypes.LONG
            + val FLOAT = JTypes.FLOAT
            + val DOUBLE = JTypes.DOUBLE
            + val DECIMAL = JTypes.DECIMAL
          • val DATE = SqlTimeTypeInfo.DATE
          • val TIME = SqlTimeTypeInfo.TIME
          • val TIMESTAMP = SqlTimeTypeInfo.TIMESTAMP
            + val SQL_DATE = JTypes.SQL_DATE
            + val SQL_TIME = JTypes.SQL_TIME
            + val SQL_TIMESTAMP = JTypes.SQL_TIMESTAMP
            val INTERVAL_MONTHS = TimeIntervalTypeInfo.INTERVAL_MONTHS
            val INTERVAL_MILLIS = TimeIntervalTypeInfo.INTERVAL_MILLIS

          + /**
          + * Generates RowTypeInfo with default names (f1, f2 ..).
          + * same as ``new RowTypeInfo(types)``
          + *
          + * @param types of Row fields. e.g. ROW(Types.STRING, Types.INT)
          + */
          + def ROW[T](types: TypeInformation[_]*)(implicit m: Manifest[T]) = {
          — End diff –

          Can you explain what the manifest is needed for?

          Show
          githubbot ASF GitHub Bot added a comment - Github user StephanEwen commented on a diff in the pull request: https://github.com/apache/flink/pull/3127#discussion_r106664092 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala — @@ -17,29 +17,51 @@ */ package org.apache.flink.table.api -import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, SqlTimeTypeInfo} +import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.typeutils.TimeIntervalTypeInfo +import org.apache.flink.api.java.typeutils. {Types => JTypes} /** This class enumerates all supported types of the Table API. */ -object Types { +object Types extends JTypes { val STRING = BasicTypeInfo.STRING_TYPE_INFO val BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO + val STRING = JTypes.STRING + val BOOLEAN = JTypes.BOOLEAN val BYTE = BasicTypeInfo.BYTE_TYPE_INFO val SHORT = BasicTypeInfo.SHORT_TYPE_INFO val INT = BasicTypeInfo.INT_TYPE_INFO val LONG = BasicTypeInfo.LONG_TYPE_INFO val FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO val DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO val DECIMAL = BasicTypeInfo.BIG_DEC_TYPE_INFO + val BYTE = JTypes.BYTE + val SHORT = JTypes.SHORT + val INT = JTypes.INT + val LONG = JTypes.LONG + val FLOAT = JTypes.FLOAT + val DOUBLE = JTypes.DOUBLE + val DECIMAL = JTypes.DECIMAL val DATE = SqlTimeTypeInfo.DATE val TIME = SqlTimeTypeInfo.TIME val TIMESTAMP = SqlTimeTypeInfo.TIMESTAMP + val SQL_DATE = JTypes.SQL_DATE + val SQL_TIME = JTypes.SQL_TIME + val SQL_TIMESTAMP = JTypes.SQL_TIMESTAMP val INTERVAL_MONTHS = TimeIntervalTypeInfo.INTERVAL_MONTHS val INTERVAL_MILLIS = TimeIntervalTypeInfo.INTERVAL_MILLIS + /** + * Generates RowTypeInfo with default names (f1, f2 ..). + * same as ``new RowTypeInfo(types)`` + * + * @param types of Row fields. e.g. ROW(Types.STRING, Types.INT) + */ + def ROW [T] (types: TypeInformation [_] *)(implicit m: Manifest [T] ) = { — End diff – Can you explain what the manifest is needed for?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tonycox commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3127#discussion_r106668425

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala —
          @@ -17,29 +17,51 @@
          */
          package org.apache.flink.table.api

          -import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, SqlTimeTypeInfo}

          +import org.apache.flink.api.common.typeinfo.TypeInformation
          import org.apache.flink.table.typeutils.TimeIntervalTypeInfo
          +import org.apache.flink.api.java.typeutils.

          {Types => JTypes}

          /**

          • This class enumerates all supported types of the Table API.
            */
            -object Types {
            +object Types extends JTypes {
          • val STRING = BasicTypeInfo.STRING_TYPE_INFO
          • val BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO
            + val STRING = JTypes.STRING
            + val BOOLEAN = JTypes.BOOLEAN
          • val BYTE = BasicTypeInfo.BYTE_TYPE_INFO
          • val SHORT = BasicTypeInfo.SHORT_TYPE_INFO
          • val INT = BasicTypeInfo.INT_TYPE_INFO
          • val LONG = BasicTypeInfo.LONG_TYPE_INFO
          • val FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO
          • val DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO
          • val DECIMAL = BasicTypeInfo.BIG_DEC_TYPE_INFO
            + val BYTE = JTypes.BYTE
            + val SHORT = JTypes.SHORT
            + val INT = JTypes.INT
            + val LONG = JTypes.LONG
            + val FLOAT = JTypes.FLOAT
            + val DOUBLE = JTypes.DOUBLE
            + val DECIMAL = JTypes.DECIMAL
          • val DATE = SqlTimeTypeInfo.DATE
          • val TIME = SqlTimeTypeInfo.TIME
          • val TIMESTAMP = SqlTimeTypeInfo.TIMESTAMP
            + val SQL_DATE = JTypes.SQL_DATE
            + val SQL_TIME = JTypes.SQL_TIME
            + val SQL_TIMESTAMP = JTypes.SQL_TIMESTAMP
            val INTERVAL_MONTHS = TimeIntervalTypeInfo.INTERVAL_MONTHS
            val INTERVAL_MILLIS = TimeIntervalTypeInfo.INTERVAL_MILLIS

          + /**
          + * Generates RowTypeInfo with default names (f1, f2 ..).
          + * same as ``new RowTypeInfo(types)``
          + *
          + * @param types of Row fields. e.g. ROW(Types.STRING, Types.INT)
          + */
          + def ROW[T](types: TypeInformation[_]*)(implicit m: Manifest[T]) = {
          — End diff –

          compilation error will come up if we don't use it
          ```
          Error:(62, 7) double definition:
          method ROW:(fields: (String, org.apache.flink.api.common.typeinfo.TypeInformation[_])*)org.apache.flink.api.java.typeutils.RowTypeInfo and
          method ROW:(types: org.apache.flink.api.common.typeinfo.TypeInformation[_]*)org.apache.flink.api.java.typeutils.RowTypeInfo at line 52
          have same type after erasure: (fields: Seq)org.apache.flink.api.java.typeutils.RowTypeInfo
          def ROW(fields: (String, TypeInformation[_])*) = {
          ```

          Show
          githubbot ASF GitHub Bot added a comment - Github user tonycox commented on a diff in the pull request: https://github.com/apache/flink/pull/3127#discussion_r106668425 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala — @@ -17,29 +17,51 @@ */ package org.apache.flink.table.api -import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, SqlTimeTypeInfo} +import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.typeutils.TimeIntervalTypeInfo +import org.apache.flink.api.java.typeutils. {Types => JTypes} /** This class enumerates all supported types of the Table API. */ -object Types { +object Types extends JTypes { val STRING = BasicTypeInfo.STRING_TYPE_INFO val BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO + val STRING = JTypes.STRING + val BOOLEAN = JTypes.BOOLEAN val BYTE = BasicTypeInfo.BYTE_TYPE_INFO val SHORT = BasicTypeInfo.SHORT_TYPE_INFO val INT = BasicTypeInfo.INT_TYPE_INFO val LONG = BasicTypeInfo.LONG_TYPE_INFO val FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO val DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO val DECIMAL = BasicTypeInfo.BIG_DEC_TYPE_INFO + val BYTE = JTypes.BYTE + val SHORT = JTypes.SHORT + val INT = JTypes.INT + val LONG = JTypes.LONG + val FLOAT = JTypes.FLOAT + val DOUBLE = JTypes.DOUBLE + val DECIMAL = JTypes.DECIMAL val DATE = SqlTimeTypeInfo.DATE val TIME = SqlTimeTypeInfo.TIME val TIMESTAMP = SqlTimeTypeInfo.TIMESTAMP + val SQL_DATE = JTypes.SQL_DATE + val SQL_TIME = JTypes.SQL_TIME + val SQL_TIMESTAMP = JTypes.SQL_TIMESTAMP val INTERVAL_MONTHS = TimeIntervalTypeInfo.INTERVAL_MONTHS val INTERVAL_MILLIS = TimeIntervalTypeInfo.INTERVAL_MILLIS + /** + * Generates RowTypeInfo with default names (f1, f2 ..). + * same as ``new RowTypeInfo(types)`` + * + * @param types of Row fields. e.g. ROW(Types.STRING, Types.INT) + */ + def ROW [T] (types: TypeInformation [_] *)(implicit m: Manifest [T] ) = { — End diff – compilation error will come up if we don't use it ``` Error:(62, 7) double definition: method ROW:(fields: (String, org.apache.flink.api.common.typeinfo.TypeInformation [_] )*)org.apache.flink.api.java.typeutils.RowTypeInfo and method ROW:(types: org.apache.flink.api.common.typeinfo.TypeInformation [_] *)org.apache.flink.api.java.typeutils.RowTypeInfo at line 52 have same type after erasure: (fields: Seq)org.apache.flink.api.java.typeutils.RowTypeInfo def ROW(fields: (String, TypeInformation [_] )*) = { ```
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StephanEwen commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3127#discussion_r106717641

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala —
          @@ -17,29 +17,51 @@
          */
          package org.apache.flink.table.api

          -import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, SqlTimeTypeInfo}

          +import org.apache.flink.api.common.typeinfo.TypeInformation
          import org.apache.flink.table.typeutils.TimeIntervalTypeInfo
          +import org.apache.flink.api.java.typeutils.

          {Types => JTypes}

          /**

          • This class enumerates all supported types of the Table API.
            */
            -object Types {
            +object Types extends JTypes {
          • val STRING = BasicTypeInfo.STRING_TYPE_INFO
          • val BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO
            + val STRING = JTypes.STRING
            + val BOOLEAN = JTypes.BOOLEAN
          • val BYTE = BasicTypeInfo.BYTE_TYPE_INFO
          • val SHORT = BasicTypeInfo.SHORT_TYPE_INFO
          • val INT = BasicTypeInfo.INT_TYPE_INFO
          • val LONG = BasicTypeInfo.LONG_TYPE_INFO
          • val FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO
          • val DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO
          • val DECIMAL = BasicTypeInfo.BIG_DEC_TYPE_INFO
            + val BYTE = JTypes.BYTE
            + val SHORT = JTypes.SHORT
            + val INT = JTypes.INT
            + val LONG = JTypes.LONG
            + val FLOAT = JTypes.FLOAT
            + val DOUBLE = JTypes.DOUBLE
            + val DECIMAL = JTypes.DECIMAL
          • val DATE = SqlTimeTypeInfo.DATE
          • val TIME = SqlTimeTypeInfo.TIME
          • val TIMESTAMP = SqlTimeTypeInfo.TIMESTAMP
            + val SQL_DATE = JTypes.SQL_DATE
            + val SQL_TIME = JTypes.SQL_TIME
            + val SQL_TIMESTAMP = JTypes.SQL_TIMESTAMP
            val INTERVAL_MONTHS = TimeIntervalTypeInfo.INTERVAL_MONTHS
            val INTERVAL_MILLIS = TimeIntervalTypeInfo.INTERVAL_MILLIS

          + /**
          + * Generates RowTypeInfo with default names (f1, f2 ..).
          + * same as ``new RowTypeInfo(types)``
          + *
          + * @param types of Row fields. e.g. ROW(Types.STRING, Types.INT)
          + */
          + def ROW[T](types: TypeInformation[_]*)(implicit m: Manifest[T]) = {
          — End diff –

          How about calling it `ROW` and `ROW_NAMED` or so? I think just adding another parameter is hacky...

          Show
          githubbot ASF GitHub Bot added a comment - Github user StephanEwen commented on a diff in the pull request: https://github.com/apache/flink/pull/3127#discussion_r106717641 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala — @@ -17,29 +17,51 @@ */ package org.apache.flink.table.api -import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, SqlTimeTypeInfo} +import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.typeutils.TimeIntervalTypeInfo +import org.apache.flink.api.java.typeutils. {Types => JTypes} /** This class enumerates all supported types of the Table API. */ -object Types { +object Types extends JTypes { val STRING = BasicTypeInfo.STRING_TYPE_INFO val BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO + val STRING = JTypes.STRING + val BOOLEAN = JTypes.BOOLEAN val BYTE = BasicTypeInfo.BYTE_TYPE_INFO val SHORT = BasicTypeInfo.SHORT_TYPE_INFO val INT = BasicTypeInfo.INT_TYPE_INFO val LONG = BasicTypeInfo.LONG_TYPE_INFO val FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO val DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO val DECIMAL = BasicTypeInfo.BIG_DEC_TYPE_INFO + val BYTE = JTypes.BYTE + val SHORT = JTypes.SHORT + val INT = JTypes.INT + val LONG = JTypes.LONG + val FLOAT = JTypes.FLOAT + val DOUBLE = JTypes.DOUBLE + val DECIMAL = JTypes.DECIMAL val DATE = SqlTimeTypeInfo.DATE val TIME = SqlTimeTypeInfo.TIME val TIMESTAMP = SqlTimeTypeInfo.TIMESTAMP + val SQL_DATE = JTypes.SQL_DATE + val SQL_TIME = JTypes.SQL_TIME + val SQL_TIMESTAMP = JTypes.SQL_TIMESTAMP val INTERVAL_MONTHS = TimeIntervalTypeInfo.INTERVAL_MONTHS val INTERVAL_MILLIS = TimeIntervalTypeInfo.INTERVAL_MILLIS + /** + * Generates RowTypeInfo with default names (f1, f2 ..). + * same as ``new RowTypeInfo(types)`` + * + * @param types of Row fields. e.g. ROW(Types.STRING, Types.INT) + */ + def ROW [T] (types: TypeInformation [_] *)(implicit m: Manifest [T] ) = { — End diff – How about calling it `ROW` and `ROW_NAMED` or so? I think just adding another parameter is hacky...
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tonycox commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3127#discussion_r106862331

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala —
          @@ -17,29 +17,51 @@
          */
          package org.apache.flink.table.api

          -import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, SqlTimeTypeInfo}

          +import org.apache.flink.api.common.typeinfo.TypeInformation
          import org.apache.flink.table.typeutils.TimeIntervalTypeInfo
          +import org.apache.flink.api.java.typeutils.

          {Types => JTypes}

          /**

          • This class enumerates all supported types of the Table API.
            */
            -object Types {
            +object Types extends JTypes {
          • val STRING = BasicTypeInfo.STRING_TYPE_INFO
          • val BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO
            + val STRING = JTypes.STRING
            + val BOOLEAN = JTypes.BOOLEAN
          • val BYTE = BasicTypeInfo.BYTE_TYPE_INFO
          • val SHORT = BasicTypeInfo.SHORT_TYPE_INFO
          • val INT = BasicTypeInfo.INT_TYPE_INFO
          • val LONG = BasicTypeInfo.LONG_TYPE_INFO
          • val FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO
          • val DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO
          • val DECIMAL = BasicTypeInfo.BIG_DEC_TYPE_INFO
            + val BYTE = JTypes.BYTE
            + val SHORT = JTypes.SHORT
            + val INT = JTypes.INT
            + val LONG = JTypes.LONG
            + val FLOAT = JTypes.FLOAT
            + val DOUBLE = JTypes.DOUBLE
            + val DECIMAL = JTypes.DECIMAL
          • val DATE = SqlTimeTypeInfo.DATE
          • val TIME = SqlTimeTypeInfo.TIME
          • val TIMESTAMP = SqlTimeTypeInfo.TIMESTAMP
            + val SQL_DATE = JTypes.SQL_DATE
            + val SQL_TIME = JTypes.SQL_TIME
            + val SQL_TIMESTAMP = JTypes.SQL_TIMESTAMP
            val INTERVAL_MONTHS = TimeIntervalTypeInfo.INTERVAL_MONTHS
            val INTERVAL_MILLIS = TimeIntervalTypeInfo.INTERVAL_MILLIS

          + /**
          + * Generates RowTypeInfo with default names (f1, f2 ..).
          + * same as ``new RowTypeInfo(types)``
          + *
          + * @param types of Row fields. e.g. ROW(Types.STRING, Types.INT)
          + */
          + def ROW[T](types: TypeInformation[_]*)(implicit m: Manifest[T]) = {
          — End diff –

          I think it's good to keep same name, but I can rename as you suggest

          Show
          githubbot ASF GitHub Bot added a comment - Github user tonycox commented on a diff in the pull request: https://github.com/apache/flink/pull/3127#discussion_r106862331 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala — @@ -17,29 +17,51 @@ */ package org.apache.flink.table.api -import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, SqlTimeTypeInfo} +import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.typeutils.TimeIntervalTypeInfo +import org.apache.flink.api.java.typeutils. {Types => JTypes} /** This class enumerates all supported types of the Table API. */ -object Types { +object Types extends JTypes { val STRING = BasicTypeInfo.STRING_TYPE_INFO val BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO + val STRING = JTypes.STRING + val BOOLEAN = JTypes.BOOLEAN val BYTE = BasicTypeInfo.BYTE_TYPE_INFO val SHORT = BasicTypeInfo.SHORT_TYPE_INFO val INT = BasicTypeInfo.INT_TYPE_INFO val LONG = BasicTypeInfo.LONG_TYPE_INFO val FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO val DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO val DECIMAL = BasicTypeInfo.BIG_DEC_TYPE_INFO + val BYTE = JTypes.BYTE + val SHORT = JTypes.SHORT + val INT = JTypes.INT + val LONG = JTypes.LONG + val FLOAT = JTypes.FLOAT + val DOUBLE = JTypes.DOUBLE + val DECIMAL = JTypes.DECIMAL val DATE = SqlTimeTypeInfo.DATE val TIME = SqlTimeTypeInfo.TIME val TIMESTAMP = SqlTimeTypeInfo.TIMESTAMP + val SQL_DATE = JTypes.SQL_DATE + val SQL_TIME = JTypes.SQL_TIME + val SQL_TIMESTAMP = JTypes.SQL_TIMESTAMP val INTERVAL_MONTHS = TimeIntervalTypeInfo.INTERVAL_MONTHS val INTERVAL_MILLIS = TimeIntervalTypeInfo.INTERVAL_MILLIS + /** + * Generates RowTypeInfo with default names (f1, f2 ..). + * same as ``new RowTypeInfo(types)`` + * + * @param types of Row fields. e.g. ROW(Types.STRING, Types.INT) + */ + def ROW [T] (types: TypeInformation [_] *)(implicit m: Manifest [T] ) = { — End diff – I think it's good to keep same name, but I can rename as you suggest
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tonycox commented on the issue:

          https://github.com/apache/flink/pull/3127

          @StephanEwen I fixed

          Show
          githubbot ASF GitHub Bot added a comment - Github user tonycox commented on the issue: https://github.com/apache/flink/pull/3127 @StephanEwen I fixed
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user DmytroShkvyra commented on the issue:

          https://github.com/apache/flink/pull/3127

          Hi @StephanEwen could you continue review this PR?

          Show
          githubbot ASF GitHub Bot added a comment - Github user DmytroShkvyra commented on the issue: https://github.com/apache/flink/pull/3127 Hi @StephanEwen could you continue review this PR?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StephanEwen commented on the issue:

          https://github.com/apache/flink/pull/3127

          Looks good, thank you!
          Merging this...

          Show
          githubbot ASF GitHub Bot added a comment - Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/3127 Looks good, thank you! Merging this...
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

          https://github.com/apache/flink/pull/3127

          Show
          githubbot ASF GitHub Bot added a comment - Github user asfgit closed the pull request at: https://github.com/apache/flink/pull/3127
          Hide
          StephanEwen Stephan Ewen added a comment -

          Fixed via eb4db536dd7b03f33849adb4b5f1926fcc57efd8

          Show
          StephanEwen Stephan Ewen added a comment - Fixed via eb4db536dd7b03f33849adb4b5f1926fcc57efd8

            People

            • Assignee:
              tonycox Anton Solovev
              Reporter:
              tonycox Anton Solovev
            • Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development