You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/05/25 06:58:51 UTC

[GitHub] [flink] JingsongLi opened a new pull request #12316: [FLINK-17896][hive] HiveCatalog can work with new table factory because of is_generic

JingsongLi opened a new pull request #12316:
URL: https://github.com/apache/flink/pull/12316


   
   ## What is the purpose of the change
   
   ```
   [ERROR] Could not execute SQL statement. Reason:
   org.apache.flink.table.api.ValidationException: Unsupported options found for connector 'print'.Unsupported options:is_genericSupported options:connector
   print-identifier
   property-version
   standard-error
   ```
   
   This's because HiveCatalog put is_generic property into generic tables, but the new factory will not delete it.
   
   ## Brief change log
   
   - Implement `HiveDynamicTableFactory` to remove `is_generic` flag before create table source&sink
   
   ## Verifying this change
   
   `HiveCatalogITCase.testNewTableFactory`
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): no
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: no
     - The serializers: no
     - The runtime per-record code paths (performance sensitive): no
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: no
     - The S3 file system connector: no
   
   ## Documentation
   
     - Does this pull request introduce a new feature? no


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] JingsongLi commented on a change in pull request #12316: [FLINK-17896][hive] HiveCatalog can not work with new table factory because of is_generic

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #12316:
URL: https://github.com/apache/flink/pull/12316#discussion_r429824642



##########
File path: flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveDynamicTableFactory.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.connectors.hive;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.flink.table.catalog.config.CatalogConfig.IS_GENERIC;
+
+/**
+ * A dynamic table factory implementation for Hive catalog. Now it only support generic tables.
+ * Hive tables should be resolved by {@link HiveTableFactory}.
+ */
+public class HiveDynamicTableFactory implements
+		DynamicTableSourceFactory,
+		DynamicTableSinkFactory {
+
+	@Override
+	public String factoryIdentifier() {
+		throw new UnsupportedOperationException("Hive factory is only work for catalog.");
+	}
+
+	@Override
+	public Set<ConfigOption<?>> requiredOptions() {
+		throw new UnsupportedOperationException("Hive factory is only work for catalog.");
+	}
+
+	@Override
+	public Set<ConfigOption<?>> optionalOptions() {
+		throw new UnsupportedOperationException("Hive factory is only work for catalog.");
+	}
+
+	private static CatalogTable removeIsGenericFlag(CatalogTable table) {
+		Map<String, String> newOptions = new HashMap<>(table.getOptions());
+		boolean isGeneric = Boolean.parseBoolean(newOptions.remove(IS_GENERIC));

Review comment:
       JIRA: https://issues.apache.org/jira/browse/FLINK-17922




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #12316: [FLINK-17896][hive] HiveCatalog can work with new table factory because of is_generic

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #12316:
URL: https://github.com/apache/flink/pull/12316#issuecomment-633419287


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2dfb5f41be178ba4f03d8f2627894d3cbc2ea519",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2dfb5f41be178ba4f03d8f2627894d3cbc2ea519",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2dfb5f41be178ba4f03d8f2627894d3cbc2ea519 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] sjwiesman commented on pull request #12316: [FLINK-17896][hive] HiveCatalog can not work with new table factory because of is_generic

Posted by GitBox <gi...@apache.org>.
sjwiesman commented on pull request #12316:
URL: https://github.com/apache/flink/pull/12316#issuecomment-633783610


   Cherry picked onto release-1.11 and deployed successfully. +1


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] twalthr commented on a change in pull request #12316: [FLINK-17896][hive] HiveCatalog can work with new table factory because of is_generic

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #12316:
URL: https://github.com/apache/flink/pull/12316#discussion_r429769415



##########
File path: flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveDynamicTableFactory.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.connectors.hive;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.flink.table.catalog.config.CatalogConfig.IS_GENERIC;
+
+/**
+ * A dynamic table factory implementation for Hive catalog. Now it only support generic tables.
+ * Hive tables should be resolved by {@link HiveTableFactory}.
+ */
+public class HiveDynamicTableFactory implements
+		DynamicTableSourceFactory,
+		DynamicTableSinkFactory {
+
+	@Override
+	public String factoryIdentifier() {
+		throw new UnsupportedOperationException("Hive factory is only work for catalog.");
+	}
+
+	@Override
+	public Set<ConfigOption<?>> requiredOptions() {
+		throw new UnsupportedOperationException("Hive factory is only work for catalog.");
+	}
+
+	@Override
+	public Set<ConfigOption<?>> optionalOptions() {
+		throw new UnsupportedOperationException("Hive factory is only work for catalog.");
+	}
+
+	private static CatalogTable removeIsGenericFlag(CatalogTable table) {
+		Map<String, String> newOptions = new HashMap<>(table.getOptions());
+		boolean isGeneric = Boolean.parseBoolean(newOptions.remove(IS_GENERIC));

Review comment:
       Shouldn't this issue be solved earlier? Actually, the Hive CatalogTable should already remove this property and return it via the `isGeneric` method.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12316: [FLINK-17896][hive] HiveCatalog can not work with new table factory because of is_generic

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12316:
URL: https://github.com/apache/flink/pull/12316#issuecomment-633419287


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2dfb5f41be178ba4f03d8f2627894d3cbc2ea519",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=2118",
       "triggerID" : "2dfb5f41be178ba4f03d8f2627894d3cbc2ea519",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2dfb5f41be178ba4f03d8f2627894d3cbc2ea519 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=2118) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12316: [FLINK-17896][hive] HiveCatalog can work with new table factory because of is_generic

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12316:
URL: https://github.com/apache/flink/pull/12316#issuecomment-633419287


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2dfb5f41be178ba4f03d8f2627894d3cbc2ea519",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=2118",
       "triggerID" : "2dfb5f41be178ba4f03d8f2627894d3cbc2ea519",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2dfb5f41be178ba4f03d8f2627894d3cbc2ea519 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=2118) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] JingsongLi commented on a change in pull request #12316: [FLINK-17896][hive] HiveCatalog can not work with new table factory because of is_generic

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #12316:
URL: https://github.com/apache/flink/pull/12316#discussion_r429801254



##########
File path: flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveDynamicTableFactory.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.connectors.hive;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.flink.table.catalog.config.CatalogConfig.IS_GENERIC;
+
+/**
+ * A dynamic table factory implementation for Hive catalog. Now it only support generic tables.
+ * Hive tables should be resolved by {@link HiveTableFactory}.
+ */
+public class HiveDynamicTableFactory implements
+		DynamicTableSourceFactory,
+		DynamicTableSinkFactory {
+
+	@Override
+	public String factoryIdentifier() {
+		throw new UnsupportedOperationException("Hive factory is only work for catalog.");
+	}
+
+	@Override
+	public Set<ConfigOption<?>> requiredOptions() {
+		throw new UnsupportedOperationException("Hive factory is only work for catalog.");
+	}
+
+	@Override
+	public Set<ConfigOption<?>> optionalOptions() {
+		throw new UnsupportedOperationException("Hive factory is only work for catalog.");
+	}
+
+	private static CatalogTable removeIsGenericFlag(CatalogTable table) {
+		Map<String, String> newOptions = new HashMap<>(table.getOptions());
+		boolean isGeneric = Boolean.parseBoolean(newOptions.remove(IS_GENERIC));

Review comment:
       Yes, this is our plan. Whether it's `createTable` or `getTable` of `HiveCatalog`, there should be two types of objects that are consistent: `HiveCatalogTable` and Flink `CatalogTableImpl`.
   But `HiveCatalogTable` should implement `toProperties` too, it can add a `"connector"="hive"`.
   And I think it is not a small change, we can finish this refactoring in 1.12.
   
   Now the underlying rules are:
   - HiveCatalog.createTable , hive table has false of `isGeneric` flag, generic table has no flag.
   - HiveCatalog.getTable , generic table has true of `isGeneric` flag, hive table has no flag.
   It is very difficult to understand.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] JingsongLi commented on pull request #12316: [FLINK-17896][hive] HiveCatalog can not work with new table factory because of is_generic

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on pull request #12316:
URL: https://github.com/apache/flink/pull/12316#issuecomment-633833725


   Thanks all for your review~


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] JingsongLi closed pull request #12316: [FLINK-17896][hive] HiveCatalog can not work with new table factory because of is_generic

Posted by GitBox <gi...@apache.org>.
JingsongLi closed pull request #12316:
URL: https://github.com/apache/flink/pull/12316


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12316: [FLINK-17896][hive] HiveCatalog can not work with new table factory because of is_generic

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12316:
URL: https://github.com/apache/flink/pull/12316#issuecomment-633413192


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit 021c972a5d802fb8e39b7d6bb4e0f1e647ce6992 (Fri Oct 16 10:55:16 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] twalthr commented on a change in pull request #12316: [FLINK-17896][hive] HiveCatalog can not work with new table factory because of is_generic

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #12316:
URL: https://github.com/apache/flink/pull/12316#discussion_r429795154



##########
File path: flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveDynamicTableFactory.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.connectors.hive;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.flink.table.catalog.config.CatalogConfig.IS_GENERIC;
+
+/**
+ * A dynamic table factory implementation for Hive catalog. Now it only support generic tables.
+ * Hive tables should be resolved by {@link HiveTableFactory}.
+ */
+public class HiveDynamicTableFactory implements
+		DynamicTableSourceFactory,
+		DynamicTableSinkFactory {
+
+	@Override
+	public String factoryIdentifier() {
+		throw new UnsupportedOperationException("Hive factory is only work for catalog.");
+	}
+
+	@Override
+	public Set<ConfigOption<?>> requiredOptions() {
+		throw new UnsupportedOperationException("Hive factory is only work for catalog.");
+	}
+
+	@Override
+	public Set<ConfigOption<?>> optionalOptions() {
+		throw new UnsupportedOperationException("Hive factory is only work for catalog.");
+	}
+
+	private static CatalogTable removeIsGenericFlag(CatalogTable table) {
+		Map<String, String> newOptions = new HashMap<>(table.getOptions());
+		boolean isGeneric = Boolean.parseBoolean(newOptions.remove(IS_GENERIC));

Review comment:
       From a logical perspective, the `HiveCatalog` should deal with this additional property while storing and retrieving a `CatalogTable`. The `isGeneric` should not travel through the stack of regular Flink connector options. It is an internal property from the `HiveCatalog` through a custom `HiveCatalogTable` to a custom `HiveFactory`. It should be stored as a member variable in a `HiveCatalogTable`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #12316: [FLINK-17896][hive] HiveCatalog can work with new table factory because of is_generic

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #12316:
URL: https://github.com/apache/flink/pull/12316#issuecomment-633413192


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit 2dfb5f41be178ba4f03d8f2627894d3cbc2ea519 (Mon May 25 07:01:25 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] JingsongLi commented on a change in pull request #12316: [FLINK-17896][hive] HiveCatalog can work with new table factory because of is_generic

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #12316:
URL: https://github.com/apache/flink/pull/12316#discussion_r429772225



##########
File path: flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveDynamicTableFactory.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.connectors.hive;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.flink.table.catalog.config.CatalogConfig.IS_GENERIC;
+
+/**
+ * A dynamic table factory implementation for Hive catalog. Now it only support generic tables.
+ * Hive tables should be resolved by {@link HiveTableFactory}.
+ */
+public class HiveDynamicTableFactory implements
+		DynamicTableSourceFactory,
+		DynamicTableSinkFactory {
+
+	@Override
+	public String factoryIdentifier() {
+		throw new UnsupportedOperationException("Hive factory is only work for catalog.");
+	}
+
+	@Override
+	public Set<ConfigOption<?>> requiredOptions() {
+		throw new UnsupportedOperationException("Hive factory is only work for catalog.");
+	}
+
+	@Override
+	public Set<ConfigOption<?>> optionalOptions() {
+		throw new UnsupportedOperationException("Hive factory is only work for catalog.");
+	}
+
+	private static CatalogTable removeIsGenericFlag(CatalogTable table) {
+		Map<String, String> newOptions = new HashMap<>(table.getOptions());
+		boolean isGeneric = Boolean.parseBoolean(newOptions.remove(IS_GENERIC));

Review comment:
       You mean when take table out of the catalog? But `HiveFactory` can not know whether it's a generic table or a hive table.
   Now the `CatalogTable` of hive table is not a standard Flink `CatalogTable`, it dose not have legacy "connector.type" or "connector" property. But I think we can make it Flink style.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12316: [FLINK-17896][hive] HiveCatalog can not work with new table factory because of is_generic

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12316:
URL: https://github.com/apache/flink/pull/12316#issuecomment-633419287






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org