Skip to content

Commit

Permalink
NIFI-11259 Added Redesigned Kafka Processors and Services
Browse files Browse the repository at this point in the history
- Added ConsumeKafka and PublishKafka Processors with Kafka3ConnectionService
- Added Kafka 3 integration tests

This closes #8463

Co-authored-by: David Handermann <[email protected]>
Signed-off-by: David Handermann <[email protected]>
  • Loading branch information
greyp9 and exceptionfactory committed Jun 24, 2024
1 parent 55d2f39 commit 24925ed
Show file tree
Hide file tree
Showing 152 changed files with 12,209 additions and 2 deletions.
18 changes: 18 additions & 0 deletions nifi-assembly/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -298,6 +298,24 @@ language governing permissions and limitations under the License. -->
<version>2.0.0-SNAPSHOT</version>
<type>nar</type>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-kafka-service-api-nar</artifactId>
<version>2.0.0-SNAPSHOT</version>
<type>nar</type>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-kafka-3-service-nar</artifactId>
<version>2.0.0-SNAPSHOT</version>
<type>nar</type>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-kafka-nar</artifactId>
<version>2.0.0-SNAPSHOT</version>
<type>nar</type>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-confluent-platform-nar</artifactId>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,89 @@
<?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 https://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>

<parent>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-kafka-bundle</artifactId>
<version>2.0.0-SNAPSHOT</version>
</parent>

<artifactId>nifi-kafka-2-6-integration</artifactId>
<packaging>jar</packaging>

<dependencies>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-schema-registry-service-api</artifactId>
<version>2.0.0-SNAPSHOT</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-ssl-context-service-api</artifactId>
<version>2.0.0-SNAPSHOT</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-kerberos-user-service-api</artifactId>
<version>2.0.0-SNAPSHOT</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-record-serialization-services</artifactId>
<version>2.0.0-SNAPSHOT</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-mock</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-utils</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.testcontainers</groupId>
<artifactId>junit-jupiter</artifactId>
<version>${testcontainers.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.testcontainers</groupId>
<artifactId>kafka</artifactId>
<version>${testcontainers.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-kafka-2-6-processors</artifactId>
<version>2.0.0-SNAPSHOT</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-record-serialization-service-api</artifactId>
<version>2.0.0-SNAPSHOT</version>
<scope>test</scope>
</dependency>
</dependencies>
</project>
Original file line number Diff line number Diff line change
@@ -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.nifi.processors.kafka.consume.it;

import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute;
import org.apache.nifi.processors.kafka.pubsub.ConsumeKafka_2_6;
import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.ProvenanceEventType;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;

import java.net.URI;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.UUID;
import java.util.concurrent.ExecutionException;

import static org.junit.jupiter.api.Assertions.assertEquals;

public class ConsumeKafkaDemarcator_2_6_IT extends ConsumeKafka_2_6_BaseIT {
private static final String TEST_RECORD_VALUE = "recordA,recordB,recordC,recordD,recordE,recordF";
private static final int EXPECTED_TOKENS = 6;

private TestRunner runner;

@BeforeEach
void setUp() {
runner = TestRunners.newTestRunner(ConsumeKafka_2_6.class);
final URI uri = URI.create(kafka.getBootstrapServers());
runner.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, String.format("%s:%s", uri.getHost(), uri.getPort()));
runner.setValidateExpressionUsage(false);
runner.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
}

@Test
public void testConsumeDemarcated() throws ExecutionException, InterruptedException {
final String topic = UUID.randomUUID().toString();
final String groupId = topic.substring(0, 4);
runner.setProperty("group.id", groupId);
runner.setProperty("topic", topic);
runner.setProperty("message-demarcator", ",");
runner.setProperty("separate-by-key", Boolean.FALSE.toString());
runner.run(1, false, true);

final Collection<ProducerRecord<String, String>> records = new ArrayList<>();
final String[] values = TEST_RECORD_VALUE.split(",");
for (String value : values) {
records.add(new ProducerRecord<>(topic, value));
}
produce(topic, records);

final long pollUntil = System.currentTimeMillis() + DURATION_POLL.toMillis();
while (System.currentTimeMillis() < pollUntil) {
runner.run(1, false, false);
}

runner.run(1, true, false);
runner.assertTransferCount("success", 1);

final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship("success");
assertEquals(1, flowFiles.size());
final MockFlowFile flowFile = flowFiles.getFirst();
assertEquals(TEST_RECORD_VALUE, flowFile.getContent());
flowFile.assertAttributeEquals(KafkaFlowFileAttribute.KAFKA_TOPIC, topic);
flowFile.assertAttributeEquals(KafkaFlowFileAttribute.KAFKA_PARTITION, Integer.toString(0));
flowFile.assertAttributeEquals(KafkaFlowFileAttribute.KAFKA_OFFSET, Long.toString(0));
flowFile.assertAttributeEquals(KafkaFlowFileAttribute.KAFKA_MAX_OFFSET, Long.toString(EXPECTED_TOKENS - 1));
flowFile.assertAttributeEquals(KafkaFlowFileAttribute.KAFKA_COUNT, Long.toString(EXPECTED_TOKENS));
flowFile.assertAttributeExists(KafkaFlowFileAttribute.KAFKA_TIMESTAMP);

final List<ProvenanceEventRecord> provenanceEvents = runner.getProvenanceEvents();
assertEquals(1, provenanceEvents.size());
final ProvenanceEventRecord provenanceEvent = provenanceEvents.getFirst();
assertEquals(ProvenanceEventType.RECEIVE, provenanceEvent.getEventType());
}

@Test
public void testConsumeDemarcatedSeparateByKey() throws ExecutionException, InterruptedException {
final String topic = UUID.randomUUID().toString();
final String groupId = topic.substring(0, 4);
runner.setProperty("group.id", groupId);
runner.setProperty("topic", topic);
runner.setProperty("message-demarcator", ",");
runner.setProperty("separate-by-key", Boolean.TRUE.toString());
runner.run(1, false, true);

final Collection<ProducerRecord<String, String>> records = new ArrayList<>();
final String[] values = TEST_RECORD_VALUE.split(",");
boolean key = false;
for (String value : values) {
records.add(new ProducerRecord<>(topic, Boolean.toString(key), value));
key = !key;
}
produce(topic, records);

final long pollUntil = System.currentTimeMillis() + DURATION_POLL.toMillis();
while (System.currentTimeMillis() < pollUntil) {
runner.run(1, false, false);
}

runner.run(1, true, false);
runner.assertTransferCount("success", 2); // key=false, key=true

final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship("success");
assertEquals(2, flowFiles.size());
for (MockFlowFile flowFile : flowFiles) {
if (Boolean.TRUE.toString().equals(flowFile.getAttribute(KafkaFlowFileAttribute.KAFKA_KEY))) {
assertEquals("recordB,recordD,recordF", flowFile.getContent());
flowFile.assertAttributeEquals(KafkaFlowFileAttribute.KAFKA_OFFSET, Long.toString(1));
flowFile.assertAttributeEquals(KafkaFlowFileAttribute.KAFKA_MAX_OFFSET, Long.toString(EXPECTED_TOKENS - 1));
flowFile.assertAttributeEquals(KafkaFlowFileAttribute.KAFKA_COUNT, Long.toString(EXPECTED_TOKENS / 2));
} else if (Boolean.FALSE.toString().equals(flowFile.getAttribute(KafkaFlowFileAttribute.KAFKA_KEY))) {
assertEquals("recordA,recordC,recordE", flowFile.getContent());
flowFile.assertAttributeEquals(KafkaFlowFileAttribute.KAFKA_OFFSET, Long.toString(0));
flowFile.assertAttributeEquals(KafkaFlowFileAttribute.KAFKA_MAX_OFFSET, Long.toString(EXPECTED_TOKENS - 2));
flowFile.assertAttributeEquals(KafkaFlowFileAttribute.KAFKA_COUNT, Long.toString(EXPECTED_TOKENS / 2));
} else {
Assertions.fail("expected KafkaFlowFileAttribute.KAFKA_KEY");
}
flowFile.assertAttributeEquals(KafkaFlowFileAttribute.KAFKA_TOPIC, topic);
flowFile.assertAttributeEquals(KafkaFlowFileAttribute.KAFKA_PARTITION, Integer.toString(0));
flowFile.assertAttributeExists(KafkaFlowFileAttribute.KAFKA_TIMESTAMP);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,105 @@
/*
* 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.nifi.processors.kafka.consume.it;

import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.ArrayNode;
import org.apache.commons.io.IOUtils;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute;
import org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_2_6;
import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.ProvenanceEventType;
import org.apache.nifi.reporting.InitializationException;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.jupiter.api.Test;

import java.io.IOException;
import java.net.URI;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Objects;
import java.util.concurrent.ExecutionException;

import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
import static org.junit.jupiter.api.Assertions.assertTrue;

public class ConsumeKafkaRecord_2_6_IT extends ConsumeKafka_2_6_BaseIT {
private static final String TEST_RESOURCE = "org/apache/nifi/processors/kafka/publish/ff.json";
private static final int TEST_RECORD_COUNT = 3;

private static final String TOPIC = ConsumeKafkaRecord_2_6_IT.class.getName();
private static final String GROUP_ID = ConsumeKafkaRecord_2_6_IT.class.getSimpleName();

@Test
public void testKafkaTestContainerProduceConsumeOne() throws ExecutionException, InterruptedException, IOException, InitializationException {
final TestRunner runner = TestRunners.newTestRunner(ConsumeKafkaRecord_2_6.class);
runner.setValidateExpressionUsage(false);
final URI uri = URI.create(kafka.getBootstrapServers());
addRecordReaderService(runner);
addRecordWriterService(runner);
runner.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, String.format("%s:%s", uri.getHost(), uri.getPort()));
runner.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
runner.setProperty("topic", TOPIC);
runner.setProperty("group.id", GROUP_ID);
runner.run(1, false, true);

final String message = new String(IOUtils.toByteArray(Objects.requireNonNull(
getClass().getClassLoader().getResource(TEST_RESOURCE))), StandardCharsets.UTF_8);
produceOne(TOPIC, null, null, message, Collections.emptyList());
final long pollUntil = System.currentTimeMillis() + DURATION_POLL.toMillis();
while (System.currentTimeMillis() < pollUntil) {
runner.run(1, false, false);
}

runner.run(1, true, false);
runner.assertTransferCount("success", 1);

final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship("success");
assertEquals(1, flowFiles.size());
final MockFlowFile flowFile = flowFiles.getFirst();

flowFile.assertAttributeEquals(KafkaFlowFileAttribute.KAFKA_TOPIC, TOPIC);
flowFile.assertAttributeEquals(KafkaFlowFileAttribute.KAFKA_PARTITION, Integer.toString(0));
flowFile.assertAttributeEquals(KafkaFlowFileAttribute.KAFKA_OFFSET, Long.toString(0));
flowFile.assertAttributeExists(KafkaFlowFileAttribute.KAFKA_TIMESTAMP);
flowFile.assertAttributeEquals("record.count", Long.toString(3));

final List<ProvenanceEventRecord> provenanceEvents = runner.getProvenanceEvents();
assertEquals(1, provenanceEvents.size());
final ProvenanceEventRecord provenanceEvent = provenanceEvents.getFirst();
assertEquals(ProvenanceEventType.RECEIVE, provenanceEvent.getEventType());

// [{"id":1,"name":"A"},{"id":2,"name":"B"},{"id":3,"name":"C"}]
final JsonNode jsonNodeTree = objectMapper.readTree(flowFile.getContent());
assertInstanceOf(ArrayNode.class, jsonNodeTree);
final ArrayNode arrayNode = (ArrayNode) jsonNodeTree;
final Iterator<JsonNode> elements = arrayNode.elements();
assertEquals(TEST_RECORD_COUNT, arrayNode.size());
while (elements.hasNext()) {
final JsonNode jsonNode = elements.next();
assertTrue(Arrays.asList(1, 2, 3).contains(jsonNode.get("id").asInt()));
assertTrue(Arrays.asList("A", "B", "C").contains(jsonNode.get("name").asText()));
}
}
}
Loading

0 comments on commit 24925ed

Please sign in to comment.