You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2021/02/04 13:32:55 UTC

[GitHub] [pulsar] lhotari commented on a change in pull request #9448: Pulsar IO - KafkaSource - allow to manage Avro Encoded messages

lhotari commented on a change in pull request #9448:
URL: https://github.com/apache/pulsar/pull/9448#discussion_r570214378



##########
File path: pom.xml
##########
@@ -152,6 +152,7 @@ flexible messaging model and an intuitive client API.</description>
     <jcip.version>1.0</jcip.version>
     <prometheus-jmx.version>0.14.0</prometheus-jmx.version>
     <confluent.version>5.3.2</confluent.version>
+    <kafka.confluent.schemaregstryclient.version>3.3.1</kafka.confluent.schemaregstryclient.version>

Review comment:
       typo (regstry -> registry)

##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/generic/GenericAvroWriter.java
##########
@@ -52,4 +67,29 @@ public GenericAvroWriter(Schema schema) {
             this.byteArrayOutputStream.reset();
         }
     }
+
+    /**
+     * This is an adapter from Pulsar GenericRecord to Avro classes.
+     */
+    private class GenericRecordAdapter extends SpecificRecordBase {

Review comment:
       shouldn't this be a `private static class` ?

##########
File path: tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/AvroKafkaSourceTest.java
##########
@@ -0,0 +1,496 @@
+/**
+ * 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.pulsar.tests.integration.io;
+
+import com.google.gson.Gson;
+import lombok.Cleanup;
+import lombok.Data;
+import lombok.extern.slf4j.Slf4j;
+import net.jodah.failsafe.Failsafe;
+import net.jodah.failsafe.RetryPolicy;
+import org.apache.avro.io.DatumWriter;
+import org.apache.avro.io.EncoderFactory;
+import org.apache.avro.io.JsonEncoder;
+import org.apache.avro.reflect.ReflectData;
+import org.apache.avro.reflect.ReflectDatumWriter;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.schema.Field;
+import org.apache.pulsar.client.api.schema.GenericRecord;
+import org.apache.pulsar.tests.integration.docker.ContainerExecException;
+import org.apache.pulsar.tests.integration.docker.ContainerExecResult;
+import org.apache.pulsar.tests.integration.functions.PulsarFunctionsTestBase;
+import org.apache.pulsar.tests.integration.topologies.PulsarCluster;
+import org.testcontainers.containers.Container.ExecResult;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.KafkaContainer;
+import org.testcontainers.containers.wait.strategy.Wait;
+import org.testcontainers.images.builder.Transferable;
+import org.testcontainers.shaded.com.google.common.collect.ImmutableMap;
+import org.testcontainers.utility.DockerImageName;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.time.Duration;
+import java.util.*;
+import java.util.concurrent.TimeUnit;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.policies.data.SourceStatus;
+
+import static org.testng.Assert.*;
+
+/**
+ * A tester for testing kafka source with Avro Messages.
+ * This test starts a PulsarCluster, a container with a Kafka Broker
+ * and a container with the SchemaRegistry.
+ * It populates a Kafka topic with Avro encoded messages with schema
+ * and then it verifies that the records are correclty received
+ * but a Pulsar Consumer
+ */
+@Slf4j
+public class AvroKafkaSourceTest extends PulsarFunctionsTestBase {
+
+    private static final String SOURCE_TYPE = "kafka";
+
+    final Duration ONE_MINUTE = Duration.ofMinutes(1);
+    final Duration TEN_SECONDS = Duration.ofSeconds(10);
+
+    final RetryPolicy statusRetryPolicy = new RetryPolicy()
+            .withMaxDuration(ONE_MINUTE)
+            .withDelay(TEN_SECONDS)
+            .onRetry(e -> log.error("Retry ... "));
+
+    private final String kafkaTopicName = "kafkasourcetopic";
+
+    private EnhancedKafkaContainer kafkaContainer;
+    private SchemaRegistryContainer schemaRegistryContainer;
+
+    protected final Map<String, Object> sourceConfig;
+    protected final String kafkaContainerName = "kafkacontainer";
+    protected final String schemaRegistryContainerName = "schemaregistry";
+
+    public AvroKafkaSourceTest() {
+        sourceConfig = new HashMap<>();
+    }
+
+    @Test(groups = "source")
+    public void test() throws Exception {
+        if (pulsarCluster == null) {
+            super.setupCluster();
+            super.setupFunctionWorkers();
+        }
+        startKafkaContainers(pulsarCluster);
+        try {
+            testSource();
+        } finally {
+            stopKafkaContainers(pulsarCluster);
+        }
+    }
+
+    private String getBootstrapServersOnDockerNetwork() {
+        return kafkaContainerName + ":9093";
+    }
+
+
+    public void startKafkaContainers(PulsarCluster cluster) throws Exception {
+        this.kafkaContainer = createKafkaContainer(cluster);
+        cluster.startService(kafkaContainerName, kafkaContainer);
+        log.info("creating schema registry kafka {}",  getBootstrapServersOnDockerNetwork());
+        this.schemaRegistryContainer = new SchemaRegistryContainer(getBootstrapServersOnDockerNetwork());
+        cluster.startService(schemaRegistryContainerName, schemaRegistryContainer);
+        sourceConfig.put("bootstrapServers", getBootstrapServersOnDockerNetwork());
+        sourceConfig.put("groupId", "test-source-group");
+        sourceConfig.put("fetchMinBytes", 1L);
+        sourceConfig.put("autoCommitIntervalMs", 10L);
+        sourceConfig.put("sessionTimeoutMs", 10000L);
+        sourceConfig.put("heartbeatIntervalMs", 5000L);
+        sourceConfig.put("topic", kafkaTopicName);
+        sourceConfig.put("consumerConfigProperties",
+                ImmutableMap.of("schema.registry.url", getRegistryAddressInDockerNetwork())
+        );
+    }
+
+    private class EnhancedKafkaContainer extends KafkaContainer {
+
+        public EnhancedKafkaContainer(DockerImageName dockerImageName) {
+            super(dockerImageName);
+        }
+
+        @Override
+        public String getBootstrapServers() {
+            // we have to override this function
+            // because we want the Kafka Broker to advertise itself
+            // with the docker network address
+            // otherwise the Kafka Schema Registry won't work
+            return "PLAINTEXT://" + kafkaContainerName + ":9093";
+        }
+
+    }
+
+    protected EnhancedKafkaContainer createKafkaContainer(PulsarCluster cluster) {
+        return (EnhancedKafkaContainer) new EnhancedKafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:6.0.1"))
+                .withEmbeddedZookeeper()
+                .withCreateContainerCmdModifier(createContainerCmd -> createContainerCmd
+                        .withName(kafkaContainerName)
+                );
+    }
+
+    public void stopKafkaContainers(PulsarCluster cluster) {
+        if (null != schemaRegistryContainer) {
+            cluster.stopService(schemaRegistryContainerName, schemaRegistryContainer);
+        }
+        if (null != kafkaContainer) {
+            cluster.stopService(kafkaContainerName, kafkaContainer);
+        }
+    }
+
+    public void prepareSource() throws Exception {
+        log.info("creating topic");
+        ExecResult execResult = kafkaContainer.execInContainer(
+            "/usr/bin/kafka-topics",
+            "--create",
+            "--zookeeper",
+                getZooKeeperAddressInDockerNetwork(),
+            "--partitions",
+            "1",
+            "--replication-factor",
+            "1",
+            "--topic",
+            kafkaTopicName);
+        assertTrue(
+            execResult.getStdout().contains("Created topic"),
+            execResult.getStdout());
+
+    }
+
+    private String getZooKeeperAddressInDockerNetwork() {
+        return kafkaContainerName +":2181";
+    }
+
+    private <T extends GenericContainer> void testSource()  throws Exception {

Review comment:
       it the type parameter (`<T extends GenericContainer>`) required? 

##########
File path: pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/sink/PulsarSinkTest.java
##########
@@ -170,8 +171,30 @@ public void testVoidOutputClasses() throws Exception {
         PulsarSink pulsarSink = new PulsarSink(getPulsarClient(), pulsarConfig, new HashMap<>(), mock(ComponentStatsManager.class), Thread.currentThread().getContextClassLoader());
 
         try {
-            Schema schema = pulsarSink.initializeSchema();
+            PulsarSink.InitSchemaResult initSchemaResult = pulsarSink.initializeSchema();
+            Schema schema = initSchemaResult.schema;
             assertNull(schema);
+            // void type do not require the sink runtime to be created
+            assertTrue(!initSchemaResult.requireSink);
+        } catch (Exception ex) {
+            ex.printStackTrace();
+            assertNull(ex);
+            fail();

Review comment:
       does TestNG support passing the exception as the argument to fail? Could `ex.printStackTrace()` be omitted in that case?

##########
File path: pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/PulsarSchemaCache.java
##########
@@ -0,0 +1,70 @@
+/**
+ * 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.pulsar.io.kafka;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.schema.Field;
+import org.apache.pulsar.client.api.schema.GenericRecord;
+import org.apache.pulsar.client.impl.schema.generic.GenericAvroSchema;
+import org.apache.pulsar.common.schema.SchemaInfo;
+import org.apache.pulsar.common.schema.SchemaType;
+
+import java.nio.charset.StandardCharsets;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.stream.Collectors;
+
+@Slf4j
+class PulsarSchemaCache<T> {
+
+    @Data
+    @AllArgsConstructor
+    public static final class CachedSchema<T> {
+        private final Schema<T> schema;
+        private final List<Field> fields;
+    }
+
+    private IdentityHashMap<org.apache.avro.Schema, CachedSchema<T>> cache = new IdentityHashMap<>();

Review comment:
       Why not use ConcurrentHashMap so that the `get` method wouldn't have to use `synchronized`?

##########
File path: pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java
##########
@@ -118,17 +117,18 @@ public void close() throws InterruptedException {
 
     public void start() {
         runnerThread = new Thread(() -> {
-            LOG.info("Starting kafka source");
+            LOG.info("Starting kafka source on {}", kafkaSourceConfig.getTopic());
             consumer.subscribe(Collections.singletonList(kafkaSourceConfig.getTopic()));
             LOG.info("Kafka source started.");
-            ConsumerRecords<String, byte[]> consumerRecords;
+            ConsumerRecords<String, KV> consumerRecords;
             while (running) {
                 consumerRecords = consumer.poll(1000);
                 CompletableFuture<?>[] futures = new CompletableFuture<?>[consumerRecords.count()];
                 int index = 0;
-                for (ConsumerRecord<String, byte[]> consumerRecord : consumerRecords) {
-                    LOG.debug("Record received from kafka, key: {}. value: {}", consumerRecord.key(), consumerRecord.value());
+                for (ConsumerRecord<String, KV> consumerRecord : consumerRecords) {
+                    LOG.info("Record received from kafka, key: {}. value: {}", consumerRecord.key(), consumerRecord.value());

Review comment:
       for performance reasonsin log statements, use debug level and wrap with `if (LOG.isDebugEnabled()) { ... }`

##########
File path: pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/sink/PulsarSinkTest.java
##########
@@ -170,8 +171,30 @@ public void testVoidOutputClasses() throws Exception {
         PulsarSink pulsarSink = new PulsarSink(getPulsarClient(), pulsarConfig, new HashMap<>(), mock(ComponentStatsManager.class), Thread.currentThread().getContextClassLoader());
 
         try {
-            Schema schema = pulsarSink.initializeSchema();
+            PulsarSink.InitSchemaResult initSchemaResult = pulsarSink.initializeSchema();
+            Schema schema = initSchemaResult.schema;
             assertNull(schema);
+            // void type do not require the sink runtime to be created
+            assertTrue(!initSchemaResult.requireSink);
+        } catch (Exception ex) {
+            ex.printStackTrace();
+            assertNull(ex);

Review comment:
       what is the meaning of `assertNull(ex)`?

##########
File path: pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/PulsarSchemaCache.java
##########
@@ -0,0 +1,70 @@
+/**
+ * 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.pulsar.io.kafka;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.schema.Field;
+import org.apache.pulsar.client.api.schema.GenericRecord;
+import org.apache.pulsar.client.impl.schema.generic.GenericAvroSchema;
+import org.apache.pulsar.common.schema.SchemaInfo;
+import org.apache.pulsar.common.schema.SchemaType;
+
+import java.nio.charset.StandardCharsets;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.stream.Collectors;
+
+@Slf4j
+class PulsarSchemaCache<T> {
+
+    @Data
+    @AllArgsConstructor
+    public static final class CachedSchema<T> {
+        private final Schema<T> schema;
+        private final List<Field> fields;
+    }
+
+    private IdentityHashMap<org.apache.avro.Schema, CachedSchema<T>> cache = new IdentityHashMap<>();
+
+    public synchronized CachedSchema<T> get(org.apache.avro.Schema avroSchema) {
+        if (cache.size() > 100) {
+            // very simple auto cleanup
+            // schema do not change very often, we just want this map to grow

Review comment:
       why does the cleanup happen at lookup time?




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