This is an automated email from the ASF dual-hosted git repository.

tzulitai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink-statefun.git

commit 1fe23c0ffd915cfe7090502505dee2e4379365b1
Author: Tzu-Li (Gordon) Tai <tzuli...@apache.org>
AuthorDate: Wed Feb 26 18:19:06 2020 +0800

    [FLINK-16123] [e2e] Add RoutableKafkaE2E
    
    End-to-end test based on the {@link RoutableKafkaVerificationModule} 
application.
    
    This test writes some records to Kafka, with target function id as key
    (UTF8 String) and MessageWithAddress messages as value, without the
    "from" field set. The routable Kafka ingress should automatically route
    them to the correct function instances, which tag the input messages
    with their own address, and then forwards it back to Kafka. The test
    verifies that the tagged outputs written back to Kafka are correct.
    
    This closes #37.
---
 .../e2e/routablekafka/RoutableKafkaE2E.java        | 142 +++++++++++++++++++++
 .../src/test/resources/Dockerfile                  |  21 +++
 .../src/test/resources/log4j.properties            |  24 ++++
 .../routable-kafka-ingress-module/module.yaml      |  39 ++++++
 4 files changed, 226 insertions(+)

diff --git 
a/statefun-e2e-tests/statefun-routable-kafka-e2e/src/test/java/org/apache/flink/statefun/e2e/routablekafka/RoutableKafkaE2E.java
 
b/statefun-e2e-tests/statefun-routable-kafka-e2e/src/test/java/org/apache/flink/statefun/e2e/routablekafka/RoutableKafkaE2E.java
new file mode 100644
index 0000000..9f4eda1
--- /dev/null
+++ 
b/statefun-e2e-tests/statefun-routable-kafka-e2e/src/test/java/org/apache/flink/statefun/e2e/routablekafka/RoutableKafkaE2E.java
@@ -0,0 +1,142 @@
+/*
+ * 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.statefun.e2e.routablekafka;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import java.util.Collections;
+import java.util.Properties;
+import org.apache.flink.statefun.e2e.common.StatefulFunctionsAppContainers;
+import org.apache.flink.statefun.e2e.common.kafka.KafkaIOVerifier;
+import org.apache.flink.statefun.e2e.common.kafka.KafkaProtobufSerializer;
+import 
org.apache.flink.statefun.e2e.routablekafka.generated.RoutableKafkaVerification.FnAddress;
+import 
org.apache.flink.statefun.e2e.routablekafka.generated.RoutableKafkaVerification.MessageWithAddress;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.junit.Rule;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.KafkaContainer;
+
+/**
+ * End-to-end test based on the {@link RoutableKafkaVerificationModule} 
application.
+ *
+ * <p>This test writes some records to Kafka, with target function id as key 
(UTF8 String) and
+ * {@link MessageWithAddress} messages as value, without the {@code from} 
field set. The routable
+ * Kafka ingress should automatically route them to the correct function 
instances, which tag the
+ * input messages with their own address, and then forwards it back to Kafka 
(see {@link
+ * FnSelfAddressTagger} function). The test verifies that the tagged outputs 
written back to Kafka
+ * are correct.
+ */
+public class RoutableKafkaE2E {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(RoutableKafkaE2E.class);
+
+  private static final String CONFLUENT_PLATFORM_VERSION = "5.0.3";
+  private static final String KAFKA_HOST = "kafka-broker";
+
+  @Rule
+  public KafkaContainer kafka =
+      new 
KafkaContainer(CONFLUENT_PLATFORM_VERSION).withNetworkAliases(KAFKA_HOST);
+
+  @Rule
+  public StatefulFunctionsAppContainers verificationApp =
+      new StatefulFunctionsAppContainers("routable-kafka-verification", 1)
+          .dependsOn(kafka)
+          .exposeMasterLogs(LOG)
+          .withBuildContextFileFromClasspath(
+              "routable-kafka-ingress-module", 
"/routable-kafka-ingress-module/")
+          .withModuleGlobalConfiguration(
+              Constants.KAFKA_BOOTSTRAP_SERVERS_CONF, KAFKA_HOST + ":9092");
+
+  @Test(timeout = 60_000L)
+  public void run() {
+    final String kafkaAddress = kafka.getBootstrapServers();
+
+    final Producer<String, MessageWithAddress> messageProducer =
+        kafkaKeyedMessagesProducer(kafkaAddress);
+    final Consumer<String, MessageWithAddress> taggedMessageConsumer =
+        kafkaTaggedMessagesConsumer(kafkaAddress);
+
+    final KafkaIOVerifier<String, MessageWithAddress, String, 
MessageWithAddress> verifier =
+        new KafkaIOVerifier<>(messageProducer, taggedMessageConsumer);
+
+    assertThat(
+        verifier.sending(
+            producerRecord("messages-1", "key-1", message("foo")),
+            producerRecord("messages-1", "key-2", message("bar")),
+            producerRecord("messages-2", "key-1", message("hello"))),
+        verifier.resultsInOrder(
+            is(taggedMessage(fnAddress(Constants.FUNCTION_NAMESPACE, "t0", 
"key-1"), "foo")),
+            is(taggedMessage(fnAddress(Constants.FUNCTION_NAMESPACE, "t1", 
"key-1"), "foo")),
+            is(taggedMessage(fnAddress(Constants.FUNCTION_NAMESPACE, "t0", 
"key-2"), "bar")),
+            is(taggedMessage(fnAddress(Constants.FUNCTION_NAMESPACE, "t1", 
"key-2"), "bar")),
+            is(taggedMessage(fnAddress(Constants.FUNCTION_NAMESPACE, "t1", 
"key-1"), "hello"))));
+  }
+
+  private static Producer<String, MessageWithAddress> 
kafkaKeyedMessagesProducer(
+      String bootstrapServers) {
+    Properties props = new Properties();
+    props.put("bootstrap.servers", bootstrapServers);
+
+    return new KafkaProducer<>(
+        props, new StringSerializer(), new 
KafkaProtobufSerializer<>(MessageWithAddress.parser()));
+  }
+
+  private Consumer<String, MessageWithAddress> kafkaTaggedMessagesConsumer(
+      String bootstrapServers) {
+    Properties consumerProps = new Properties();
+    consumerProps.setProperty("bootstrap.servers", bootstrapServers);
+    consumerProps.setProperty("group.id", "routable-kafka");
+    consumerProps.setProperty("auto.offset.reset", "earliest");
+
+    KafkaConsumer<String, MessageWithAddress> consumer =
+        new KafkaConsumer<>(
+            consumerProps,
+            new StringDeserializer(),
+            new KafkaProtobufSerializer<>(MessageWithAddress.parser()));
+    
consumer.subscribe(Collections.singletonList(KafkaIO.TAGGED_MESSAGES_TOPIC_NAME));
+
+    return consumer;
+  }
+
+  private static ProducerRecord<String, MessageWithAddress> producerRecord(
+      String topic, String key, MessageWithAddress message) {
+    return new ProducerRecord<>(topic, key, message);
+  }
+
+  private static MessageWithAddress message(String message) {
+    return MessageWithAddress.newBuilder().setMessage(message).build();
+  }
+
+  private static MessageWithAddress taggedMessage(FnAddress fromTag, String 
message) {
+    return 
MessageWithAddress.newBuilder().setFrom(fromTag).setMessage(message).build();
+  }
+
+  private static FnAddress fnAddress(String namespace, String type, String id) 
{
+    return 
FnAddress.newBuilder().setNamespace(namespace).setType(type).setId(id).build();
+  }
+}
diff --git 
a/statefun-e2e-tests/statefun-routable-kafka-e2e/src/test/resources/Dockerfile 
b/statefun-e2e-tests/statefun-routable-kafka-e2e/src/test/resources/Dockerfile
new file mode 100644
index 0000000..2b025c7
--- /dev/null
+++ 
b/statefun-e2e-tests/statefun-routable-kafka-e2e/src/test/resources/Dockerfile
@@ -0,0 +1,21 @@
+# 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.
+
+FROM statefun
+
+RUN mkdir -p /opt/statefun/modules/statefun-routable-kafka-e2e
+COPY statefun-routable-kafka-e2e*.jar 
/opt/statefun/modules/statefun-routable-kafka-e2e/
+COPY routable-kafka-ingress-module/ 
/opt/statefun/modules/statefun-routable-kafka-e2e/
+COPY flink-conf.yaml $FLINK_HOME/conf/flink-conf.yaml
diff --git 
a/statefun-e2e-tests/statefun-routable-kafka-e2e/src/test/resources/log4j.properties
 
b/statefun-e2e-tests/statefun-routable-kafka-e2e/src/test/resources/log4j.properties
new file mode 100644
index 0000000..fb965d3
--- /dev/null
+++ 
b/statefun-e2e-tests/statefun-routable-kafka-e2e/src/test/resources/log4j.properties
@@ -0,0 +1,24 @@
+################################################################################
+#  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.
+################################################################################
+
+log4j.rootLogger=INFO, console
+
+# Log all infos in the given file
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x 
- %m%n
diff --git 
a/statefun-e2e-tests/statefun-routable-kafka-e2e/src/test/resources/routable-kafka-ingress-module/module.yaml
 
b/statefun-e2e-tests/statefun-routable-kafka-e2e/src/test/resources/routable-kafka-ingress-module/module.yaml
new file mode 100644
index 0000000..6e0c7f5
--- /dev/null
+++ 
b/statefun-e2e-tests/statefun-routable-kafka-e2e/src/test/resources/routable-kafka-ingress-module/module.yaml
@@ -0,0 +1,39 @@
+# 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.
+
+module:
+  meta:
+    type: remote
+  spec:
+    ingresses:
+      - ingress:
+          meta:
+            type: 
org.apache.flink.statefun.sdk.kafka/routable-protobuf-kafka-connector
+            id: org.apache.flink.statefun.e2e/messages
+          spec:
+            address: kafka-broker:9092
+            consumerGroupId: routable-kafka-e2e
+            startupPosition:
+              type: earliest
+            topics:
+              - topic: messages-1
+                typeUrl: 
com.googleapis/org.apache.flink.statefun.e2e.routablekafka.MessageWithAddress
+                targets:
+                  - org.apache.flink.e2e.routablekafka/t0
+                  - org.apache.flink.e2e.routablekafka/t1
+              - topic: messages-2
+                typeUrl: 
com.googleapis/org.apache.flink.statefun.e2e.routablekafka.MessageWithAddress
+                targets:
+                  - org.apache.flink.e2e.routablekafka/t1

Reply via email to