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/12/22 13:24:43 UTC

[GitHub] [flink] SteNicholas opened a new pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

SteNicholas opened a new pull request #14464:
URL: https://github.com/apache/flink/pull/14464


   ## What is the purpose of the change
   
   *Currently FLIP-107 supports reading meta from the Debezium format. According to FLIP-107, metadata should support to be exposed for the Canal JSON format.*
   
   ## Brief change log
   
     - *Let `CanalJsonDeserializationSchema` access and convert those additional fields to metadata columns.*
   
   ## Verifying this change
   
     - *`CanalJsonSerDeSchemaTest` adds `testDeserializationWithMetadata` to whether deserialization of `CanalJsonDeserializationSchema` could read metadata for the Canal JSON format.*
   
   ## 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, Kubernetes/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] wangfeigithub edited a comment on pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   
   ![image](https://user-images.githubusercontent.com/29723968/103872018-c44e8300-5108-11eb-86d2-518cdf87d70d.png)
   
   ![image](https://user-images.githubusercontent.com/29723968/103872094-e0eabb00-5108-11eb-8987-71ebed05732a.png)
   
   
   
   ![image](https://user-images.githubusercontent.com/29723968/103871920-a1bc6a00-5108-11eb-931f-3977272ddea3.png)
   


----------------------------------------------------------------
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] SteNicholas commented on pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   @wuchong I have kept code spotless with Maven for conflicts resolution and modified the document. Please help to review again.


----------------------------------------------------------------
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] wuchong commented on a change in pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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



##########
File path: docs/dev/table/connectors/formats/canal.md
##########
@@ -142,6 +142,79 @@ SELECT * FROM topic_products;
 </div>
 </div>
 
+Available Metadata
+------------------
+
+The following format metadata can be exposed as read-only (`VIRTUAL`) columns in a table definition.
+
+<span class="label label-danger">Attention</span> Format metadata fields are only available if the
+corresponding connector forwards format metadata. Currently, only the Kafka connector is able to expose
+metadata fields for its value format.
+
+<table class="table table-bordered">
+    <thead>
+    <tr>
+      <th class="text-left" style="width: 25%">Key</th>
+      <th class="text-center" style="width: 40%">Data Type</th>
+      <th class="text-center" style="width: 40%">Description</th>
+    </tr>
+    </thead>
+    <tbody>
+    <tr>
+      <td><code>database</code></td>
+      <td><code>STRING NULL</code></td>
+      <td>The originating database. Corresponds to the <code>database</code> field in the
+      Canal record if available.</td>
+    </tr>
+    <tr>
+      <td><code>table</code></td>
+      <td><code>STRING NULL</code></td>
+      <td>The originating database table. Corresponds to the <code>table</code> field in the
+      Canal record if available.</td>
+    </tr>
+    <tr>
+      <td><code>sql-type</code></td>
+      <td><code>MAP&lt;STRING, INT&gt; NULL</code></td>
+      <td>Map of various sql types. Corresponds to the <code>sqlType</code> field in the 
+      Canal record if available.</td>
+    </tr>
+    <tr>
+      <td><code>pk-names</code></td>
+      <td><code>ARRAY&lt;STRING&gt; NULL</code></td>
+      <td>Array of primary key names. Corresponds to the <code>pkNames</code> field in the 
+      Canal record if available.</td>
+    </tr>
+    <tr>
+      <td><code>ingestion-timestamp</code></td>
+      <td><code>TIMESTAMP(3) WITH LOCAL TIME ZONE NULL</code></td>
+      <td>The timestamp at which the connector processed the event. Corresponds to the <code>ts</code>
+      field in the Canal record.</td>
+    </tr>
+    </tbody>
+</table>
+
+The following example shows how to access Canal metadata fields in Kafka:
+
+<div class="codetabs" markdown="1">
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+CREATE TABLE KafkaTable (
+  `origin_database` STRING METADATA FROM 'value.database' VIRTUAL,
+  `origin_table` STRING METADATA FROM 'value.table' VIRTUAL,

Review comment:
       Could you list all the metadata columns? I think that would be helpful, esp. for the complex types. 

##########
File path: docs/dev/table/connectors/formats/canal.md
##########
@@ -142,6 +142,79 @@ SELECT * FROM topic_products;
 </div>
 </div>
 
+Available Metadata
+------------------
+
+The following format metadata can be exposed as read-only (`VIRTUAL`) columns in a table definition.
+
+<span class="label label-danger">Attention</span> Format metadata fields are only available if the
+corresponding connector forwards format metadata. Currently, only the Kafka connector is able to expose
+metadata fields for its value format.
+
+<table class="table table-bordered">
+    <thead>
+    <tr>
+      <th class="text-left" style="width: 25%">Key</th>
+      <th class="text-center" style="width: 40%">Data Type</th>
+      <th class="text-center" style="width: 40%">Description</th>
+    </tr>
+    </thead>
+    <tbody>
+    <tr>
+      <td><code>database</code></td>
+      <td><code>STRING NULL</code></td>
+      <td>The originating database. Corresponds to the <code>database</code> field in the
+      Canal record if available.</td>
+    </tr>
+    <tr>
+      <td><code>table</code></td>
+      <td><code>STRING NULL</code></td>
+      <td>The originating database table. Corresponds to the <code>table</code> field in the
+      Canal record if available.</td>
+    </tr>
+    <tr>
+      <td><code>sql-type</code></td>
+      <td><code>MAP&lt;STRING, INT&gt; NULL</code></td>
+      <td>Map of various sql types. Corresponds to the <code>sqlType</code> field in the 
+      Canal record if available.</td>
+    </tr>
+    <tr>
+      <td><code>pk-names</code></td>
+      <td><code>ARRAY&lt;STRING&gt; NULL</code></td>
+      <td>Array of primary key names. Corresponds to the <code>pkNames</code> field in the 
+      Canal record if available.</td>
+    </tr>
+    <tr>
+      <td><code>ingestion-timestamp</code></td>
+      <td><code>TIMESTAMP(3) WITH LOCAL TIME ZONE NULL</code></td>
+      <td>The timestamp at which the connector processed the event. Corresponds to the <code>ts</code>
+      field in the Canal record.</td>
+    </tr>
+    </tbody>
+</table>
+
+The following example shows how to access Canal metadata fields in Kafka:
+
+<div class="codetabs" markdown="1">
+<div data-lang="SQL" markdown="1">
+{% highlight sql %}
+CREATE TABLE KafkaTable (
+  `origin_database` STRING METADATA FROM 'value.database' VIRTUAL,
+  `origin_table` STRING METADATA FROM 'value.table' VIRTUAL,
+  `user_id` BIGINT,
+  `item_id` BIGINT,
+  `behavior` STRING

Review comment:
       I think we don't need to add backquotes around the column names, because they are not keywords. 

##########
File path: flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java
##########
@@ -198,4 +198,130 @@ public void testKafkaDebeziumChangelogSource() throws Exception {
 		tableResult.getJobClient().get().cancel().get(); // stop the job
 		deleteTestTopic(topic);
 	}
+
+	@Test
+	public void testKafkaCanalChangelogSource() throws Exception {
+		final String topic = "changelog_canal";
+		createTestTopic(topic, 1, 1);
+
+		// enables MiniBatch processing to verify MiniBatch + FLIP-95, see FLINK-18769
+		Configuration tableConf = tEnv.getConfig().getConfiguration();
+		tableConf.setString("table.exec.mini-batch.enabled", "true");
+		tableConf.setString("table.exec.mini-batch.allow-latency", "1s");
+		tableConf.setString("table.exec.mini-batch.size", "5000");
+		tableConf.setString("table.optimizer.agg-phase-strategy", "TWO_PHASE");
+
+		// ---------- Write the Canal json into Kafka -------------------
+		List<String> lines = readLines("canal-data.txt");
+		DataStreamSource<String> stream = env.fromCollection(lines);
+		SerializationSchema<String> serSchema = new SimpleStringSchema();
+		FlinkKafkaPartitioner<String> partitioner = new FlinkFixedPartitioner<>();
+
+		// the producer must not produce duplicates
+		Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings);
+		producerProperties.setProperty("retries", "0");
+		producerProperties.putAll(secureProps);
+		kafkaServer.produceIntoKafka(stream, topic, serSchema, producerProperties, partitioner);
+		try {
+			env.execute("Write sequence");
+		} catch (Exception e) {
+			throw new Exception("Failed to write canal data to Kafka.", e);
+		}
+
+		// ---------- Produce an event time stream into Kafka -------------------
+		String bootstraps = standardProps.getProperty("bootstrap.servers");
+		String sourceDDL = String.format(
+			"CREATE TABLE canal_source (" +
+				// test format metadata
+				" origin_ts STRING METADATA FROM 'value.ingestion-timestamp' VIRTUAL," + // unused
+				" origin_table STRING METADATA FROM 'value.table' VIRTUAL," +

Review comment:
       Could you also test the Map and Array metadatas? 




----------------------------------------------------------------
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] wangfeigithub commented on pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   ![image](https://user-images.githubusercontent.com/29723968/103871825-82bdd800-5108-11eb-93ad-55a9b0c7a016.png)
   ![Uploading image.png…]()
   


----------------------------------------------------------------
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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387",
       "triggerID" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389",
       "triggerID" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360) 
   * 44f3c38cdbc31df14dafa54cf9b80264da281256 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381) 
   * 2aa16a8e76b7ebf4705e2554d13ea014af50e446 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387) 
   * e81abd70d44d6ce7221d4cd2f44d31deab78ea9a UNKNOWN
   * 6956309e3928df5f4e7717a978bd91eee472f6ed Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389) 
   
   <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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387",
       "triggerID" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360) 
   * 44f3c38cdbc31df14dafa54cf9b80264da281256 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381) 
   * 2aa16a8e76b7ebf4705e2554d13ea014af50e446 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387) 
   * e81abd70d44d6ce7221d4cd2f44d31deab78ea9a UNKNOWN
   * 6956309e3928df5f4e7717a978bd91eee472f6ed 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] wangfeigithub commented on pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   Nicholas Jiang Jark Wu  I found a bug in the canal code. 'canal-json.table.include' does not filter out the binlog of the specified table correctly, which will cause an error in the parsing section. For example, if I want to read the binlog of canal-json.table.include = 'a' table, there is a source field of int in table a, but at this time if table b also has a source field of string, An error will be reported.


----------------------------------------------------------------
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] SteNicholas commented on pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   > Btw, could you add documentation for this feature?
   
   No problem.


----------------------------------------------------------------
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] SteNicholas commented on pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   > Hi @SteNicholas , I would like to discuss the metadata keys first. What do you think think just use the keys and types?
   > 
   > * `pk_names ARRAY<STRING>`
   > * `sql_types MAP<STRING, INT>`
   > * `table STRING`
   > * `database STRING`
   > * `ingestion_timestamp TIMESTAMP(3) WITH LOCAL TIME ZONE`
   
   IMO, just using the above keys and types is friendly for users to read metadata of the Canal JSON format.


----------------------------------------------------------------
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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387",
       "triggerID" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389",
       "triggerID" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11428",
       "triggerID" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11430",
       "triggerID" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11456",
       "triggerID" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f3b88d98c525b635179b6a9098a2634a1ffb42c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11483",
       "triggerID" : "3f3b88d98c525b635179b6a9098a2634a1ffb42c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5a5ef1f0bb5e1dcc094ad5454890d13ddf55c646",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11484",
       "triggerID" : "5a5ef1f0bb5e1dcc094ad5454890d13ddf55c646",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d6ec8319ec7906aeda62d9c60887f6b9fb2f505f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d6ec8319ec7906aeda62d9c60887f6b9fb2f505f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e81abd70d44d6ce7221d4cd2f44d31deab78ea9a UNKNOWN
   * 554e0a2c2ada42836d65397b163bad657806e8c2 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11456) 
   * 3f3b88d98c525b635179b6a9098a2634a1ffb42c Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11483) 
   * 5a5ef1f0bb5e1dcc094ad5454890d13ddf55c646 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11484) 
   * d6ec8319ec7906aeda62d9c60887f6b9fb2f505f 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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356) 
   
   <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] SteNicholas removed a comment on pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

Posted by GitBox <gi...@apache.org>.
SteNicholas removed a comment on pull request #14464:
URL: https://github.com/apache/flink/pull/14464#issuecomment-751155184


   > The change looks good in general. Could you add a IT case in `KafkaChangelogTableITCase` to test kafka+canal-json with metadata accessing?
   
   OK. I would like to add a IT case in `KafkaChangelogTableITCase`.


----------------------------------------------------------------
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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387",
       "triggerID" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389",
       "triggerID" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11428",
       "triggerID" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11430",
       "triggerID" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11456",
       "triggerID" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f3b88d98c525b635179b6a9098a2634a1ffb42c",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11483",
       "triggerID" : "3f3b88d98c525b635179b6a9098a2634a1ffb42c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5a5ef1f0bb5e1dcc094ad5454890d13ddf55c646",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11484",
       "triggerID" : "5a5ef1f0bb5e1dcc094ad5454890d13ddf55c646",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d6ec8319ec7906aeda62d9c60887f6b9fb2f505f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d6ec8319ec7906aeda62d9c60887f6b9fb2f505f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e81abd70d44d6ce7221d4cd2f44d31deab78ea9a UNKNOWN
   * 3f3b88d98c525b635179b6a9098a2634a1ffb42c Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11483) 
   * 5a5ef1f0bb5e1dcc094ad5454890d13ddf55c646 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11484) 
   * d6ec8319ec7906aeda62d9c60887f6b9fb2f505f 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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * bc5759c4d2710fd618ae0ea4541152e87d0dad65 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351) 
   * 4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356) 
   
   <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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * bc5759c4d2710fd618ae0ea4541152e87d0dad65 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351) 
   * 4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5 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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387",
       "triggerID" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360) 
   * 44f3c38cdbc31df14dafa54cf9b80264da281256 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381) 
   * 2aa16a8e76b7ebf4705e2554d13ea014af50e446 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387) 
   
   <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] wuchong commented on a change in pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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



##########
File path: docs/dev/table/connectors/formats/canal.zh.md
##########
@@ -200,7 +200,7 @@ The following example shows how to access Canal metadata fields in Kafka:
 CREATE TABLE KafkaTable (
   origin_database STRING METADATA FROM 'value.database' VIRTUAL,
   origin_table STRING METADATA FROM 'value.table' VIRTUAL,
-  origin_sql_type MAP&lt;STRING, INT&gt; METADATA FROM 'value.sql-type' VIRTUAL,
+  origin_sql_type MAP<STRING, INT> METADATA FROM 'value.sql-type' VIRTUAL,
   origin_pk_names ARRAY&lt;STRING&gt; METADATA FROM 'value.pk-names' VIRTUAL,

Review comment:
       Please also modify others and preview the page in local.




----------------------------------------------------------------
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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360) 
   
   <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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387",
       "triggerID" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389",
       "triggerID" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11428",
       "triggerID" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11430",
       "triggerID" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11456",
       "triggerID" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f3b88d98c525b635179b6a9098a2634a1ffb42c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11483",
       "triggerID" : "3f3b88d98c525b635179b6a9098a2634a1ffb42c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5a5ef1f0bb5e1dcc094ad5454890d13ddf55c646",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5a5ef1f0bb5e1dcc094ad5454890d13ddf55c646",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e81abd70d44d6ce7221d4cd2f44d31deab78ea9a UNKNOWN
   * 554e0a2c2ada42836d65397b163bad657806e8c2 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11456) 
   * 3f3b88d98c525b635179b6a9098a2634a1ffb42c Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11483) 
   * 5a5ef1f0bb5e1dcc094ad5454890d13ddf55c646 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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387",
       "triggerID" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389",
       "triggerID" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11428",
       "triggerID" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11430",
       "triggerID" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11456",
       "triggerID" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f3b88d98c525b635179b6a9098a2634a1ffb42c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11483",
       "triggerID" : "3f3b88d98c525b635179b6a9098a2634a1ffb42c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5a5ef1f0bb5e1dcc094ad5454890d13ddf55c646",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11484",
       "triggerID" : "5a5ef1f0bb5e1dcc094ad5454890d13ddf55c646",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d6ec8319ec7906aeda62d9c60887f6b9fb2f505f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11497",
       "triggerID" : "d6ec8319ec7906aeda62d9c60887f6b9fb2f505f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e81abd70d44d6ce7221d4cd2f44d31deab78ea9a UNKNOWN
   * 5a5ef1f0bb5e1dcc094ad5454890d13ddf55c646 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11484) 
   * d6ec8319ec7906aeda62d9c60887f6b9fb2f505f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11497) 
   
   <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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360) 
   * 44f3c38cdbc31df14dafa54cf9b80264da281256 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381) 
   
   <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] wuchong commented on pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   Hi @SteNicholas , I would like to discuss the metadata keys first. What do you think think just use the keys and types?
   
   - `pk_names ARRAY<STRING>`
   - `sql_types MAP<STRING, INT>`
   - `table STRING`
   - `database STRING`
   - `ingestion_timestamp TIMESTAMP(3) WITH LOCAL TIME ZONE`
   
   
   


----------------------------------------------------------------
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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387",
       "triggerID" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389",
       "triggerID" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e81abd70d44d6ce7221d4cd2f44d31deab78ea9a UNKNOWN
   * 6956309e3928df5f4e7717a978bd91eee472f6ed Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389) 
   
   <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] wuchong commented on pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   Btw, could you add documentation for this feature?


----------------------------------------------------------------
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] wuchong commented on a change in pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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



##########
File path: docs/dev/table/connectors/formats/canal.zh.md
##########
@@ -198,11 +198,14 @@ The following example shows how to access Canal metadata fields in Kafka:
 <div data-lang="SQL" markdown="1">
 {% highlight sql %}
 CREATE TABLE KafkaTable (
-  `origin_database` STRING METADATA FROM 'value.database' VIRTUAL,
-  `origin_table` STRING METADATA FROM 'value.table' VIRTUAL,
-  `user_id` BIGINT,
-  `item_id` BIGINT,
-  `behavior` STRING
+  origin_database STRING METADATA FROM 'value.database' VIRTUAL,
+  origin_table STRING METADATA FROM 'value.table' VIRTUAL,
+  origin_sql_type MAP&lt;STRING, INT&gt; METADATA FROM 'value.sql-type' VIRTUAL,

Review comment:
       Don't need to escape the character. 




----------------------------------------------------------------
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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387",
       "triggerID" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389",
       "triggerID" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e81abd70d44d6ce7221d4cd2f44d31deab78ea9a UNKNOWN
   * 6956309e3928df5f4e7717a978bd91eee472f6ed Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389) 
   * f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8 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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387",
       "triggerID" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389",
       "triggerID" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11428",
       "triggerID" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e81abd70d44d6ce7221d4cd2f44d31deab78ea9a UNKNOWN
   * 6956309e3928df5f4e7717a978bd91eee472f6ed Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389) 
   * f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11428) 
   * 73d00ec389bcf20486ae71758ab613e8cfe7cb03 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] wuchong commented on a change in pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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



##########
File path: flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java
##########
@@ -218,4 +218,168 @@ public void testKafkaDebeziumChangelogSource() throws Exception {
         tableResult.getJobClient().get().cancel().get(); // stop the job
         deleteTestTopic(topic);
     }
+
+    @Test
+    public void testKafkaCanalChangelogSource() throws Exception {
+        final String topic = "changelog_canal";
+        createTestTopic(topic, 1, 1);
+
+        // enables MiniBatch processing to verify MiniBatch + FLIP-95, see FLINK-18769
+        Configuration tableConf = tEnv.getConfig().getConfiguration();
+        tableConf.setString("table.exec.mini-batch.enabled", "true");
+        tableConf.setString("table.exec.mini-batch.allow-latency", "1s");
+        tableConf.setString("table.exec.mini-batch.size", "5000");
+        tableConf.setString("table.optimizer.agg-phase-strategy", "TWO_PHASE");
+
+        // ---------- Write the Canal json into Kafka -------------------
+        List<String> lines = readLines("canal-data.txt");
+        DataStreamSource<String> stream = env.fromCollection(lines);
+        SerializationSchema<String> serSchema = new SimpleStringSchema();
+        FlinkKafkaPartitioner<String> partitioner = new FlinkFixedPartitioner<>();
+
+        // the producer must not produce duplicates
+        Properties producerProperties =
+                FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings);
+        producerProperties.setProperty("retries", "0");
+        producerProperties.putAll(secureProps);
+        kafkaServer.produceIntoKafka(stream, topic, serSchema, producerProperties, partitioner);
+        try {
+            env.execute("Write sequence");
+        } catch (Exception e) {
+            throw new Exception("Failed to write canal data to Kafka.", e);
+        }
+
+        // ---------- Produce an event time stream into Kafka -------------------
+        String bootstraps = standardProps.getProperty("bootstrap.servers");
+        String sourceDDL =
+                String.format(
+                        "CREATE TABLE canal_source ("
+                                +
+                                // test format metadata
+                                " origin_database STRING METADATA FROM 'value.database' VIRTUAL,"
+                                + " origin_table STRING METADATA FROM 'value.table' VIRTUAL,"
+                                + " origin_sql_type MAP<STRING, INT> METADATA FROM 'value.sql-type' VIRTUAL,"
+                                + " origin_pk_names ARRAY<STRING> METADATA FROM 'value.pk-names' VIRTUAL,"
+                                + " origin_ts TIMESTAMP(3) METADATA FROM 'value.ingestion-timestamp' VIRTUAL,"
+                                + " id INT NOT NULL,"
+                                + " name STRING,"
+                                + " description STRING,"
+                                + " weight DECIMAL(10,3),"
+                                +
+                                // test connector metadata
+                                " origin_topic STRING METADATA FROM 'topic' VIRTUAL,"
+                                + " origin_partition STRING METADATA FROM 'partition' VIRTUAL"
+                                + // unused
+                                ") WITH ("
+                                + " 'connector' = 'kafka',"
+                                + " 'topic' = '%s',"
+                                + " 'properties.bootstrap.servers' = '%s',"
+                                + " 'scan.startup.mode' = 'earliest-offset',"
+                                + " 'value.format' = 'canal-json'"
+                                + ")",
+                        topic, bootstraps);
+        String sinkDDL =
+                "CREATE TABLE sink ("
+                        + " origin_topic STRING,"
+                        + " origin_database STRING,"
+                        + " origin_table STRING,"
+                        + " origin_sql_type MAP<STRING, INT>,"
+                        + " origin_pk_names ARRAY<STRING>,"
+                        + " origin_ts TIMESTAMP(3),"
+                        + " name STRING,"
+                        + " PRIMARY KEY (name) NOT ENFORCED"
+                        + ") WITH ("
+                        + " 'connector' = 'values',"
+                        + " 'sink-insert-only' = 'false'"
+                        + ")";
+        tEnv.executeSql(sourceDDL);
+        tEnv.executeSql(sinkDDL);
+        TableResult tableResult =
+                tEnv.executeSql(
+                        "INSERT INTO sink "
+                                + "SELECT origin_topic, origin_database, origin_table, origin_sql_type, "
+                                + "origin_pk_names, origin_ts, name "
+                                + "FROM canal_source");
+
+        // Canal captures change data on the `products2` table:
+        //
+        // CREATE TABLE products2 (
+        //  id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,
+        //  name VARCHAR(255),
+        //  description VARCHAR(512),
+        //  weight FLOAT
+        // );
+        // ALTER TABLE products2 AUTO_INCREMENT = 101;
+        //
+        // INSERT INTO products2
+        // VALUES (default,"scooter","Small 2-wheel scooter",3.14),
+        //        (default,"car battery","12V car battery",8.1),
+        //        (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40
+        // to
+        // #3",0.8),
+        //        (default,"hammer","12oz carpenter's hammer",0.75),
+        //        (default,"hammer","14oz carpenter's hammer",0.875),
+        //        (default,"hammer","16oz carpenter's hammer",1.0),
+        //        (default,"rocks","box of assorted rocks",5.3),
+        //        (default,"jacket","water resistent black wind breaker",0.1),
+        //        (default,"spare tire","24 inch spare tire",22.2);
+        // UPDATE products2 SET description='18oz carpenter hammer' WHERE id=106;
+        // UPDATE products2 SET weight='5.1' WHERE id=107;
+        // INSERT INTO products2 VALUES (default,"jacket","water resistent white wind breaker",0.2);
+        // INSERT INTO products2 VALUES (default,"scooter","Big 2-wheel scooter ",5.18);
+        // UPDATE products2 SET description='new water resistent white wind breaker', weight='0.5'
+        // WHERE
+        // id=110;
+        // UPDATE products2 SET weight='5.17' WHERE id=111;
+        // DELETE FROM products2 WHERE id=111;
+        // UPDATE products2 SET weight='5.17' WHERE id=102 or id = 101;
+        // DELETE FROM products2 WHERE id=102 or id = 103;
+        //
+        // > SELECT * FROM products2;
+        // +-----+--------------------+---------------------------------------------------------+--------+
+        // | id  | name               | description                                             |
+        // weight
+        // |
+        // +-----+--------------------+---------------------------------------------------------+--------+
+        // | 101 | scooter            | Small 2-wheel scooter                                   |
+        // 5.17
+        // |
+        // | 104 | hammer             | 12oz carpenter's hammer                                 |
+        // 0.75
+        // |
+        // | 105 | hammer             | 14oz carpenter's hammer                                 |
+        // 0.875
+        // |
+        // | 106 | hammer             | 18oz carpenter hammer                                   |
+        //   1
+        // |
+        // | 107 | rocks              | box of assorted rocks                                   |
+        // 5.1
+        // |
+        // | 108 | jacket             | water resistent black wind breaker                      |

Review comment:
       Could you beautify the format? The same to the debezium one. 




----------------------------------------------------------------
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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ddcee436fcf2904fb1b3241eb4eccfd36f6553be Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181) 
   
   <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] wuchong commented on pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   @wangfeigithub thanks for reporting this. Could you create an JIRA issue for this?


----------------------------------------------------------------
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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387",
       "triggerID" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360) 
   * 44f3c38cdbc31df14dafa54cf9b80264da281256 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381) 
   * 2aa16a8e76b7ebf4705e2554d13ea014af50e446 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387) 
   * e81abd70d44d6ce7221d4cd2f44d31deab78ea9a 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] wangfeigithub edited a comment on pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   ![image](https://user-images.githubusercontent.com/29723968/103871825-82bdd800-5108-11eb-93ad-55a9b0c7a016.png)
   
   ![image](https://user-images.githubusercontent.com/29723968/103871920-a1bc6a00-5108-11eb-931f-3977272ddea3.png)
   


----------------------------------------------------------------
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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387",
       "triggerID" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389",
       "triggerID" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11428",
       "triggerID" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11430",
       "triggerID" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11456",
       "triggerID" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e81abd70d44d6ce7221d4cd2f44d31deab78ea9a UNKNOWN
   * 554e0a2c2ada42836d65397b163bad657806e8c2 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11456) 
   
   <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] SteNicholas commented on pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   > The change looks good in general. Could you add a IT case in `KafkaChangelogTableITCase` to test kafka+canal-json with metadata accessing?
   
   OK. I would like to add a IT case in `KafkaChangelogTableITCase`.


----------------------------------------------------------------
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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387",
       "triggerID" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389",
       "triggerID" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11428",
       "triggerID" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11430",
       "triggerID" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11456",
       "triggerID" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f3b88d98c525b635179b6a9098a2634a1ffb42c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3f3b88d98c525b635179b6a9098a2634a1ffb42c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e81abd70d44d6ce7221d4cd2f44d31deab78ea9a UNKNOWN
   * 554e0a2c2ada42836d65397b163bad657806e8c2 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11456) 
   * 3f3b88d98c525b635179b6a9098a2634a1ffb42c 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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387",
       "triggerID" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389",
       "triggerID" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11428",
       "triggerID" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11430",
       "triggerID" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11456",
       "triggerID" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e81abd70d44d6ce7221d4cd2f44d31deab78ea9a UNKNOWN
   * 73d00ec389bcf20486ae71758ab613e8cfe7cb03 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11430) 
   * 554e0a2c2ada42836d65397b163bad657806e8c2 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11456) 
   
   <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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387",
       "triggerID" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389",
       "triggerID" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11428",
       "triggerID" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11430",
       "triggerID" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e81abd70d44d6ce7221d4cd2f44d31deab78ea9a UNKNOWN
   * 6956309e3928df5f4e7717a978bd91eee472f6ed Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389) 
   * f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11428) 
   * 73d00ec389bcf20486ae71758ab613e8cfe7cb03 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11430) 
   
   <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] wuchong commented on pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   I helped to beautify the format. Will merge this once Azure is passed. 


----------------------------------------------------------------
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] SteNicholas commented on pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   @wuchong Thanks for helping to beautify the format. I have set the time zone for the Canal JSON format metadata `ingestion-timestamp` to solve the Azure failure. Please help to merge once Azure is passed.


----------------------------------------------------------------
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] wuchong merged pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

Posted by GitBox <gi...@apache.org>.
wuchong merged pull request #14464:
URL: https://github.com/apache/flink/pull/14464


   


----------------------------------------------------------------
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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ddcee436fcf2904fb1b3241eb4eccfd36f6553be 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] SteNicholas commented on pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   @wuchong I have added the document about the metadata for Canal JSON format. Please help to review the document together.


----------------------------------------------------------------
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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ddcee436fcf2904fb1b3241eb4eccfd36f6553be Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181) 
   * bc5759c4d2710fd618ae0ea4541152e87d0dad65 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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387",
       "triggerID" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389",
       "triggerID" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11428",
       "triggerID" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11430",
       "triggerID" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11456",
       "triggerID" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f3b88d98c525b635179b6a9098a2634a1ffb42c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11483",
       "triggerID" : "3f3b88d98c525b635179b6a9098a2634a1ffb42c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5a5ef1f0bb5e1dcc094ad5454890d13ddf55c646",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11484",
       "triggerID" : "5a5ef1f0bb5e1dcc094ad5454890d13ddf55c646",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e81abd70d44d6ce7221d4cd2f44d31deab78ea9a UNKNOWN
   * 554e0a2c2ada42836d65397b163bad657806e8c2 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11456) 
   * 3f3b88d98c525b635179b6a9098a2634a1ffb42c Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11483) 
   * 5a5ef1f0bb5e1dcc094ad5454890d13ddf55c646 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11484) 
   
   <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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387",
       "triggerID" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389",
       "triggerID" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11428",
       "triggerID" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11430",
       "triggerID" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11456",
       "triggerID" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f3b88d98c525b635179b6a9098a2634a1ffb42c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11483",
       "triggerID" : "3f3b88d98c525b635179b6a9098a2634a1ffb42c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5a5ef1f0bb5e1dcc094ad5454890d13ddf55c646",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11484",
       "triggerID" : "5a5ef1f0bb5e1dcc094ad5454890d13ddf55c646",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d6ec8319ec7906aeda62d9c60887f6b9fb2f505f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11497",
       "triggerID" : "d6ec8319ec7906aeda62d9c60887f6b9fb2f505f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e81abd70d44d6ce7221d4cd2f44d31deab78ea9a UNKNOWN
   * d6ec8319ec7906aeda62d9c60887f6b9fb2f505f Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11497) 
   
   <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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   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 ddcee436fcf2904fb1b3241eb4eccfd36f6553be (Tue Dec 22 13:26: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] flinkbot edited a comment on pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387",
       "triggerID" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389",
       "triggerID" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 44f3c38cdbc31df14dafa54cf9b80264da281256 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381) 
   * 2aa16a8e76b7ebf4705e2554d13ea014af50e446 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387) 
   * e81abd70d44d6ce7221d4cd2f44d31deab78ea9a UNKNOWN
   * 6956309e3928df5f4e7717a978bd91eee472f6ed Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389) 
   
   <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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ddcee436fcf2904fb1b3241eb4eccfd36f6553be Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181) 
   
   <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] SteNicholas removed a comment on pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

Posted by GitBox <gi...@apache.org>.
SteNicholas removed a comment on pull request #14464:
URL: https://github.com/apache/flink/pull/14464#issuecomment-751480880


   > Btw, could you add documentation for this feature?
   
   No problem.


----------------------------------------------------------------
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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387",
       "triggerID" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389",
       "triggerID" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11428",
       "triggerID" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11430",
       "triggerID" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e81abd70d44d6ce7221d4cd2f44d31deab78ea9a UNKNOWN
   * 73d00ec389bcf20486ae71758ab613e8cfe7cb03 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11430) 
   * 554e0a2c2ada42836d65397b163bad657806e8c2 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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387",
       "triggerID" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389",
       "triggerID" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11428",
       "triggerID" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11430",
       "triggerID" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e81abd70d44d6ce7221d4cd2f44d31deab78ea9a UNKNOWN
   * f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11428) 
   * 73d00ec389bcf20486ae71758ab613e8cfe7cb03 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11430) 
   * 554e0a2c2ada42836d65397b163bad657806e8c2 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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387",
       "triggerID" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389",
       "triggerID" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2aa16a8e76b7ebf4705e2554d13ea014af50e446 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387) 
   * e81abd70d44d6ce7221d4cd2f44d31deab78ea9a UNKNOWN
   * 6956309e3928df5f4e7717a978bd91eee472f6ed Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389) 
   
   <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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356) 
   * 9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780 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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356) 
   * 9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360) 
   
   <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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387",
       "triggerID" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389",
       "triggerID" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11428",
       "triggerID" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11430",
       "triggerID" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e81abd70d44d6ce7221d4cd2f44d31deab78ea9a UNKNOWN
   * f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11428) 
   * 73d00ec389bcf20486ae71758ab613e8cfe7cb03 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11430) 
   
   <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] wuchong commented on a change in pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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



##########
File path: flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java
##########
@@ -0,0 +1,226 @@
+/*
+ * 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.formats.json.canal;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.formats.json.TimestampFormat;
+import org.apache.flink.formats.json.canal.CanalJsonDeserializationSchema.MetadataConverter;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.DecodingFormat;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.utils.DataTypeUtils;
+import org.apache.flink.types.RowKind;
+
+import javax.annotation.Nullable;
+
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * {@link DecodingFormat} for Canal using JSON encoding.
+ */
+public class CanalJsonDecodingFormat implements DecodingFormat<DeserializationSchema<RowData>> {
+
+	// --------------------------------------------------------------------------------------------
+	// Mutable attributes
+	// --------------------------------------------------------------------------------------------
+
+	private List<String> metadataKeys;
+
+	// --------------------------------------------------------------------------------------------
+	// Canal-specific attributes
+	// --------------------------------------------------------------------------------------------
+
+	private final @Nullable String database;
+
+	private final @Nullable String table;
+
+	private final boolean ignoreParseErrors;
+
+	private final TimestampFormat timestampFormat;
+
+	public CanalJsonDecodingFormat(
+		String database,
+		String table,
+		boolean ignoreParseErrors,
+		TimestampFormat timestampFormat) {
+		this.database = database;
+		this.table = table;
+		this.ignoreParseErrors = ignoreParseErrors;
+		this.timestampFormat = timestampFormat;
+		this.metadataKeys = Collections.emptyList();
+	}
+
+	@Override
+	public DeserializationSchema<RowData> createRuntimeDecoder(
+		DynamicTableSource.Context context,
+		DataType physicalDataType) {

Review comment:
       Add indent. 

##########
File path: flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDecodingFormat.java
##########
@@ -0,0 +1,226 @@
+/*
+ * 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.formats.json.canal;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.formats.json.TimestampFormat;
+import org.apache.flink.formats.json.canal.CanalJsonDeserializationSchema.MetadataConverter;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.DecodingFormat;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.utils.DataTypeUtils;
+import org.apache.flink.types.RowKind;
+
+import javax.annotation.Nullable;
+
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * {@link DecodingFormat} for Canal using JSON encoding.
+ */
+public class CanalJsonDecodingFormat implements DecodingFormat<DeserializationSchema<RowData>> {
+
+	// --------------------------------------------------------------------------------------------
+	// Mutable attributes
+	// --------------------------------------------------------------------------------------------
+
+	private List<String> metadataKeys;
+
+	// --------------------------------------------------------------------------------------------
+	// Canal-specific attributes
+	// --------------------------------------------------------------------------------------------
+
+	private final @Nullable String database;
+
+	private final @Nullable String table;
+
+	private final boolean ignoreParseErrors;
+
+	private final TimestampFormat timestampFormat;
+
+	public CanalJsonDecodingFormat(
+		String database,
+		String table,
+		boolean ignoreParseErrors,
+		TimestampFormat timestampFormat) {
+		this.database = database;
+		this.table = table;
+		this.ignoreParseErrors = ignoreParseErrors;
+		this.timestampFormat = timestampFormat;
+		this.metadataKeys = Collections.emptyList();
+	}
+
+	@Override
+	public DeserializationSchema<RowData> createRuntimeDecoder(
+		DynamicTableSource.Context context,
+		DataType physicalDataType) {
+		final List<ReadableMetadata> readableMetadata = metadataKeys.stream()
+			.map(k ->
+				Stream.of(ReadableMetadata.values())
+					.filter(rm -> rm.key.equals(k))
+					.findFirst()
+					.orElseThrow(IllegalStateException::new))
+			.collect(Collectors.toList());
+		final List<DataTypes.Field> metadataFields = readableMetadata.stream()
+			.map(m -> DataTypes.FIELD(m.key, m.dataType))
+			.collect(Collectors.toList());
+		final DataType producedDataType = DataTypeUtils.appendRowFields(physicalDataType, metadataFields);
+		final TypeInformation<RowData> producedTypeInfo = context.createTypeInformation(producedDataType);
+		return CanalJsonDeserializationSchema.builder(physicalDataType, readableMetadata, producedTypeInfo)
+			.setDatabase(database)
+			.setTable(table)
+			.setIgnoreParseErrors(ignoreParseErrors)
+			.setTimestampFormat(timestampFormat)
+			.build();
+	}
+
+	@Override
+	public Map<String, DataType> listReadableMetadata() {
+		final Map<String, DataType> metadataMap = new LinkedHashMap<>();
+		Stream.of(ReadableMetadata.values()).forEachOrdered(m -> metadataMap.put(m.key, m.dataType));
+		return metadataMap;
+	}
+
+	@Override
+	public void applyReadableMetadata(List<String> metadataKeys) {
+		this.metadataKeys = metadataKeys;
+	}
+
+	@Override
+	public ChangelogMode getChangelogMode() {
+		return ChangelogMode.newBuilder()
+			.addContainedKind(RowKind.INSERT)
+			.addContainedKind(RowKind.UPDATE_BEFORE)
+			.addContainedKind(RowKind.UPDATE_AFTER)
+			.addContainedKind(RowKind.DELETE)
+			.build();
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// Metadata handling
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * List of metadata that can be read with this format.
+	 */
+	enum ReadableMetadata {
+		DATABASE(
+			"database",
+			DataTypes.STRING().nullable(),
+			DataTypes.FIELD("database", DataTypes.STRING()),
+			new MetadataConverter() {
+				private static final long serialVersionUID = 1L;
+
+				@Override
+				public Object convert(GenericRowData row, int pos) {
+					return row.getString(pos);
+				}
+			}
+		),
+
+		TABLE(
+			"table",
+			DataTypes.STRING().nullable(),
+			DataTypes.FIELD("table", DataTypes.STRING()),
+			new MetadataConverter() {
+				private static final long serialVersionUID = 1L;
+
+				@Override
+				public Object convert(GenericRowData row, int pos) {
+					return row.getString(pos);
+				}
+			}
+		),
+
+		SQL_TYPE(
+			"sql-type",
+			DataTypes.STRING().nullable(),
+			DataTypes.FIELD("sqlType", DataTypes.STRING()),
+			new MetadataConverter() {
+				private static final long serialVersionUID = 1L;
+
+				@Override
+				public Object convert(GenericRowData row, int pos) {
+					return row.getString(pos);
+				}
+			}
+		),
+
+		PK_NAMES(
+			"pk-names",
+			DataTypes.STRING().nullable(),
+			DataTypes.FIELD("pkNames", DataTypes.ARRAY(DataTypes.STRING())),
+			new MetadataConverter() {
+				private static final long serialVersionUID = 1L;
+
+				@Override
+				public Object convert(GenericRowData row, int pos) {
+					return row.getArray(pos);
+				}
+			}
+		),
+
+		INGESTION_TIMESTAMP(
+			"ingestion-timestamp",
+			DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(),
+			DataTypes.FIELD("ts", DataTypes.BIGINT()),
+			new MetadataConverter() {
+				private static final long serialVersionUID = 1L;
+
+				@Override
+				public Object convert(GenericRowData row, int pos) {
+					if (row.isNullAt(pos)) {
+						return null;
+					}
+					return TimestampData.fromEpochMillis(row.getLong(pos));
+				}
+			}
+		);
+
+		final String key;
+
+		final DataType dataType;
+
+		final DataTypes.Field requiredJsonField;
+
+		final MetadataConverter converter;
+
+		ReadableMetadata(
+			String key,
+			DataType dataType,
+			DataTypes.Field requiredJsonField,
+			MetadataConverter converter) {

Review comment:
       Add indent.

##########
File path: flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/canal/CanalJsonDeserializationSchema.java
##########
@@ -262,24 +296,71 @@ public boolean equals(Object o) {
 			return false;
 		}
 		CanalJsonDeserializationSchema that = (CanalJsonDeserializationSchema) o;
-		return ignoreParseErrors == that.ignoreParseErrors &&
-			fieldCount == that.fieldCount &&
-			Objects.equals(jsonDeserializer, that.jsonDeserializer) &&
-			Objects.equals(resultTypeInfo, that.resultTypeInfo);
+		return Objects.equals(jsonDeserializer, that.jsonDeserializer)
+			&& hasMetadata == that.hasMetadata
+			&& Objects.equals(producedTypeInfo, that.producedTypeInfo)
+			&& Objects.equals(database, that.database)
+			&& Objects.equals(table, that.table)
+			&& ignoreParseErrors == that.ignoreParseErrors
+			&& fieldCount == that.fieldCount;
 	}
 
 	@Override
 	public int hashCode() {
-		return Objects.hash(jsonDeserializer, resultTypeInfo, ignoreParseErrors, fieldCount);
+		return Objects.hash(jsonDeserializer, hasMetadata, producedTypeInfo, database, table, ignoreParseErrors, fieldCount);
 	}
 
-	private static RowType createJsonRowType(DataType databaseSchema) {
+	// --------------------------------------------------------------------------------------------
+
+	private static RowType createJsonRowType(DataType physicalDataType, List<ReadableMetadata> readableMetadata) {
 		// Canal JSON contains other information, e.g. "ts", "sql", but we don't need them
-		return (RowType) DataTypes.ROW(
-			DataTypes.FIELD("data", DataTypes.ARRAY(databaseSchema)),
-			DataTypes.FIELD("old", DataTypes.ARRAY(databaseSchema)),
+		DataType root = DataTypes.ROW(
+			DataTypes.FIELD("data", DataTypes.ARRAY(physicalDataType)),
+			DataTypes.FIELD("old", DataTypes.ARRAY(physicalDataType)),
 			DataTypes.FIELD("type", DataTypes.STRING()),
-			DataTypes.FIELD("database", DataTypes.STRING()),
-			DataTypes.FIELD("table", DataTypes.STRING())).getLogicalType();
+			ReadableMetadata.DATABASE.requiredJsonField,
+			ReadableMetadata.TABLE.requiredJsonField);
+		// append fields that are required for reading metadata in the root
+		final List<DataTypes.Field> rootMetadataFields = readableMetadata.stream()
+			.filter(m -> m != ReadableMetadata.DATABASE && m != ReadableMetadata.TABLE)
+			.map(m -> m.requiredJsonField)
+			.distinct()
+			.collect(Collectors.toList());
+		return (RowType) DataTypeUtils.appendRowFields(root, rootMetadataFields).getLogicalType();
+	}
+
+	private static MetadataConverter[] createMetadataConverters(
+		RowType jsonRowType,
+		List<ReadableMetadata> requestedMetadata) {

Review comment:
       Add indent.




----------------------------------------------------------------
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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360) 
   * 44f3c38cdbc31df14dafa54cf9b80264da281256 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381) 
   * 2aa16a8e76b7ebf4705e2554d13ea014af50e446 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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ddcee436fcf2904fb1b3241eb4eccfd36f6553be Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181) 
   * bc5759c4d2710fd618ae0ea4541152e87d0dad65 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351) 
   
   <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] wuchong commented on pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   I'm also fine with that. Will review it tomorrow. 


----------------------------------------------------------------
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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387",
       "triggerID" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389",
       "triggerID" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11428",
       "triggerID" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11430",
       "triggerID" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11456",
       "triggerID" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f3b88d98c525b635179b6a9098a2634a1ffb42c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11483",
       "triggerID" : "3f3b88d98c525b635179b6a9098a2634a1ffb42c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e81abd70d44d6ce7221d4cd2f44d31deab78ea9a UNKNOWN
   * 554e0a2c2ada42836d65397b163bad657806e8c2 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11456) 
   * 3f3b88d98c525b635179b6a9098a2634a1ffb42c Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11483) 
   
   <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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360) 
   * 44f3c38cdbc31df14dafa54cf9b80264da281256 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] SteNicholas edited a comment on pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   @wuchong  I have added a IT case in `KafkaChangelogTableITCase` to test Kafka and Canal JSON with metadata accessing.
   About the Canal JSON metadata, I prefer to the following keys and types:
   
    - `database STRING`
    - `table STRING`
    - `sql-type MAP<STRING, INT>`
    - `pk-names ARRAY<STRING>`
    - `ingestion-timestamp TIMESTAMP(3) WITH LOCAL TIME ZONE`
    
   The name of keys are a little different from yours. What do you think about the above names of keys?


----------------------------------------------------------------
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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   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 d6ec8319ec7906aeda62d9c60887f6b9fb2f505f (Fri May 28 07:01:42 UTC 2021)
   
   **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] SteNicholas removed a comment on pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

Posted by GitBox <gi...@apache.org>.
SteNicholas removed a comment on pull request #14464:
URL: https://github.com/apache/flink/pull/14464#issuecomment-751155066


   > Hi @SteNicholas , I would like to discuss the metadata keys first. What do you think think just use the keys and types?
   > 
   > * `pk_names ARRAY<STRING>`
   > * `sql_types MAP<STRING, INT>`
   > * `table STRING`
   > * `database STRING`
   > * `ingestion_timestamp TIMESTAMP(3) WITH LOCAL TIME ZONE`
   
   @wuchong IMO, just using the above keys and types is friendly for users to read metadata of the Canal JSON format.


----------------------------------------------------------------
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] SteNicholas commented on pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   @wuchong  I have added a IT case in `KafkaChangelogTableITCase` to test Kafka and Canal JSON with metadata accessing.
   About the Canal JSON metadata, I prefer to the following keys and types:
    - `database STRING`
    - `table STRING`
    - `sql-type MAP<STRING, INT>`
    - `pk-names ARRAY<STRING>`
    - `ingestion-timestamp TIMESTAMP(3) WITH LOCAL TIME ZONE`
    The name of keys are a little different from yours. What do you think about the above names of keys?


----------------------------------------------------------------
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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11356",
       "triggerID" : "4ded1f32d30bf98ad7ac4753a2831acf2e78a4f5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11360",
       "triggerID" : "9abb79a2bb6e487d3f1deb0cfc4ecfff74a7d780",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11381",
       "triggerID" : "44f3c38cdbc31df14dafa54cf9b80264da281256",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11387",
       "triggerID" : "2aa16a8e76b7ebf4705e2554d13ea014af50e446",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e81abd70d44d6ce7221d4cd2f44d31deab78ea9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11389",
       "triggerID" : "6956309e3928df5f4e7717a978bd91eee472f6ed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11428",
       "triggerID" : "f868d49d3c71f7b55d1c6a3dc601ac1561bdd9e8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11430",
       "triggerID" : "73d00ec389bcf20486ae71758ab613e8cfe7cb03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11456",
       "triggerID" : "554e0a2c2ada42836d65397b163bad657806e8c2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f3b88d98c525b635179b6a9098a2634a1ffb42c",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11483",
       "triggerID" : "3f3b88d98c525b635179b6a9098a2634a1ffb42c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5a5ef1f0bb5e1dcc094ad5454890d13ddf55c646",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11484",
       "triggerID" : "5a5ef1f0bb5e1dcc094ad5454890d13ddf55c646",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d6ec8319ec7906aeda62d9c60887f6b9fb2f505f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11497",
       "triggerID" : "d6ec8319ec7906aeda62d9c60887f6b9fb2f505f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e81abd70d44d6ce7221d4cd2f44d31deab78ea9a UNKNOWN
   * 3f3b88d98c525b635179b6a9098a2634a1ffb42c Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11483) 
   * 5a5ef1f0bb5e1dcc094ad5454890d13ddf55c646 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11484) 
   * d6ec8319ec7906aeda62d9c60887f6b9fb2f505f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11497) 
   
   <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 #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11181",
       "triggerID" : "ddcee436fcf2904fb1b3241eb4eccfd36f6553be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351",
       "triggerID" : "bc5759c4d2710fd618ae0ea4541152e87d0dad65",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * bc5759c4d2710fd618ae0ea4541152e87d0dad65 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11351) 
   
   <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] SteNicholas edited a comment on pull request #14464: [FLINK-20385][canal][json] Allow to read metadata for canal-json format

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


   > Hi @SteNicholas , I would like to discuss the metadata keys first. What do you think think just use the keys and types?
   > 
   > * `pk_names ARRAY<STRING>`
   > * `sql_types MAP<STRING, INT>`
   > * `table STRING`
   > * `database STRING`
   > * `ingestion_timestamp TIMESTAMP(3) WITH LOCAL TIME ZONE`
   
   @wuchong IMO, just using the above keys and types is friendly for users to read metadata of the Canal JSON format.


----------------------------------------------------------------
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