Details

    • Type: New Feature
    • Status: Resolved
    • Priority: Major
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: 1.3.0
    • Component/s: Table API & SQL
    • Labels:

      Description

      For debugging and maybe for visualization in future (e.g. in a shell) it would be good to have the possibilty to get the names of Table columns. At the moment the user has no idea how the table columns are named; if they need to be matched with POJO fields for example.

      My suggestion:

      Schema s = table.schema();
      TypeInformation<?> type = s.getType(1);
      TypeInformation<?> type = s.getType("col");
      String s = s.getColumnName(1);
      String[] s = s.getColumnNames();
      

        Issue Links

          Activity

          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user wuchong opened a pull request:

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

          FLINK-4686 [table] Add possibility to get column names

          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.

          • [ x] 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)
          • [x ] 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
          • [x ] 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

          This PR introduces `Schema` to allow us get column names and types.

          ```java
          Schema s = table.schema();
          TypeInformation<?> type = s.getType(1);
          TypeInformation<?> type = s.getType("col");
          String s = s.getColumnName(1);
          String[] s = s.getColumnNames();
          ```

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

          $ git pull https://github.com/wuchong/flink schema-FLINK-4686

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

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


          commit 9b40f85e9ebbe2524b440d6abc9722380467266c
          Author: Jark Wu <wuchong.wc@alibaba-inc.com>
          Date: 2016-09-27T09:05:27Z

          FLINK-4686 [table] Add possibility to get column names


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user wuchong opened a pull request: https://github.com/apache/flink/pull/2553 FLINK-4686 [table] Add possibility to get column names 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. [ x] 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) [x ] 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 [x ] 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 This PR introduces `Schema` to allow us get column names and types. ```java Schema s = table.schema(); TypeInformation<?> type = s.getType(1); TypeInformation<?> type = s.getType("col"); String s = s.getColumnName(1); String[] s = s.getColumnNames(); ``` You can merge this pull request into a Git repository by running: $ git pull https://github.com/wuchong/flink schema- FLINK-4686 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/2553.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 #2553 commit 9b40f85e9ebbe2524b440d6abc9722380467266c Author: Jark Wu <wuchong.wc@alibaba-inc.com> Date: 2016-09-27T09:05:27Z FLINK-4686 [table] Add possibility to get column names
          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/2553#discussion_r80929290

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/Schema.scala —
          @@ -0,0 +1,89 @@
          +/*
          + * 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.table
          +
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +
          +/**
          + * A Schema represents a Table's structure
          + */
          +class Schema(
          + val columnNames: Array[String],
          — End diff –

          private?

          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/2553#discussion_r80929290 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/Schema.scala — @@ -0,0 +1,89 @@ +/* + * 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.table + +import org.apache.flink.api.common.typeinfo.TypeInformation + +/** + * A Schema represents a Table's structure + */ +class Schema( + val columnNames: Array [String] , — End diff – private?
          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/2553#discussion_r80931476

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/Schema.scala —
          @@ -0,0 +1,89 @@
          +/*
          + * 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.table
          +
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +
          +/**
          + * A Schema represents a Table's structure
          + */
          +class Schema(
          + val columnNames: Array[String],
          + val columnTypes: Array[TypeInformation[_]]) {
          +
          + if (columnNames.length != columnTypes.length)

          { + throw new TableException( + "Number of column indexes and column names must be equal.") + }

          +
          + // check uniqueness of field names
          + if (columnNames.toSet.size != columnTypes.length)

          { + throw new TableException( + "Table column names must be unique.") + }

          +
          + val columnNameToIndex: Map[String, Int] = columnNames.zipWithIndex.toMap
          +
          + /**
          + * Returns all type informations as an array
          + */
          + def getTypes: Array[TypeInformation[_]] = columnTypes
          +
          + /**
          + * Returns the specified type information for the given column index
          + *
          + * @param columnIndex the index of the field
          + */
          + def getType(columnIndex: Int): TypeInformation[_] = columnTypes(columnIndex)
          — End diff –

          No index check here.

          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/2553#discussion_r80931476 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/Schema.scala — @@ -0,0 +1,89 @@ +/* + * 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.table + +import org.apache.flink.api.common.typeinfo.TypeInformation + +/** + * A Schema represents a Table's structure + */ +class Schema( + val columnNames: Array [String] , + val columnTypes: Array[TypeInformation [_] ]) { + + if (columnNames.length != columnTypes.length) { + throw new TableException( + "Number of column indexes and column names must be equal.") + } + + // check uniqueness of field names + if (columnNames.toSet.size != columnTypes.length) { + throw new TableException( + "Table column names must be unique.") + } + + val columnNameToIndex: Map [String, Int] = columnNames.zipWithIndex.toMap + + /** + * Returns all type informations as an array + */ + def getTypes: Array[TypeInformation [_] ] = columnTypes + + /** + * Returns the specified type information for the given column index + * + * @param columnIndex the index of the field + */ + def getType(columnIndex: Int): TypeInformation [_] = columnTypes(columnIndex) — End diff – No index check here.
          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/2553#discussion_r80930567

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/Schema.scala —
          @@ -0,0 +1,89 @@
          +/*
          + * 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.table
          +
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +
          +/**
          + * A Schema represents a Table's structure
          + */
          +class Schema(
          — End diff –

          @fhueske are you fine with the name? Or maybe `TableSchema`?

          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/2553#discussion_r80930567 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/Schema.scala — @@ -0,0 +1,89 @@ +/* + * 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.table + +import org.apache.flink.api.common.typeinfo.TypeInformation + +/** + * A Schema represents a Table's structure + */ +class Schema( — End diff – @fhueske are you fine with the name? Or maybe `TableSchema`?
          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/2553#discussion_r80932356

          — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/SchemaTest.java —
          @@ -0,0 +1,84 @@
          +/*
          + * 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.batch;
          +
          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
          +import org.apache.flink.api.common.typeinfo.TypeInformation;
          +import org.apache.flink.api.table.FieldNotFoundException;
          +import org.apache.flink.api.table.Schema;
          +import org.apache.flink.api.table.TableException;
          +import org.junit.Test;
          +
          +import static org.junit.Assert.*;
          +
          +public class SchemaTest {
          +
          + @Test
          + public void testSchema() {
          — End diff –

          Could you add a test that tests getting a schema from a existing table, too?

          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/2553#discussion_r80932356 — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/SchemaTest.java — @@ -0,0 +1,84 @@ +/* + * 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.batch; + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.table.FieldNotFoundException; +import org.apache.flink.api.table.Schema; +import org.apache.flink.api.table.TableException; +import org.junit.Test; + +import static org.junit.Assert.*; + +public class SchemaTest { + + @Test + public void testSchema() { — End diff – Could you add a test that tests getting a schema from a existing table, too?
          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/2553#discussion_r80930670

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/Schema.scala —
          @@ -0,0 +1,89 @@
          +/*
          + * 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.table
          +
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +
          +/**
          + * A Schema represents a Table's structure
          + */
          +class Schema(
          + val columnNames: Array[String],
          + val columnTypes: Array[TypeInformation[_]]) {
          +
          + if (columnNames.length != columnTypes.length)

          { + throw new TableException( + "Number of column indexes and column names must be equal.") + }

          +
          + // check uniqueness of field names
          + if (columnNames.toSet.size != columnTypes.length)

          { + throw new TableException( + "Table column names must be unique.") + }

          +
          + val columnNameToIndex: Map[String, Int] = columnNames.zipWithIndex.toMap
          +
          + /**
          + * Returns all type informations as an array
          + */
          + def getTypes: Array[TypeInformation[_]] = columnTypes
          +
          + /**
          + * Returns the specified type information for the given column index
          + *
          — End diff –

          wrong indention

          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/2553#discussion_r80930670 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/Schema.scala — @@ -0,0 +1,89 @@ +/* + * 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.table + +import org.apache.flink.api.common.typeinfo.TypeInformation + +/** + * A Schema represents a Table's structure + */ +class Schema( + val columnNames: Array [String] , + val columnTypes: Array[TypeInformation [_] ]) { + + if (columnNames.length != columnTypes.length) { + throw new TableException( + "Number of column indexes and column names must be equal.") + } + + // check uniqueness of field names + if (columnNames.toSet.size != columnTypes.length) { + throw new TableException( + "Table column names must be unique.") + } + + val columnNameToIndex: Map [String, Int] = columnNames.zipWithIndex.toMap + + /** + * Returns all type informations as an array + */ + def getTypes: Array[TypeInformation [_] ] = columnTypes + + /** + * Returns the specified type information for the given column index + * — End diff – wrong indention
          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/2553#discussion_r80931239

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/Schema.scala —
          @@ -0,0 +1,89 @@
          +/*
          + * 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.table
          +
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +
          +/**
          + * A Schema represents a Table's structure
          + */
          +class Schema(
          + val columnNames: Array[String],
          + val columnTypes: Array[TypeInformation[_]]) {
          +
          + if (columnNames.length != columnTypes.length)

          { + throw new TableException( + "Number of column indexes and column names must be equal.") + }

          +
          + // check uniqueness of field names
          + if (columnNames.toSet.size != columnTypes.length)

          { + throw new TableException( + "Table column names must be unique.") + }

          +
          + val columnNameToIndex: Map[String, Int] = columnNames.zipWithIndex.toMap
          +
          + /**
          + * Returns all type informations as an array
          + */
          + def getTypes: Array[TypeInformation[_]] = columnTypes
          +
          + /**
          + * Returns the specified type information for the given column index
          + *
          + * @param columnIndex the index of the field
          + */
          + def getType(columnIndex: Int): TypeInformation[_] = columnTypes(columnIndex)
          +
          + /**
          + * Returns the specified type information for the given column name
          + *
          + * @param columnName the name of the field
          + */
          + def getType(columnName: String): TypeInformation[_] = {
          + if (columnNameToIndex.contains(columnName))

          { + columnTypes(columnNameToIndex(columnName)) + }

          else {
          + throw FieldNotFoundException(s"Table doesn't have the column : $columnName")
          — End diff –

          You could also return an `Option`. Then we don't need a new exception type.

          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/2553#discussion_r80931239 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/Schema.scala — @@ -0,0 +1,89 @@ +/* + * 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.table + +import org.apache.flink.api.common.typeinfo.TypeInformation + +/** + * A Schema represents a Table's structure + */ +class Schema( + val columnNames: Array [String] , + val columnTypes: Array[TypeInformation [_] ]) { + + if (columnNames.length != columnTypes.length) { + throw new TableException( + "Number of column indexes and column names must be equal.") + } + + // check uniqueness of field names + if (columnNames.toSet.size != columnTypes.length) { + throw new TableException( + "Table column names must be unique.") + } + + val columnNameToIndex: Map [String, Int] = columnNames.zipWithIndex.toMap + + /** + * Returns all type informations as an array + */ + def getTypes: Array[TypeInformation [_] ] = columnTypes + + /** + * Returns the specified type information for the given column index + * + * @param columnIndex the index of the field + */ + def getType(columnIndex: Int): TypeInformation [_] = columnTypes(columnIndex) + + /** + * Returns the specified type information for the given column name + * + * @param columnName the name of the field + */ + def getType(columnName: String): TypeInformation [_] = { + if (columnNameToIndex.contains(columnName)) { + columnTypes(columnNameToIndex(columnName)) + } else { + throw FieldNotFoundException(s"Table doesn't have the column : $columnName") — End diff – You could also return an `Option`. Then we don't need a new exception type.
          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/2553#discussion_r80931530

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/Schema.scala —
          @@ -0,0 +1,89 @@
          +/*
          + * 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.table
          +
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +
          +/**
          + * A Schema represents a Table's structure
          + */
          +class Schema(
          + val columnNames: Array[String],
          + val columnTypes: Array[TypeInformation[_]]) {
          +
          + if (columnNames.length != columnTypes.length)

          { + throw new TableException( + "Number of column indexes and column names must be equal.") + }

          +
          + // check uniqueness of field names
          + if (columnNames.toSet.size != columnTypes.length)

          { + throw new TableException( + "Table column names must be unique.") + }

          +
          + val columnNameToIndex: Map[String, Int] = columnNames.zipWithIndex.toMap
          +
          + /**
          + * Returns all type informations as an array
          + */
          + def getTypes: Array[TypeInformation[_]] = columnTypes
          +
          + /**
          + * Returns the specified type information for the given column index
          + *
          + * @param columnIndex the index of the field
          + */
          + def getType(columnIndex: Int): TypeInformation[_] = columnTypes(columnIndex)
          +
          + /**
          + * Returns the specified type information for the given column name
          + *
          + * @param columnName the name of the field
          + */
          + def getType(columnName: String): TypeInformation[_] = {
          + if (columnNameToIndex.contains(columnName))

          { + columnTypes(columnNameToIndex(columnName)) + }

          else

          { + throw FieldNotFoundException(s"Table doesn't have the column : $columnName") + }

          + }
          +
          + /**
          + * Returns all column names as an array.
          + */
          + def getColumnNames: Array[String] = columnNames
          +
          + /**
          + * Return the specified column name for the given column index.
          + *
          + * @param columnIndex the index of the field
          + */
          + def getColumnName(columnIndex: Int): String = columnNames(columnIndex)
          — End diff –

          No index check here. I would use a `Option` here.

          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/2553#discussion_r80931530 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/Schema.scala — @@ -0,0 +1,89 @@ +/* + * 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.table + +import org.apache.flink.api.common.typeinfo.TypeInformation + +/** + * A Schema represents a Table's structure + */ +class Schema( + val columnNames: Array [String] , + val columnTypes: Array[TypeInformation [_] ]) { + + if (columnNames.length != columnTypes.length) { + throw new TableException( + "Number of column indexes and column names must be equal.") + } + + // check uniqueness of field names + if (columnNames.toSet.size != columnTypes.length) { + throw new TableException( + "Table column names must be unique.") + } + + val columnNameToIndex: Map [String, Int] = columnNames.zipWithIndex.toMap + + /** + * Returns all type informations as an array + */ + def getTypes: Array[TypeInformation [_] ] = columnTypes + + /** + * Returns the specified type information for the given column index + * + * @param columnIndex the index of the field + */ + def getType(columnIndex: Int): TypeInformation [_] = columnTypes(columnIndex) + + /** + * Returns the specified type information for the given column name + * + * @param columnName the name of the field + */ + def getType(columnName: String): TypeInformation [_] = { + if (columnNameToIndex.contains(columnName)) { + columnTypes(columnNameToIndex(columnName)) + } else { + throw FieldNotFoundException(s"Table doesn't have the column : $columnName") + } + } + + /** + * Returns all column names as an array. + */ + def getColumnNames: Array [String] = columnNames + + /** + * Return the specified column name for the given column index. + * + * @param columnIndex the index of the field + */ + def getColumnName(columnIndex: Int): String = columnNames(columnIndex) — End diff – No index check here. I would use a `Option` here.
          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/2553#discussion_r80930699

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/Schema.scala —
          @@ -0,0 +1,89 @@
          +/*
          + * 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.table
          +
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +
          +/**
          + * A Schema represents a Table's structure
          + */
          +class Schema(
          + val columnNames: Array[String],
          + val columnTypes: Array[TypeInformation[_]]) {
          +
          + if (columnNames.length != columnTypes.length)

          { + throw new TableException( + "Number of column indexes and column names must be equal.") + }

          +
          + // check uniqueness of field names
          + if (columnNames.toSet.size != columnTypes.length)

          { + throw new TableException( + "Table column names must be unique.") + }

          +
          + val columnNameToIndex: Map[String, Int] = columnNames.zipWithIndex.toMap
          +
          + /**
          + * Returns all type informations as an array
          + */
          + def getTypes: Array[TypeInformation[_]] = columnTypes
          +
          + /**
          + * Returns the specified type information for the given column index
          + *
          + * @param columnIndex the index of the field
          + */
          + def getType(columnIndex: Int): TypeInformation[_] = columnTypes(columnIndex)
          +
          + /**
          + * Returns the specified type information for the given column name
          + *
          — End diff –

          wrong indention

          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/2553#discussion_r80930699 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/Schema.scala — @@ -0,0 +1,89 @@ +/* + * 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.table + +import org.apache.flink.api.common.typeinfo.TypeInformation + +/** + * A Schema represents a Table's structure + */ +class Schema( + val columnNames: Array [String] , + val columnTypes: Array[TypeInformation [_] ]) { + + if (columnNames.length != columnTypes.length) { + throw new TableException( + "Number of column indexes and column names must be equal.") + } + + // check uniqueness of field names + if (columnNames.toSet.size != columnTypes.length) { + throw new TableException( + "Table column names must be unique.") + } + + val columnNameToIndex: Map [String, Int] = columnNames.zipWithIndex.toMap + + /** + * Returns all type informations as an array + */ + def getTypes: Array[TypeInformation [_] ] = columnTypes + + /** + * Returns the specified type information for the given column index + * + * @param columnIndex the index of the field + */ + def getType(columnIndex: Int): TypeInformation [_] = columnTypes(columnIndex) + + /** + * Returns the specified type information for the given column name + * — End diff – wrong indention
          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/2553#discussion_r80931954

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/table.scala —
          @@ -66,6 +66,17 @@ class Table(
          def getRelNode: RelNode = logicalPlan.toRelNode(relBuilder)

          /**
          + * Returns the schema of this Table
          + */
          + def schema: Schema = new Schema(logicalPlan.output.map(_.name).toArray,
          — End diff –

          I would call this `getSchema` to be consistent.

          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/2553#discussion_r80931954 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/table.scala — @@ -66,6 +66,17 @@ class Table( def getRelNode: RelNode = logicalPlan.toRelNode(relBuilder) /** + * Returns the schema of this Table + */ + def schema: Schema = new Schema(logicalPlan.output.map(_.name).toArray, — End diff – I would call this `getSchema` to be consistent.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user wuchong commented on the issue:

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

          Hi @twalthr , I addressed the comments. Thank you for your suggestion. I add some tests that getting schema from tables in existing ITCases.

          Show
          githubbot ASF GitHub Bot added a comment - Github user wuchong commented on the issue: https://github.com/apache/flink/pull/2553 Hi @twalthr , I addressed the comments. Thank you for your suggestion. I add some tests that getting schema from tables in existing ITCases.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user twalthr commented on the issue:

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

          Thanks @wuchong. I think we can convert the modified ITCases to TableTestBase tests. I will try it and merge this.

          Show
          githubbot ASF GitHub Bot added a comment - Github user twalthr commented on the issue: https://github.com/apache/flink/pull/2553 Thanks @wuchong. I think we can convert the modified ITCases to TableTestBase tests. I will try it and merge this.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

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

          Show
          githubbot ASF GitHub Bot added a comment - Github user asfgit closed the pull request at: https://github.com/apache/flink/pull/2553
          Hide
          twalthr Timo Walther added a comment -

          Fixed in 1.3.0: 6ac579455115d1062cade8a0453417c9a6ae6a83

          Show
          twalthr Timo Walther added a comment - Fixed in 1.3.0: 6ac579455115d1062cade8a0453417c9a6ae6a83

            People

            • Assignee:
              jark Jark Wu
              Reporter:
              twalthr Timo Walther
            • Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development