Skip to content

Commit

Permalink
[FLINK-34440][formats] Dynamic protobuf format
Browse files Browse the repository at this point in the history
  • Loading branch information
dmariassy committed Nov 21, 2024
1 parent d7c87f2 commit d8f85de
Show file tree
Hide file tree
Showing 45 changed files with 4,798 additions and 2 deletions.
266 changes: 266 additions & 0 deletions flink-formats/flink-protobuf-confluent-registry/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,266 @@
<?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>
<artifactId>flink-formats</artifactId>
<groupId>org.apache.flink</groupId>
<version>2.0-SNAPSHOT</version>
</parent>
<build>
<plugins>
<plugin>
<groupId>com.github.os72</groupId>
<artifactId>protoc-jar-maven-plugin</artifactId>
<version>3.11.4</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>run</goal>
</goals>
<configuration>
<protocArtifact>com.google.protobuf:protoc:${protoc.version}</protocArtifact>
<includeMavenTypes>direct</includeMavenTypes>
<inputDirectories>
<include>src/test/resources/proto</include>
</inputDirectories>
<outputTargets>
<outputTarget>
<type>java</type>
<addSources>none</addSources>
<outputDirectory>target/test-proto-sources</outputDirectory>
</outputTarget>
</outputTargets>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>build-helper-maven-plugin</artifactId>
<version>3.0.0</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>add-test-source</goal>
</goals>
<configuration>
<sources>
<source>target/test-proto-sources</source>
</sources>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
<modelVersion>4.0.0</modelVersion>

<artifactId>flink-protobuf-confluent-registry</artifactId>

<name>Flink : Formats : Profobuf confluent registry</name>

<properties>
<confluent.version>7.5.3</confluent.version>
<schema.registry.version>7.5.0</schema.registry.version>
<connect.version>3.5.1</connect.version>
</properties>

<repositories>
<repository>
<id>confluent</id>
<url>https://packages.confluent.io/maven/</url>
</repository>
</repositories>

<dependencies>

<dependency>
<groupId>io.confluent</groupId>
<artifactId>kafka-protobuf-provider</artifactId>
<version>${schema.registry.version}</version>
<optional>${flink.markBundledAsOptional}</optional>
</dependency>

<dependency>
<groupId>io.confluent</groupId>
<artifactId>kafka-protobuf-types</artifactId>
<version>${schema.registry.version}</version>
<optional>${flink.markBundledAsOptional}</optional>
</dependency>

<dependency>
<groupId>io.confluent</groupId>
<artifactId>kafka-protobuf-serializer</artifactId>
<version>${schema.registry.version}</version>
<optional>${flink.markBundledAsOptional}</optional>
</dependency>

<dependency>
<groupId>com.google.protobuf</groupId>
<artifactId>protobuf-java</artifactId>
<version>${protoc.version}</version>
</dependency>

<dependency>
<groupId>io.confluent</groupId>
<artifactId>kafka-schema-registry-client</artifactId>
<version>${confluent.version}</version>
<exclusions>
<exclusion>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
</exclusion>
<exclusion>
<groupId>org.lz4</groupId>
<artifactId>lz4-java</artifactId>
</exclusion>
<exclusion>
<groupId>io.swagger</groupId>
<artifactId>swagger-core</artifactId>
</exclusion>
<exclusion>
<groupId>com.google.guava</groupId>
<artifactId>failureaccess</artifactId>
</exclusion>
<exclusion>
<groupId>com.google.guava</groupId>
<artifactId>listenablefuture</artifactId>
</exclusion>
<exclusion>
<groupId>io.swagger.core.v3</groupId>
<artifactId>swagger-annotations</artifactId>
</exclusion>
<exclusion>
<groupId>com.google.errorprone</groupId>
<artifactId>error_prone_annotations</artifactId>
</exclusion>
<exclusion>
<groupId>com.google.j2objc</groupId>
<artifactId>j2objc-annotations</artifactId>
</exclusion>
<exclusion>
<groupId>com.google.code.findbugs</groupId>
<artifactId>jsr305</artifactId>
</exclusion>
<exclusion>
<groupId>org.checkerframework</groupId>
<artifactId>checker-qual</artifactId>
</exclusion>
</exclusions>
</dependency>

<dependency>
<groupId>com.github.os72</groupId>
<artifactId>protoc-jar</artifactId>
<version>3.11.4</version>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-core</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-api-java</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-protobuf</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>

<!-- Table ecosystem and filesystem connector -->

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-common</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
<optional>true</optional>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-files</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
<optional>true</optional>
</dependency>

<!-- test dependencies -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-api-java</artifactId>
<version>${project.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>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-common</artifactId>
<version>${project.version}</version>
<scope>test</scope>
<type>test-jar</type>
</dependency>

<dependency>
<groupId>io.confluent</groupId>
<artifactId>kafka-connect-protobuf-converter</artifactId>
<version>${schema.registry.version}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>connect-api</artifactId>
<version>${connect.version}</version>
<scope>test</scope>
</dependency>
<!-- ArchUnit test dependencies -->

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-architecture-tests-test</artifactId>
<exclusions>
<exclusion>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
</exclusion>
</exclusions>
<scope>test</scope>
</dependency>
</dependencies>

</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
/*
* 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.protobuf.registry.confluent;

import org.apache.flink.table.types.logical.RowType;

public interface MutableRowTypeSchema {
/**
* Sets the row type information for the deserialization schema. While the "vanilla"
* implementations of this interface will never need to call this method, it is useful for the
* debezium format, which needs to decorate the user's row type to match the debezium payload
* format.
*
* @param rowType The row type information.
*/
void setRowType(RowType rowType);

RowType getRowType();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
/*
* 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.protobuf.registry.confluent;

import org.apache.flink.api.common.serialization.DeserializationSchema;
import org.apache.flink.table.data.RowData;

public interface ProtobufConfluentDeserializationSchema
extends DeserializationSchema<RowData>, MutableRowTypeSchema {}
Loading

0 comments on commit d8f85de

Please sign in to comment.