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



##########
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 @@
+/*

Review comment:
       It seems that the package name is not properly encoded into 
subdirectories.
   Part of the directory name of this file is 
`org.apache.flink.glue.schema.registry.test`, but it should be 
`org/apache/flink/glue/schema/registry/test`. This might be difficult to see in 
some IDEs, as they are replacing this directory structure with the dot-notation.

##########
File path: 
flink-end-to-end-tests/flink-glue-schema-registry-test/src/main/java/org.apache.flink.glue.schema.registry.test/GlueSchemaRegistryExampleTest.java
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.api.java.utils.ParameterTool;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
+
+/** Test driver for {@link GlueSchemaRegistryExample#main}. */
+public class GlueSchemaRegistryExampleTest {
+    private static final Logger LOG = 
LoggerFactory.getLogger(GlueSchemaRegistryExampleTest.class);
+
+    public static void main(String[] args) throws Exception {
+        LOG.info("System properties: {}", System.getProperties());
+        final ParameterTool parameterTool = ParameterTool.fromArgs(args);
+
+        String inputStream = parameterTool.getRequired("input-stream");
+        String outputStream = parameterTool.getRequired("output-stream");
+
+        GSRKinesisPubsubClient pubsub = new 
GSRKinesisPubsubClient(parameterTool.getProperties());
+        pubsub.createTopic(inputStream, 2, parameterTool.getProperties());
+        pubsub.createTopic(outputStream, 2, parameterTool.getProperties());
+
+        // The example job needs to start after streams are created and run in 
parallel to the
+        // validation logic.
+        // The thread that runs the job won't terminate, we don't have a job 
reference to cancel it.
+        // Once results are validated, the driver main thread will exit; 
job/cluster will be
+        // terminated from script.
+        final AtomicReference<Exception> executeException = new 
AtomicReference<>();
+        Thread executeThread =
+                new Thread(
+                        () -> {
+                            try {
+                                GlueSchemaRegistryExample.main(args);
+                                // this message won't appear in the log,
+                                // job is terminated when shutting down cluster
+                                LOG.info("executed program");
+                            } catch (Exception e) {
+                                executeException.set(e);
+                            }
+                        });
+        executeThread.start();
+
+        List<GenericRecord> messages = getRecords();
+        for (GenericRecord msg : messages) {
+            
pubsub.sendMessage(GlueSchemaRegistryExample.getSchema().toString(), 
inputStream, msg);
+        }
+        LOG.info("generated records");
+
+        Deadline deadline = Deadline.fromNow(Duration.ofSeconds(60));
+        List<Object> results = pubsub.readAllMessages(outputStream);
+        while (deadline.hasTimeLeft()
+                && executeException.get() == null
+                && results.size() < messages.size()) {
+            LOG.info("waiting for results..");
+            Thread.sleep(1000);
+            results = pubsub.readAllMessages(outputStream);
+        }
+
+        if (executeException.get() != null) {
+            throw executeException.get();
+        }
+
+        LOG.info("results: {}", results);
+        Assert.assertEquals(
+                "Results received from '" + outputStream + "': " + results,
+                messages.size(),
+                results.size());
+
+        List<GenericRecord> expectedResults = getRecords();
+
+        for (Object expectedResult : expectedResults) {
+            Assert.assertTrue(results.contains(expectedResult));
+        }
+
+        // TODO: main thread needs to create job or CLI fails with:
+        // "The program didn't contain a Flink job. Perhaps you forgot to call 
execute() on the
+        // execution environment."
+        System.out.println("test finished");

Review comment:
       use `LOG`?

##########
File path: 
flink-formats/flink-avro-glue-schema-registry/src/main/java/org/apache/flink/formats/avro/glue/schema/registry/GlueSchemaRegistryInputStreamDeserializer.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.utils.MutableByteArrayInputStream;
+
+import com.amazonaws.services.schemaregistry.deserializers.AWSDeserializer;
+import 
com.amazonaws.services.schemaregistry.exception.AWSSchemaRegistryException;
+import org.apache.avro.Schema;
+import org.apache.avro.SchemaParseException;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+
+/**
+ * AWS Glue Schema Registry input stream de-serializer to accept input stream 
and extract schema
+ * from it and remove schema registry information in the input stream.
+ */
+public class GlueSchemaRegistryInputStreamDeserializer {
+    private final AWSDeserializer awsDeserializer;
+
+    /**
+     * Constructor accepts configuration map for AWS Deserializer.
+     *
+     * @param configs configuration map
+     */
+    public GlueSchemaRegistryInputStreamDeserializer(Map<String, Object> 
configs) {
+        awsDeserializer =
+                AWSDeserializer.builder()
+                        
.credentialProvider(DefaultCredentialsProvider.builder().build())
+                        .configs(configs)
+                        .build();
+    }
+
+    public GlueSchemaRegistryInputStreamDeserializer(AWSDeserializer 
awsDeserializer) {
+        this.awsDeserializer = awsDeserializer;
+    }
+
+    /**
+     * Get schema and remove extra Schema Registry information within input 
stream.
+     *
+     * @param in input stream
+     * @return schema of object within input stream
+     * @throws IOException Exception during decompression
+     */
+    public Schema getSchemaAndDeserializedStream(InputStream in) throws 
IOException {
+        byte[] inputBytes = new byte[in.available()];
+        in.read(inputBytes);
+        in.reset();
+
+        MutableByteArrayInputStream mutableByteArrayInputStream = 
(MutableByteArrayInputStream) in;
+        String schemaDefinition = 
awsDeserializer.getSchema(inputBytes).getSchemaDefinition();
+        byte[] deserializedBytes = awsDeserializer.getActualData(inputBytes);
+        mutableByteArrayInputStream.setBuffer(deserializedBytes);
+
+        Schema schema;
+        try {
+            schema = (new Schema.Parser()).parse(schemaDefinition);

Review comment:
       if I'm not mistaken, this parser initialization and schema parsing is 
done for every `RegistryAvroDeserializationSchema.deserialize()` call.
   I guess this is necessary when deserializing GenericRecord Avro records, but 
for SpecificRecord we only need to deserialize the schema once?

##########
File path: 
flink-formats/flink-avro-glue-schema-registry/src/test/java/org/apache/flink/formats/avro/glue/schema/registry/GlueSchemaRegistryAvroSchemaCoderTest.java
##########
@@ -0,0 +1,287 @@
+/*
+ * 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 
com.amazonaws.services.schemaregistry.caching.AWSSchemaRegistrySerializerCache;
+import com.amazonaws.services.schemaregistry.common.AWSSchemaRegistryClient;
+import 
com.amazonaws.services.schemaregistry.common.configs.GlueSchemaRegistryConfiguration;
+import 
com.amazonaws.services.schemaregistry.exception.AWSSchemaRegistryException;
+import 
com.amazonaws.services.schemaregistry.serializers.GlueSchemaRegistrySerializationFacade;
+import com.amazonaws.services.schemaregistry.utils.AWSSchemaRegistryConstants;
+import org.apache.avro.Schema;
+import org.hamcrest.Matchers;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.EnumSource;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.services.glue.model.EntityNotFoundException;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.lang.reflect.Field;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyMap;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.doCallRealMethod;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;

Review comment:
       Per the "Apache Flink Code Style and Quality Guide" 
(https://flink.apache.org/contributing/code-style-and-quality-common.html) the 
use of mockito is not recommended in tests. Check the document, and also the 
slides linked there for more details.

##########
File path: flink-end-to-end-tests/test-scripts/test_glue_schema_registry.sh
##########
@@ -0,0 +1,72 @@
+#!/usr/bin/env bash

Review comment:
       I'm feeling a bit sorry that I didn't tell you beforehand, but the Flink 
community decided recently to stop adding new bash-based e2e tests: 
https://lists.apache.org/thread.html/rdc2894c67c6da3fa92f85ec2cde1d5a0c551748050431d36c13bf7a3%40%3Cdev.flink.apache.org%3E
   We have a Java e2e test framework that is going to replace the bash tests in 
the long run. See `SQLClientSchemaRegistryITCase` or some other examples there.

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

Review comment:
       why do we have to skip the maven enforcer rules?




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to