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/11 09:27:17 UTC

[GitHub] [flink] docete opened a new pull request #12076: [FLINK-15950][table] Remove registration of TableSource/TableSink in Table Env

docete opened a new pull request #12076:
URL: https://github.com/apache/flink/pull/12076


   ## What is the purpose of the change
   
   This PR is a workaround to remove all registration of TableSource/TableSink in Table Env. We introduces utility methods to register TableSource/TableSink since descriptor APIs is not enough to fulfill some testing scenarios.
   
   ## Brief change log
   
   - b4d44da introduces utility methods to register table source and sink
   - 809c003 - c0a733e removes all registration of TableSource/TableSink in Table Env
   - b3cc76c  removes deprecated registerTableSource and registerTableSink APIs in TableEnvironment and related classes
   
   ## Verifying this change
   
   This change is already covered by existing tests.
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (yes / **no**)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / **no**)
     - The serializers: (yes / **no** / don't know)
     - The runtime per-record code paths (performance sensitive): (yes / **no** / don't know)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes / **no** / don't know)
     - The S3 file system connector: (yes / **no** / don't know)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (yes / **no**)
     - If yes, how is the feature documented? (not applicable / docs / **JavaDocs** / not documented)
   


----------------------------------------------------------------
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 #12076: [FLINK-15950][table] Remove registration of TableSource/TableSink in Table Env

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "525efab0792cd8b81842e5964454ee373f34d029",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=974",
       "triggerID" : "525efab0792cd8b81842e5964454ee373f34d029",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6a96fc1c177316b66544bde05677a7037e94fff",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1295",
       "triggerID" : "f6a96fc1c177316b66544bde05677a7037e94fff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d27eed53d8c72a0533d681ca1ac354499606a395",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d27eed53d8c72a0533d681ca1ac354499606a395",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f6a96fc1c177316b66544bde05677a7037e94fff Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1295) 
   * d27eed53d8c72a0533d681ca1ac354499606a395 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] flinkbot commented on pull request #12076: [FLINK-15950][table] Remove registration of TableSource/TableSink in Table Env

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "525efab0792cd8b81842e5964454ee373f34d029",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "525efab0792cd8b81842e5964454ee373f34d029",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 525efab0792cd8b81842e5964454ee373f34d029 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] KurtYoung commented on a change in pull request #12076: [FLINK-15950][table] Remove registration of TableSource/TableSink in Table Env

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



##########
File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentInternal.java
##########
@@ -79,4 +81,31 @@
 	 */
 	String explainInternal(List<Operation> operations, ExplainDetail... extraDetails);
 
+
+	/**
+	 * Registers an external {@link TableSource} in this {@link TableEnvironment}'s catalog.
+	 * Registered tables can be referenced in SQL queries.
+	 *
+	 * <p>Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will
+	 * be inaccessible in the current session. To make the permanent object available again one can drop the
+	 * corresponding temporary object.
+	 *
+	 * @param name        The name under which the {@link TableSource} is registered.
+	 * @param tableSource The {@link TableSource} to register.
+	 */
+	void registerTableSource(String name, TableSource<?> tableSource);

Review comment:
       Be consistent with other internal methods, add a `Internal` suffix




----------------------------------------------------------------
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 #12076: [FLINK-15950][table] Remove registration of TableSource/TableSink in Table Env

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "525efab0792cd8b81842e5964454ee373f34d029",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=974",
       "triggerID" : "525efab0792cd8b81842e5964454ee373f34d029",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6a96fc1c177316b66544bde05677a7037e94fff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1295",
       "triggerID" : "f6a96fc1c177316b66544bde05677a7037e94fff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d27eed53d8c72a0533d681ca1ac354499606a395",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d27eed53d8c72a0533d681ca1ac354499606a395",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c74ce8a4dcfd150fe7619aef8b35be0091520a2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1430",
       "triggerID" : "4c74ce8a4dcfd150fe7619aef8b35be0091520a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "848b68881f0994f98a20a5c9e85bc4d8fb7a1781",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1444",
       "triggerID" : "848b68881f0994f98a20a5c9e85bc4d8fb7a1781",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d27eed53d8c72a0533d681ca1ac354499606a395 UNKNOWN
   * 4c74ce8a4dcfd150fe7619aef8b35be0091520a2 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1430) 
   * 848b68881f0994f98a20a5c9e85bc4d8fb7a1781 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1444) 
   
   <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] KurtYoung commented on a change in pull request #12076: [FLINK-15950][table] Remove registration of TableSource/TableSink in Table Env

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



##########
File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/utils/TableEnvUtils.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.table.utils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.api.internal.TableEnvironmentInternal;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogManager;
+import org.apache.flink.table.catalog.ConnectorCatalogTable;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.catalog.UnresolvedIdentifier;
+import org.apache.flink.table.sinks.TableSink;
+import org.apache.flink.table.sources.TableSource;
+
+import java.util.Optional;
+
+/**
+ * Utility methods to register table source/sink as a Table in {@link TableEnvironment}'s catalog.
+ */
+@Internal
+public class TableEnvUtils {
+	/**
+	 * Register a table source as a Table under the given name in this
+	 * {@link TableEnvironment}'s catalog.
+	 */
+	public static void registerTableSource(
+			TableEnvironment tEnv,
+			String name,
+			TableSource<?> tableSource,
+			boolean isBatch) {
+		if (tEnv instanceof TableEnvironmentInternal) {

Review comment:
       check and throw exception first, and do other normal logic without this big brace

##########
File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/utils/TableEnvUtils.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.table.utils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.api.internal.TableEnvironmentInternal;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogManager;
+import org.apache.flink.table.catalog.ConnectorCatalogTable;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.catalog.UnresolvedIdentifier;
+import org.apache.flink.table.sinks.TableSink;
+import org.apache.flink.table.sources.TableSource;
+
+import java.util.Optional;
+
+/**
+ * Utility methods to register table source/sink as a Table in {@link TableEnvironment}'s catalog.
+ */
+@Internal
+public class TableEnvUtils {
+	/**
+	 * Register a table source as a Table under the given name in this
+	 * {@link TableEnvironment}'s catalog.
+	 */
+	public static void registerTableSource(

Review comment:
       registerTableSourceInternal

##########
File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/utils/TableEnvUtils.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.table.utils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.api.internal.TableEnvironmentInternal;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogManager;
+import org.apache.flink.table.catalog.ConnectorCatalogTable;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.catalog.UnresolvedIdentifier;
+import org.apache.flink.table.sinks.TableSink;
+import org.apache.flink.table.sources.TableSource;
+
+import java.util.Optional;
+
+/**
+ * Utility methods to register table source/sink as a Table in {@link TableEnvironment}'s catalog.
+ */
+@Internal
+public class TableEnvUtils {

Review comment:
       How about adding these two methods to `TableEnvironmentInternal`?

##########
File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/utils/TableEnvUtils.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.table.utils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.api.internal.TableEnvironmentInternal;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogManager;
+import org.apache.flink.table.catalog.ConnectorCatalogTable;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.catalog.UnresolvedIdentifier;
+import org.apache.flink.table.sinks.TableSink;
+import org.apache.flink.table.sources.TableSource;
+
+import java.util.Optional;
+
+/**
+ * Utility methods to register table source/sink as a Table in {@link TableEnvironment}'s catalog.
+ */
+@Internal
+public class TableEnvUtils {
+	/**
+	 * Register a table source as a Table under the given name in this
+	 * {@link TableEnvironment}'s catalog.
+	 */
+	public static void registerTableSource(
+			TableEnvironment tEnv,
+			String name,
+			TableSource<?> tableSource,
+			boolean isBatch) {
+		if (tEnv instanceof TableEnvironmentInternal) {
+			CatalogManager catalogManager = ((TableEnvironmentInternal) tEnv).getCatalogManager();

Review comment:
       You can keep all the logic of original `registerTableSourceInternal`, e.g. it will call `validateTableSource` when register

##########
File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/utils/TableEnvUtils.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.table.utils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.api.internal.TableEnvironmentInternal;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogManager;
+import org.apache.flink.table.catalog.ConnectorCatalogTable;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.catalog.UnresolvedIdentifier;
+import org.apache.flink.table.sinks.TableSink;
+import org.apache.flink.table.sources.TableSource;
+
+import java.util.Optional;
+
+/**
+ * Utility methods to register table source/sink as a Table in {@link TableEnvironment}'s catalog.
+ */
+@Internal
+public class TableEnvUtils {
+	/**
+	 * Register a table source as a Table under the given name in this
+	 * {@link TableEnvironment}'s catalog.
+	 */
+	public static void registerTableSource(
+			TableEnvironment tEnv,
+			String name,
+			TableSource<?> tableSource,
+			boolean isBatch) {
+		if (tEnv instanceof TableEnvironmentInternal) {
+			CatalogManager catalogManager = ((TableEnvironmentInternal) tEnv).getCatalogManager();
+			ObjectIdentifier objectIdentifier = catalogManager.qualifyIdentifier(UnresolvedIdentifier.of(name));
+			Optional<CatalogBaseTable> table = getTemporaryTable(catalogManager, objectIdentifier);
+
+			if (table.isPresent()) {
+				if (table.get() instanceof ConnectorCatalogTable<?, ?>) {
+					ConnectorCatalogTable<?, ?> sourceSinkTable = (ConnectorCatalogTable<?, ?>) table.get();
+					if (sourceSinkTable.getTableSource().isPresent()) {
+						throw new ValidationException(String.format(
+							"Table '%s' already exists. Please choose a different name.", name));
+					} else {
+						ConnectorCatalogTable sourceAndSink = ConnectorCatalogTable.sourceAndSink(
+							tableSource,
+							sourceSinkTable.getTableSink().get(),
+							isBatch);
+						catalogManager.dropTemporaryTable(objectIdentifier, false);
+						catalogManager.createTemporaryTable(sourceAndSink, objectIdentifier, false);
+					}
+				} else {
+					throw new ValidationException(String.format(
+						"Table '%s' already exists. Please choose a different name.", name));
+				}
+			} else {
+				ConnectorCatalogTable source = ConnectorCatalogTable.source(tableSource, isBatch);
+				catalogManager.createTemporaryTable(source, objectIdentifier, false);
+			}
+		} else {
+			throw new ValidationException(String.format(
+				"This is a workaround to remove registration of TableSource in Table Env. We " +
+					"need TableEnvironmentInternal#getCatalogManager to register a table source " +
+					"as a Table under the given name '%s' in this TableEnvironment's catalog.",
+				name));
+		}
+	}
+
+	/**
+	 * Register a table sink as a Table under the given name in this
+	 * {@link TableEnvironment}'s catalog.
+	 */
+	public static void registerTableSink(

Review comment:
       registerTableSinkInternal




----------------------------------------------------------------
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] docete commented on a change in pull request #12076: [FLINK-15950][table] Remove registration of TableSource/TableSink in Table Env

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



##########
File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/utils/TableEnvUtils.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.table.utils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.api.internal.TableEnvironmentInternal;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogManager;
+import org.apache.flink.table.catalog.ConnectorCatalogTable;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.catalog.UnresolvedIdentifier;
+import org.apache.flink.table.sinks.TableSink;
+import org.apache.flink.table.sources.TableSource;
+
+import java.util.Optional;
+
+/**
+ * Utility methods to register table source/sink as a Table in {@link TableEnvironment}'s catalog.
+ */
+@Internal
+public class TableEnvUtils {

Review comment:
       OK. I will add `registerTableSource` and `registerTableSink` to `TableEnvironmentInternal `.
   




----------------------------------------------------------------
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] docete commented on pull request #12076: [FLINK-15950][table] Remove registration of TableSource/TableSink in Table Env

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


   @flinkbot run azure


----------------------------------------------------------------
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 #12076: [FLINK-15950][table] Remove registration of TableSource/TableSink in Table Env

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "525efab0792cd8b81842e5964454ee373f34d029",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=974",
       "triggerID" : "525efab0792cd8b81842e5964454ee373f34d029",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6a96fc1c177316b66544bde05677a7037e94fff",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1295",
       "triggerID" : "f6a96fc1c177316b66544bde05677a7037e94fff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d27eed53d8c72a0533d681ca1ac354499606a395",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d27eed53d8c72a0533d681ca1ac354499606a395",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c74ce8a4dcfd150fe7619aef8b35be0091520a2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1430",
       "triggerID" : "4c74ce8a4dcfd150fe7619aef8b35be0091520a2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f6a96fc1c177316b66544bde05677a7037e94fff Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1295) 
   * d27eed53d8c72a0533d681ca1ac354499606a395 UNKNOWN
   * 4c74ce8a4dcfd150fe7619aef8b35be0091520a2 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1430) 
   
   <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 #12076: [FLINK-17748][table] Remove registration of TableSource/TableSink in Table Env

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


   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 848b68881f0994f98a20a5c9e85bc4d8fb7a1781 (Fri Oct 16 10:31:28 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] flinkbot edited a comment on pull request #12076: [FLINK-15950][table] Remove registration of TableSource/TableSink in Table Env

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "525efab0792cd8b81842e5964454ee373f34d029",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=974",
       "triggerID" : "525efab0792cd8b81842e5964454ee373f34d029",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6a96fc1c177316b66544bde05677a7037e94fff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1295",
       "triggerID" : "f6a96fc1c177316b66544bde05677a7037e94fff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d27eed53d8c72a0533d681ca1ac354499606a395",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d27eed53d8c72a0533d681ca1ac354499606a395",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c74ce8a4dcfd150fe7619aef8b35be0091520a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1430",
       "triggerID" : "4c74ce8a4dcfd150fe7619aef8b35be0091520a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "848b68881f0994f98a20a5c9e85bc4d8fb7a1781",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1444",
       "triggerID" : "848b68881f0994f98a20a5c9e85bc4d8fb7a1781",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d27eed53d8c72a0533d681ca1ac354499606a395 UNKNOWN
   * 848b68881f0994f98a20a5c9e85bc4d8fb7a1781 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1444) 
   
   <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 #12076: [FLINK-15950][table] Remove registration of TableSource/TableSink in Table Env

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "525efab0792cd8b81842e5964454ee373f34d029",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=974",
       "triggerID" : "525efab0792cd8b81842e5964454ee373f34d029",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6a96fc1c177316b66544bde05677a7037e94fff",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1295",
       "triggerID" : "f6a96fc1c177316b66544bde05677a7037e94fff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d27eed53d8c72a0533d681ca1ac354499606a395",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d27eed53d8c72a0533d681ca1ac354499606a395",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c74ce8a4dcfd150fe7619aef8b35be0091520a2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1430",
       "triggerID" : "4c74ce8a4dcfd150fe7619aef8b35be0091520a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "848b68881f0994f98a20a5c9e85bc4d8fb7a1781",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1444",
       "triggerID" : "848b68881f0994f98a20a5c9e85bc4d8fb7a1781",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f6a96fc1c177316b66544bde05677a7037e94fff Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1295) 
   * d27eed53d8c72a0533d681ca1ac354499606a395 UNKNOWN
   * 4c74ce8a4dcfd150fe7619aef8b35be0091520a2 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1430) 
   * 848b68881f0994f98a20a5c9e85bc4d8fb7a1781 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1444) 
   
   <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] docete commented on pull request #12076: [FLINK-15950][table] Remove registration of TableSource/TableSink in Table Env

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


   @KurtYoung Could you have a look at this if you have time?


----------------------------------------------------------------
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 #12076: [FLINK-15950][table] Remove registration of TableSource/TableSink in Table Env

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "525efab0792cd8b81842e5964454ee373f34d029",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=974",
       "triggerID" : "525efab0792cd8b81842e5964454ee373f34d029",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6a96fc1c177316b66544bde05677a7037e94fff",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1295",
       "triggerID" : "f6a96fc1c177316b66544bde05677a7037e94fff",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f6a96fc1c177316b66544bde05677a7037e94fff Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1295) 
   
   <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 commented on pull request #12076: [FLINK-15950][table] Remove registration of TableSource/TableSink in Table Env

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


   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 b3cc76c11c4feb07ccd2eda1850b23550597d61f (Mon May 11 09:32:59 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
    * **This pull request references an unassigned [Jira ticket](https://issues.apache.org/jira/browse/FLINK-15950).** According to the [code contribution guide](https://flink.apache.org/contributing/contribute-code.html), tickets need to be assigned before starting with the implementation work.
   
   
   <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] flinkbot edited a comment on pull request #12076: [FLINK-15950][table] Remove registration of TableSource/TableSink in Table Env

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "525efab0792cd8b81842e5964454ee373f34d029",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=974",
       "triggerID" : "525efab0792cd8b81842e5964454ee373f34d029",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6a96fc1c177316b66544bde05677a7037e94fff",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f6a96fc1c177316b66544bde05677a7037e94fff",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 525efab0792cd8b81842e5964454ee373f34d029 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=974) 
   * f6a96fc1c177316b66544bde05677a7037e94fff 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] flinkbot edited a comment on pull request #12076: [FLINK-15950][table] Remove registration of TableSource/TableSink in Table Env

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "525efab0792cd8b81842e5964454ee373f34d029",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=974",
       "triggerID" : "525efab0792cd8b81842e5964454ee373f34d029",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6a96fc1c177316b66544bde05677a7037e94fff",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1295",
       "triggerID" : "f6a96fc1c177316b66544bde05677a7037e94fff",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 525efab0792cd8b81842e5964454ee373f34d029 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=974) 
   * f6a96fc1c177316b66544bde05677a7037e94fff Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1295) 
   
   <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 #12076: [FLINK-15950][table] Remove registration of TableSource/TableSink in Table Env

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "525efab0792cd8b81842e5964454ee373f34d029",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=974",
       "triggerID" : "525efab0792cd8b81842e5964454ee373f34d029",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6a96fc1c177316b66544bde05677a7037e94fff",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1295",
       "triggerID" : "f6a96fc1c177316b66544bde05677a7037e94fff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d27eed53d8c72a0533d681ca1ac354499606a395",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d27eed53d8c72a0533d681ca1ac354499606a395",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c74ce8a4dcfd150fe7619aef8b35be0091520a2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4c74ce8a4dcfd150fe7619aef8b35be0091520a2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f6a96fc1c177316b66544bde05677a7037e94fff Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1295) 
   * d27eed53d8c72a0533d681ca1ac354499606a395 UNKNOWN
   * 4c74ce8a4dcfd150fe7619aef8b35be0091520a2 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] KurtYoung commented on pull request #12076: [FLINK-17748][table] Remove registration of TableSource/TableSink in Table Env

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


   I fixed the compilation error and python tests, AZP: https://dev.azure.com/ykt836/Flink/_build/results?buildId=7&view=results


----------------------------------------------------------------
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 #12076: [FLINK-15950][table] Remove registration of TableSource/TableSink in Table Env

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "525efab0792cd8b81842e5964454ee373f34d029",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=974",
       "triggerID" : "525efab0792cd8b81842e5964454ee373f34d029",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f6a96fc1c177316b66544bde05677a7037e94fff",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1295",
       "triggerID" : "f6a96fc1c177316b66544bde05677a7037e94fff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d27eed53d8c72a0533d681ca1ac354499606a395",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d27eed53d8c72a0533d681ca1ac354499606a395",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c74ce8a4dcfd150fe7619aef8b35be0091520a2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1430",
       "triggerID" : "4c74ce8a4dcfd150fe7619aef8b35be0091520a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "848b68881f0994f98a20a5c9e85bc4d8fb7a1781",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "848b68881f0994f98a20a5c9e85bc4d8fb7a1781",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f6a96fc1c177316b66544bde05677a7037e94fff Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1295) 
   * d27eed53d8c72a0533d681ca1ac354499606a395 UNKNOWN
   * 4c74ce8a4dcfd150fe7619aef8b35be0091520a2 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1430) 
   * 848b68881f0994f98a20a5c9e85bc4d8fb7a1781 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] KurtYoung closed pull request #12076: [FLINK-17748][table] Remove registration of TableSource/TableSink in Table Env

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


   


----------------------------------------------------------------
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 #12076: [FLINK-15950][table] Remove registration of TableSource/TableSink in Table Env

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "525efab0792cd8b81842e5964454ee373f34d029",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=974",
       "triggerID" : "525efab0792cd8b81842e5964454ee373f34d029",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 525efab0792cd8b81842e5964454ee373f34d029 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=974) 
   
   <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