You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "mas-chen (via GitHub)" <gi...@apache.org> on 2023/03/24 07:45:15 UTC

[GitHub] [flink-connector-kafka] mas-chen opened a new pull request, #16: [FLINK-30859] Externalize confluent avro related code

mas-chen opened a new pull request, #16:
URL: https://github.com/apache/flink-connector-kafka/pull/16

   1st major part of FLINK-30859 to externalize confluent Avro related code to remove flink-connector-kafka code safely from apache/flink:master


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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink-connector-kafka] mas-chen commented on a diff in pull request #16: [FLINK-30859] Externalize confluent avro related code

Posted by "mas-chen (via GitHub)" <gi...@apache.org>.
mas-chen commented on code in PR #16:
URL: https://github.com/apache/flink-connector-kafka/pull/16#discussion_r1158835180


##########
flink-formats-confluent/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java:
##########


Review Comment:
   Just the debezium parts which will require a dependency on Flink-json I believe



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink-connector-kafka] tzulitai commented on a diff in pull request #16: [FLINK-30859] Externalize confluent avro related code

Posted by "tzulitai (via GitHub)" <gi...@apache.org>.
tzulitai commented on code in PR #16:
URL: https://github.com/apache/flink-connector-kafka/pull/16#discussion_r1157799805


##########
flink-formats-confluent/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java:
##########


Review Comment:
   I'm not so sure but: this makes me wonder if `DebeziumJson` should be moved here as well.
   Strictly speaking its tied to Kafka as well. But it's a bit awkward, since that format is bundled as part of `flink-json` ...



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink-connector-kafka] mas-chen commented on a diff in pull request #16: [FLINK-30859] Externalize confluent avro related code

Posted by "mas-chen (via GitHub)" <gi...@apache.org>.
mas-chen commented on code in PR #16:
URL: https://github.com/apache/flink-connector-kafka/pull/16#discussion_r1158044725


##########
flink-formats-confluent/flink-avro-confluent-registry/pom.xml:
##########
@@ -0,0 +1,184 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-formats-confluent</artifactId>
+		<version>4.0-SNAPSHOT</version>
+	</parent>
+	<modelVersion>4.0.0</modelVersion>
+
+	<artifactId>flink-avro-confluent-registry</artifactId>
+
+	<name>Flink : Formats : Avro confluent registry</name>
+
+	<properties>
+		<kafka.version>3.2.3</kafka.version>

Review Comment:
   Good catch! Same with the Confluent version



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink-connector-kafka] mas-chen commented on a diff in pull request #16: [FLINK-30859] Externalize confluent avro related code

Posted by "mas-chen (via GitHub)" <gi...@apache.org>.
mas-chen commented on code in PR #16:
URL: https://github.com/apache/flink-connector-kafka/pull/16#discussion_r1159166591


##########
flink-formats-confluent/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java:
##########


Review Comment:
   I took all of the Flink-json and deleted the irrelevant code, keeping only debezium related code. I found that to be an easier approach when sync'ing the commits



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink-connector-kafka] tzulitai commented on a diff in pull request #16: [FLINK-30859] Externalize confluent avro related code

Posted by "tzulitai (via GitHub)" <gi...@apache.org>.
tzulitai commented on code in PR #16:
URL: https://github.com/apache/flink-connector-kafka/pull/16#discussion_r1160139972


##########
flink-formats-kafka/flink-json-debezium/pom.xml:
##########
@@ -0,0 +1,141 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-formats-kafka</artifactId>
+		<version>4.1-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>flink-json-debezium</artifactId>

Review Comment:
   re: backwards compatibility
   
   I also just realized that the Flink SQL Client bundles flink-json as well. So we can't really separate the debezium-json formats out of flink-json ... otherwise it becomes a behavior breaking change (users suddenly have to explicitly include this new module why previously it just works out of the box).



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink-connector-kafka] tzulitai closed pull request #16: [FLINK-30859] Externalize confluent avro related code

Posted by "tzulitai (via GitHub)" <gi...@apache.org>.
tzulitai closed pull request #16: [FLINK-30859] Externalize confluent avro related code
URL: https://github.com/apache/flink-connector-kafka/pull/16


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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink-connector-kafka] tzulitai commented on a diff in pull request #16: [FLINK-30859] Externalize confluent avro related code

Posted by "tzulitai (via GitHub)" <gi...@apache.org>.
tzulitai commented on code in PR #16:
URL: https://github.com/apache/flink-connector-kafka/pull/16#discussion_r1158843434


##########
flink-formats-confluent/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java:
##########


Review Comment:
   👍 makes sense



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink-connector-kafka] tzulitai commented on a diff in pull request #16: [FLINK-30859] Externalize confluent avro related code

Posted by "tzulitai (via GitHub)" <gi...@apache.org>.
tzulitai commented on code in PR #16:
URL: https://github.com/apache/flink-connector-kafka/pull/16#discussion_r1160139972


##########
flink-formats-kafka/flink-json-debezium/pom.xml:
##########
@@ -0,0 +1,141 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-formats-kafka</artifactId>
+		<version>4.1-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>flink-json-debezium</artifactId>

Review Comment:
   re: backwards compatibility
   
   I also just realized that the Flink SQL Client bundles flink-json as well. So we can't really separate the debezium-json formats out of flink-json ... otherwise it becomes a behavior breaking change (users suddenly have to explicitly include this new module while previously it just works out of the box).



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink-connector-kafka] mas-chen commented on a diff in pull request #16: [FLINK-30859] Externalize confluent avro related code

Posted by "mas-chen (via GitHub)" <gi...@apache.org>.
mas-chen commented on code in PR #16:
URL: https://github.com/apache/flink-connector-kafka/pull/16#discussion_r1147223589


##########
flink-formats-confluent/flink-avro-confluent-registry/pom.xml:
##########
@@ -135,30 +135,36 @@ under the License.
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-table-api-java</artifactId>
-			<version>${project.version}</version>
+			<version>${flink.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-table-runtime</artifactId>
-			<version>${project.version}</version>
+			<version>${flink.version}</version>
 			<scope>test</scope>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-table-common</artifactId>
-			<version>${project.version}</version>
+			<version>${flink.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-avro</artifactId>
-			<version>${project.version}</version>
+			<version>${flink.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
 		</dependency>
+		<dependency>

Review Comment:
   required for unit test



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink-connector-kafka] mas-chen commented on a diff in pull request #16: [FLINK-30859] Externalize confluent avro related code

Posted by "mas-chen (via GitHub)" <gi...@apache.org>.
mas-chen commented on code in PR #16:
URL: https://github.com/apache/flink-connector-kafka/pull/16#discussion_r1158044388


##########
flink-formats-confluent/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java:
##########


Review Comment:
   It is awkward. I think it makes sense and the Debezium Flink documentation does covers both Avro + Json so we should port both over



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink-connector-kafka] tzulitai commented on a diff in pull request #16: [FLINK-30859] Externalize confluent avro related code

Posted by "tzulitai (via GitHub)" <gi...@apache.org>.
tzulitai commented on code in PR #16:
URL: https://github.com/apache/flink-connector-kafka/pull/16#discussion_r1160329477


##########
flink-confluent-schema-registry-e2e-tests/pom.xml:
##########
@@ -0,0 +1,136 @@
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-connector-kafka-parent</artifactId>
+		<version>4.1-SNAPSHOT</version>
+	</parent>
+	<modelVersion>4.0.0</modelVersion>
+
+	<artifactId>flink-confluent-schema-registry</artifactId>
+	<name>Flink : E2E Tests : Confluent schema registry</name>
+
+	<properties>
+		<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+	</properties>
+
+	<repositories>
+		<repository>
+			<id>confluent</id>
+			<url>https://packages.confluent.io/maven/</url>
+		</repository>
+	</repositories>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java</artifactId>
+			<version>${flink.version}</version>
+			<scope>provided</scope>
+		</dependency>
+		<!-- https://mvnrepository.com/artifact/org.apache.flink/flink-connector-kafka -->

Review Comment:
   This comment can be removed.



##########
flink-formats-kafka/pom.xml:
##########
@@ -0,0 +1,41 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.flink</groupId>
+    <artifactId>flink-connector-kafka-parent</artifactId>
+    <version>4.1-SNAPSHOT</version>
+  </parent>
+
+  <packaging>pom</packaging>
+
+  <artifactId>flink-formats-kafka</artifactId>
+  <name>Flink : Formats : Confluent</name>

Review Comment:
   This should be renamed to `Flink : Formats : Kafka` as well.



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink-connector-kafka] mas-chen commented on pull request #16: [FLINK-30859] Externalize confluent avro related code

Posted by "mas-chen (via GitHub)" <gi...@apache.org>.
mas-chen commented on PR #16:
URL: https://github.com/apache/flink-connector-kafka/pull/16#issuecomment-1483223799

   BTW I noticed that the NOTICE files are outdated. I hope you don't mind that I change it in the same PR here: [a554982](https://github.com/apache/flink-connector-kafka/pull/16/commits/a55498297a3d20c0ccaf078cd8629ea22e23f820)


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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink-connector-kafka] zentol commented on a diff in pull request #16: [FLINK-30859] Externalize confluent avro related code

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #16:
URL: https://github.com/apache/flink-connector-kafka/pull/16#discussion_r1160135014


##########
flink-formats-kafka/flink-json-debezium/pom.xml:
##########
@@ -0,0 +1,141 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-formats-kafka</artifactId>
+		<version>4.1-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>flink-json-debezium</artifactId>

Review Comment:
   I'm wondering if there is a significant benefit to hanving this as a separate module.
   
   for backwards-compatibility alone we'd need to bundle it in the kafka-connector (as it was previously provided by the distribution by default), and with no additional dependencies being added I don't see a real downside.



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink-connector-kafka] mas-chen commented on a diff in pull request #16: [FLINK-30859] Externalize confluent avro related code

Posted by "mas-chen (via GitHub)" <gi...@apache.org>.
mas-chen commented on code in PR #16:
URL: https://github.com/apache/flink-connector-kafka/pull/16#discussion_r1160320646


##########
flink-formats-kafka/flink-json-debezium/pom.xml:
##########
@@ -0,0 +1,141 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-formats-kafka</artifactId>
+		<version>4.1-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>flink-json-debezium</artifactId>

Review Comment:
   Gordon and I discussed offline. For the purposes of finishing off the externalization and externalizing the required confluent avro, we will table this discussion to outside of this PR. I filed https://issues.apache.org/jira/browse/FLINK-31747 to track 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink-connector-kafka] tzulitai commented on a diff in pull request #16: [FLINK-30859] Externalize confluent avro related code

Posted by "tzulitai (via GitHub)" <gi...@apache.org>.
tzulitai commented on code in PR #16:
URL: https://github.com/apache/flink-connector-kafka/pull/16#discussion_r1157777291


##########
docs/content.zh/docs/connectors/table/formats/avro-confluent.md:
##########


Review Comment:
   Does the "Debezium" format docs need to be migrated here as well?



##########
flink-formats-confluent/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java:
##########


Review Comment:
   I'm not so sure but: this makes me wonder if `debezium json` should be moved here as well.
   Strictly speaking its tied to Kafka as well. But it's a bit awkward, since that format is bundled as part of `flink-json` ...



##########
flink-formats-confluent/flink-avro-confluent-registry/pom.xml:
##########
@@ -0,0 +1,184 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-formats-confluent</artifactId>
+		<version>4.0-SNAPSHOT</version>
+	</parent>
+	<modelVersion>4.0.0</modelVersion>
+
+	<artifactId>flink-avro-confluent-registry</artifactId>
+
+	<name>Flink : Formats : Avro confluent registry</name>
+
+	<properties>
+		<kafka.version>3.2.3</kafka.version>

Review Comment:
   The `main` branch recently had this upgraded to 3.4.0 for `flink-connector-kafka`.
   We'd probably want this version to be shared across all modules.



##########
flink-formats-confluent/pom.xml:
##########
@@ -0,0 +1,41 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.flink</groupId>
+    <artifactId>flink-connector-kafka-parent</artifactId>
+    <version>4.0-SNAPSHOT</version>
+  </parent>
+
+  <packaging>pom</packaging>
+
+  <artifactId>flink-formats-confluent</artifactId>
+  <name>Flink : Formats : Confluent</name>

Review Comment:
   Not so sure of the naming. Should we name this `Flink : Formats : Kafka` instead?
   I know that the current formats we have is all Confluent-specific, but naming it more generally might make sense if we add other Kafka-related formats down the road.



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink-connector-kafka] tzulitai commented on a diff in pull request #16: [FLINK-30859] Externalize confluent avro related code

Posted by "tzulitai (via GitHub)" <gi...@apache.org>.
tzulitai commented on code in PR #16:
URL: https://github.com/apache/flink-connector-kafka/pull/16#discussion_r1157803740


##########
flink-formats-confluent/pom.xml:
##########
@@ -0,0 +1,41 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.flink</groupId>
+    <artifactId>flink-connector-kafka-parent</artifactId>
+    <version>4.0-SNAPSHOT</version>
+  </parent>
+
+  <packaging>pom</packaging>
+
+  <artifactId>flink-formats-confluent</artifactId>
+  <name>Flink : Formats : Confluent</name>

Review Comment:
   For example, the debezium formats are not strictly tied to Confluent concepts.



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink-connector-kafka] tzulitai commented on a diff in pull request #16: [FLINK-30859] Externalize confluent avro related code

Posted by "tzulitai (via GitHub)" <gi...@apache.org>.
tzulitai commented on code in PR #16:
URL: https://github.com/apache/flink-connector-kafka/pull/16#discussion_r1158798008


##########
flink-formats-confluent/flink-avro-confluent-registry/src/main/java/org/apache/flink/formats/avro/registry/confluent/debezium/DebeziumAvroFormatFactory.java:
##########


Review Comment:
   Do you intend to port all of `flink-json` over or just the Debezium parts?



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink-connector-kafka] mas-chen commented on a diff in pull request #16: [FLINK-30859] Externalize confluent avro related code

Posted by "mas-chen (via GitHub)" <gi...@apache.org>.
mas-chen commented on code in PR #16:
URL: https://github.com/apache/flink-connector-kafka/pull/16#discussion_r1160320646


##########
flink-formats-kafka/flink-json-debezium/pom.xml:
##########
@@ -0,0 +1,141 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-formats-kafka</artifactId>
+		<version>4.1-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>flink-json-debezium</artifactId>

Review Comment:
   Gordon and I discussed offline. For the purposes of finishing off the externalization and externalizing the required confluent avro, we will table this discussion to outside of this PR since it requires careful thought due to backward compatibility. I filed https://issues.apache.org/jira/browse/FLINK-31747 to track 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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