You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by wuchong <gi...@git.apache.org> on 2016/09/27 09:08:45 UTC

[GitHub] flink pull request #2553: [FLINK-4686] [table] Add possibility to get column...

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 <wu...@alibaba-inc.com>
Date:   2016-09-27T09:05:27Z

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

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2553: [FLINK-4686] [table] Add possibility to get column...

Posted by twalthr <gi...@git.apache.org>.
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?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2553: [FLINK-4686] [table] Add possibility to get column...

Posted by twalthr <gi...@git.apache.org>.
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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2553: [FLINK-4686] [table] Add possibility to get column...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2553: [FLINK-4686] [table] Add possibility to get column...

Posted by twalthr <gi...@git.apache.org>.
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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2553: [FLINK-4686] [table] Add possibility to get column...

Posted by twalthr <gi...@git.apache.org>.
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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #2553: [FLINK-4686] [table] Add possibility to get column names

Posted by twalthr <gi...@git.apache.org>.
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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2553: [FLINK-4686] [table] Add possibility to get column...

Posted by twalthr <gi...@git.apache.org>.
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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2553: [FLINK-4686] [table] Add possibility to get column...

Posted by twalthr <gi...@git.apache.org>.
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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #2553: [FLINK-4686] [table] Add possibility to get column names

Posted by wuchong <gi...@git.apache.org>.
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. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2553: [FLINK-4686] [table] Add possibility to get column...

Posted by twalthr <gi...@git.apache.org>.
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`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2553: [FLINK-4686] [table] Add possibility to get column...

Posted by twalthr <gi...@git.apache.org>.
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?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2553: [FLINK-4686] [table] Add possibility to get column...

Posted by twalthr <gi...@git.apache.org>.
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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---