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 2021/01/28 15:52:39 UTC

[GitHub] [flink] dannycranmer commented on a change in pull request #14737: [FLINK-19667] Add AWS Glue Schema Registry integration

dannycranmer commented on a change in pull request #14737:
URL: https://github.com/apache/flink/pull/14737#discussion_r566133368



##########
File path: flink-end-to-end-tests/flink-glue-schema-registry-test/pom.xml
##########
@@ -0,0 +1,150 @@
+<?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-end-to-end-tests</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.13-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+	<modelVersion>4.0.0</modelVersion>
+
+	<artifactId>flink-glue-schema-registry-test_${scala.binary.version}</artifactId>
+	<name>Flink : E2E Tests : AWS Glue Schema Registry</name>
+	<packaging>jar</packaging>
+
+	<properties>
+		<httpclient.version>4.5.9</httpclient.version>
+		<httpcore.version>4.4.11</httpcore.version>
+		<aws.sdk.version>1.11.754</aws.sdk.version>
+		<aws.sdkv2.version>2.15.32</aws.sdkv2.version>

Review comment:
       nit: Consider updating, at the time of writing:
   - v1 @ 1.11.943
   - v2 @ 2.15.70

##########
File path: flink-end-to-end-tests/flink-glue-schema-registry-test/pom.xml
##########
@@ -0,0 +1,150 @@
+<?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-end-to-end-tests</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.13-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+	<modelVersion>4.0.0</modelVersion>
+
+	<artifactId>flink-glue-schema-registry-test_${scala.binary.version}</artifactId>
+	<name>Flink : E2E Tests : AWS Glue Schema Registry</name>
+	<packaging>jar</packaging>
+
+	<properties>
+		<httpclient.version>4.5.9</httpclient.version>
+		<httpcore.version>4.4.11</httpcore.version>
+		<aws.sdk.version>1.11.754</aws.sdk.version>
+		<aws.sdkv2.version>2.15.32</aws.sdkv2.version>
+		<enforcer.skip>true</enforcer.skip>
+	</properties>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-kinesis_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-kinesis-test_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-avro</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-avro-glue-schema-registry</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>junit</groupId>
+			<artifactId>junit</artifactId>
+			<version>${junit.version}</version>
+			<scope>compile</scope>

Review comment:
       nit: you should not need `<version>` and `<scope>` tags here. I assume you meant to include junit as a compile scoped dependency (compile is default)

##########
File path: flink-end-to-end-tests/flink-glue-schema-registry-test/pom.xml
##########
@@ -0,0 +1,150 @@
+<?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-end-to-end-tests</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.13-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+	<modelVersion>4.0.0</modelVersion>
+
+	<artifactId>flink-glue-schema-registry-test_${scala.binary.version}</artifactId>

Review comment:
       Correct me if I am wrong, but I do not think you need additional artifacts per scala version. Suggest dropping `_${scala.binary.version}`

##########
File path: flink-end-to-end-tests/flink-glue-schema-registry-test/src/main/java/org.apache.flink.glue.schema.registry.test/GSRKinesisPubsubClient.java
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.glue.schema.registry.test;
+
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.kinesis.shaded.com.amazonaws.services.kinesis.model.GetRecordsResult;
+import org.apache.flink.kinesis.shaded.com.amazonaws.services.kinesis.model.Record;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
+
+import com.amazonaws.AmazonClientException;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.EnvironmentVariableCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.services.kinesis.AmazonKinesis;
+import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder;
+import com.amazonaws.services.kinesis.model.PutRecordRequest;
+import com.amazonaws.services.kinesis.model.PutRecordResult;
+import com.amazonaws.services.kinesis.model.ResourceNotFoundException;
+import com.amazonaws.services.schemaregistry.common.AWSDeserializerInput;
+import com.amazonaws.services.schemaregistry.common.AWSSerializerInput;
+import com.amazonaws.services.schemaregistry.deserializers.AWSDeserializer;
+import com.amazonaws.services.schemaregistry.serializers.avro.AWSAvroSerializer;
+import com.amazonaws.services.schemaregistry.utils.AWSSchemaRegistryConstants;
+import com.amazonaws.services.schemaregistry.utils.AvroRecordType;
+import org.apache.avro.generic.GenericRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+
+import java.nio.ByteBuffer;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.UUID;
+
+/**
+ * Simple client to publish and retrieve messages, using the AWS Kinesis SDK, Flink Kinesis
+ * Connectors and Glue Schema Registry classes.
+ */
+public class GSRKinesisPubsubClient {
+    private static final Logger LOG = LoggerFactory.getLogger(GSRKinesisPubsubClient.class);
+
+    private final AmazonKinesis kinesisClient;
+    private final Properties properties;
+
+    public GSRKinesisPubsubClient(Properties properties) {
+        this.kinesisClient = createClientWithCredentials(properties);
+        this.properties = properties;
+    }
+
+    public void createTopic(String stream, int shards, Properties props) throws Exception {
+        try {
+            kinesisClient.describeStream(stream);
+            kinesisClient.deleteStream(stream);
+        } catch (ResourceNotFoundException rnfe) {
+            // expected when stream doesn't exist
+        }
+
+        kinesisClient.createStream(stream, shards);
+        Deadline deadline = Deadline.fromNow(Duration.ofSeconds(5));
+        while (deadline.hasTimeLeft()) {
+            try {
+                Thread.sleep(250); // sleep for a bit for stream to be created
+                if (kinesisClient.describeStream(stream).getStreamDescription().getShards().size()
+                        != shards) {
+                    // not fully created yet

Review comment:
       Consider removing these comments as per the "Golden Rule":
   - Golden rule: Comment as much as necessary to support code understanding, but don’t add redundant information.
   
   https://flink.apache.org/contributing/code-style-and-quality-common.html#comments-and-code-readability
   

##########
File path: flink-formats/flink-avro-glue-schema-registry/src/main/java/org/apache/flink/formats/avro/glue/schema/registry/GlueSchemaRegistryAvroSerializationSchema.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.avro.glue.schema.registry;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.formats.avro.RegistryAvroSerializationSchema;
+import org.apache.flink.formats.avro.SchemaCoder;
+
+import lombok.SneakyThrows;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.Encoder;
+import org.apache.avro.specific.SpecificRecord;
+
+import javax.annotation.Nullable;
+
+import java.io.ByteArrayOutputStream;
+import java.util.Map;
+
+/**
+ * AWS Glue Schema Registry Serialization schema to serialize to Avro binary format for Flink
+ * Producer user.
+ *
+ * @param <T> the type to be serialized
+ */
+public class GlueSchemaRegistryAvroSerializationSchema<T>
+        extends RegistryAvroSerializationSchema<T> {
+    /**
+     * Creates an Avro serialization schema.
+     *
+     * @param recordClazz class to serialize. Should be one of: {@link SpecificRecord}, {@link
+     *     GenericRecord}.
+     * @param reader reader's Avro schema. Should be provided if recordClazz is {@link
+     *     GenericRecord}
+     * @param schemaCoderProvider schema coder provider which reads writer schema from AWS Glue
+     *     Schema Registry
+     */
+    private GlueSchemaRegistryAvroSerializationSchema(
+            Class<T> recordClazz,
+            @Nullable Schema reader,
+            SchemaCoder.SchemaCoderProvider schemaCoderProvider) {
+        super(recordClazz, reader, schemaCoderProvider);
+    }
+
+    @VisibleForTesting
+    protected GlueSchemaRegistryAvroSerializationSchema(
+            Class<T> recordClazz, @Nullable Schema reader, SchemaCoder schemaCoder) {
+        // Pass null schema coder provider
+        super(recordClazz, reader, null);
+        this.schemaCoder = schemaCoder;
+    }
+
+    /**
+     * Creates {@link GlueSchemaRegistryAvroSerializationSchema} that serializes {@link
+     * GenericRecord} using provided schema.
+     *
+     * @param schema the schema that will be used for serialization
+     * @param transportName topic name or stream name etc.
+     * @param configs configuration map of AWS Glue Schema Registry
+     * @return serialized record in form of byte array
+     */
+    public static GlueSchemaRegistryAvroSerializationSchema<GenericRecord> forGeneric(
+            Schema schema, String transportName, Map<String, Object> configs) {
+        return new GlueSchemaRegistryAvroSerializationSchema<>(
+                GenericRecord.class,
+                schema,
+                new GlueSchemaRegistryAvroSchemaCoderProvider(transportName, configs));
+    }
+
+    /**
+     * Creates {@link GlueSchemaRegistryAvroSerializationSchema} that serializes {@link
+     * SpecificRecord} using provided schema.
+     *
+     * @param clazz the type to be serialized
+     * @param transportName topic name or stream name etc.
+     * @param configs configuration map of Amazon Schema Registry
+     * @return serialized record in form of byte array
+     */
+    public static <T extends SpecificRecord>
+            GlueSchemaRegistryAvroSerializationSchema<T> forSpecific(
+                    Class<T> clazz, String transportName, Map<String, Object> configs) {
+        return new GlueSchemaRegistryAvroSerializationSchema<>(
+                clazz, null, new GlueSchemaRegistryAvroSchemaCoderProvider(transportName, configs));
+    }
+
+    /**
+     * Serializes the incoming element to a byte array containing bytes of AWS Glue Schema registry
+     * information.
+     *
+     * @param object The incoming element to be serialized
+     * @return The serialized bytes.
+     */
+    @SneakyThrows
+    @Override
+    public byte[] serialize(T object) {
+        checkAvroInitialized();
+
+        if (object == null) {
+            return null;
+        } else {
+            ByteArrayOutputStream outputStream = getOutputStream();
+            outputStream.reset();
+            Encoder encoder = getEncoder();
+            getDatumWriter().write(object, encoder);
+            schemaCoder.writeSchema(getSchema(), outputStream);
+            encoder.flush();
+
+            return outputStream.toByteArray();
+        }
+    }

Review comment:
       Is there a reason to override this? It looks almost identical to the version in `RegistryAvroSerializationSchema`

##########
File path: flink-formats/flink-avro-glue-schema-registry/pom.xml
##########
@@ -0,0 +1,99 @@
+<?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>1.13-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+	<modelVersion>4.0.0</modelVersion>
+
+	<artifactId>flink-avro-glue-schema-registry</artifactId>
+	<name>Flink : Formats : Avro AWS Glue Schema Registry</name>
+	<packaging>jar</packaging>
+
+	<properties>
+		<glue.schema.registry.version>1.0.0</glue.schema.registry.version>
+		<junit.jupiter.version>5.6.2</junit.jupiter.version>
+		<enforcer.skip>true</enforcer.skip>
+	</properties>
+
+	<dependencies>
+		<!-- core dependencies -->
+
+		<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-avro</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-clients_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+		</dependency>

Review comment:
       Is this actually required? Module still builds without it, usually provided by user application if required.

##########
File path: flink-end-to-end-tests/flink-glue-schema-registry-test/src/main/java/org.apache.flink.glue.schema.registry.test/GSRKinesisPubsubClient.java
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.glue.schema.registry.test;
+
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.kinesis.shaded.com.amazonaws.services.kinesis.model.GetRecordsResult;
+import org.apache.flink.kinesis.shaded.com.amazonaws.services.kinesis.model.Record;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
+
+import com.amazonaws.AmazonClientException;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.EnvironmentVariableCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.services.kinesis.AmazonKinesis;
+import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder;
+import com.amazonaws.services.kinesis.model.PutRecordRequest;
+import com.amazonaws.services.kinesis.model.PutRecordResult;
+import com.amazonaws.services.kinesis.model.ResourceNotFoundException;
+import com.amazonaws.services.schemaregistry.common.AWSDeserializerInput;
+import com.amazonaws.services.schemaregistry.common.AWSSerializerInput;
+import com.amazonaws.services.schemaregistry.deserializers.AWSDeserializer;
+import com.amazonaws.services.schemaregistry.serializers.avro.AWSAvroSerializer;
+import com.amazonaws.services.schemaregistry.utils.AWSSchemaRegistryConstants;
+import com.amazonaws.services.schemaregistry.utils.AvroRecordType;
+import org.apache.avro.generic.GenericRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+
+import java.nio.ByteBuffer;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.UUID;
+
+/**
+ * Simple client to publish and retrieve messages, using the AWS Kinesis SDK, Flink Kinesis
+ * Connectors and Glue Schema Registry classes.
+ */
+public class GSRKinesisPubsubClient {
+    private static final Logger LOG = LoggerFactory.getLogger(GSRKinesisPubsubClient.class);
+
+    private final AmazonKinesis kinesisClient;
+    private final Properties properties;
+
+    public GSRKinesisPubsubClient(Properties properties) {
+        this.kinesisClient = createClientWithCredentials(properties);
+        this.properties = properties;
+    }
+
+    public void createTopic(String stream, int shards, Properties props) throws Exception {
+        try {
+            kinesisClient.describeStream(stream);

Review comment:
       Why do you need to describeStream here?

##########
File path: flink-formats/flink-avro-glue-schema-registry/src/main/java/org/apache/flink/formats/avro/glue/schema/registry/GlueSchemaRegistryAvroSchemaCoder.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.avro.glue.schema.registry;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.formats.avro.SchemaCoder;
+
+import org.apache.avro.Schema;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Map;
+
+/**
+ * Schema coder that allows reading schema that is somehow embedded into serialized record. Used by
+ * {@link GlueSchemaRegistryAvroDeserializationSchema} and {@link
+ * GlueSchemaRegistryAvroSerializationSchema}.
+ */
+public class GlueSchemaRegistryAvroSchemaCoder implements SchemaCoder {
+    private GlueSchemaRegistryInputStreamDeserializer glueSchemaRegistryInputStreamDeserializer;
+    private GlueSchemaRegistryOutputStreamSerializer glueSchemaRegistryOutputStreamSerializer;
+
+    /**
+     * Constructor accepts transport name and configuration map for AWS Glue Schema Registry.
+     *
+     * @param transportName topic name or stream name etc.
+     * @param configs configurations for AWS Glue Schema Registry
+     */
+    public GlueSchemaRegistryAvroSchemaCoder(
+            final String transportName, final Map<String, Object> configs) {
+        glueSchemaRegistryInputStreamDeserializer =
+                new GlueSchemaRegistryInputStreamDeserializer(configs);
+        glueSchemaRegistryOutputStreamSerializer =
+                new GlueSchemaRegistryOutputStreamSerializer(transportName, configs);
+    }
+
+    @VisibleForTesting
+    protected GlueSchemaRegistryAvroSchemaCoder(
+            final GlueSchemaRegistryInputStreamDeserializer
+                    glueSchemaRegistryInputStreamDeserializer) {
+        this.glueSchemaRegistryInputStreamDeserializer = glueSchemaRegistryInputStreamDeserializer;
+    }
+
+    @VisibleForTesting
+    protected GlueSchemaRegistryAvroSchemaCoder(
+            final GlueSchemaRegistryOutputStreamSerializer
+                    glueSchemaRegistryOutputStreamSerializer) {
+        this.glueSchemaRegistryOutputStreamSerializer = glueSchemaRegistryOutputStreamSerializer;
+    }
+
+    @Override
+    public Schema readSchema(InputStream in) throws IOException {
+        return glueSchemaRegistryInputStreamDeserializer.getSchemaAndDeserializedStream(in);
+    }
+
+    @Override
+    public void writeSchema(Schema schema, OutputStream out) throws IOException {
+        byte[] data = ((ByteArrayOutputStream) out).toByteArray();

Review comment:
       Are you sure this will always be a `ByteArrayOutputStream`? Since this is a public method suggest a check:
   - `Preconditions.checkArgument(out instanceof ByteArrayOutputStream, "..");`

##########
File path: flink-formats/flink-avro-glue-schema-registry/pom.xml
##########
@@ -0,0 +1,99 @@
+<?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>1.13-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+	<modelVersion>4.0.0</modelVersion>
+
+	<artifactId>flink-avro-glue-schema-registry</artifactId>
+	<name>Flink : Formats : Avro AWS Glue Schema Registry</name>
+	<packaging>jar</packaging>
+
+	<properties>
+		<glue.schema.registry.version>1.0.0</glue.schema.registry.version>
+		<junit.jupiter.version>5.6.2</junit.jupiter.version>
+		<enforcer.skip>true</enforcer.skip>
+	</properties>
+
+	<dependencies>
+		<!-- core dependencies -->
+
+		<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-avro</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-clients_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>software.amazon.glue</groupId>
+			<artifactId>schema-registry-serde</artifactId>
+			<version>${glue.schema.registry.version}</version>
+		</dependency>
+
+		<!-- test dependencies -->
+
+		<dependency>
+			<groupId>org.junit.jupiter</groupId>
+			<artifactId>junit-jupiter-api</artifactId>
+			<version>${junit.jupiter.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.junit.jupiter</groupId>
+			<artifactId>junit-jupiter-params</artifactId>
+			<version>${junit.jupiter.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mockito</groupId>
+			<artifactId>mockito-junit-jupiter</artifactId>
+			<version>${mockito.version}</version>
+			<scope>test</scope>
+		</dependency>

Review comment:
       Why are we not using the `junit` version provided by flink? Is there a reason to upgrade a single module? Are you sure results and reports would aggregate correctly on CI? I am worried we might miss or break something by using a different test runner for this module.

##########
File path: flink-end-to-end-tests/flink-glue-schema-registry-test/src/main/java/org.apache.flink.glue.schema.registry.test/GSRKinesisPubsubClient.java
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.glue.schema.registry.test;
+
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.kinesis.shaded.com.amazonaws.services.kinesis.model.GetRecordsResult;
+import org.apache.flink.kinesis.shaded.com.amazonaws.services.kinesis.model.Record;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
+
+import com.amazonaws.AmazonClientException;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.EnvironmentVariableCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.services.kinesis.AmazonKinesis;
+import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder;
+import com.amazonaws.services.kinesis.model.PutRecordRequest;
+import com.amazonaws.services.kinesis.model.PutRecordResult;
+import com.amazonaws.services.kinesis.model.ResourceNotFoundException;
+import com.amazonaws.services.schemaregistry.common.AWSDeserializerInput;
+import com.amazonaws.services.schemaregistry.common.AWSSerializerInput;
+import com.amazonaws.services.schemaregistry.deserializers.AWSDeserializer;
+import com.amazonaws.services.schemaregistry.serializers.avro.AWSAvroSerializer;
+import com.amazonaws.services.schemaregistry.utils.AWSSchemaRegistryConstants;
+import com.amazonaws.services.schemaregistry.utils.AvroRecordType;
+import org.apache.avro.generic.GenericRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+
+import java.nio.ByteBuffer;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.UUID;
+
+/**
+ * Simple client to publish and retrieve messages, using the AWS Kinesis SDK, Flink Kinesis
+ * Connectors and Glue Schema Registry classes.
+ */
+public class GSRKinesisPubsubClient {
+    private static final Logger LOG = LoggerFactory.getLogger(GSRKinesisPubsubClient.class);
+
+    private final AmazonKinesis kinesisClient;
+    private final Properties properties;
+
+    public GSRKinesisPubsubClient(Properties properties) {
+        this.kinesisClient = createClientWithCredentials(properties);
+        this.properties = properties;
+    }
+
+    public void createTopic(String stream, int shards, Properties props) throws Exception {

Review comment:
       `createStream`? I would consider updating the method name or add Javadoc to indicate that an existing stream will be deleted

##########
File path: flink-end-to-end-tests/flink-glue-schema-registry-test/pom.xml
##########
@@ -0,0 +1,150 @@
+<?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-end-to-end-tests</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.13-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+	<modelVersion>4.0.0</modelVersion>
+
+	<artifactId>flink-glue-schema-registry-test_${scala.binary.version}</artifactId>
+	<name>Flink : E2E Tests : AWS Glue Schema Registry</name>
+	<packaging>jar</packaging>
+
+	<properties>
+		<httpclient.version>4.5.9</httpclient.version>
+		<httpcore.version>4.4.11</httpcore.version>
+		<aws.sdk.version>1.11.754</aws.sdk.version>
+		<aws.sdkv2.version>2.15.32</aws.sdkv2.version>
+		<enforcer.skip>true</enforcer.skip>
+	</properties>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-kinesis_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-kinesis-test_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-avro</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-avro-glue-schema-registry</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>junit</groupId>
+			<artifactId>junit</artifactId>
+			<version>${junit.version}</version>
+			<scope>compile</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-kinesis_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>compile</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.httpcomponents</groupId>
+			<artifactId>httpclient</artifactId>
+			<version>${httpclient.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.httpcomponents</groupId>
+			<artifactId>httpcore</artifactId>
+			<version>${httpcore.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.httpcomponents</groupId>
+			<artifactId>httpcore</artifactId>
+			<version>${httpcore.version}</version>
+		</dependency>

Review comment:
       Duplicate dependency

##########
File path: flink-formats/flink-avro-glue-schema-registry/src/main/java/org/apache/flink/formats/avro/glue/schema/registry/GlueSchemaRegistryAvroSerializationSchema.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.avro.glue.schema.registry;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.formats.avro.RegistryAvroSerializationSchema;
+import org.apache.flink.formats.avro.SchemaCoder;
+
+import lombok.SneakyThrows;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.Encoder;
+import org.apache.avro.specific.SpecificRecord;
+
+import javax.annotation.Nullable;
+
+import java.io.ByteArrayOutputStream;
+import java.util.Map;
+
+/**
+ * AWS Glue Schema Registry Serialization schema to serialize to Avro binary format for Flink
+ * Producer user.
+ *
+ * @param <T> the type to be serialized
+ */
+public class GlueSchemaRegistryAvroSerializationSchema<T>
+        extends RegistryAvroSerializationSchema<T> {
+    /**
+     * Creates an Avro serialization schema.
+     *
+     * @param recordClazz class to serialize. Should be one of: {@link SpecificRecord}, {@link
+     *     GenericRecord}.
+     * @param reader reader's Avro schema. Should be provided if recordClazz is {@link
+     *     GenericRecord}
+     * @param schemaCoderProvider schema coder provider which reads writer schema from AWS Glue
+     *     Schema Registry
+     */
+    private GlueSchemaRegistryAvroSerializationSchema(
+            Class<T> recordClazz,
+            @Nullable Schema reader,
+            SchemaCoder.SchemaCoderProvider schemaCoderProvider) {
+        super(recordClazz, reader, schemaCoderProvider);
+    }
+
+    @VisibleForTesting
+    protected GlueSchemaRegistryAvroSerializationSchema(
+            Class<T> recordClazz, @Nullable Schema reader, SchemaCoder schemaCoder) {
+        // Pass null schema coder provider
+        super(recordClazz, reader, null);
+        this.schemaCoder = schemaCoder;
+    }
+
+    /**
+     * Creates {@link GlueSchemaRegistryAvroSerializationSchema} that serializes {@link
+     * GenericRecord} using provided schema.
+     *
+     * @param schema the schema that will be used for serialization
+     * @param transportName topic name or stream name etc.
+     * @param configs configuration map of AWS Glue Schema Registry
+     * @return serialized record in form of byte array
+     */
+    public static GlueSchemaRegistryAvroSerializationSchema<GenericRecord> forGeneric(
+            Schema schema, String transportName, Map<String, Object> configs) {
+        return new GlueSchemaRegistryAvroSerializationSchema<>(
+                GenericRecord.class,
+                schema,
+                new GlueSchemaRegistryAvroSchemaCoderProvider(transportName, configs));
+    }
+
+    /**
+     * Creates {@link GlueSchemaRegistryAvroSerializationSchema} that serializes {@link
+     * SpecificRecord} using provided schema.
+     *
+     * @param clazz the type to be serialized
+     * @param transportName topic name or stream name etc.
+     * @param configs configuration map of Amazon Schema Registry
+     * @return serialized record in form of byte array
+     */
+    public static <T extends SpecificRecord>
+            GlueSchemaRegistryAvroSerializationSchema<T> forSpecific(
+                    Class<T> clazz, String transportName, Map<String, Object> configs) {
+        return new GlueSchemaRegistryAvroSerializationSchema<>(
+                clazz, null, new GlueSchemaRegistryAvroSchemaCoderProvider(transportName, configs));
+    }
+
+    /**
+     * Serializes the incoming element to a byte array containing bytes of AWS Glue Schema registry
+     * information.
+     *
+     * @param object The incoming element to be serialized
+     * @return The serialized bytes.
+     */
+    @SneakyThrows

Review comment:
       See parent implementation. Wrap or declare the exception rather than sneaky throws. 

##########
File path: flink-formats/flink-avro-glue-schema-registry/src/main/java/org/apache/flink/formats/avro/glue/schema/registry/GlueSchemaRegistryAvroSchemaCoderProvider.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.avro.glue.schema.registry;
+
+import org.apache.flink.formats.avro.SchemaCoder;
+
+import lombok.NonNull;

Review comment:
       I think wrong annotation 

##########
File path: flink-formats/flink-avro-glue-schema-registry/pom.xml
##########
@@ -0,0 +1,99 @@
+<?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>1.13-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+	<modelVersion>4.0.0</modelVersion>
+
+	<artifactId>flink-avro-glue-schema-registry</artifactId>
+	<name>Flink : Formats : Avro AWS Glue Schema Registry</name>
+	<packaging>jar</packaging>
+
+	<properties>
+		<glue.schema.registry.version>1.0.0</glue.schema.registry.version>
+		<junit.jupiter.version>5.6.2</junit.jupiter.version>
+		<enforcer.skip>true</enforcer.skip>
+	</properties>
+
+	<dependencies>
+		<!-- core dependencies -->
+
+		<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-avro</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+		</dependency>

Review comment:
       This should probably be `<scope>provided</scope>`

##########
File path: flink-formats/flink-avro-glue-schema-registry/pom.xml
##########
@@ -0,0 +1,99 @@
+<?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>1.13-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+	<modelVersion>4.0.0</modelVersion>
+
+	<artifactId>flink-avro-glue-schema-registry</artifactId>
+	<name>Flink : Formats : Avro AWS Glue Schema Registry</name>
+	<packaging>jar</packaging>
+
+	<properties>
+		<glue.schema.registry.version>1.0.0</glue.schema.registry.version>
+		<junit.jupiter.version>5.6.2</junit.jupiter.version>
+		<enforcer.skip>true</enforcer.skip>
+	</properties>
+
+	<dependencies>

Review comment:
       I have taken a look at the dependency footprint of this module and it looks like there is too much pulled in:
   
   Why do we need Kafka dependencies?
   - `+- org.apache.kafka:connect-json:jar:2.5.0:compile`
   - `+- org.apache.kafka:connect-api:jar:2.5.0:compile`
   - `+- org.apache.kafka:kafka-streams:jar:2.5.0:compile`
   - `+- org.apache.kafka:kafka-clients:jar:2.5.0:compile`
   
   Pulling lombok as a compile dependency looks wrong, is this scoped correctly in upstream module?
   - `+- org.projectlombok:lombok:jar:1.18.2:compile`
   - `\- org.projectlombok:lombok-utils:jar:1.18.12:compile`
   
   As mentioned, we should use the standard junit framework for flink:
   - `+- org.junit.jupiter:junit-jupiter-api:jar:5.6.2:test`
   
   ```
   [INFO] --- maven-dependency-plugin:3.1.1:tree (default-cli) @ flink-avro-glue-schema-registry ---
   [INFO] org.apache.flink:flink-avro-glue-schema-registry:jar:1.13-SNAPSHOT
   [INFO] +- org.apache.flink:flink-core:jar:1.13-SNAPSHOT:provided
   [INFO] |  +- org.apache.flink:flink-annotations:jar:1.13-SNAPSHOT:provided
   [INFO] |  +- org.apache.flink:flink-metrics-core:jar:1.13-SNAPSHOT:provided
   [INFO] |  +- org.apache.flink:flink-shaded-asm-7:jar:7.1-12.0:provided
   [INFO] |  +- org.apache.commons:commons-lang3:jar:3.3.2:compile
   [INFO] |  +- com.esotericsoftware.kryo:kryo:jar:2.24.0:provided
   [INFO] |  |  \- com.esotericsoftware.minlog:minlog:jar:1.2:provided
   [INFO] |  +- commons-collections:commons-collections:jar:3.2.2:provided
   [INFO] |  +- org.apache.commons:commons-compress:jar:1.20:compile
   [INFO] |  \- org.apache.flink:flink-shaded-guava:jar:18.0-12.0:compile
   [INFO] +- org.apache.flink:flink-avro:jar:1.13-SNAPSHOT:compile
   [INFO] |  \- org.apache.avro:avro:jar:1.10.0:compile
   [INFO] |     +- com.fasterxml.jackson.core:jackson-core:jar:2.12.1:compile
   [INFO] |     \- com.fasterxml.jackson.core:jackson-databind:jar:2.12.1:compile
   [INFO] |        \- com.fasterxml.jackson.core:jackson-annotations:jar:2.12.1:compile
   [INFO] +- org.apache.flink:flink-streaming-java_2.11:jar:1.13-SNAPSHOT:compile
   [INFO] |  +- org.apache.flink:flink-file-sink-common:jar:1.13-SNAPSHOT:compile
   [INFO] |  +- org.apache.flink:flink-runtime_2.11:jar:1.13-SNAPSHOT:compile
   [INFO] |  |  +- org.apache.flink:flink-queryable-state-client-java:jar:1.13-SNAPSHOT:compile
   [INFO] |  |  +- org.apache.flink:flink-hadoop-fs:jar:1.13-SNAPSHOT:compile
   [INFO] |  |  +- commons-io:commons-io:jar:2.7:compile
   [INFO] |  |  +- org.apache.flink:flink-shaded-netty:jar:4.1.49.Final-12.0:compile
   [INFO] |  |  +- org.apache.flink:flink-shaded-jackson:jar:2.10.1-12.0:compile
   [INFO] |  |  +- org.apache.flink:flink-shaded-zookeeper-3:jar:3.4.14-12.0:compile
   [INFO] |  |  +- org.javassist:javassist:jar:3.24.0-GA:compile
   [INFO] |  |  +- org.scala-lang:scala-library:jar:2.11.12:compile
   [INFO] |  |  +- com.typesafe.akka:akka-actor_2.11:jar:2.5.21:compile
   [INFO] |  |  |  +- com.typesafe:config:jar:1.3.0:compile
   [INFO] |  |  |  \- org.scala-lang.modules:scala-java8-compat_2.11:jar:0.7.0:compile
   [INFO] |  |  +- com.typesafe.akka:akka-stream_2.11:jar:2.5.21:compile
   [INFO] |  |  |  +- org.reactivestreams:reactive-streams:jar:1.0.2:compile
   [INFO] |  |  |  \- com.typesafe:ssl-config-core_2.11:jar:0.3.7:compile
   [INFO] |  |  |     \- org.scala-lang.modules:scala-parser-combinators_2.11:jar:1.1.1:compile
   [INFO] |  |  +- com.typesafe.akka:akka-protobuf_2.11:jar:2.5.21:compile
   [INFO] |  |  +- com.typesafe.akka:akka-slf4j_2.11:jar:2.5.21:compile
   [INFO] |  |  +- org.clapper:grizzled-slf4j_2.11:jar:1.3.2:compile
   [INFO] |  |  +- com.github.scopt:scopt_2.11:jar:3.5.0:compile
   [INFO] |  |  +- org.xerial.snappy:snappy-java:jar:1.1.4:compile
   [INFO] |  |  +- com.twitter:chill_2.11:jar:0.7.6:compile
   [INFO] |  |  |  \- com.twitter:chill-java:jar:0.7.6:compile
   [INFO] |  |  \- org.lz4:lz4-java:jar:1.6.0:compile
   [INFO] |  +- org.apache.flink:flink-java:jar:1.13-SNAPSHOT:compile
   [INFO] |  \- org.apache.commons:commons-math3:jar:3.5:compile
   [INFO] +- org.apache.flink:flink-clients_2.11:jar:1.13-SNAPSHOT:compile
   [INFO] |  +- org.apache.flink:flink-optimizer_2.11:jar:1.13-SNAPSHOT:compile
   [INFO] |  \- commons-cli:commons-cli:jar:1.3.1:compile
   [INFO] +- software.amazon.glue:schema-registry-serde:jar:1.0.0:compile
   [INFO] |  +- software.amazon.glue:schema-registry-common:jar:1.0.0:compile
   [INFO] |  |  +- software.amazon.awssdk:glue:jar:2.15.32:compile
   [INFO] |  |  |  +- software.amazon.awssdk:protocol-core:jar:2.15.32:compile
   [INFO] |  |  |  +- software.amazon.awssdk:auth:jar:2.15.32:compile
   [INFO] |  |  |  |  \- software.amazon.eventstream:eventstream:jar:1.0.1:compile
   [INFO] |  |  |  +- software.amazon.awssdk:http-client-spi:jar:2.15.32:compile
   [INFO] |  |  |  +- software.amazon.awssdk:regions:jar:2.15.32:compile
   [INFO] |  |  |  +- software.amazon.awssdk:aws-core:jar:2.15.32:compile
   [INFO] |  |  |  +- software.amazon.awssdk:metrics-spi:jar:2.15.32:compile
   [INFO] |  |  |  +- software.amazon.awssdk:apache-client:jar:2.15.32:runtime
   [INFO] |  |  |  |  +- org.apache.httpcomponents:httpclient:jar:4.5.3:runtime
   [INFO] |  |  |  |  |  +- commons-logging:commons-logging:jar:1.1.3:runtime
   [INFO] |  |  |  |  |  \- commons-codec:commons-codec:jar:1.13:runtime
   [INFO] |  |  |  |  \- org.apache.httpcomponents:httpcore:jar:4.4.6:runtime
   [INFO] |  |  |  \- software.amazon.awssdk:netty-nio-client:jar:2.15.32:runtime
   [INFO] |  |  |     +- io.netty:netty-codec-http:jar:4.1.53.Final:runtime
   [INFO] |  |  |     +- io.netty:netty-codec-http2:jar:4.1.53.Final:runtime
   [INFO] |  |  |     +- io.netty:netty-codec:jar:4.1.53.Final:runtime
   [INFO] |  |  |     +- io.netty:netty-transport:jar:4.1.53.Final:runtime
   [INFO] |  |  |     |  \- io.netty:netty-resolver:jar:4.1.53.Final:runtime
   [INFO] |  |  |     +- io.netty:netty-common:jar:4.1.53.Final:runtime
   [INFO] |  |  |     +- io.netty:netty-buffer:jar:4.1.53.Final:runtime
   [INFO] |  |  |     +- io.netty:netty-handler:jar:4.1.53.Final:runtime
   [INFO] |  |  |     +- io.netty:netty-transport-native-epoll:jar:linux-x86_64:4.1.53.Final:runtime
   [INFO] |  |  |     |  \- io.netty:netty-transport-native-unix-common:jar:4.1.53.Final:runtime
   [INFO] |  |  |     \- com.typesafe.netty:netty-reactive-streams-http:jar:2.0.4:runtime
   [INFO] |  |  |        \- com.typesafe.netty:netty-reactive-streams:jar:2.0.4:runtime
   [INFO] |  |  +- software.amazon.awssdk:aws-json-protocol:jar:2.15.30:compile
   [INFO] |  |  +- software.amazon.awssdk:cloudwatch:jar:2.15.30:compile
   [INFO] |  |  |  \- software.amazon.awssdk:aws-query-protocol:jar:2.15.30:compile
   [INFO] |  |  +- software.amazon.awssdk:sdk-core:jar:2.15.30:compile
   [INFO] |  |  |  \- software.amazon.awssdk:profiles:jar:2.15.30:compile
   [INFO] |  |  +- org.apache.kafka:kafka-clients:jar:2.5.0:compile
   [INFO] |  |  |  \- com.github.luben:zstd-jni:jar:1.4.4-7:compile
   [INFO] |  |  +- org.apache.kafka:kafka-streams:jar:2.5.0:compile
   [INFO] |  |  |  +- org.apache.kafka:connect-json:jar:2.5.0:compile
   [INFO] |  |  |  |  +- org.apache.kafka:connect-api:jar:2.5.0:compile
   [INFO] |  |  |  |  \- com.fasterxml.jackson.datatype:jackson-datatype-jdk8:jar:2.12.1:compile
   [INFO] |  |  |  \- org.rocksdb:rocksdbjni:jar:5.18.3:compile
   [INFO] |  |  \- com.google.guava:guava:jar:29.0-jre:compile
   [INFO] |  |     +- com.google.guava:failureaccess:jar:1.0.1:compile
   [INFO] |  |     +- com.google.guava:listenablefuture:jar:9999.0-empty-to-avoid-conflict-with-guava:compile
   [INFO] |  |     +- org.checkerframework:checker-qual:jar:2.11.1:compile
   [INFO] |  |     +- com.google.errorprone:error_prone_annotations:jar:2.3.4:compile
   [INFO] |  |     \- com.google.j2objc:j2objc-annotations:jar:1.3:compile
   [INFO] |  +- software.amazon.awssdk:arns:jar:2.15.26:compile
   [INFO] |  |  +- software.amazon.awssdk:annotations:jar:2.15.26:compile
   [INFO] |  |  \- software.amazon.awssdk:utils:jar:2.15.26:compile
   [INFO] |  +- org.projectlombok:lombok:jar:1.18.2:compile
   [INFO] |  \- org.projectlombok:lombok-utils:jar:1.18.12:compile
   [INFO] +- org.junit.jupiter:junit-jupiter-api:jar:5.6.2:test
   [INFO] |  +- org.apiguardian:apiguardian-api:jar:1.1.0:test
   [INFO] |  +- org.opentest4j:opentest4j:jar:1.2.0:test
   [INFO] |  \- org.junit.platform:junit-platform-commons:jar:1.6.2:test
   [INFO] +- org.junit.jupiter:junit-jupiter-params:jar:5.6.2:test
   [INFO] +- org.mockito:mockito-junit-jupiter:jar:2.21.0:test
   [INFO] +- org.slf4j:slf4j-api:jar:1.7.15:provided
   [INFO] +- org.apache.flink:flink-test-utils-junit:jar:1.13-SNAPSHOT:test
   [INFO] +- org.apache.flink:force-shading:jar:1.13-SNAPSHOT:compile
   [INFO] +- com.google.code.findbugs:jsr305:jar:1.3.9:compile
   [INFO] +- junit:junit:jar:4.12:test
   [INFO] |  \- org.hamcrest:hamcrest-core:jar:1.3:test
   [INFO] +- org.mockito:mockito-core:jar:2.21.0:test
   [INFO] |  +- net.bytebuddy:byte-buddy:jar:1.8.15:test
   [INFO] |  +- net.bytebuddy:byte-buddy-agent:jar:1.8.15:test
   [INFO] |  \- org.objenesis:objenesis:jar:2.1:provided
   [INFO] +- org.powermock:powermock-module-junit4:jar:2.0.4:test
   [INFO] |  \- org.powermock:powermock-module-junit4-common:jar:2.0.4:test
   [INFO] |     +- org.powermock:powermock-reflect:jar:2.0.4:test
   [INFO] |     \- org.powermock:powermock-core:jar:2.0.4:test
   [INFO] +- org.powermock:powermock-api-mockito2:jar:2.0.4:test
   [INFO] |  \- org.powermock:powermock-api-support:jar:2.0.4:test
   [INFO] +- org.hamcrest:hamcrest-all:jar:1.3:test
   [INFO] +- org.apache.logging.log4j:log4j-slf4j-impl:jar:2.12.1:test
   [INFO] +- org.apache.logging.log4j:log4j-api:jar:2.12.1:test
   [INFO] +- org.apache.logging.log4j:log4j-core:jar:2.12.1:test
   [INFO] \- org.apache.logging.log4j:log4j-1.2-api:jar:2.12.1: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.

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