From e4aebaadab12000464489d70e0b14fd70f6a02f0 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Thu, 12 Nov 2020 15:48:30 +0100 Subject: [PATCH 01/27] #177 works now --- .../KafkaToKafkaDeduplicationDockerTest.scala | 184 +++++++++++++ .../DeduplicateKafkaSinkTransformer.scala | 252 ++++++++++++++++++ ...licateKafkaSinkTransformerAttributes.scala | 32 +++ ...eduplicateKafkaSinkTransformerLoader.scala | 22 ++ .../kafka/kafka010/JsonUtils.scala | 102 +++++++ .../kafka/kafka010/KafkaSourceOffset.scala | 46 ++++ 6 files changed, 638 insertions(+) create mode 100644 driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala create mode 100644 ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala create mode 100644 ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformerAttributes.scala create mode 100644 ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformerLoader.scala create mode 100644 ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/kafka010/JsonUtils.scala create mode 100644 ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/kafka010/KafkaSourceOffset.scala diff --git a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala new file mode 100644 index 00000000..ec1eb61e --- /dev/null +++ b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala @@ -0,0 +1,184 @@ +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed 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 za.co.absa.hyperdrive.driver.drivers + +import java.io.{BufferedWriter, File, FileWriter, OutputStreamWriter} +import java.nio.file.{Files, Paths} +import java.time.Duration +import java.util.UUID.randomUUID +import java.util.{Collections, Properties} + +import org.apache.avro.Schema.{Parser, Type} +import org.apache.avro.generic.{GenericData, GenericRecord} +import org.apache.avro.util.Utf8 +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.kafka.clients.consumer.KafkaConsumer +import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} +import org.scalatest.{BeforeAndAfter, FlatSpec, Matchers} +import za.co.absa.abris.avro.read.confluent.SchemaManagerFactory +import za.co.absa.commons.io.TempDirectory +import za.co.absa.commons.spark.SparkTestBase + + +/** + * This e2e test requires a Docker installation on the executing machine. + */ +class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with SparkTestBase with BeforeAndAfter { + + private val fs = FileSystem.get(spark.sparkContext.hadoopConfiguration) + private val baseDir = TempDirectory("hyperdriveE2eTest").deleteOnExit() + private val baseDirPath = baseDir.path.toUri + private val checkpointDir = s"$baseDirPath/checkpoint" + + behavior of "CommandLineIngestionDriver" + private def schemaString(topic: String) = raw"""{"type": "record", "name": "$topic", "fields": [ + {"type": "string", "name": "record_id"}, + {"type": ["string", "null"], "name": "value_field"} + ]}""" + + private def sendData(producer: KafkaProducer[GenericRecord, GenericRecord], from: Int, to: Int, topic: String) = { + val schema = new Parser().parse(schemaString(topic)) + for (i <- from until to) { + val valueRecord = new GenericData.Record(schema) + valueRecord.put("record_id", i.toString) + valueRecord.put("value_field", s"valueHello_$i") + + val producerRecord = new ProducerRecord[GenericRecord, GenericRecord](topic, valueRecord) + producer.send(producerRecord) + } + } + + private def writeToFile(filename: String, content: String) = { + val path = new Path(filename) + val out = fs.create(path) + out.writeBytes(content) + out.flush() + out.close() + } + + it should "execute the whole kafka-to-kafka pipeline" in { + // given + val kafkaSchemaRegistryWrapper = new KafkaSchemaRegistryWrapper + + val sourceTopic = "deduplication_src" + val destinationTopic = "deduplication_dest" + val allRecords = 250 + val duplicatedRange1 = (0, 75) + val duplicatedRange2 = (100, 175) + val producer = createProducer(kafkaSchemaRegistryWrapper) + sendData(producer, 0, allRecords, sourceTopic) + sendData(producer, duplicatedRange1._1, duplicatedRange1._2, destinationTopic) + sendData(producer, duplicatedRange2._1, duplicatedRange2._2, destinationTopic) + + val offset = raw"""v1 + |{"batchWatermarkMs":0,"batchTimestampMs":1605193344093,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"200"}} + |{"$sourceTopic":{"0":250}}""".stripMargin + val sources = raw"""0v1 + |{"$sourceTopic":{"0":0}}""".stripMargin + val metadata = raw"""{"id":"7a9e78ae-3473-469c-a906-c78ffaf0f3c9"}""" + + fs.mkdirs(new Path(s"$checkpointDir/$sourceTopic/commits")) + writeToFile(s"$checkpointDir/$sourceTopic/offsets/0", offset) + writeToFile(s"$checkpointDir/$sourceTopic/sources/0/0", sources) + writeToFile(s"$checkpointDir/$sourceTopic/metadata", metadata) + + Thread.sleep(3000) + + val driverConfig = Map( + // Pipeline settings + "component.ingestor" -> "spark", + "component.reader" -> "za.co.absa.hyperdrive.ingestor.implementation.reader.kafka.KafkaStreamReader", + "component.transformer.id.0" -> "[avro.decoder]", + "component.transformer.class.[avro.decoder]" -> "za.co.absa.hyperdrive.ingestor.implementation.transformer.avro.confluent.ConfluentAvroDecodingTransformer", + "component.transformer.id.1" -> "[kafka.deduplicator]", + "component.transformer.class.[kafka.deduplicator]" -> "za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka.DeduplicateKafkaSinkTransformer", + "component.transformer.id.2" -> "[avro.encoder]", + "component.transformer.class.[avro.encoder]" -> "za.co.absa.hyperdrive.ingestor.implementation.transformer.avro.confluent.ConfluentAvroEncodingTransformer", + "component.writer" -> "za.co.absa.hyperdrive.ingestor.implementation.writer.kafka.KafkaStreamWriter", + + // Spark settings + "ingestor.spark.app.name" -> "ingestor-app", + + // Source(Kafka) settings + "reader.kafka.topic" -> sourceTopic, + "reader.kafka.brokers" -> kafkaSchemaRegistryWrapper.kafkaUrl, + + // Avro Decoder (ABRiS) settings + "transformer.[avro.decoder].schema.registry.url" -> kafkaSchemaRegistryWrapper.schemaRegistryUrl, + "transformer.[avro.decoder].value.schema.id" -> "latest", + "transformer.[avro.decoder].value.schema.naming.strategy" -> "topic.name", + + // comma separated list of columns to select + "transformer.[kafka.deduplicator].id.column" -> "record_id", + "transformer.[kafka.deduplicator].schema.registry.url" -> "${transformer.[avro.decoder].schema.registry.url}", + + // Avro Encoder (ABRiS) settings + "transformer.[avro.encoder].schema.registry.url" -> "${transformer.[avro.decoder].schema.registry.url}", + "transformer.[avro.encoder].value.schema.naming.strategy" -> "topic.name", + + // Sink(Kafka) settings + "writer.common.checkpoint.location" -> (checkpointDir + "/${reader.kafka.topic}"), + "writer.common.trigger.type" -> "Once", + "writer.kafka.topic" -> destinationTopic, + "writer.kafka.brokers" -> "${reader.kafka.brokers}" + ) + val driverConfigArray = driverConfig.map { case (key, value) => s"$key=$value" }.toArray + + // when + CommandLineIngestionDriver.main(driverConfigArray) + + // then + fs.exists(new Path(s"$checkpointDir/$sourceTopic")) shouldBe true + + val consumer = createConsumer(kafkaSchemaRegistryWrapper) + consumer.subscribe(Collections.singletonList(destinationTopic)) + import scala.collection.JavaConverters._ + val records = consumer.poll(Duration.ofMillis(500L)).asScala.toList + records.size shouldBe allRecords + + val valueFieldNames = records.head.value().getSchema.getFields.asScala.map(_.name()) + valueFieldNames should contain theSameElementsAs List("record_id", "value_field") + val actual = records.map(_.value().get("record_id")) + val expected = List.range(0, allRecords).map(i => new Utf8(i.toString)) + actual should contain theSameElementsAs expected + } + + after { + SchemaManagerFactory.resetSRClientInstance() + } + + def createProducer(kafkaSchemaRegistryWrapper: KafkaSchemaRegistryWrapper): KafkaProducer[GenericRecord, GenericRecord] = { + val props = new Properties() + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaSchemaRegistryWrapper.kafka.getBootstrapServers) + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "io.confluent.kafka.serializers.KafkaAvroSerializer") + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "io.confluent.kafka.serializers.KafkaAvroSerializer") + props.put(ProducerConfig.CLIENT_ID_CONFIG, "KafkaToKafkaProducer") + props.put(ProducerConfig.ACKS_CONFIG, "1") + kafkaSchemaRegistryWrapper.createProducer(props) + } + + def createConsumer(kafkaSchemaRegistryWrapper: KafkaSchemaRegistryWrapper): KafkaConsumer[GenericRecord, GenericRecord] = { + import org.apache.kafka.clients.consumer.ConsumerConfig + val props = new Properties() + props.put(ConsumerConfig.GROUP_ID_CONFIG, randomUUID.toString) + props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaSchemaRegistryWrapper.kafka.getBootstrapServers) + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "io.confluent.kafka.serializers.KafkaAvroDeserializer") + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "io.confluent.kafka.serializers.KafkaAvroDeserializer") + kafkaSchemaRegistryWrapper.createConsumer(props) + } + +} diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala new file mode 100644 index 00000000..19cba3f7 --- /dev/null +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala @@ -0,0 +1,252 @@ +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed 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 za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka + +import java.time.Duration +import java.util +import java.util.{Collections, Properties, UUID} + +import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig +import org.apache.avro.generic.GenericRecord +import org.apache.avro.util.Utf8 +import org.apache.commons.configuration2.Configuration +import org.apache.hadoop.fs.Path +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRebalanceListener, ConsumerRecord, KafkaConsumer} +import org.apache.kafka.common.TopicPartition +import org.apache.logging.log4j.LogManager +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog} +import za.co.absa.hyperdrive.ingestor.api.transformer.{StreamTransformer, StreamTransformerFactory} +import za.co.absa.hyperdrive.ingestor.api.utils.ConfigUtils.{getOrThrow, getPropertySubset} +import za.co.absa.hyperdrive.ingestor.api.utils.StreamWriterUtil +import za.co.absa.hyperdrive.ingestor.api.writer.StreamWriterCommonAttributes +import za.co.absa.hyperdrive.ingestor.implementation.reader.kafka.KafkaStreamReader +import za.co.absa.hyperdrive.ingestor.implementation.transformer.avro.confluent.{ConfluentAvroDecodingTransformer, ConfluentAvroEncodingTransformer} +import za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka.kafka010.KafkaSourceOffset +import za.co.absa.hyperdrive.ingestor.implementation.writer.kafka.KafkaStreamWriter + +import scala.collection.JavaConverters._ +import scala.collection.mutable + + +private[transformer] class DeduplicateKafkaSinkTransformer( + val readerSchemaRegistryUrl: String, + val readerTopic: String, + val readerBrokers: String, + val readerExtraOptions: Map[String, String], + val writerSchemaRegistryUrl: String, + val writerTopic: String, + val writerBrokers: String, + val writerExtraOptions: Map[String, String], + val checkpointLocation: String, + val idColumn: String) extends StreamTransformer { + private val logger = LogManager.getLogger + private val timeout = Duration.ofSeconds(5L) + + override def transform(dataFrame: DataFrame): DataFrame = { + val spark = dataFrame.sparkSession + val offsetLog = new OffsetSeqLog(spark, new Path(checkpointLocation, "offsets").toString) + val commitLog = new CommitLog(spark, new Path(checkpointLocation, "commits").toString) + val latestOffsetLog = offsetLog.getLatest().map(_._1) + val latestCommitLog = commitLog.getLatest().map(_._1) + + if (latestOffsetLog != latestCommitLog) { + deduplicateDataFrame(dataFrame, offsetLog, commitLog) + } else { + dataFrame + } + } + + private def deduplicateDataFrame(dataFrame: DataFrame, offsetLog: OffsetSeqLog, commitLog: CommitLog) = { + logger.info("Deduplicate rows after retry") + val sourceConsumer = createConsumer(readerBrokers, readerExtraOptions, readerSchemaRegistryUrl) + seekToOffsets(sourceConsumer, offsetLog, commitLog) + + val sourceRecordsCount = try { + getAllAvailableMessagesCount(sourceConsumer) + } catch { + case throwable: Throwable => logger.error(s"An unexpected error occurred while consuming from topic $readerTopic", throwable) + throw throwable + } finally { + sourceConsumer.close() + } + + val sinkConsumer = createConsumer(writerBrokers, writerExtraOptions, writerSchemaRegistryUrl) + val sinkTopicPartitions = getTopicPartitions(sinkConsumer, writerTopic) + val latestSinkRecords = try { + sinkTopicPartitions.map(topicPartition => { + getAtLeastNLatestRecords(sinkConsumer, topicPartition, sourceRecordsCount) + }) + } catch { + case throwable: Throwable => logger.error(s"An unexpected error occurred while consuming from topic $writerTopic", throwable) + throw throwable + } finally { + sinkConsumer.close() + } + + val publishedIds = latestSinkRecords.flatten.map(record => { + try { + record.value().get(idColumn).asInstanceOf[Utf8].toString + } catch { + case throwable: Throwable => logger.error(s"Could not get $idColumn from record, schema is ${record.value().getSchema}", throwable) + throw throwable + } + }) + import org.apache.spark.sql.functions._ + dataFrame.filter(not(col(idColumn).isInCollection(publishedIds))) + } + + private def seekToOffsets(consumer: KafkaConsumer[GenericRecord, GenericRecord], offsetLog: OffsetSeqLog, commitLog: CommitLog) = { + val sourceTopicPartitionOffsetsOpt = getTopicPartitionsFromOffsets(offsetLog, commitLog) + consumer.subscribe(Collections.singletonList(readerTopic), new ConsumerRebalanceListener { + override def onPartitionsRevoked(partitions: util.Collection[TopicPartition]): Unit = {} + + override def onPartitionsAssigned(partitions: util.Collection[TopicPartition]): Unit = { + sourceTopicPartitionOffsetsOpt match { + case Some(topicPartitionOffsets) => topicPartitionOffsets.foreach { + case (topicPartition, offset) => consumer.seek(topicPartition, offset) + } + case None => + val partitions = getTopicPartitions(consumer, readerTopic) + consumer.seekToBeginning(partitions.asJava) + } + } + }) + } + + private def getTopicPartitions(consumer: KafkaConsumer[GenericRecord, GenericRecord], topic: String) = { + consumer.partitionsFor(topic).asScala.map(p => new TopicPartition(p.topic(), p.partition())) + } + + private def getAtLeastNLatestRecords(consumer: KafkaConsumer[GenericRecord, GenericRecord], topicPartition: TopicPartition, numberOfRecords: Int) = { + consumer.assign(Seq(topicPartition).asJava) + val endOffsets = consumer.endOffsets(Seq(topicPartition).asJava).asScala + if (endOffsets.size != 1) { + throw new IllegalStateException(s"Expected exactly 1 end offset, got ${endOffsets}") + } + val partition = endOffsets.keys.head + val offset = endOffsets.values.head + + var records: Seq[ConsumerRecord[GenericRecord, GenericRecord]] = Seq() + var offsetLowerBound = offset + while(records.size < numberOfRecords && offsetLowerBound != 0) { + offsetLowerBound = Math.max(0, offsetLowerBound - numberOfRecords) + consumer.seek(partition, offsetLowerBound) + records = getAllAvailableMessages(consumer) + } + + records + } + + private def getAllAvailableMessagesCount(consumer: KafkaConsumer[GenericRecord, GenericRecord]): Int = { + import scala.util.control.Breaks._ + var recordsCount = 0 + breakable { + while (true) { + val currentRecordsCount = consumer.poll(timeout).count() + if (currentRecordsCount == 0) { + break() + } + recordsCount += currentRecordsCount + } + } + recordsCount + } + + private def getAllAvailableMessages(consumer: KafkaConsumer[GenericRecord, GenericRecord]) = { + import scala.util.control.Breaks._ + var records: Seq[ConsumerRecord[GenericRecord, GenericRecord]] = mutable.Seq() + breakable { + while (true) { + val newRecords = consumer.poll(timeout).asScala.toSeq + if (newRecords.isEmpty) { + break() + } + records ++= newRecords + } + } + records + } + + private def createConsumer(brokers: String, extraOptions: Map[String, String], schemaRegistryUrl: String) = { + val props = new Properties() + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokers) + props.put(ConsumerConfig.CLIENT_ID_CONFIG, s"hyperdrive_consumer_${UUID.randomUUID().toString}") + props.put(ConsumerConfig.GROUP_ID_CONFIG, s"hyperdrive_group_${UUID.randomUUID().toString}") + extraOptions.foreach { + case (key, value) => props.put(key, value) + } + props.put(AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, schemaRegistryUrl) + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "io.confluent.kafka.serializers.KafkaAvroDeserializer") + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "io.confluent.kafka.serializers.KafkaAvroDeserializer") + new KafkaConsumer[GenericRecord, GenericRecord](props) + } + + def getTopicPartitionsFromOffsets(offsetLog: OffsetSeqLog, commitLog: CommitLog): Option[Map[TopicPartition, Long]] = { + val offsetSeqOpt = commitLog.getLatest().map(_._1) + .flatMap(batchId => offsetLog.get(batchId)) + .map(offsetLog => offsetLog.offsets) + val result = if (offsetSeqOpt.isDefined) { + if (offsetSeqOpt.get.size == 1) { + if (offsetSeqOpt.get.head.isDefined) { + Some(KafkaSourceOffset.getPartitionOffsets(offsetSeqOpt.get.head.get)) + } else { + throw new IllegalStateException("Offset must be defined, got None") + } + } else { + throw new IllegalStateException(s"Cannot support more than 1 source, got ${offsetSeqOpt.toString}") + } + } else { + None + } + result + } +} + +object DeduplicateKafkaSinkTransformer extends StreamTransformerFactory with DeduplicateKafkaSinkTransformerAttributes { + + override def apply(config: Configuration): StreamTransformer = { + val readerSchemaRegistryUrl = config.getString(schemaRegistryUrl) + val readerTopic = config.getString(KafkaStreamReader.KEY_TOPIC) + val readerBrokers = config.getString(KafkaStreamReader.KEY_BROKERS) + val readerExtraOptions = getPropertySubset(config, KafkaStreamReader.getExtraConfigurationPrefix.get) + + val writerSchemaRegistryUrl = config.getString(schemaRegistryUrl) + val writerTopic = config.getString(KafkaStreamWriter.KEY_TOPIC) + val writerBrokers = config.getString(KafkaStreamWriter.KEY_BROKERS) + val writerExtraOptions = getPropertySubset(config, KafkaStreamWriter.optionalConfKey) + + val checkpointLocation = StreamWriterUtil.getCheckpointLocation(config) + + val idColumn = getOrThrow(IdColumn, config, errorMessage = s"Destination directory not found. Is '${IdColumn}' defined?") + new DeduplicateKafkaSinkTransformer(readerSchemaRegistryUrl, readerTopic, readerBrokers, readerExtraOptions, + writerSchemaRegistryUrl, writerTopic, writerBrokers, writerExtraOptions, + checkpointLocation, idColumn) + } + + override def getMappingFromRetainedGlobalConfigToLocalConfig(globalConfig: Configuration): Map[String, String] = { +// TODO: What about subsets? + Set( + KafkaStreamReader.KEY_TOPIC, + KafkaStreamReader.KEY_BROKERS, + KafkaStreamWriter.KEY_TOPIC, + KafkaStreamWriter.KEY_BROKERS, + StreamWriterCommonAttributes.keyCheckpointBaseLocation + ).map(e => e -> e).toMap + } +} + + diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformerAttributes.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformerAttributes.scala new file mode 100644 index 00000000..e188b1bb --- /dev/null +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformerAttributes.scala @@ -0,0 +1,32 @@ +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed 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 za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka + +import za.co.absa.hyperdrive.ingestor.api.{HasComponentAttributes, PropertyMetadata} + +trait DeduplicateKafkaSinkTransformerAttributes extends HasComponentAttributes { + + val IdColumn = "id.column" + val schemaRegistryUrl = "schema.registry.url" + + override def getName: String = "Deduplicate Kafka Sink Transformer" + + override def getDescription: String = "" + + override def getProperties: Map[String, PropertyMetadata] = Map( + IdColumn -> PropertyMetadata("Id column", Some(""), required = true) + ) +} diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformerLoader.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformerLoader.scala new file mode 100644 index 00000000..85c8363b --- /dev/null +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformerLoader.scala @@ -0,0 +1,22 @@ +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed 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 za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka + +import za.co.absa.hyperdrive.ingestor.api.transformer.{StreamTransformerFactory, StreamTransformerFactoryProvider} + +class DeduplicateKafkaSinkTransformerLoader extends StreamTransformerFactoryProvider { + override def getComponentFactory: StreamTransformerFactory = DeduplicateKafkaSinkTransformer +} diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/kafka010/JsonUtils.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/kafka010/JsonUtils.scala new file mode 100644 index 00000000..dc9dd69e --- /dev/null +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/kafka010/JsonUtils.scala @@ -0,0 +1,102 @@ + +package za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka.kafka010 + +import scala.collection.mutable.HashMap +import scala.util.control.NonFatal + +import org.apache.kafka.common.TopicPartition +import org.json4s.NoTypeHints +import org.json4s.jackson.Serialization + +/** + * Utilities for converting Kafka related objects to and from json. + */ +private[kafka] object JsonUtils { + private implicit val formats = Serialization.formats(NoTypeHints) + + /** + * Read TopicPartitions from json string + */ + def partitions(str: String): Array[TopicPartition] = { + try { + Serialization.read[Map[String, Seq[Int]]](str).flatMap { case (topic, parts) => + parts.map { part => + new TopicPartition(topic, part) + } + }.toArray + } catch { + case NonFatal(x) => + throw new IllegalArgumentException( + s"""Expected e.g. {"topicA":[0,1],"topicB":[0,1]}, got $str""") + } + } + + /** + * Write TopicPartitions as json string + */ + def partitions(partitions: Iterable[TopicPartition]): String = { + val result = new HashMap[String, List[Int]] + partitions.foreach { tp => + val parts: List[Int] = result.getOrElse(tp.topic, Nil) + result += tp.topic -> (tp.partition::parts) + } + Serialization.write(result) + } + + /** + * Read per-TopicPartition offsets from json string + */ + def partitionOffsets(str: String): Map[TopicPartition, Long] = { + try { + Serialization.read[Map[String, Map[Int, Long]]](str).flatMap { case (topic, partOffsets) => + partOffsets.map { case (part, offset) => + new TopicPartition(topic, part) -> offset + } + } + } catch { + case NonFatal(x) => + throw new IllegalArgumentException( + s"""Expected e.g. {"topicA":{"0":23,"1":-1},"topicB":{"0":-2}}, got $str""") + } + } + + def partitionTimestamps(str: String): Map[TopicPartition, Long] = { + try { + Serialization.read[Map[String, Map[Int, Long]]](str).flatMap { case (topic, partTimestamps) => + partTimestamps.map { case (part, timestamp) => + new TopicPartition(topic, part) -> timestamp + } + } + } catch { + case NonFatal(x) => + throw new IllegalArgumentException( + s"""Expected e.g. {"topicA": {"0": 123456789, "1": 123456789}, + |"topicB": {"0": 123456789, "1": 123456789}}, got $str""".stripMargin) + } + } + + /** + * Write per-TopicPartition offsets as json string + */ + def partitionOffsets(partitionOffsets: Map[TopicPartition, Long]): String = { + val result = new HashMap[String, HashMap[Int, Long]]() + implicit val order = new Ordering[TopicPartition] { + override def compare(x: TopicPartition, y: TopicPartition): Int = { + Ordering.Tuple2[String, Int].compare((x.topic, x.partition), (y.topic, y.partition)) + } + } + val partitions = partitionOffsets.keySet.toSeq.sorted // sort for more determinism + partitions.foreach { tp => + val off = partitionOffsets(tp) + val parts = result.getOrElse(tp.topic, new HashMap[Int, Long]) + parts += tp.partition -> off + result += tp.topic -> parts + } + Serialization.write(result) + } + + def partitionTimestamps(topicTimestamps: Map[TopicPartition, Long]): String = { + // For now it's same as partitionOffsets + partitionOffsets(topicTimestamps) + } +} diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/kafka010/KafkaSourceOffset.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/kafka010/KafkaSourceOffset.scala new file mode 100644 index 00000000..54fa0c52 --- /dev/null +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/kafka010/KafkaSourceOffset.scala @@ -0,0 +1,46 @@ + +package za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka.kafka010 + +import org.apache.kafka.common.TopicPartition +import org.apache.spark.sql.execution.streaming.{Offset, SerializedOffset} +import org.apache.spark.sql.sources.v2.reader.streaming.PartitionOffset +/** + * An [[Offset]] for the [[KafkaSource]]. This one tracks all partitions of subscribed topics and + * their offsets. + */ +case class KafkaSourceOffset(partitionToOffsets: Map[TopicPartition, Long]) extends Offset { + + override val json = JsonUtils.partitionOffsets(partitionToOffsets) +} + +private[kafka] +case class KafkaSourcePartitionOffset(topicPartition: TopicPartition, partitionOffset: Long) + extends PartitionOffset + +/** Companion object of the [[KafkaSourceOffset]] */ +private[kafka] object KafkaSourceOffset { + + def getPartitionOffsets(offset: Offset): Map[TopicPartition, Long] = { + offset match { + case o: KafkaSourceOffset => o.partitionToOffsets + case so: SerializedOffset => KafkaSourceOffset(so).partitionToOffsets + case _ => + throw new IllegalArgumentException( + s"Invalid conversion from offset of ${offset.getClass} to KafkaSourceOffset") + } + } + + /** + * Returns [[KafkaSourceOffset]] from a variable sequence of (topic, partitionId, offset) + * tuples. + */ + def apply(offsetTuples: (String, Int, Long)*): KafkaSourceOffset = { + KafkaSourceOffset(offsetTuples.map { case(t, p, o) => (new TopicPartition(t, p), o) }.toMap) + } + + /** + * Returns [[KafkaSourceOffset]] from a JSON [[SerializedOffset]] + */ + def apply(offset: SerializedOffset): KafkaSourceOffset = + KafkaSourceOffset(JsonUtils.partitionOffsets(offset.json)) +} From 1325b9e5a160aa0b6cf7dc56949c58b7609106c7 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Fri, 13 Nov 2020 17:33:17 +0100 Subject: [PATCH 02/27] refactored --- .../KafkaToKafkaDeduplicationDockerTest.scala | 2 + .../DeduplicateKafkaSinkTransformer.scala | 49 ++++++++++--------- 2 files changed, 28 insertions(+), 23 deletions(-) diff --git a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala index ec1eb61e..b699ad2b 100644 --- a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala +++ b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala @@ -36,6 +36,8 @@ import za.co.absa.commons.spark.SparkTestBase /** * This e2e test requires a Docker installation on the executing machine. */ +// TODO: Add testcase with multiple partitions +// TODO: Add testcase with at least one committed microbatch class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with SparkTestBase with BeforeAndAfter { private val fs = FileSystem.get(spark.sparkContext.hadoopConfiguration) diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala index 19cba3f7..88c880a1 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala @@ -73,29 +73,14 @@ private[transformer] class DeduplicateKafkaSinkTransformer( private def deduplicateDataFrame(dataFrame: DataFrame, offsetLog: OffsetSeqLog, commitLog: CommitLog) = { logger.info("Deduplicate rows after retry") val sourceConsumer = createConsumer(readerBrokers, readerExtraOptions, readerSchemaRegistryUrl) - seekToOffsets(sourceConsumer, offsetLog, commitLog) - - val sourceRecordsCount = try { - getAllAvailableMessagesCount(sourceConsumer) - } catch { - case throwable: Throwable => logger.error(s"An unexpected error occurred while consuming from topic $readerTopic", throwable) - throw throwable - } finally { - sourceConsumer.close() - } - + seekToLatestCommittedOffsets(sourceConsumer, offsetLog, commitLog) + val sourceRecordsCount = consumeAndClose(sourceConsumer, getAllAvailableMessagesCount) val sinkConsumer = createConsumer(writerBrokers, writerExtraOptions, writerSchemaRegistryUrl) val sinkTopicPartitions = getTopicPartitions(sinkConsumer, writerTopic) - val latestSinkRecords = try { - sinkTopicPartitions.map(topicPartition => { - getAtLeastNLatestRecords(sinkConsumer, topicPartition, sourceRecordsCount) + val latestSinkRecords = consumeAndClose(sinkConsumer, + (consumer: KafkaConsumer[GenericRecord, GenericRecord]) => sinkTopicPartitions.map { + topicPartition => getAtLeastNLatestRecords(consumer, topicPartition, sourceRecordsCount) }) - } catch { - case throwable: Throwable => logger.error(s"An unexpected error occurred while consuming from topic $writerTopic", throwable) - throw throwable - } finally { - sinkConsumer.close() - } val publishedIds = latestSinkRecords.flatten.map(record => { try { @@ -109,8 +94,23 @@ private[transformer] class DeduplicateKafkaSinkTransformer( dataFrame.filter(not(col(idColumn).isInCollection(publishedIds))) } - private def seekToOffsets(consumer: KafkaConsumer[GenericRecord, GenericRecord], offsetLog: OffsetSeqLog, commitLog: CommitLog) = { - val sourceTopicPartitionOffsetsOpt = getTopicPartitionsFromOffsets(offsetLog, commitLog) + private def consumeAndClose[T](consumer: KafkaConsumer[GenericRecord, GenericRecord], consume: KafkaConsumer[GenericRecord, GenericRecord] => T) = { + try { + consume(consumer) + } catch { + case throwable: Throwable => logger.error(s"An unexpected error occurred while consuming", throwable) + throw throwable + } finally { + consumer.close() + } + } + + /** + * Determines the latest committed offsets by inspecting structured streaming's offset log and commit log. + * If no committed offsets are available, seeks to beginning. + */ + private def seekToLatestCommittedOffsets(consumer: KafkaConsumer[GenericRecord, GenericRecord], offsetLog: OffsetSeqLog, commitLog: CommitLog): Unit = { + val sourceTopicPartitionOffsetsOpt = getTopicPartitionsFromLatestCommittedOffsets(offsetLog, commitLog) consumer.subscribe(Collections.singletonList(readerTopic), new ConsumerRebalanceListener { override def onPartitionsRevoked(partitions: util.Collection[TopicPartition]): Unit = {} @@ -151,6 +151,7 @@ private[transformer] class DeduplicateKafkaSinkTransformer( records } + // TODO: Move to KafkaUtils. Test with MockConsumer private def getAllAvailableMessagesCount(consumer: KafkaConsumer[GenericRecord, GenericRecord]): Int = { import scala.util.control.Breaks._ var recordsCount = 0 @@ -166,6 +167,7 @@ private[transformer] class DeduplicateKafkaSinkTransformer( recordsCount } +// TODO: Move to KafkaUtils. Test with MockConsumer private def getAllAvailableMessages(consumer: KafkaConsumer[GenericRecord, GenericRecord]) = { import scala.util.control.Breaks._ var records: Seq[ConsumerRecord[GenericRecord, GenericRecord]] = mutable.Seq() @@ -195,7 +197,8 @@ private[transformer] class DeduplicateKafkaSinkTransformer( new KafkaConsumer[GenericRecord, GenericRecord](props) } - def getTopicPartitionsFromOffsets(offsetLog: OffsetSeqLog, commitLog: CommitLog): Option[Map[TopicPartition, Long]] = { +// TODO: Move this to util class and test there + private def getTopicPartitionsFromLatestCommittedOffsets(offsetLog: OffsetSeqLog, commitLog: CommitLog): Option[Map[TopicPartition, Long]] = { val offsetSeqOpt = commitLog.getLatest().map(_._1) .flatMap(batchId => offsetLog.get(batchId)) .map(offsetLog => offsetLog.offsets) From b53ddce7b87c8f94d25a730e6e7d1beb33c66fa1 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Mon, 16 Nov 2020 08:47:01 +0100 Subject: [PATCH 03/27] wip --- .../deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala index 88c880a1..5d938d95 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala @@ -131,6 +131,7 @@ private[transformer] class DeduplicateKafkaSinkTransformer( consumer.partitionsFor(topic).asScala.map(p => new TopicPartition(p.topic(), p.partition())) } +// TODO: How to test while loop? Not possible with MockConsumer because it resets messages after each poll. E2E-Test? private def getAtLeastNLatestRecords(consumer: KafkaConsumer[GenericRecord, GenericRecord], topicPartition: TopicPartition, numberOfRecords: Int) = { consumer.assign(Seq(topicPartition).asJava) val endOffsets = consumer.endOffsets(Seq(topicPartition).asJava).asScala From e042a0dcc74efb4bdae5be4e17adfc0c65bd61fb Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Wed, 18 Nov 2020 10:22:46 +0100 Subject: [PATCH 04/27] Update test for multiple id columns --- .../KafkaToKafkaDeduplicationDockerTest.scala | 22 ++++++++++++++++--- 1 file changed, 19 insertions(+), 3 deletions(-) diff --git a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala index b699ad2b..051f146c 100644 --- a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala +++ b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala @@ -46,17 +46,33 @@ class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with Sp private val checkpointDir = s"$baseDirPath/checkpoint" behavior of "CommandLineIngestionDriver" - private def schemaString(topic: String) = raw"""{"type": "record", "name": "$topic", "fields": [ + + private val hyperdriveIdSchemaString = + raw"""{"type": "record", "name": "hyperdrive_id_record", "fields": [ + |{"type": "long", "name": "source_offset", "nullable": true}, + |{"type": "long", "name": "source_partition", "nullable": true} + |]} + |""".stripMargin + + private def schemaString(topic: String) = raw""" + {"type": "record", "name": "$topic", "fields": [ {"type": "string", "name": "record_id"}, - {"type": ["string", "null"], "name": "value_field"} + {"type": ["string", "null"], "name": "value_field"}, + {"type": "hyperdrive_id_record", "name": "hyperdrive_id"} ]}""" private def sendData(producer: KafkaProducer[GenericRecord, GenericRecord], from: Int, to: Int, topic: String) = { - val schema = new Parser().parse(schemaString(topic)) + val parser = new Parser() + val hyperdriveIdSchema = parser.parse(hyperdriveIdSchemaString) + val schema = parser.parse(schemaString(topic)) for (i <- from until to) { val valueRecord = new GenericData.Record(schema) valueRecord.put("record_id", i.toString) valueRecord.put("value_field", s"valueHello_$i") + val hyperdriveIdRecord = new GenericData.Record(hyperdriveIdSchema) + hyperdriveIdRecord.put("source_offset", i.toLong) + hyperdriveIdRecord.put("source_partition", 1L) + valueRecord.put("hyperdrive_id", hyperdriveIdRecord) val producerRecord = new ProducerRecord[GenericRecord, GenericRecord](topic, valueRecord) producer.send(producerRecord) From c2764010c17d88b4c666e3d34507808f121ee030 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Wed, 18 Nov 2020 11:52:01 +0100 Subject: [PATCH 05/27] Works with composite id --- .../KafkaToKafkaDeduplicationDockerTest.scala | 14 +++--- .../DeduplicateKafkaSinkTransformer.scala | 50 +++++++++++++++---- 2 files changed, 47 insertions(+), 17 deletions(-) diff --git a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala index 051f146c..0137ba29 100644 --- a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala +++ b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala @@ -56,7 +56,8 @@ class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with Sp private def schemaString(topic: String) = raw""" {"type": "record", "name": "$topic", "fields": [ - {"type": "string", "name": "record_id"}, + {"type": "int", "name": "record_id"}, + {"type": "string", "name": "record_id2"}, {"type": ["string", "null"], "name": "value_field"}, {"type": "hyperdrive_id_record", "name": "hyperdrive_id"} ]}""" @@ -67,7 +68,8 @@ class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with Sp val schema = parser.parse(schemaString(topic)) for (i <- from until to) { val valueRecord = new GenericData.Record(schema) - valueRecord.put("record_id", i.toString) + valueRecord.put("record_id", i/5) + valueRecord.put("record_id2", (i % 5).toString) valueRecord.put("value_field", s"valueHello_$i") val hyperdriveIdRecord = new GenericData.Record(hyperdriveIdSchema) hyperdriveIdRecord.put("source_offset", i.toLong) @@ -140,7 +142,7 @@ class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with Sp "transformer.[avro.decoder].value.schema.naming.strategy" -> "topic.name", // comma separated list of columns to select - "transformer.[kafka.deduplicator].id.column" -> "record_id", + "transformer.[kafka.deduplicator].id.column" -> "record_id,record_id2", "transformer.[kafka.deduplicator].schema.registry.url" -> "${transformer.[avro.decoder].schema.registry.url}", // Avro Encoder (ABRiS) settings @@ -168,9 +170,9 @@ class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with Sp records.size shouldBe allRecords val valueFieldNames = records.head.value().getSchema.getFields.asScala.map(_.name()) - valueFieldNames should contain theSameElementsAs List("record_id", "value_field") - val actual = records.map(_.value().get("record_id")) - val expected = List.range(0, allRecords).map(i => new Utf8(i.toString)) + valueFieldNames should contain theSameElementsAs List("record_id", "record_id2", "value_field", "hyperdrive_id") + val actual = records.map(_.value().get("value_field")) + val expected = List.range(0, allRecords).map(i => new Utf8(s"valueHello_$i")) actual should contain theSameElementsAs expected } diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala index 5d938d95..dc421375 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala @@ -30,11 +30,10 @@ import org.apache.logging.log4j.LogManager import org.apache.spark.sql.DataFrame import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog} import za.co.absa.hyperdrive.ingestor.api.transformer.{StreamTransformer, StreamTransformerFactory} -import za.co.absa.hyperdrive.ingestor.api.utils.ConfigUtils.{getOrThrow, getPropertySubset} +import za.co.absa.hyperdrive.ingestor.api.utils.ConfigUtils.{getOrThrow, getSeqOrThrow, getPropertySubset} import za.co.absa.hyperdrive.ingestor.api.utils.StreamWriterUtil import za.co.absa.hyperdrive.ingestor.api.writer.StreamWriterCommonAttributes import za.co.absa.hyperdrive.ingestor.implementation.reader.kafka.KafkaStreamReader -import za.co.absa.hyperdrive.ingestor.implementation.transformer.avro.confluent.{ConfluentAvroDecodingTransformer, ConfluentAvroEncodingTransformer} import za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka.kafka010.KafkaSourceOffset import za.co.absa.hyperdrive.ingestor.implementation.writer.kafka.KafkaStreamWriter @@ -52,9 +51,9 @@ private[transformer] class DeduplicateKafkaSinkTransformer( val writerBrokers: String, val writerExtraOptions: Map[String, String], val checkpointLocation: String, - val idColumn: String) extends StreamTransformer { + val idColumnNames: Seq[String]) extends StreamTransformer { private val logger = LogManager.getLogger - private val timeout = Duration.ofSeconds(5L) + private val timeout = Duration.ofSeconds(5L) // TODO: Make it configurable override def transform(dataFrame: DataFrame): DataFrame = { val spark = dataFrame.sparkSession @@ -74,24 +73,53 @@ private[transformer] class DeduplicateKafkaSinkTransformer( logger.info("Deduplicate rows after retry") val sourceConsumer = createConsumer(readerBrokers, readerExtraOptions, readerSchemaRegistryUrl) seekToLatestCommittedOffsets(sourceConsumer, offsetLog, commitLog) - val sourceRecordsCount = consumeAndClose(sourceConsumer, getAllAvailableMessagesCount) + val sourceRecords = consumeAndClose(sourceConsumer, getAllAvailableMessages) + val sourceIds = sourceRecords.map(record => { + try { + getIdColumnsFromSourceRecord(record) + } catch { + case throwable: Throwable => logger.error(s"Could not get $idColumnNames from record, schema is ${record.value().getSchema}", throwable) + throw throwable + } + }) val sinkConsumer = createConsumer(writerBrokers, writerExtraOptions, writerSchemaRegistryUrl) val sinkTopicPartitions = getTopicPartitions(sinkConsumer, writerTopic) val latestSinkRecords = consumeAndClose(sinkConsumer, (consumer: KafkaConsumer[GenericRecord, GenericRecord]) => sinkTopicPartitions.map { - topicPartition => getAtLeastNLatestRecords(consumer, topicPartition, sourceRecordsCount) + topicPartition => getAtLeastNLatestRecords(consumer, topicPartition, sourceRecords.size) }) val publishedIds = latestSinkRecords.flatten.map(record => { try { - record.value().get(idColumn).asInstanceOf[Utf8].toString + getIdColumnsFromSinkRecord(record) } catch { - case throwable: Throwable => logger.error(s"Could not get $idColumn from record, schema is ${record.value().getSchema}", throwable) + case throwable: Throwable => logger.error(s"Could not get $idColumnNames from record, schema is ${record.value().getSchema}", throwable) throw throwable } }) + + val duplicatedIds = sourceIds.intersect(publishedIds) + import org.apache.spark.sql.functions._ - dataFrame.filter(not(col(idColumn).isInCollection(publishedIds))) + val idColumns = idColumnNames.map(col) + val duplicatedIdsLit = duplicatedIds.map(duplicatedId => struct(duplicatedId.map(lit):_*)) + dataFrame.filter(not(struct(idColumns:_*).isInCollection(duplicatedIdsLit))) + } + + private def getIdColumnsFromSourceRecord(record: ConsumerRecord[GenericRecord, GenericRecord]): Seq[Any] = { + idColumnNames.map(idColumn => record.value().get(idColumn)) + .map { + case utf8: Utf8 => utf8.toString + case v => v + } + } + + private def getIdColumnsFromSinkRecord(record: ConsumerRecord[GenericRecord, GenericRecord]): Seq[Any] = { + idColumnNames.map(idColumn => record.value().get(idColumn)) + .map { + case utf8: Utf8 => utf8.toString + case v => v + } } private def consumeAndClose[T](consumer: KafkaConsumer[GenericRecord, GenericRecord], consume: KafkaConsumer[GenericRecord, GenericRecord] => T) = { @@ -235,10 +263,10 @@ object DeduplicateKafkaSinkTransformer extends StreamTransformerFactory with Ded val checkpointLocation = StreamWriterUtil.getCheckpointLocation(config) - val idColumn = getOrThrow(IdColumn, config, errorMessage = s"Destination directory not found. Is '${IdColumn}' defined?") + val idColumns = getSeqOrThrow(IdColumn, config, errorMessage = s"Destination directory not found. Is '${IdColumn}' defined?") new DeduplicateKafkaSinkTransformer(readerSchemaRegistryUrl, readerTopic, readerBrokers, readerExtraOptions, writerSchemaRegistryUrl, writerTopic, writerBrokers, writerExtraOptions, - checkpointLocation, idColumn) + checkpointLocation, idColumns) } override def getMappingFromRetainedGlobalConfigToLocalConfig(globalConfig: Configuration): Map[String, String] = { From a537cc4dcc2f10db344c228aa6da906c3de899ab Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Wed, 18 Nov 2020 17:02:04 +0100 Subject: [PATCH 06/27] works with nested columns --- .../KafkaToKafkaDeduplicationDockerTest.scala | 17 +++---- .../DeduplicateKafkaSinkTransformer.scala | 36 +++++++------ ...licateKafkaSinkTransformerAttributes.scala | 6 ++- .../implementation/utils/AvroUtil.scala | 29 +++++++++++ .../implementation/utils/TestAvroUtil.scala | 50 +++++++++++++++++++ 5 files changed, 108 insertions(+), 30 deletions(-) create mode 100644 ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/AvroUtil.scala create mode 100644 ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestAvroUtil.scala diff --git a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala index 0137ba29..c9d83b56 100644 --- a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala +++ b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala @@ -49,7 +49,7 @@ class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with Sp private val hyperdriveIdSchemaString = raw"""{"type": "record", "name": "hyperdrive_id_record", "fields": [ - |{"type": "long", "name": "source_offset", "nullable": true}, + |{"type": "string", "name": "source_offset", "nullable": true}, |{"type": "long", "name": "source_partition", "nullable": true} |]} |""".stripMargin @@ -57,7 +57,6 @@ class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with Sp private def schemaString(topic: String) = raw""" {"type": "record", "name": "$topic", "fields": [ {"type": "int", "name": "record_id"}, - {"type": "string", "name": "record_id2"}, {"type": ["string", "null"], "name": "value_field"}, {"type": "hyperdrive_id_record", "name": "hyperdrive_id"} ]}""" @@ -67,13 +66,12 @@ class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with Sp val hyperdriveIdSchema = parser.parse(hyperdriveIdSchemaString) val schema = parser.parse(schemaString(topic)) for (i <- from until to) { + val hyperdriveIdRecord = new GenericData.Record(hyperdriveIdSchema) + hyperdriveIdRecord.put("source_offset", (i / 5).toString) + hyperdriveIdRecord.put("source_partition", (i % 5).toLong) val valueRecord = new GenericData.Record(schema) - valueRecord.put("record_id", i/5) - valueRecord.put("record_id2", (i % 5).toString) + valueRecord.put("record_id", i) valueRecord.put("value_field", s"valueHello_$i") - val hyperdriveIdRecord = new GenericData.Record(hyperdriveIdSchema) - hyperdriveIdRecord.put("source_offset", i.toLong) - hyperdriveIdRecord.put("source_partition", 1L) valueRecord.put("hyperdrive_id", hyperdriveIdRecord) val producerRecord = new ProducerRecord[GenericRecord, GenericRecord](topic, valueRecord) @@ -142,7 +140,8 @@ class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with Sp "transformer.[avro.decoder].value.schema.naming.strategy" -> "topic.name", // comma separated list of columns to select - "transformer.[kafka.deduplicator].id.column" -> "record_id,record_id2", + "transformer.[kafka.deduplicator].source.id.columns" -> "hyperdrive_id.source_offset,hyperdrive_id.source_partition", + "transformer.[kafka.deduplicator].destination.id.columns" -> "${transformer.[kafka.deduplicator].source.id.columns}", "transformer.[kafka.deduplicator].schema.registry.url" -> "${transformer.[avro.decoder].schema.registry.url}", // Avro Encoder (ABRiS) settings @@ -170,7 +169,7 @@ class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with Sp records.size shouldBe allRecords val valueFieldNames = records.head.value().getSchema.getFields.asScala.map(_.name()) - valueFieldNames should contain theSameElementsAs List("record_id", "record_id2", "value_field", "hyperdrive_id") + valueFieldNames should contain theSameElementsAs List("record_id", "value_field", "hyperdrive_id") val actual = records.map(_.value().get("value_field")) val expected = List.range(0, allRecords).map(i => new Utf8(s"valueHello_$i")) actual should contain theSameElementsAs expected diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala index dc421375..a0462495 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala @@ -28,13 +28,15 @@ import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRebalanceListe import org.apache.kafka.common.TopicPartition import org.apache.logging.log4j.LogManager import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog} import za.co.absa.hyperdrive.ingestor.api.transformer.{StreamTransformer, StreamTransformerFactory} -import za.co.absa.hyperdrive.ingestor.api.utils.ConfigUtils.{getOrThrow, getSeqOrThrow, getPropertySubset} +import za.co.absa.hyperdrive.ingestor.api.utils.ConfigUtils.{getOrThrow, getPropertySubset, getSeqOrThrow} import za.co.absa.hyperdrive.ingestor.api.utils.StreamWriterUtil import za.co.absa.hyperdrive.ingestor.api.writer.StreamWriterCommonAttributes import za.co.absa.hyperdrive.ingestor.implementation.reader.kafka.KafkaStreamReader import za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka.kafka010.KafkaSourceOffset +import za.co.absa.hyperdrive.ingestor.implementation.utils.AvroUtil import za.co.absa.hyperdrive.ingestor.implementation.writer.kafka.KafkaStreamWriter import scala.collection.JavaConverters._ @@ -51,7 +53,9 @@ private[transformer] class DeduplicateKafkaSinkTransformer( val writerBrokers: String, val writerExtraOptions: Map[String, String], val checkpointLocation: String, - val idColumnNames: Seq[String]) extends StreamTransformer { + val sourceIdColumnNames: Seq[String], + val destinationIdColumnNames: Seq[String] +) extends StreamTransformer { private val logger = LogManager.getLogger private val timeout = Duration.ofSeconds(5L) // TODO: Make it configurable @@ -76,9 +80,9 @@ private[transformer] class DeduplicateKafkaSinkTransformer( val sourceRecords = consumeAndClose(sourceConsumer, getAllAvailableMessages) val sourceIds = sourceRecords.map(record => { try { - getIdColumnsFromSourceRecord(record) + getIdColumnsFromRecord(record, sourceIdColumnNames) } catch { - case throwable: Throwable => logger.error(s"Could not get $idColumnNames from record, schema is ${record.value().getSchema}", throwable) + case throwable: Throwable => logger.error(s"Could not get $sourceIdColumnNames from record, schema is ${record.value().getSchema}", throwable) throw throwable } }) @@ -91,9 +95,9 @@ private[transformer] class DeduplicateKafkaSinkTransformer( val publishedIds = latestSinkRecords.flatten.map(record => { try { - getIdColumnsFromSinkRecord(record) + getIdColumnsFromRecord(record, destinationIdColumnNames) } catch { - case throwable: Throwable => logger.error(s"Could not get $idColumnNames from record, schema is ${record.value().getSchema}", throwable) + case throwable: Throwable => logger.error(s"Could not get $destinationIdColumnNames from record, schema is ${record.value().getSchema}", throwable) throw throwable } }) @@ -101,21 +105,13 @@ private[transformer] class DeduplicateKafkaSinkTransformer( val duplicatedIds = sourceIds.intersect(publishedIds) import org.apache.spark.sql.functions._ - val idColumns = idColumnNames.map(col) + val idColumns = sourceIdColumnNames.map(col) // TODO: Make idColumns for dataframe configurable. Take into account rename trsf val duplicatedIdsLit = duplicatedIds.map(duplicatedId => struct(duplicatedId.map(lit):_*)) dataFrame.filter(not(struct(idColumns:_*).isInCollection(duplicatedIdsLit))) } - private def getIdColumnsFromSourceRecord(record: ConsumerRecord[GenericRecord, GenericRecord]): Seq[Any] = { - idColumnNames.map(idColumn => record.value().get(idColumn)) - .map { - case utf8: Utf8 => utf8.toString - case v => v - } - } - - private def getIdColumnsFromSinkRecord(record: ConsumerRecord[GenericRecord, GenericRecord]): Seq[Any] = { - idColumnNames.map(idColumn => record.value().get(idColumn)) + private def getIdColumnsFromRecord(record: ConsumerRecord[GenericRecord, GenericRecord], idColumnNames: Seq[String]): Seq[Any] = { + idColumnNames.map(idColumn => AvroUtil.getRecursively(record.value(), UnresolvedAttribute.parseAttributeName(idColumn).toList)) .map { case utf8: Utf8 => utf8.toString case v => v @@ -263,10 +259,12 @@ object DeduplicateKafkaSinkTransformer extends StreamTransformerFactory with Ded val checkpointLocation = StreamWriterUtil.getCheckpointLocation(config) - val idColumns = getSeqOrThrow(IdColumn, config, errorMessage = s"Destination directory not found. Is '${IdColumn}' defined?") + val sourceIdColumns = getSeqOrThrow(SourceIdColumns, config) + val destinationIdColumns = getSeqOrThrow(DestinationIdColumns, config) +// TODO: Check same length sourceId, destinationId new DeduplicateKafkaSinkTransformer(readerSchemaRegistryUrl, readerTopic, readerBrokers, readerExtraOptions, writerSchemaRegistryUrl, writerTopic, writerBrokers, writerExtraOptions, - checkpointLocation, idColumns) + checkpointLocation, sourceIdColumns, destinationIdColumns) } override def getMappingFromRetainedGlobalConfigToLocalConfig(globalConfig: Configuration): Map[String, String] = { diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformerAttributes.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformerAttributes.scala index e188b1bb..e941a053 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformerAttributes.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformerAttributes.scala @@ -19,7 +19,8 @@ import za.co.absa.hyperdrive.ingestor.api.{HasComponentAttributes, PropertyMetad trait DeduplicateKafkaSinkTransformerAttributes extends HasComponentAttributes { - val IdColumn = "id.column" + val SourceIdColumns = "source.id.columns" + val DestinationIdColumns = "destination.id.columns" val schemaRegistryUrl = "schema.registry.url" override def getName: String = "Deduplicate Kafka Sink Transformer" @@ -27,6 +28,7 @@ trait DeduplicateKafkaSinkTransformerAttributes extends HasComponentAttributes { override def getDescription: String = "" override def getProperties: Map[String, PropertyMetadata] = Map( - IdColumn -> PropertyMetadata("Id column", Some(""), required = true) + SourceIdColumns -> PropertyMetadata("Id column", Some(""), required = true), + DestinationIdColumns -> PropertyMetadata("Id column", Some(""), required = true) ) } diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/AvroUtil.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/AvroUtil.scala new file mode 100644 index 00000000..a84529ad --- /dev/null +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/AvroUtil.scala @@ -0,0 +1,29 @@ +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed 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 za.co.absa.hyperdrive.ingestor.implementation.utils + +import org.apache.avro.generic.GenericRecord + +import scala.annotation.tailrec + +private[hyperdrive] object AvroUtil { + + @tailrec + def getRecursively(record: GenericRecord, keys: List[String]): Any = keys match { + case key :: Nil => record.get(key) + case head :: tail => getRecursively(record.get(head).asInstanceOf[GenericRecord], tail) + } +} diff --git a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestAvroUtil.scala b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestAvroUtil.scala new file mode 100644 index 00000000..33fdf78a --- /dev/null +++ b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestAvroUtil.scala @@ -0,0 +1,50 @@ +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed 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 za.co.absa.hyperdrive.ingestor.implementation.utils + +import org.apache.avro.Schema.Parser +import org.apache.avro.generic.{GenericData, GenericRecordBuilder} +import org.scalatest.{BeforeAndAfter, FlatSpec, Matchers} + +class TestAvroUtil extends FlatSpec with Matchers with BeforeAndAfter { + "getRecursively" should "get recursively" in { + // given + val rootSchemaString = raw""" + {"type": "record", "name": "schemaName", "fields": [ + {"type": "int", "name": "record_id"}, + {"type": ["null", "schemaName"], "name": "child_record", "nullable": true} + ]}""" + + val parser = new Parser() + val schema = parser.parse(rootSchemaString) + + val childRecord2 = new GenericData.Record(schema) + childRecord2.put("record_id", 3) + childRecord2.put("child_record", null) + val childRecord1 = new GenericData.Record(schema) + childRecord1.put("record_id", 2) + childRecord1.put("child_record", childRecord2) + val rootRecord = new GenericData.Record(schema) + rootRecord.put("record_id", 1) + rootRecord.put("child_record", childRecord1) + + // when, then + AvroUtil.getRecursively(rootRecord, List("record_id")).asInstanceOf[Int] shouldBe 1 + AvroUtil.getRecursively(rootRecord, List("child_record", "record_id")).asInstanceOf[Int] shouldBe 2 + AvroUtil.getRecursively(rootRecord, List("child_record", "child_record", "record_id")).asInstanceOf[Int] shouldBe 3 + } + +} From a2aac7146660d3bce0fa66075eb0056d80e6c925 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Thu, 19 Nov 2020 11:30:33 +0100 Subject: [PATCH 07/27] #177: Configure multiple kafka partitions on testcase --- .../KafkaToKafkaDeduplicationDockerTest.scala | 38 +++++++++++++------ 1 file changed, 26 insertions(+), 12 deletions(-) diff --git a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala index c9d83b56..453241d2 100644 --- a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala +++ b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala @@ -15,16 +15,16 @@ package za.co.absa.hyperdrive.driver.drivers -import java.io.{BufferedWriter, File, FileWriter, OutputStreamWriter} -import java.nio.file.{Files, Paths} import java.time.Duration +import java.util import java.util.UUID.randomUUID import java.util.{Collections, Properties} -import org.apache.avro.Schema.{Parser, Type} +import org.apache.avro.Schema.Parser import org.apache.avro.generic.{GenericData, GenericRecord} import org.apache.avro.util.Utf8 import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.kafka.clients.admin.{AdminClient, AdminClientConfig, NewTopic} import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} import org.scalatest.{BeforeAndAfter, FlatSpec, Matchers} @@ -36,7 +36,6 @@ import za.co.absa.commons.spark.SparkTestBase /** * This e2e test requires a Docker installation on the executing machine. */ -// TODO: Add testcase with multiple partitions // TODO: Add testcase with at least one committed microbatch class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with SparkTestBase with BeforeAndAfter { @@ -61,7 +60,7 @@ class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with Sp {"type": "hyperdrive_id_record", "name": "hyperdrive_id"} ]}""" - private def sendData(producer: KafkaProducer[GenericRecord, GenericRecord], from: Int, to: Int, topic: String) = { + private def sendData(producer: KafkaProducer[GenericRecord, GenericRecord], from: Int, to: Int, topic: String, partitions: Int) = { val parser = new Parser() val hyperdriveIdSchema = parser.parse(hyperdriveIdSchemaString) val schema = parser.parse(schemaString(topic)) @@ -74,7 +73,8 @@ class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with Sp valueRecord.put("value_field", s"valueHello_$i") valueRecord.put("hyperdrive_id", hyperdriveIdRecord) - val producerRecord = new ProducerRecord[GenericRecord, GenericRecord](topic, valueRecord) + val partition = i % partitions + val producerRecord = new ProducerRecord[GenericRecord, GenericRecord](topic, partition, null, valueRecord) producer.send(producerRecord) } } @@ -87,25 +87,39 @@ class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with Sp out.close() } + private def createTopic(kafkaSchemaRegistryWrapper: KafkaSchemaRegistryWrapper, topicName: String, partitions: Int): Unit = { + val config = new Properties() + config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaSchemaRegistryWrapper.kafka.getBootstrapServers) + val localKafkaAdmin = AdminClient.create(config) + val replication = 1.toShort + val topic = new NewTopic(topicName, partitions, replication) + val topicCreationFut = localKafkaAdmin.createTopics(util.Arrays.asList(topic)).all() + while(!topicCreationFut.isDone) {} + } + it should "execute the whole kafka-to-kafka pipeline" in { // given val kafkaSchemaRegistryWrapper = new KafkaSchemaRegistryWrapper - + kafkaSchemaRegistryWrapper.kafka.start() val sourceTopic = "deduplication_src" val destinationTopic = "deduplication_dest" + val sourceTopicPartitions = 5 + val destinationTopicPartitions = 3 val allRecords = 250 val duplicatedRange1 = (0, 75) val duplicatedRange2 = (100, 175) val producer = createProducer(kafkaSchemaRegistryWrapper) - sendData(producer, 0, allRecords, sourceTopic) - sendData(producer, duplicatedRange1._1, duplicatedRange1._2, destinationTopic) - sendData(producer, duplicatedRange2._1, duplicatedRange2._2, destinationTopic) + createTopic(kafkaSchemaRegistryWrapper, sourceTopic, sourceTopicPartitions) + sendData(producer, 0, allRecords, sourceTopic, sourceTopicPartitions) + createTopic(kafkaSchemaRegistryWrapper, destinationTopic, destinationTopicPartitions) + sendData(producer, duplicatedRange1._1, duplicatedRange1._2, destinationTopic, destinationTopicPartitions) + sendData(producer, duplicatedRange2._1, duplicatedRange2._2, destinationTopic, destinationTopicPartitions) val offset = raw"""v1 |{"batchWatermarkMs":0,"batchTimestampMs":1605193344093,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"200"}} - |{"$sourceTopic":{"0":250}}""".stripMargin + |{"$sourceTopic":{"0":50, "1":50, "2":50, "3":50, "4":50}}""".stripMargin val sources = raw"""0v1 - |{"$sourceTopic":{"0":0}}""".stripMargin + |{"$sourceTopic":{"0":0, "1":0, "2":0, "3":0, "4":0}}""".stripMargin val metadata = raw"""{"id":"7a9e78ae-3473-469c-a906-c78ffaf0f3c9"}""" fs.mkdirs(new Path(s"$checkpointDir/$sourceTopic/commits")) From 13db638fe8ffbef18fdb32cb3e73e5a375409df2 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Wed, 25 Nov 2020 17:23:57 +0100 Subject: [PATCH 08/27] Attempt real test case --- .../KafkaToKafkaDeduplicationDockerTest.scala | 137 ++++++++++-------- .../DeduplicateKafkaSinkTransformer.scala | 41 ++++-- 2 files changed, 101 insertions(+), 77 deletions(-) diff --git a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala index 453241d2..8c1420e3 100644 --- a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala +++ b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala @@ -31,12 +31,11 @@ import org.scalatest.{BeforeAndAfter, FlatSpec, Matchers} import za.co.absa.abris.avro.read.confluent.SchemaManagerFactory import za.co.absa.commons.io.TempDirectory import za.co.absa.commons.spark.SparkTestBase - +import za.co.absa.abris.avro.registry.SchemaSubject /** * This e2e test requires a Docker installation on the executing machine. */ -// TODO: Add testcase with at least one committed microbatch class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with SparkTestBase with BeforeAndAfter { private val fs = FileSystem.get(spark.sparkContext.hadoopConfiguration) @@ -53,40 +52,27 @@ class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with Sp |]} |""".stripMargin - private def schemaString(topic: String) = raw""" - {"type": "record", "name": "$topic", "fields": [ - {"type": "int", "name": "record_id"}, - {"type": ["string", "null"], "name": "value_field"}, - {"type": "hyperdrive_id_record", "name": "hyperdrive_id"} - ]}""" - - private def sendData(producer: KafkaProducer[GenericRecord, GenericRecord], from: Int, to: Int, topic: String, partitions: Int) = { - val parser = new Parser() - val hyperdriveIdSchema = parser.parse(hyperdriveIdSchemaString) - val schema = parser.parse(schemaString(topic)) - for (i <- from until to) { - val hyperdriveIdRecord = new GenericData.Record(hyperdriveIdSchema) - hyperdriveIdRecord.put("source_offset", (i / 5).toString) - hyperdriveIdRecord.put("source_partition", (i % 5).toLong) - val valueRecord = new GenericData.Record(schema) - valueRecord.put("record_id", i) - valueRecord.put("value_field", s"valueHello_$i") - valueRecord.put("hyperdrive_id", hyperdriveIdRecord) - - val partition = i % partitions - val producerRecord = new ProducerRecord[GenericRecord, GenericRecord](topic, partition, null, valueRecord) - producer.send(producerRecord) + private def schemaV1String(name: String) = + raw"""{"type": "record", "name": "$name", "fields": [ + |{"type": "int", "name": "record_id"}, + |{"type": "string", "name": "value_field", "nullable": false} + |]}""".stripMargin + + private def schemaV2String(name: String) = + raw"""{"type": "record", "name": "$name", "fields": [ + |{"type": "int", "name": "record_id"}, + |{"type": ["null", "string"], "name": "value_field", "nullable": true} + |]}""".stripMargin + + private def sendData(producer: KafkaProducer[GenericRecord, GenericRecord], records: Seq[GenericRecord], topic: String, partitions: Int): Unit = { + records.zipWithIndex.foreach { + case (record, i) => + val partition = i % partitions + val producerRecord = new ProducerRecord[GenericRecord, GenericRecord](topic, partition, null, record) + producer.send(producerRecord) } } - private def writeToFile(filename: String, content: String) = { - val path = new Path(filename) - val out = fs.create(path) - out.writeBytes(content) - out.flush() - out.close() - } - private def createTopic(kafkaSchemaRegistryWrapper: KafkaSchemaRegistryWrapper, topicName: String, partitions: Int): Unit = { val config = new Properties() config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaSchemaRegistryWrapper.kafka.getBootstrapServers) @@ -100,32 +86,37 @@ class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with Sp it should "execute the whole kafka-to-kafka pipeline" in { // given val kafkaSchemaRegistryWrapper = new KafkaSchemaRegistryWrapper - kafkaSchemaRegistryWrapper.kafka.start() val sourceTopic = "deduplication_src" val destinationTopic = "deduplication_dest" val sourceTopicPartitions = 5 val destinationTopicPartitions = 3 - val allRecords = 250 - val duplicatedRange1 = (0, 75) - val duplicatedRange2 = (100, 175) + val schemaManager = SchemaManagerFactory.create(Map("schema.registry.url" -> kafkaSchemaRegistryWrapper.schemaRegistryUrl)) + val subject = SchemaSubject.usingTopicNameStrategy(sourceTopic) + val parserV1 = new Parser() + val schemaV1 = parserV1.parse(schemaV1String(sourceTopic)) + val parserV2 = new Parser() + val schemaV2 = parserV2.parse(schemaV2String(sourceTopic)) + val schemaV1Id = schemaManager.register(subject, schemaV1) + val schemaV2Id = schemaManager.register(subject, schemaV2) + val producer = createProducer(kafkaSchemaRegistryWrapper) createTopic(kafkaSchemaRegistryWrapper, sourceTopic, sourceTopicPartitions) - sendData(producer, 0, allRecords, sourceTopic, sourceTopicPartitions) createTopic(kafkaSchemaRegistryWrapper, destinationTopic, destinationTopicPartitions) - sendData(producer, duplicatedRange1._1, duplicatedRange1._2, destinationTopic, destinationTopicPartitions) - sendData(producer, duplicatedRange2._1, duplicatedRange2._2, destinationTopic, destinationTopicPartitions) - val offset = raw"""v1 - |{"batchWatermarkMs":0,"batchTimestampMs":1605193344093,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"200"}} - |{"$sourceTopic":{"0":50, "1":50, "2":50, "3":50, "4":50}}""".stripMargin - val sources = raw"""0v1 - |{"$sourceTopic":{"0":0, "1":0, "2":0, "3":0, "4":0}}""".stripMargin - val metadata = raw"""{"id":"7a9e78ae-3473-469c-a906-c78ffaf0f3c9"}""" - - fs.mkdirs(new Path(s"$checkpointDir/$sourceTopic/commits")) - writeToFile(s"$checkpointDir/$sourceTopic/offsets/0", offset) - writeToFile(s"$checkpointDir/$sourceTopic/sources/0/0", sources) - writeToFile(s"$checkpointDir/$sourceTopic/metadata", metadata) + val recordsV1 = (0 until 50).map(i => { + val valueRecord = new GenericData.Record(schemaV1) + valueRecord.put("record_id", i) + valueRecord.put("value_field", s"valueHello_$i") + valueRecord + }) + val recordsV2 = (50 until 100).map(i => { + val valueRecord = new GenericData.Record(schemaV2) + valueRecord.put("record_id", i) + valueRecord.put("value_field", null) + valueRecord + }) + sendData(producer, recordsV1, sourceTopic, sourceTopicPartitions) + sendData(producer, recordsV2, sourceTopic, sourceTopicPartitions) Thread.sleep(3000) @@ -133,28 +124,36 @@ class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with Sp // Pipeline settings "component.ingestor" -> "spark", "component.reader" -> "za.co.absa.hyperdrive.ingestor.implementation.reader.kafka.KafkaStreamReader", - "component.transformer.id.0" -> "[avro.decoder]", + "component.transformer.id.0" -> "[column.copy]", + "component.transformer.class.[column.copy]" -> "za.co.absa.hyperdrive.ingestor.implementation.transformer.column.copy.ColumnCopyStreamTransformer", + "component.transformer.id.1" -> "[avro.decoder]", "component.transformer.class.[avro.decoder]" -> "za.co.absa.hyperdrive.ingestor.implementation.transformer.avro.confluent.ConfluentAvroDecodingTransformer", - "component.transformer.id.1" -> "[kafka.deduplicator]", + "component.transformer.id.2" -> "[kafka.deduplicator]", "component.transformer.class.[kafka.deduplicator]" -> "za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka.DeduplicateKafkaSinkTransformer", - "component.transformer.id.2" -> "[avro.encoder]", + "component.transformer.id.3" -> "[avro.encoder]", "component.transformer.class.[avro.encoder]" -> "za.co.absa.hyperdrive.ingestor.implementation.transformer.avro.confluent.ConfluentAvroEncodingTransformer", "component.writer" -> "za.co.absa.hyperdrive.ingestor.implementation.writer.kafka.KafkaStreamWriter", // Spark settings "ingestor.spark.app.name" -> "ingestor-app", + "ingestor.spark.termination.timeout" -> "60000", // Source(Kafka) settings "reader.kafka.topic" -> sourceTopic, "reader.kafka.brokers" -> kafkaSchemaRegistryWrapper.kafkaUrl, + "reader.option.maxOffsetsPerTrigger" -> "20", + + "transformer.[column.copy].columns.copy.from" -> "offset, partition", + "transformer.[column.copy].columns.copy.to" -> "hyperdrive_id.source_offset, hyperdrive_id.source_partition", // Avro Decoder (ABRiS) settings "transformer.[avro.decoder].schema.registry.url" -> kafkaSchemaRegistryWrapper.schemaRegistryUrl, - "transformer.[avro.decoder].value.schema.id" -> "latest", + "transformer.[avro.decoder].value.schema.id" -> s"$schemaV1Id", "transformer.[avro.decoder].value.schema.naming.strategy" -> "topic.name", + "transformer.[avro.decoder].keep.columns" -> "hyperdrive_id", // comma separated list of columns to select - "transformer.[kafka.deduplicator].source.id.columns" -> "hyperdrive_id.source_offset,hyperdrive_id.source_partition", + "transformer.[kafka.deduplicator].source.id.columns" -> "offset,partition", "transformer.[kafka.deduplicator].destination.id.columns" -> "${transformer.[kafka.deduplicator].source.id.columns}", "transformer.[kafka.deduplicator].schema.registry.url" -> "${transformer.[avro.decoder].schema.registry.url}", @@ -164,28 +163,44 @@ class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with Sp // Sink(Kafka) settings "writer.common.checkpoint.location" -> (checkpointDir + "/${reader.kafka.topic}"), - "writer.common.trigger.type" -> "Once", + "writer.common.trigger.type" -> "ProcessingTime", "writer.kafka.topic" -> destinationTopic, "writer.kafka.brokers" -> "${reader.kafka.brokers}" ) val driverConfigArray = driverConfig.map { case (key, value) => s"$key=$value" }.toArray // when - CommandLineIngestionDriver.main(driverConfigArray) + var exceptionWasThrown = false + try { + CommandLineIngestionDriver.main(driverConfigArray) + } catch { + case e: Exception => + exceptionWasThrown = true + val retryConfig = driverConfig ++ Map( + "transformer.[avro.decoder].value.schema.id" -> s"$schemaV2Id", + "writer.common.trigger.type" -> "Once", + "reader.option.maxOffsetsPerTrigger" -> "9999" + ) + val retryConfigArray = retryConfig.map { case (key, value) => s"$key=$value"}.toArray + CommandLineIngestionDriver.main(retryConfigArray) + } + + exceptionWasThrown shouldBe true // then fs.exists(new Path(s"$checkpointDir/$sourceTopic")) shouldBe true + val allRecords = recordsV1.size + recordsV2.size val consumer = createConsumer(kafkaSchemaRegistryWrapper) consumer.subscribe(Collections.singletonList(destinationTopic)) import scala.collection.JavaConverters._ - val records = consumer.poll(Duration.ofMillis(500L)).asScala.toList - records.size shouldBe allRecords + val records = consumer.poll(Duration.ofMillis(1000L)).asScala.toList +// records.size shouldBe allRecords val valueFieldNames = records.head.value().getSchema.getFields.asScala.map(_.name()) valueFieldNames should contain theSameElementsAs List("record_id", "value_field", "hyperdrive_id") val actual = records.map(_.value().get("value_field")) - val expected = List.range(0, allRecords).map(i => new Utf8(s"valueHello_$i")) + val expected = (0 until allRecords).map(i => new Utf8(s"valueHello_$i")) actual should contain theSameElementsAs expected } diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala index a0462495..fcd9783c 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala @@ -19,7 +19,7 @@ import java.time.Duration import java.util import java.util.{Collections, Properties, UUID} -import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig +//import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig import org.apache.avro.generic.GenericRecord import org.apache.avro.util.Utf8 import org.apache.commons.configuration2.Configuration @@ -106,16 +106,25 @@ private[transformer] class DeduplicateKafkaSinkTransformer( import org.apache.spark.sql.functions._ val idColumns = sourceIdColumnNames.map(col) // TODO: Make idColumns for dataframe configurable. Take into account rename trsf - val duplicatedIdsLit = duplicatedIds.map(duplicatedId => struct(duplicatedId.map(lit):_*)) - dataFrame.filter(not(struct(idColumns:_*).isInCollection(duplicatedIdsLit))) + val duplicatedIdsLit = duplicatedIds.map(duplicatedId => struct(duplicatedId.map(lit): _*)) + dataFrame.filter(not(struct(idColumns: _*).isInCollection(duplicatedIdsLit))) } private def getIdColumnsFromRecord(record: ConsumerRecord[GenericRecord, GenericRecord], idColumnNames: Seq[String]): Seq[Any] = { - idColumnNames.map(idColumn => AvroUtil.getRecursively(record.value(), UnresolvedAttribute.parseAttributeName(idColumn).toList)) - .map { - case utf8: Utf8 => utf8.toString - case v => v - } + idColumnNames.map { + case "topic" => record.topic() + case "offset" => record.offset() + case "partition" => record.partition() + case "timestamp" => record.timestamp() + case "timestampType" => record.timestampType() + case "serializedKeySize" => record.serializedKeySize() + case "serializedValueSize" => record.serializedValueSize() + case "headers" => record.headers() + case idColumn => AvroUtil.getRecursively(record.value(), UnresolvedAttribute.parseAttributeName(idColumn).toList) + }.map { + case utf8: Utf8 => utf8.toString + case v => v + } } private def consumeAndClose[T](consumer: KafkaConsumer[GenericRecord, GenericRecord], consume: KafkaConsumer[GenericRecord, GenericRecord] => T) = { @@ -155,7 +164,7 @@ private[transformer] class DeduplicateKafkaSinkTransformer( consumer.partitionsFor(topic).asScala.map(p => new TopicPartition(p.topic(), p.partition())) } -// TODO: How to test while loop? Not possible with MockConsumer because it resets messages after each poll. E2E-Test? + // TODO: How to test while loop? Not possible with MockConsumer because it resets messages after each poll. E2E-Test? private def getAtLeastNLatestRecords(consumer: KafkaConsumer[GenericRecord, GenericRecord], topicPartition: TopicPartition, numberOfRecords: Int) = { consumer.assign(Seq(topicPartition).asJava) val endOffsets = consumer.endOffsets(Seq(topicPartition).asJava).asScala @@ -167,7 +176,7 @@ private[transformer] class DeduplicateKafkaSinkTransformer( var records: Seq[ConsumerRecord[GenericRecord, GenericRecord]] = Seq() var offsetLowerBound = offset - while(records.size < numberOfRecords && offsetLowerBound != 0) { + while (records.size < numberOfRecords && offsetLowerBound != 0) { offsetLowerBound = Math.max(0, offsetLowerBound - numberOfRecords) consumer.seek(partition, offsetLowerBound) records = getAllAvailableMessages(consumer) @@ -192,7 +201,7 @@ private[transformer] class DeduplicateKafkaSinkTransformer( recordsCount } -// TODO: Move to KafkaUtils. Test with MockConsumer + // TODO: Move to KafkaUtils. Test with MockConsumer private def getAllAvailableMessages(consumer: KafkaConsumer[GenericRecord, GenericRecord]) = { import scala.util.control.Breaks._ var records: Seq[ConsumerRecord[GenericRecord, GenericRecord]] = mutable.Seq() @@ -216,13 +225,13 @@ private[transformer] class DeduplicateKafkaSinkTransformer( extraOptions.foreach { case (key, value) => props.put(key, value) } - props.put(AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, schemaRegistryUrl) + props.put("schema.registry.url", schemaRegistryUrl) props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "io.confluent.kafka.serializers.KafkaAvroDeserializer") props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "io.confluent.kafka.serializers.KafkaAvroDeserializer") new KafkaConsumer[GenericRecord, GenericRecord](props) } -// TODO: Move this to util class and test there + // TODO: Move this to util class and test there private def getTopicPartitionsFromLatestCommittedOffsets(offsetLog: OffsetSeqLog, commitLog: CommitLog): Option[Map[TopicPartition, Long]] = { val offsetSeqOpt = commitLog.getLatest().map(_._1) .flatMap(batchId => offsetLog.get(batchId)) @@ -235,7 +244,7 @@ private[transformer] class DeduplicateKafkaSinkTransformer( throw new IllegalStateException("Offset must be defined, got None") } } else { - throw new IllegalStateException(s"Cannot support more than 1 source, got ${offsetSeqOpt.toString}") + throw new IllegalStateException(s"Cannot support more than 1 source, got ${offsetSeqOpt.toString}") } } else { None @@ -261,14 +270,14 @@ object DeduplicateKafkaSinkTransformer extends StreamTransformerFactory with Ded val sourceIdColumns = getSeqOrThrow(SourceIdColumns, config) val destinationIdColumns = getSeqOrThrow(DestinationIdColumns, config) -// TODO: Check same length sourceId, destinationId + // TODO: Check same length sourceId, destinationId new DeduplicateKafkaSinkTransformer(readerSchemaRegistryUrl, readerTopic, readerBrokers, readerExtraOptions, writerSchemaRegistryUrl, writerTopic, writerBrokers, writerExtraOptions, checkpointLocation, sourceIdColumns, destinationIdColumns) } override def getMappingFromRetainedGlobalConfigToLocalConfig(globalConfig: Configuration): Map[String, String] = { -// TODO: What about subsets? + // TODO: What about subsets? Set( KafkaStreamReader.KEY_TOPIC, KafkaStreamReader.KEY_BROKERS, From 42618b11cb56e7f7f46e04bc883a5ff800c01224 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Wed, 25 Nov 2020 17:59:40 +0100 Subject: [PATCH 09/27] Doesn't work with multiple partitions --- .../KafkaToKafkaDeduplicationDockerTest.scala | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala index 8c1420e3..db6e0fa9 100644 --- a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala +++ b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala @@ -41,7 +41,8 @@ class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with Sp private val fs = FileSystem.get(spark.sparkContext.hadoopConfiguration) private val baseDir = TempDirectory("hyperdriveE2eTest").deleteOnExit() private val baseDirPath = baseDir.path.toUri - private val checkpointDir = s"$baseDirPath/checkpoint" +// private val checkpointDir = s"$baseDirPath/checkpoint" + private val checkpointDir = s"/tmp/bla1/checkpoint" behavior of "CommandLineIngestionDriver" @@ -88,8 +89,10 @@ class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with Sp val kafkaSchemaRegistryWrapper = new KafkaSchemaRegistryWrapper val sourceTopic = "deduplication_src" val destinationTopic = "deduplication_dest" - val sourceTopicPartitions = 5 - val destinationTopicPartitions = 3 +// val sourceTopicPartitions = 5 + val sourceTopicPartitions = 1 +// val destinationTopicPartitions = 3 + val destinationTopicPartitions = 1 val schemaManager = SchemaManagerFactory.create(Map("schema.registry.url" -> kafkaSchemaRegistryWrapper.schemaRegistryUrl)) val subject = SchemaSubject.usingTopicNameStrategy(sourceTopic) val parserV1 = new Parser() @@ -183,6 +186,7 @@ class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with Sp ) val retryConfigArray = retryConfig.map { case (key, value) => s"$key=$value"}.toArray CommandLineIngestionDriver.main(retryConfigArray) + CommandLineIngestionDriver.main(retryConfigArray) } exceptionWasThrown shouldBe true @@ -199,8 +203,8 @@ class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with Sp val valueFieldNames = records.head.value().getSchema.getFields.asScala.map(_.name()) valueFieldNames should contain theSameElementsAs List("record_id", "value_field", "hyperdrive_id") - val actual = records.map(_.value().get("value_field")) - val expected = (0 until allRecords).map(i => new Utf8(s"valueHello_$i")) + val actual = records.map(_.value().get("record_id")) + val expected = 0 until allRecords actual should contain theSameElementsAs expected } From bfe0fda788f643563d926b718ca82286a1041465 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Thu, 26 Nov 2020 11:21:50 +0100 Subject: [PATCH 10/27] works now --- ...kaDeduplicationAfterRetryDockerTest.scala} | 64 +++++++++++-------- 1 file changed, 36 insertions(+), 28 deletions(-) rename driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/{KafkaToKafkaDeduplicationDockerTest.scala => KafkaToKafkaDeduplicationAfterRetryDockerTest.scala} (79%) diff --git a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala similarity index 79% rename from driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala rename to driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala index db6e0fa9..1663a6fb 100644 --- a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationDockerTest.scala +++ b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala @@ -22,7 +22,6 @@ import java.util.{Collections, Properties} import org.apache.avro.Schema.Parser import org.apache.avro.generic.{GenericData, GenericRecord} -import org.apache.avro.util.Utf8 import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.kafka.clients.admin.{AdminClient, AdminClientConfig, NewTopic} import org.apache.kafka.clients.consumer.KafkaConsumer @@ -32,27 +31,39 @@ import za.co.absa.abris.avro.read.confluent.SchemaManagerFactory import za.co.absa.commons.io.TempDirectory import za.co.absa.commons.spark.SparkTestBase import za.co.absa.abris.avro.registry.SchemaSubject +import za.co.absa.hyperdrive.shared.exceptions.IngestionException /** * This e2e test requires a Docker installation on the executing machine. + * In this test, 50 messages with schema v1 are written to the source topic, followed by 50 messages with schema v2. + * Schema v2 contains a forward-incompatible change, i.e. messages written with v2 cannot be read with v1. + * + * The first run is configured as a long-running job (writer.common.trigger.type=ProcessingTime) and with a maximum + * number of messages per micro-batch set to 20 (reader.option.maxOffsetsPerTrigger=20). Furthermore, the schema id is + * explicitly set for v1 (see transformer.[avro.decoder].value.schema.id). Due to the forward-incompatible change, + * it will fail at the 51st message, which was written with schema v2. At this point, 2 micro-batches (i.e. 40 messages) + * have been successfully committed, while the 3rd has failed half-way through. 50 messages have been written + * to the destination topic. + * + * To successfully rerun, the schema id needs to be set to use schema v2. In order to avoid an infinite runtime, the + * trigger is set to Once. The Deduplication transformer ensures that the 41st-50th messages are not written to the + * destination topic again. In this test, offset and partition from the source topic are used as a composite id + * to identify messages across the topics (See transformer.[kafka.deduplicator].source.id.columns + * and transformer.[kafka.deduplicator].destination.id.columns) + * + * Finally, the destination topic is expected to contain all messages from the source topic + * exactly once. Without the deduplication transformer, the 41st-50th messages would be duplicated. */ -class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with SparkTestBase with BeforeAndAfter { +class KafkaToKafkaDeduplicationAfterRetryDockerTest extends FlatSpec with Matchers with SparkTestBase with BeforeAndAfter { + import scala.collection.JavaConverters._ private val fs = FileSystem.get(spark.sparkContext.hadoopConfiguration) private val baseDir = TempDirectory("hyperdriveE2eTest").deleteOnExit() private val baseDirPath = baseDir.path.toUri -// private val checkpointDir = s"$baseDirPath/checkpoint" - private val checkpointDir = s"/tmp/bla1/checkpoint" + private val checkpointDir = s"$baseDirPath/checkpoint" behavior of "CommandLineIngestionDriver" - private val hyperdriveIdSchemaString = - raw"""{"type": "record", "name": "hyperdrive_id_record", "fields": [ - |{"type": "string", "name": "source_offset", "nullable": true}, - |{"type": "long", "name": "source_partition", "nullable": true} - |]} - |""".stripMargin - private def schemaV1String(name: String) = raw"""{"type": "record", "name": "$name", "fields": [ |{"type": "int", "name": "record_id"}, @@ -89,10 +100,8 @@ class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with Sp val kafkaSchemaRegistryWrapper = new KafkaSchemaRegistryWrapper val sourceTopic = "deduplication_src" val destinationTopic = "deduplication_dest" -// val sourceTopicPartitions = 5 - val sourceTopicPartitions = 1 -// val destinationTopicPartitions = 3 - val destinationTopicPartitions = 1 + val sourceTopicPartitions = 5 + val destinationTopicPartitions = 3 val schemaManager = SchemaManagerFactory.create(Map("schema.registry.url" -> kafkaSchemaRegistryWrapper.schemaRegistryUrl)) val subject = SchemaSubject.usingTopicNameStrategy(sourceTopic) val parserV1 = new Parser() @@ -157,7 +166,7 @@ class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with Sp // comma separated list of columns to select "transformer.[kafka.deduplicator].source.id.columns" -> "offset,partition", - "transformer.[kafka.deduplicator].destination.id.columns" -> "${transformer.[kafka.deduplicator].source.id.columns}", + "transformer.[kafka.deduplicator].destination.id.columns" -> "hyperdrive_id.source_offset, hyperdrive_id.source_partition", "transformer.[kafka.deduplicator].schema.registry.url" -> "${transformer.[avro.decoder].schema.registry.url}", // Avro Encoder (ABRiS) settings @@ -172,40 +181,39 @@ class KafkaToKafkaDeduplicationDockerTest extends FlatSpec with Matchers with Sp ) val driverConfigArray = driverConfig.map { case (key, value) => s"$key=$value" }.toArray - // when + // when, then var exceptionWasThrown = false try { CommandLineIngestionDriver.main(driverConfigArray) } catch { - case e: Exception => + case _: IngestionException => exceptionWasThrown = true + val consumer = createConsumer(kafkaSchemaRegistryWrapper) + consumer.subscribe(Collections.singletonList(destinationTopic)) + val records = consumer.poll(Duration.ofMillis(1000L)).asScala.toList + records.size shouldBe recordsV1.size val retryConfig = driverConfig ++ Map( "transformer.[avro.decoder].value.schema.id" -> s"$schemaV2Id", "writer.common.trigger.type" -> "Once", "reader.option.maxOffsetsPerTrigger" -> "9999" ) val retryConfigArray = retryConfig.map { case (key, value) => s"$key=$value"}.toArray - CommandLineIngestionDriver.main(retryConfigArray) - CommandLineIngestionDriver.main(retryConfigArray) + CommandLineIngestionDriver.main(retryConfigArray) // first rerun only retries the failed micro-batch + CommandLineIngestionDriver.main(retryConfigArray) // second rerun consumes the rest of the messages } exceptionWasThrown shouldBe true - - // then fs.exists(new Path(s"$checkpointDir/$sourceTopic")) shouldBe true - val allRecords = recordsV1.size + recordsV2.size val consumer = createConsumer(kafkaSchemaRegistryWrapper) consumer.subscribe(Collections.singletonList(destinationTopic)) - import scala.collection.JavaConverters._ val records = consumer.poll(Duration.ofMillis(1000L)).asScala.toList -// records.size shouldBe allRecords val valueFieldNames = records.head.value().getSchema.getFields.asScala.map(_.name()) valueFieldNames should contain theSameElementsAs List("record_id", "value_field", "hyperdrive_id") - val actual = records.map(_.value().get("record_id")) - val expected = 0 until allRecords - actual should contain theSameElementsAs expected + val actualRecordIds = records.map(_.value().get("record_id")) + val expectedRecordIds = 0 until recordsV1.size + recordsV2.size + actualRecordIds should contain theSameElementsAs expectedRecordIds } after { From b577e3908bcb486795cab26bcf4fa2d74f75de25 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Thu, 26 Nov 2020 14:30:20 +0100 Subject: [PATCH 11/27] Generalize kafka record selection --- .../KafkaToKafkaDeduplicationAfterRetryDockerTest.scala | 2 +- .../deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala | 5 ++++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala index 1663a6fb..8fdeb51d 100644 --- a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala +++ b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala @@ -166,7 +166,7 @@ class KafkaToKafkaDeduplicationAfterRetryDockerTest extends FlatSpec with Matche // comma separated list of columns to select "transformer.[kafka.deduplicator].source.id.columns" -> "offset,partition", - "transformer.[kafka.deduplicator].destination.id.columns" -> "hyperdrive_id.source_offset, hyperdrive_id.source_partition", + "transformer.[kafka.deduplicator].destination.id.columns" -> "value.hyperdrive_id.source_offset, value.hyperdrive_id.source_partition", "transformer.[kafka.deduplicator].schema.registry.url" -> "${transformer.[avro.decoder].schema.registry.url}", // Avro Encoder (ABRiS) settings diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala index fcd9783c..cc1cf770 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala @@ -120,7 +120,10 @@ private[transformer] class DeduplicateKafkaSinkTransformer( case "serializedKeySize" => record.serializedKeySize() case "serializedValueSize" => record.serializedValueSize() case "headers" => record.headers() - case idColumn => AvroUtil.getRecursively(record.value(), UnresolvedAttribute.parseAttributeName(idColumn).toList) + case keyColumn if keyColumn.startsWith("key.") => AvroUtil.getRecursively(record.value(), + UnresolvedAttribute.parseAttributeName(keyColumn.stripPrefix("key.")).toList) + case valueColumn if valueColumn.startsWith("value.") => AvroUtil.getRecursively(record.value(), + UnresolvedAttribute.parseAttributeName(valueColumn.stripPrefix("value.")).toList) }.map { case utf8: Utf8 => utf8.toString case v => v From 864ed48536c791fee5f5c349ffec59597e17a876 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Thu, 26 Nov 2020 17:24:14 +0100 Subject: [PATCH 12/27] Refactoring + Tests wip --- .../DeduplicateKafkaSinkTransformer.scala | 217 ++++-------------- .../kafka/kafka010/KafkaSourceOffset.scala | 4 +- .../implementation/utils/AvroUtil.scala | 23 ++ .../implementation/utils/KafkaUtil.scala | 113 +++++++++ .../kafka/KafkaStreamWriterAttributes.scala | 2 + .../implementation/utils/TestKafkaUtil.scala | 46 ++++ 6 files changed, 234 insertions(+), 171 deletions(-) create mode 100644 ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala create mode 100644 ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtil.scala diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala index cc1cf770..f84224b4 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala @@ -16,32 +16,27 @@ package za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka import java.time.Duration -import java.util -import java.util.{Collections, Properties, UUID} +import java.util.{Properties, UUID} + +import za.co.absa.hyperdrive.ingestor.implementation.utils.KafkaUtil //import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig import org.apache.avro.generic.GenericRecord -import org.apache.avro.util.Utf8 import org.apache.commons.configuration2.Configuration import org.apache.hadoop.fs.Path -import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRebalanceListener, ConsumerRecord, KafkaConsumer} -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer} import org.apache.logging.log4j.LogManager import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog} +import org.apache.spark.sql.functions.{col, lit, struct, not} import za.co.absa.hyperdrive.ingestor.api.transformer.{StreamTransformer, StreamTransformerFactory} import za.co.absa.hyperdrive.ingestor.api.utils.ConfigUtils.{getOrThrow, getPropertySubset, getSeqOrThrow} import za.co.absa.hyperdrive.ingestor.api.utils.StreamWriterUtil import za.co.absa.hyperdrive.ingestor.api.writer.StreamWriterCommonAttributes import za.co.absa.hyperdrive.ingestor.implementation.reader.kafka.KafkaStreamReader -import za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka.kafka010.KafkaSourceOffset import za.co.absa.hyperdrive.ingestor.implementation.utils.AvroUtil import za.co.absa.hyperdrive.ingestor.implementation.writer.kafka.KafkaStreamWriter -import scala.collection.JavaConverters._ -import scala.collection.mutable - private[transformer] class DeduplicateKafkaSinkTransformer( val readerSchemaRegistryUrl: String, @@ -57,7 +52,6 @@ private[transformer] class DeduplicateKafkaSinkTransformer( val destinationIdColumnNames: Seq[String] ) extends StreamTransformer { private val logger = LogManager.getLogger - private val timeout = Duration.ofSeconds(5L) // TODO: Make it configurable override def transform(dataFrame: DataFrame): DataFrame = { val spark = dataFrame.sparkSession @@ -75,58 +69,32 @@ private[transformer] class DeduplicateKafkaSinkTransformer( private def deduplicateDataFrame(dataFrame: DataFrame, offsetLog: OffsetSeqLog, commitLog: CommitLog) = { logger.info("Deduplicate rows after retry") + implicit val kafkaConsumerTimeout: Duration = Duration.ofSeconds(5L) // TODO: Make it configurable val sourceConsumer = createConsumer(readerBrokers, readerExtraOptions, readerSchemaRegistryUrl) - seekToLatestCommittedOffsets(sourceConsumer, offsetLog, commitLog) - val sourceRecords = consumeAndClose(sourceConsumer, getAllAvailableMessages) - val sourceIds = sourceRecords.map(record => { - try { - getIdColumnsFromRecord(record, sourceIdColumnNames) - } catch { - case throwable: Throwable => logger.error(s"Could not get $sourceIdColumnNames from record, schema is ${record.value().getSchema}", throwable) - throw throwable - } - }) + KafkaUtil.seekToLatestCommittedOffsets(sourceConsumer, readerTopic, offsetLog, commitLog) + val sourceRecords = consumeAndClose(sourceConsumer, KafkaUtil.getAllAvailableMessages) + val sourceIds = sourceRecords.map(extractIdFieldsFromRecord(_, sourceIdColumnNames)) + val sinkConsumer = createConsumer(writerBrokers, writerExtraOptions, writerSchemaRegistryUrl) - val sinkTopicPartitions = getTopicPartitions(sinkConsumer, writerTopic) + val sinkTopicPartitions = KafkaUtil.getTopicPartitions(sinkConsumer, writerTopic) val latestSinkRecords = consumeAndClose(sinkConsumer, (consumer: KafkaConsumer[GenericRecord, GenericRecord]) => sinkTopicPartitions.map { - topicPartition => getAtLeastNLatestRecords(consumer, topicPartition, sourceRecords.size) + topicPartition => KafkaUtil.getAtLeastNLatestRecords(consumer, topicPartition, sourceRecords.size) }) - - val publishedIds = latestSinkRecords.flatten.map(record => { - try { - getIdColumnsFromRecord(record, destinationIdColumnNames) - } catch { - case throwable: Throwable => logger.error(s"Could not get $destinationIdColumnNames from record, schema is ${record.value().getSchema}", throwable) - throw throwable - } - }) + val publishedIds = latestSinkRecords.flatten.map(extractIdFieldsFromRecord(_, destinationIdColumnNames)) val duplicatedIds = sourceIds.intersect(publishedIds) - - import org.apache.spark.sql.functions._ - val idColumns = sourceIdColumnNames.map(col) // TODO: Make idColumns for dataframe configurable. Take into account rename trsf val duplicatedIdsLit = duplicatedIds.map(duplicatedId => struct(duplicatedId.map(lit): _*)) + val idColumns = sourceIdColumnNames.map(col) dataFrame.filter(not(struct(idColumns: _*).isInCollection(duplicatedIdsLit))) } - private def getIdColumnsFromRecord(record: ConsumerRecord[GenericRecord, GenericRecord], idColumnNames: Seq[String]): Seq[Any] = { - idColumnNames.map { - case "topic" => record.topic() - case "offset" => record.offset() - case "partition" => record.partition() - case "timestamp" => record.timestamp() - case "timestampType" => record.timestampType() - case "serializedKeySize" => record.serializedKeySize() - case "serializedValueSize" => record.serializedValueSize() - case "headers" => record.headers() - case keyColumn if keyColumn.startsWith("key.") => AvroUtil.getRecursively(record.value(), - UnresolvedAttribute.parseAttributeName(keyColumn.stripPrefix("key.")).toList) - case valueColumn if valueColumn.startsWith("value.") => AvroUtil.getRecursively(record.value(), - UnresolvedAttribute.parseAttributeName(valueColumn.stripPrefix("value.")).toList) - }.map { - case utf8: Utf8 => utf8.toString - case v => v + private def extractIdFieldsFromRecord(record: ConsumerRecord[GenericRecord, GenericRecord], idColumnNames: Seq[String]): Seq[Any] = { + try { + AvroUtil.getIdColumnsFromRecord(record, idColumnNames) + } catch { + case throwable: Throwable => logger.error(s"Could not get $idColumnNames from record, schema is ${record.value().getSchema}", throwable) + throw throwable } } @@ -141,85 +109,6 @@ private[transformer] class DeduplicateKafkaSinkTransformer( } } - /** - * Determines the latest committed offsets by inspecting structured streaming's offset log and commit log. - * If no committed offsets are available, seeks to beginning. - */ - private def seekToLatestCommittedOffsets(consumer: KafkaConsumer[GenericRecord, GenericRecord], offsetLog: OffsetSeqLog, commitLog: CommitLog): Unit = { - val sourceTopicPartitionOffsetsOpt = getTopicPartitionsFromLatestCommittedOffsets(offsetLog, commitLog) - consumer.subscribe(Collections.singletonList(readerTopic), new ConsumerRebalanceListener { - override def onPartitionsRevoked(partitions: util.Collection[TopicPartition]): Unit = {} - - override def onPartitionsAssigned(partitions: util.Collection[TopicPartition]): Unit = { - sourceTopicPartitionOffsetsOpt match { - case Some(topicPartitionOffsets) => topicPartitionOffsets.foreach { - case (topicPartition, offset) => consumer.seek(topicPartition, offset) - } - case None => - val partitions = getTopicPartitions(consumer, readerTopic) - consumer.seekToBeginning(partitions.asJava) - } - } - }) - } - - private def getTopicPartitions(consumer: KafkaConsumer[GenericRecord, GenericRecord], topic: String) = { - consumer.partitionsFor(topic).asScala.map(p => new TopicPartition(p.topic(), p.partition())) - } - - // TODO: How to test while loop? Not possible with MockConsumer because it resets messages after each poll. E2E-Test? - private def getAtLeastNLatestRecords(consumer: KafkaConsumer[GenericRecord, GenericRecord], topicPartition: TopicPartition, numberOfRecords: Int) = { - consumer.assign(Seq(topicPartition).asJava) - val endOffsets = consumer.endOffsets(Seq(topicPartition).asJava).asScala - if (endOffsets.size != 1) { - throw new IllegalStateException(s"Expected exactly 1 end offset, got ${endOffsets}") - } - val partition = endOffsets.keys.head - val offset = endOffsets.values.head - - var records: Seq[ConsumerRecord[GenericRecord, GenericRecord]] = Seq() - var offsetLowerBound = offset - while (records.size < numberOfRecords && offsetLowerBound != 0) { - offsetLowerBound = Math.max(0, offsetLowerBound - numberOfRecords) - consumer.seek(partition, offsetLowerBound) - records = getAllAvailableMessages(consumer) - } - - records - } - - // TODO: Move to KafkaUtils. Test with MockConsumer - private def getAllAvailableMessagesCount(consumer: KafkaConsumer[GenericRecord, GenericRecord]): Int = { - import scala.util.control.Breaks._ - var recordsCount = 0 - breakable { - while (true) { - val currentRecordsCount = consumer.poll(timeout).count() - if (currentRecordsCount == 0) { - break() - } - recordsCount += currentRecordsCount - } - } - recordsCount - } - - // TODO: Move to KafkaUtils. Test with MockConsumer - private def getAllAvailableMessages(consumer: KafkaConsumer[GenericRecord, GenericRecord]) = { - import scala.util.control.Breaks._ - var records: Seq[ConsumerRecord[GenericRecord, GenericRecord]] = mutable.Seq() - breakable { - while (true) { - val newRecords = consumer.poll(timeout).asScala.toSeq - if (newRecords.isEmpty) { - break() - } - records ++= newRecords - } - } - records - } - private def createConsumer(brokers: String, extraOptions: Map[String, String], schemaRegistryUrl: String) = { val props = new Properties() props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokers) @@ -233,61 +122,51 @@ private[transformer] class DeduplicateKafkaSinkTransformer( props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "io.confluent.kafka.serializers.KafkaAvroDeserializer") new KafkaConsumer[GenericRecord, GenericRecord](props) } - - // TODO: Move this to util class and test there - private def getTopicPartitionsFromLatestCommittedOffsets(offsetLog: OffsetSeqLog, commitLog: CommitLog): Option[Map[TopicPartition, Long]] = { - val offsetSeqOpt = commitLog.getLatest().map(_._1) - .flatMap(batchId => offsetLog.get(batchId)) - .map(offsetLog => offsetLog.offsets) - val result = if (offsetSeqOpt.isDefined) { - if (offsetSeqOpt.get.size == 1) { - if (offsetSeqOpt.get.head.isDefined) { - Some(KafkaSourceOffset.getPartitionOffsets(offsetSeqOpt.get.head.get)) - } else { - throw new IllegalStateException("Offset must be defined, got None") - } - } else { - throw new IllegalStateException(s"Cannot support more than 1 source, got ${offsetSeqOpt.toString}") - } - } else { - None - } - result - } } object DeduplicateKafkaSinkTransformer extends StreamTransformerFactory with DeduplicateKafkaSinkTransformerAttributes { override def apply(config: Configuration): StreamTransformer = { - val readerSchemaRegistryUrl = config.getString(schemaRegistryUrl) - val readerTopic = config.getString(KafkaStreamReader.KEY_TOPIC) - val readerBrokers = config.getString(KafkaStreamReader.KEY_BROKERS) - val readerExtraOptions = getPropertySubset(config, KafkaStreamReader.getExtraConfigurationPrefix.get) +// TODO: How to get schemaRegistryUrl from transformer config? + val readerSchemaRegistryUrl = getOrThrow(schemaRegistryUrl, config) + val readerTopic = getOrThrow(KafkaStreamReader.KEY_TOPIC, config) + val readerBrokers = getOrThrow(KafkaStreamReader.KEY_BROKERS, config) + val readerExtraOptions = KafkaStreamReader.getExtraConfigurationPrefix.map(getPropertySubset(config, _)).getOrElse(Map()) - val writerSchemaRegistryUrl = config.getString(schemaRegistryUrl) - val writerTopic = config.getString(KafkaStreamWriter.KEY_TOPIC) - val writerBrokers = config.getString(KafkaStreamWriter.KEY_BROKERS) - val writerExtraOptions = getPropertySubset(config, KafkaStreamWriter.optionalConfKey) + val writerSchemaRegistryUrl = getOrThrow(schemaRegistryUrl, config) + val writerTopic = getOrThrow(KafkaStreamWriter.KEY_TOPIC, config) + val writerBrokers = getOrThrow(KafkaStreamWriter.KEY_BROKERS, config) + val writerExtraOptions = KafkaStreamWriter.getExtraConfigurationPrefix.map(getPropertySubset(config, _)).getOrElse(Map()) val checkpointLocation = StreamWriterUtil.getCheckpointLocation(config) val sourceIdColumns = getSeqOrThrow(SourceIdColumns, config) val destinationIdColumns = getSeqOrThrow(DestinationIdColumns, config) - // TODO: Check same length sourceId, destinationId + if (sourceIdColumns.size != destinationIdColumns.size) { + throw new IllegalArgumentException("The size of source id column names doesn't match the list of destination id column names " + + s"${sourceIdColumns.size} != ${destinationIdColumns.size}.") + } + new DeduplicateKafkaSinkTransformer(readerSchemaRegistryUrl, readerTopic, readerBrokers, readerExtraOptions, writerSchemaRegistryUrl, writerTopic, writerBrokers, writerExtraOptions, checkpointLocation, sourceIdColumns, destinationIdColumns) } override def getMappingFromRetainedGlobalConfigToLocalConfig(globalConfig: Configuration): Map[String, String] = { - // TODO: What about subsets? - Set( - KafkaStreamReader.KEY_TOPIC, - KafkaStreamReader.KEY_BROKERS, - KafkaStreamWriter.KEY_TOPIC, - KafkaStreamWriter.KEY_BROKERS, - StreamWriterCommonAttributes.keyCheckpointBaseLocation - ).map(e => e -> e).toMap + import scala.collection.JavaConverters._ + val readerExtraOptionsKeys = + KafkaStreamReader.getExtraConfigurationPrefix.map(globalConfig.getKeys(_).asScala.toSeq).getOrElse(Seq()) + val writerExtraOptionsKeys = + KafkaStreamWriter.getExtraConfigurationPrefix.map(globalConfig.getKeys(_).asScala.toSeq).getOrElse(Seq()) + val keys = readerExtraOptionsKeys ++ writerExtraOptionsKeys ++ + Seq( + KafkaStreamReader.KEY_TOPIC, + KafkaStreamReader.KEY_BROKERS, + KafkaStreamWriter.KEY_TOPIC, + KafkaStreamWriter.KEY_BROKERS, + StreamWriterCommonAttributes.keyCheckpointBaseLocation + ) + keys.map(e => e -> e).toMap } } diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/kafka010/KafkaSourceOffset.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/kafka010/KafkaSourceOffset.scala index 54fa0c52..22b76d2f 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/kafka010/KafkaSourceOffset.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/kafka010/KafkaSourceOffset.scala @@ -13,12 +13,12 @@ case class KafkaSourceOffset(partitionToOffsets: Map[TopicPartition, Long]) exte override val json = JsonUtils.partitionOffsets(partitionToOffsets) } -private[kafka] +private[hyperdrive] case class KafkaSourcePartitionOffset(topicPartition: TopicPartition, partitionOffset: Long) extends PartitionOffset /** Companion object of the [[KafkaSourceOffset]] */ -private[kafka] object KafkaSourceOffset { +private[hyperdrive] object KafkaSourceOffset { def getPartitionOffsets(offset: Offset): Map[TopicPartition, Long] = { offset match { diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/AvroUtil.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/AvroUtil.scala index a84529ad..d2dcbdb3 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/AvroUtil.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/AvroUtil.scala @@ -16,11 +16,34 @@ package za.co.absa.hyperdrive.ingestor.implementation.utils import org.apache.avro.generic.GenericRecord +import org.apache.avro.util.Utf8 +import org.apache.kafka.clients.consumer.ConsumerRecord +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import scala.annotation.tailrec private[hyperdrive] object AvroUtil { + def getIdColumnsFromRecord(record: ConsumerRecord[GenericRecord, GenericRecord], idColumnNames: Seq[String]): Seq[Any] = { + idColumnNames.map { + case "topic" => record.topic() + case "offset" => record.offset() + case "partition" => record.partition() + case "timestamp" => record.timestamp() + case "timestampType" => record.timestampType() + case "serializedKeySize" => record.serializedKeySize() + case "serializedValueSize" => record.serializedValueSize() + case "headers" => record.headers() + case keyColumn if keyColumn.startsWith("key.") => getRecursively(record.value(), + UnresolvedAttribute.parseAttributeName(keyColumn.stripPrefix("key.")).toList) + case valueColumn if valueColumn.startsWith("value.") => getRecursively(record.value(), + UnresolvedAttribute.parseAttributeName(valueColumn.stripPrefix("value.")).toList) + }.map { + case utf8: Utf8 => utf8.toString + case v => v + } + } + @tailrec def getRecursively(record: GenericRecord, keys: List[String]): Any = keys match { case key :: Nil => record.get(key) diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala new file mode 100644 index 00000000..09ed0688 --- /dev/null +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala @@ -0,0 +1,113 @@ +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed 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 za.co.absa.hyperdrive.ingestor.implementation.utils + +import java.time.Duration +import java.util +import java.util.Collections + +import org.apache.avro.generic.GenericRecord +import org.apache.kafka.clients.consumer.{ConsumerRebalanceListener, ConsumerRecord, KafkaConsumer} +import org.apache.kafka.common.TopicPartition +import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog} +import za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka.kafka010.KafkaSourceOffset + +import scala.collection.mutable +import scala.collection.JavaConverters._ + +object KafkaUtil { + + def getAtLeastNLatestRecords[K, V](consumer: KafkaConsumer[K, V], topicPartition: TopicPartition, numberOfRecords: Int)(implicit kafkaConsumerTimeout: Duration): Seq[ConsumerRecord[K, V]] = { + consumer.assign(Seq(topicPartition).asJava) + val endOffsets = consumer.endOffsets(Seq(topicPartition).asJava).asScala + if (endOffsets.size != 1) { + throw new IllegalStateException(s"Expected exactly 1 end offset, got ${endOffsets}") + } + val partition = endOffsets.keys.head + val offset = endOffsets.values.head + + var records: Seq[ConsumerRecord[K, V]] = Seq() + var offsetLowerBound = offset + while (records.size < numberOfRecords && offsetLowerBound != 0) { + offsetLowerBound = Math.max(0, offsetLowerBound - numberOfRecords) + consumer.seek(partition, offsetLowerBound) + records = getAllAvailableMessages(consumer) + } + + records + } + + def getAllAvailableMessages[K, V](consumer: KafkaConsumer[K, V])(implicit kafkaConsumerTimeout: Duration): Seq[ConsumerRecord[K, V]] = { + import scala.util.control.Breaks._ + var records: Seq[ConsumerRecord[K, V]] = mutable.Seq() + breakable { + while (true) { + val newRecords = consumer.poll(kafkaConsumerTimeout).asScala.toSeq + if (newRecords.isEmpty) { + break() + } + records ++= newRecords + } + } + records + } + + /** + * Determines the latest committed offsets by inspecting structured streaming's offset log and commit log. + * If no committed offsets are available, seeks to beginning. + */ + def seekToLatestCommittedOffsets[K, V](consumer: KafkaConsumer[K, V], topic: String, offsetLog: OffsetSeqLog, commitLog: CommitLog): Unit = { + val sourceTopicPartitionOffsetsOpt = getTopicPartitionsFromLatestCommittedOffsets(offsetLog, commitLog) + consumer.subscribe(Collections.singletonList(topic), new ConsumerRebalanceListener { + override def onPartitionsRevoked(partitions: util.Collection[TopicPartition]): Unit = {} + + override def onPartitionsAssigned(partitions: util.Collection[TopicPartition]): Unit = { + sourceTopicPartitionOffsetsOpt match { + case Some(topicPartitionOffsets) => topicPartitionOffsets.foreach { + case (topicPartition, offset) => consumer.seek(topicPartition, offset) + } + case None => + val partitions = getTopicPartitions(consumer, topic) + consumer.seekToBeginning(partitions.asJava) + } + } + }) + } + + def getTopicPartitions[K, V](consumer: KafkaConsumer[K, V], topic: String): Seq[TopicPartition] = { + consumer.partitionsFor(topic).asScala.map(p => new TopicPartition(p.topic(), p.partition())) + } + + private def getTopicPartitionsFromLatestCommittedOffsets(offsetLog: OffsetSeqLog, commitLog: CommitLog): Option[Map[TopicPartition, Long]] = { + val offsetSeqOpt = commitLog.getLatest().map(_._1) + .flatMap(batchId => offsetLog.get(batchId)) + .map(offsetLog => offsetLog.offsets) + val result = if (offsetSeqOpt.isDefined) { + if (offsetSeqOpt.get.size == 1) { + if (offsetSeqOpt.get.head.isDefined) { + Some(KafkaSourceOffset.getPartitionOffsets(offsetSeqOpt.get.head.get)) + } else { + throw new IllegalStateException("Offset must be defined, got None") + } + } else { + throw new IllegalStateException(s"Cannot support more than 1 source, got ${offsetSeqOpt.toString}") + } + } else { + None + } + result + } +} diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/writer/kafka/KafkaStreamWriterAttributes.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/writer/kafka/KafkaStreamWriterAttributes.scala index 073a77e9..33622194 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/writer/kafka/KafkaStreamWriterAttributes.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/writer/kafka/KafkaStreamWriterAttributes.scala @@ -45,4 +45,6 @@ trait KafkaStreamWriterAttributes extends HasComponentAttributes { StreamWriterCommonAttributes.keyTriggerProcessingTime -> StreamWriterCommonAttributes.triggerProcessingTimeMetadata, StreamWriterCommonAttributes.keyCheckpointBaseLocation -> StreamWriterCommonAttributes.checkpointBaseLocation ) + + override def getExtraConfigurationPrefix: Option[String] = Some(optionalConfKey) } diff --git a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtil.scala b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtil.scala new file mode 100644 index 00000000..4f588653 --- /dev/null +++ b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtil.scala @@ -0,0 +1,46 @@ +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed 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 za.co.absa.hyperdrive.ingestor.implementation.utils + +import java.time.Duration + +import net.manub.embeddedkafka.{EmbeddedKafka, Consumers} +import org.apache.kafka.common.serialization.{Deserializer, Serializer, StringDeserializer, StringSerializer} +import org.scalatest.{Assertion, FlatSpec, Matchers} + +class TestKafkaUtil extends FlatSpec with Matchers with EmbeddedKafka with Consumers { + + "getAllAvailableMessages" should "get all available messages" in { +// implicit val kafkaConsumerTimeout: Duration = Duration.ofMillis(500L) +// +// withRunningKafka { +// implicit val serializer: Serializer[String] = new StringSerializer() +// implicit val deserializer: Deserializer[String] = new StringDeserializer() +// val messagesCount = 10000 +// (1 to messagesCount).foreach(i => EmbeddedKafka.publishToKafka("test-topic", s"message_${i}")) +// +// +// withConsumer[String, String, Assertion](consumer => { +// val messages = KafkaUtil.getAllAvailableMessages(consumer) +// messages.size shouldBe messagesCount +// val values = messages.map(c => c.value()) +// values should contain theSameElementsAs (1 to messagesCount).map(i => s"message_$i") +// }) +// } + } +} + + From 152d0ef22212f60a8fee9ca8b22b57e714c97e47 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Fri, 27 Nov 2020 14:55:49 +0100 Subject: [PATCH 13/27] wip --- .../drivers/KafkaSchemaRegistryWrapper.scala | 7 +- ingestor-default/pom.xml | 10 ++ .../implementation/utils/KafkaUtil.scala | 3 +- .../implementation/utils/TestKafkaUtil.scala | 46 -------- .../utils/TestKafkaUtilDockerTest.scala | 108 ++++++++++++++++++ 5 files changed, 123 insertions(+), 51 deletions(-) delete mode 100644 ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtil.scala create mode 100644 ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala diff --git a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaSchemaRegistryWrapper.scala b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaSchemaRegistryWrapper.scala index b7973f97..cc2e4ae0 100644 --- a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaSchemaRegistryWrapper.scala +++ b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaSchemaRegistryWrapper.scala @@ -17,7 +17,6 @@ package za.co.absa.hyperdrive.driver.drivers import java.util.Properties -import org.apache.avro.generic.GenericRecord import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.clients.producer.KafkaProducer import org.apache.logging.log4j.LogManager @@ -29,11 +28,11 @@ case class SchemaRegistryContainer(dockerImageName: String) extends GenericConta class KafkaSchemaRegistryWrapper { private val logger = LogManager.getLogger - private val confluentPlatformVersion = "5.3.1" + private val confluentPlatformVersion = "5.3.4" // should be same as kafka.avro.serializer.version property in pom file private val schemaRegistryPort = 8081 private val commonNetwork = Network.newNetwork() - val kafka: KafkaContainer = startKafka(commonNetwork) - private val schemaRegistry = startSchemaRegistry(commonNetwork) + lazy val kafka: KafkaContainer = startKafka(commonNetwork) + lazy private val schemaRegistry = startSchemaRegistry(commonNetwork) logger.info(s"Created network with id ${commonNetwork.getId}") diff --git a/ingestor-default/pom.xml b/ingestor-default/pom.xml index 9375dc7e..13a977ba 100644 --- a/ingestor-default/pom.xml +++ b/ingestor-default/pom.xml @@ -52,5 +52,15 @@ org.apache.spark spark-sql-kafka-${kafka.spark.version}_${scala.compat.version} + + + + org.testcontainers + testcontainers + + + org.testcontainers + kafka + diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala index 09ed0688..76a95a95 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala @@ -16,7 +16,7 @@ package za.co.absa.hyperdrive.ingestor.implementation.utils import java.time.Duration -import java.util +import java.{lang, util} import java.util.Collections import org.apache.avro.generic.GenericRecord @@ -50,6 +50,7 @@ object KafkaUtil { records } +// TODO: Add from, to arguments. Caller should know what beginning and end offsets to expect def getAllAvailableMessages[K, V](consumer: KafkaConsumer[K, V])(implicit kafkaConsumerTimeout: Duration): Seq[ConsumerRecord[K, V]] = { import scala.util.control.Breaks._ var records: Seq[ConsumerRecord[K, V]] = mutable.Seq() diff --git a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtil.scala b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtil.scala deleted file mode 100644 index 4f588653..00000000 --- a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtil.scala +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Copyright 2018 ABSA Group Limited - * - * Licensed 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 za.co.absa.hyperdrive.ingestor.implementation.utils - -import java.time.Duration - -import net.manub.embeddedkafka.{EmbeddedKafka, Consumers} -import org.apache.kafka.common.serialization.{Deserializer, Serializer, StringDeserializer, StringSerializer} -import org.scalatest.{Assertion, FlatSpec, Matchers} - -class TestKafkaUtil extends FlatSpec with Matchers with EmbeddedKafka with Consumers { - - "getAllAvailableMessages" should "get all available messages" in { -// implicit val kafkaConsumerTimeout: Duration = Duration.ofMillis(500L) -// -// withRunningKafka { -// implicit val serializer: Serializer[String] = new StringSerializer() -// implicit val deserializer: Deserializer[String] = new StringDeserializer() -// val messagesCount = 10000 -// (1 to messagesCount).foreach(i => EmbeddedKafka.publishToKafka("test-topic", s"message_${i}")) -// -// -// withConsumer[String, String, Assertion](consumer => { -// val messages = KafkaUtil.getAllAvailableMessages(consumer) -// messages.size shouldBe messagesCount -// val values = messages.map(c => c.value()) -// values should contain theSameElementsAs (1 to messagesCount).map(i => s"message_$i") -// }) -// } - } -} - - diff --git a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala new file mode 100644 index 00000000..9474f19a --- /dev/null +++ b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala @@ -0,0 +1,108 @@ +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed 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 za.co.absa.hyperdrive.ingestor.implementation.utils + +import java.time.Duration +import java.util +import java.util.UUID.randomUUID +import java.util.{Collections, Properties, UUID} + +import org.apache.avro.generic.GenericRecord +import org.apache.kafka.clients.admin.{AdminClient, AdminClientConfig, NewTopic} +import org.apache.kafka.clients.consumer.KafkaConsumer +import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} +import org.apache.kafka.common.serialization.{Deserializer, Serializer, StringDeserializer, StringSerializer} +import org.scalatest.{Assertion, BeforeAndAfter, FlatSpec, Matchers} +import org.testcontainers.containers.KafkaContainer + +class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter { + + private val confluentPlatformVersion = "5.3.4" // should be same as kafka.avro.serializer.version property in pom file + private val kafka = new KafkaContainer(confluentPlatformVersion) + + before{ + kafka.start() + } + + after { + kafka.stop() + } + + "getAllAvailableMessages" should "get all available messages" in { + // given + val topic = "get-all-available-messages-topic" + val partitions = 3 + createTopic(kafka, "get-all-available-messages-topic", partitions) + val producer = createProducer(kafka) + val messagesCount = 10000 + val messages = (1 to messagesCount).map(i => s"message_${i}") + produceData(producer, messages, topic, partitions) + + val consumer = createConsumer(kafka) + consumer.subscribe(Collections.singletonList(topic)) + + // when + implicit val kafkaConsumerTimeout: Duration = Duration.ofMillis(500L) + val records = KafkaUtil.getAllAvailableMessages(consumer) + + // then + val actualMessages = records.map(_.value()) + actualMessages should contain theSameElementsAs (1 to messagesCount).map(i => s"message_$i") + } + + private def createTopic(kafkaContainer: KafkaContainer, topicName: String, partitions: Int): Unit = { + val config = new Properties() + config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaContainer.getBootstrapServers) + val localKafkaAdmin = AdminClient.create(config) + val replication = 1.toShort + val topic = new NewTopic(topicName, partitions, replication) + val topicCreationFut = localKafkaAdmin.createTopics(util.Arrays.asList(topic)).all() + while(!topicCreationFut.isDone) {} + } + + def createProducer(kafkaContainer: KafkaContainer): KafkaProducer[String, String] = { + val props = new Properties() + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaContainer.getBootstrapServers) + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, classOf[StringSerializer].getCanonicalName) + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, classOf[StringSerializer].getCanonicalName) + props.put(ProducerConfig.CLIENT_ID_CONFIG, randomUUID().toString) + props.put(ProducerConfig.ACKS_CONFIG, "1") + new KafkaProducer[String, String](props) + } + + def createConsumer(kafkaContainer: KafkaContainer): KafkaConsumer[String, String] = { + import org.apache.kafka.clients.consumer.ConsumerConfig + val props = new Properties() + props.put(ConsumerConfig.GROUP_ID_CONFIG, randomUUID.toString) + props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaContainer.getBootstrapServers) + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, classOf[StringDeserializer].getCanonicalName) + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[StringDeserializer].getCanonicalName) + new KafkaConsumer[String, String](props) + } + + private def produceData(producer: KafkaProducer[String, String], records: Seq[String], topic: String, partitions: Int): Unit = { + records.zipWithIndex.foreach { + case (record, i) => + val partition = i % partitions + val producerRecord = new ProducerRecord[String, String](topic, partition, null, record) + producer.send(producerRecord) + } + } + +} + + From b42880d09a8206be4a8b210aabb12be9318de5c8 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Fri, 27 Nov 2020 18:35:57 +0100 Subject: [PATCH 14/27] works, seems a bit slow --- .../drivers/KafkaSchemaRegistryWrapper.scala | 6 +- .../DeduplicateKafkaSinkTransformer.scala | 5 +- .../implementation/utils/KafkaUtil.scala | 61 ++++++++++++++----- .../utils/TestKafkaUtilDockerTest.scala | 23 ++++--- 4 files changed, 67 insertions(+), 28 deletions(-) diff --git a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaSchemaRegistryWrapper.scala b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaSchemaRegistryWrapper.scala index cc2e4ae0..a6515d53 100644 --- a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaSchemaRegistryWrapper.scala +++ b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaSchemaRegistryWrapper.scala @@ -28,11 +28,11 @@ case class SchemaRegistryContainer(dockerImageName: String) extends GenericConta class KafkaSchemaRegistryWrapper { private val logger = LogManager.getLogger - private val confluentPlatformVersion = "5.3.4" // should be same as kafka.avro.serializer.version property in pom file + private val confluentPlatformVersion = "5.3.1" // should be same as kafka.avro.serializer.version property in pom file private val schemaRegistryPort = 8081 private val commonNetwork = Network.newNetwork() - lazy val kafka: KafkaContainer = startKafka(commonNetwork) - lazy private val schemaRegistry = startSchemaRegistry(commonNetwork) + val kafka: KafkaContainer = startKafka(commonNetwork) + private val schemaRegistry = startSchemaRegistry(commonNetwork) logger.info(s"Created network with id ${commonNetwork.getId}") diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala index f84224b4..1787b0e3 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala @@ -72,7 +72,10 @@ private[transformer] class DeduplicateKafkaSinkTransformer( implicit val kafkaConsumerTimeout: Duration = Duration.ofSeconds(5L) // TODO: Make it configurable val sourceConsumer = createConsumer(readerBrokers, readerExtraOptions, readerSchemaRegistryUrl) KafkaUtil.seekToLatestCommittedOffsets(sourceConsumer, readerTopic, offsetLog, commitLog) - val sourceRecords = consumeAndClose(sourceConsumer, KafkaUtil.getAllAvailableMessages) + val sourceTopicPartitions = KafkaUtil.getTopicPartitionsFromLatestOffset(offsetLog) +// TODO: consumeAndClose into the util methods? + val sourceRecords = sourceTopicPartitions.map(stp => consumeAndClose(sourceConsumer, + (consumer: KafkaConsumer[GenericRecord, GenericRecord]) => KafkaUtil.getMessagesAtLeastToOffset(consumer, stp))).getOrElse(Seq()) val sourceIds = sourceRecords.map(extractIdFieldsFromRecord(_, sourceIdColumnNames)) val sinkConsumer = createConsumer(writerBrokers, writerExtraOptions, writerSchemaRegistryUrl) diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala index 76a95a95..fad75cd1 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala @@ -16,17 +16,16 @@ package za.co.absa.hyperdrive.ingestor.implementation.utils import java.time.Duration -import java.{lang, util} +import java.util import java.util.Collections -import org.apache.avro.generic.GenericRecord import org.apache.kafka.clients.consumer.{ConsumerRebalanceListener, ConsumerRecord, KafkaConsumer} import org.apache.kafka.common.TopicPartition import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog} import za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka.kafka010.KafkaSourceOffset -import scala.collection.mutable import scala.collection.JavaConverters._ +import scala.collection.mutable object KafkaUtil { @@ -44,25 +43,43 @@ object KafkaUtil { while (records.size < numberOfRecords && offsetLowerBound != 0) { offsetLowerBound = Math.max(0, offsetLowerBound - numberOfRecords) consumer.seek(partition, offsetLowerBound) - records = getAllAvailableMessages(consumer) + records = getMessagesAtLeastToOffset(consumer, Map(topicPartition -> offset)) } records } // TODO: Add from, to arguments. Caller should know what beginning and end offsets to expect - def getAllAvailableMessages[K, V](consumer: KafkaConsumer[K, V])(implicit kafkaConsumerTimeout: Duration): Seq[ConsumerRecord[K, V]] = { +// TODO: Does it also work with subscribe, or only with assign? + def getMessagesAtLeastToOffset[K, V](consumer: KafkaConsumer[K, V], toOffsets: Map[TopicPartition, Long]) + (implicit kafkaConsumerTimeout: Duration): Seq[ConsumerRecord[K, V]] = { + val endOffsets = consumer.endOffsets(toOffsets.keys.toSeq.asJava).asScala + endOffsets.foreach { case (topicPartition, offset) => + val toOffset = toOffsets(topicPartition) + if (toOffset > offset) { + throw new IllegalArgumentException(s"Requested consumption to $toOffset, but it cannot be higher than the " + + s"end offset, which is $offset on $topicPartition") + } + } import scala.util.control.Breaks._ var records: Seq[ConsumerRecord[K, V]] = mutable.Seq() breakable { while (true) { val newRecords = consumer.poll(kafkaConsumerTimeout).asScala.toSeq + records ++= newRecords if (newRecords.isEmpty) { break() } - records ++= newRecords } } + toOffsets.foreach { case (tp, toOffset) => + val offsetAfterPoll = consumer.position(tp) + if (offsetAfterPoll < toOffset) { + throw new IllegalStateException(s"Expected to reach offset $toOffset on $tp, but only reached $offsetAfterPoll." + + s" Not all messages were consumed. You may want to increase the consumer timeout") + } + } + records } @@ -71,7 +88,7 @@ object KafkaUtil { * If no committed offsets are available, seeks to beginning. */ def seekToLatestCommittedOffsets[K, V](consumer: KafkaConsumer[K, V], topic: String, offsetLog: OffsetSeqLog, commitLog: CommitLog): Unit = { - val sourceTopicPartitionOffsetsOpt = getTopicPartitionsFromLatestCommittedOffsets(offsetLog, commitLog) + val sourceTopicPartitionOffsetsOpt = getTopicPartitionsFromLatestCommit(offsetLog, commitLog) consumer.subscribe(Collections.singletonList(topic), new ConsumerRebalanceListener { override def onPartitionsRevoked(partitions: util.Collection[TopicPartition]): Unit = {} @@ -92,23 +109,35 @@ object KafkaUtil { consumer.partitionsFor(topic).asScala.map(p => new TopicPartition(p.topic(), p.partition())) } - private def getTopicPartitionsFromLatestCommittedOffsets(offsetLog: OffsetSeqLog, commitLog: CommitLog): Option[Map[TopicPartition, Long]] = { + private def getTopicPartitionsFromLatestCommit(offsetLog: OffsetSeqLog, commitLog: CommitLog): Option[Map[TopicPartition, Long]] = { val offsetSeqOpt = commitLog.getLatest().map(_._1) .flatMap(batchId => offsetLog.get(batchId)) .map(offsetLog => offsetLog.offsets) - val result = if (offsetSeqOpt.isDefined) { - if (offsetSeqOpt.get.size == 1) { - if (offsetSeqOpt.get.head.isDefined) { - Some(KafkaSourceOffset.getPartitionOffsets(offsetSeqOpt.get.head.get)) + offsetSeqOpt.map(offsetSeq => + if (offsetSeq.size == 1) { + if (offsetSeq.head.isDefined) { + KafkaSourceOffset.getPartitionOffsets(offsetSeq.head.get) } else { throw new IllegalStateException("Offset must be defined, got None") } } else { throw new IllegalStateException(s"Cannot support more than 1 source, got ${offsetSeqOpt.toString}") } - } else { - None - } - result + ) + } + + def getTopicPartitionsFromLatestOffset(offsetLog: OffsetSeqLog): Option[Map[TopicPartition, Long]] = { + val offsetSeqOpt = offsetLog.getLatest().map(_._2.offsets) + offsetSeqOpt.map(offsetSeq => + if (offsetSeq.size == 1) { + if (offsetSeq.head.isDefined) { + KafkaSourceOffset.getPartitionOffsets(offsetSeq.head.get) + } else { + throw new IllegalStateException("Offset must be defined, got None") + } + } else { + throw new IllegalStateException(s"Cannot support more than 1 source, got ${offsetSeqOpt.toString}") + } + ) } } diff --git a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala index 9474f19a..c6492594 100644 --- a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala +++ b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala @@ -16,7 +16,7 @@ package za.co.absa.hyperdrive.ingestor.implementation.utils import java.time.Duration -import java.util +import java.{lang, util} import java.util.UUID.randomUUID import java.util.{Collections, Properties, UUID} @@ -24,10 +24,13 @@ import org.apache.avro.generic.GenericRecord import org.apache.kafka.clients.admin.{AdminClient, AdminClientConfig, NewTopic} import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} +import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.serialization.{Deserializer, Serializer, StringDeserializer, StringSerializer} import org.scalatest.{Assertion, BeforeAndAfter, FlatSpec, Matchers} import org.testcontainers.containers.KafkaContainer +import scala.collection.JavaConverters._ +import scala.collection.mutable class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter { private val confluentPlatformVersion = "5.3.4" // should be same as kafka.avro.serializer.version property in pom file @@ -43,23 +46,27 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter "getAllAvailableMessages" should "get all available messages" in { // given - val topic = "get-all-available-messages-topic" + val topic = "get-all-available-messages-topic-2" val partitions = 3 - createTopic(kafka, "get-all-available-messages-topic", partitions) + createTopic(kafka, topic, partitions) val producer = createProducer(kafka) - val messagesCount = 10000 + val messagesCount = 100 val messages = (1 to messagesCount).map(i => s"message_${i}") produceData(producer, messages, topic, partitions) val consumer = createConsumer(kafka) - consumer.subscribe(Collections.singletonList(topic)) + val topicPartitions = KafkaUtil.getTopicPartitions(consumer, topic) + consumer.assign(topicPartitions.asJava) + consumer.seekToBeginning(topicPartitions.asJava) + val offsets = consumer.endOffsets(topicPartitions.asJava).asScala.toMap + val offsets2 = offsets.mapValues(l => l + 0L) // when - implicit val kafkaConsumerTimeout: Duration = Duration.ofMillis(500L) - val records = KafkaUtil.getAllAvailableMessages(consumer) + implicit val kafkaConsumerTimeout: Duration = Duration.ofMillis(1500L) + val records = KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets2) // then - val actualMessages = records.map(_.value()) + val actualMessages = records.map(_.value()).toList.sorted actualMessages should contain theSameElementsAs (1 to messagesCount).map(i => s"message_$i") } From 7476bc95e74a6feeaa89f53fa9b8b4c34e18351f Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Mon, 30 Nov 2020 18:21:56 +0100 Subject: [PATCH 15/27] Fix getMessagesAtLeastToOffset --- .../implementation/utils/KafkaUtil.scala | 26 ++--- .../utils/TestKafkaUtilDockerTest.scala | 95 +++++++++++++++---- 2 files changed, 88 insertions(+), 33 deletions(-) diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala index fad75cd1..673e3d90 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala @@ -25,7 +25,6 @@ import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog} import za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka.kafka010.KafkaSourceOffset import scala.collection.JavaConverters._ -import scala.collection.mutable object KafkaUtil { @@ -49,34 +48,27 @@ object KafkaUtil { records } -// TODO: Add from, to arguments. Caller should know what beginning and end offsets to expect -// TODO: Does it also work with subscribe, or only with assign? def getMessagesAtLeastToOffset[K, V](consumer: KafkaConsumer[K, V], toOffsets: Map[TopicPartition, Long]) (implicit kafkaConsumerTimeout: Duration): Seq[ConsumerRecord[K, V]] = { val endOffsets = consumer.endOffsets(toOffsets.keys.toSeq.asJava).asScala endOffsets.foreach { case (topicPartition, offset) => val toOffset = toOffsets(topicPartition) if (toOffset > offset) { - throw new IllegalArgumentException(s"Requested consumption to $toOffset, but it cannot be higher than the " + - s"end offset, which is $offset on $topicPartition") - } - } - import scala.util.control.Breaks._ - var records: Seq[ConsumerRecord[K, V]] = mutable.Seq() - breakable { - while (true) { - val newRecords = consumer.poll(kafkaConsumerTimeout).asScala.toSeq - records ++= newRecords - if (newRecords.isEmpty) { - break() - } + throw new IllegalArgumentException(s"Requested consumption to offsets $toOffsets, but they cannot be higher " + + s"than the end offsets, which are $endOffsets") } } + + val records = consumer.poll(kafkaConsumerTimeout).asScala.toSeq + toOffsets.foreach { case (tp, toOffset) => + if (!consumer.assignment().contains(tp)) { + throw new IllegalStateException(s"Consumer is unexpectedly not assigned to $tp. Consider increasing the consumer timeout") + } val offsetAfterPoll = consumer.position(tp) if (offsetAfterPoll < toOffset) { throw new IllegalStateException(s"Expected to reach offset $toOffset on $tp, but only reached $offsetAfterPoll." + - s" Not all messages were consumed. You may want to increase the consumer timeout") + s" Not all expected messages were consumed. Consider increasing the consumer timeout") } } diff --git a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala index c6492594..5fa7ce88 100644 --- a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala +++ b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala @@ -16,21 +16,18 @@ package za.co.absa.hyperdrive.ingestor.implementation.utils import java.time.Duration -import java.{lang, util} +import java.util import java.util.UUID.randomUUID -import java.util.{Collections, Properties, UUID} +import java.util.{Collections, Properties} -import org.apache.avro.generic.GenericRecord import org.apache.kafka.clients.admin.{AdminClient, AdminClientConfig, NewTopic} import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.serialization.{Deserializer, Serializer, StringDeserializer, StringSerializer} -import org.scalatest.{Assertion, BeforeAndAfter, FlatSpec, Matchers} +import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializer} +import org.scalatest.{BeforeAndAfter, FlatSpec, Matchers} import org.testcontainers.containers.KafkaContainer import scala.collection.JavaConverters._ -import scala.collection.mutable class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter { private val confluentPlatformVersion = "5.3.4" // should be same as kafka.avro.serializer.version property in pom file @@ -46,28 +43,93 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter "getAllAvailableMessages" should "get all available messages" in { // given - val topic = "get-all-available-messages-topic-2" + val topic = "get-all-available-messages-topic" val partitions = 3 createTopic(kafka, topic, partitions) val producer = createProducer(kafka) - val messagesCount = 100 - val messages = (1 to messagesCount).map(i => s"message_${i}") + val messages = (1 to 100).map(i => s"message_${i}") produceData(producer, messages, topic, partitions) val consumer = createConsumer(kafka) + consumer.subscribe(Collections.singletonList(topic)) val topicPartitions = KafkaUtil.getTopicPartitions(consumer, topic) - consumer.assign(topicPartitions.asJava) - consumer.seekToBeginning(topicPartitions.asJava) - val offsets = consumer.endOffsets(topicPartitions.asJava).asScala.toMap - val offsets2 = offsets.mapValues(l => l + 0L) + val offsets = consumer.endOffsets(topicPartitions.asJava).asScala.toMap.mapValues(_.asInstanceOf[Long]) // when implicit val kafkaConsumerTimeout: Duration = Duration.ofMillis(1500L) - val records = KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets2) + val records = KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets) // then val actualMessages = records.map(_.value()).toList.sorted - actualMessages should contain theSameElementsAs (1 to messagesCount).map(i => s"message_$i") + actualMessages should contain theSameElementsAs messages + } + + it should "throw an exception if partitions were not assigned" in { + // given + val topic = "get-all-available-messages-topic" + val partitions = 3 + createTopic(kafka, topic, partitions) + val producer = createProducer(kafka) + val messages = (1 to 100).map(i => s"message_${i}") + produceData(producer, messages, topic, partitions) + + val consumer = createConsumer(kafka) + consumer.subscribe(Collections.singletonList(topic)) + val topicPartitions = KafkaUtil.getTopicPartitions(consumer, topic) + val offsets = consumer.endOffsets(topicPartitions.asJava).asScala.toMap.mapValues(_.asInstanceOf[Long]) + + // when + implicit val kafkaConsumerTimeout: Duration = Duration.ofMillis(1L) + val exception = the[Exception] thrownBy KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets) + + // then + exception.getMessage should include ("Consumer is unexpectedly not assigned") + } + + it should "throw an exception if not all messages could be consumed" in { + // given + val topic = "get-all-available-messages-topic" + val partitions = 3 + createTopic(kafka, topic, partitions) + val producer = createProducer(kafka) + val messages = (1 to 100).map(i => s"message_${i}") + produceData(producer, messages, topic, partitions) + + val consumer = createConsumer(kafka) + val topicPartitions = KafkaUtil.getTopicPartitions(consumer, topic) + consumer.assign(topicPartitions.asJava) + consumer.seekToBeginning(topicPartitions.asJava) + val offsets = consumer.endOffsets(topicPartitions.asJava).asScala.toMap.mapValues(_.asInstanceOf[Long]) + + // when + implicit val kafkaConsumerTimeout: Duration = Duration.ofMillis(1L) + val exception = the[Exception] thrownBy KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets) + + // then + exception.getMessage should include ("Not all expected messages were consumed") + } + + + it should "throw an exception if requested offsets are not available" in { + // given + val topic = "get-all-available-messages-topic" + val partitions = 3 + createTopic(kafka, topic, partitions) + val producer = createProducer(kafka) + val messages = (1 to 100).map(i => s"message_${i}") + produceData(producer, messages, topic, partitions) + + val consumer = createConsumer(kafka) + consumer.subscribe(Collections.singletonList(topic)) + val topicPartitions = KafkaUtil.getTopicPartitions(consumer, topic) + val offsets = consumer.endOffsets(topicPartitions.asJava).asScala.toMap.mapValues(_ * 2L) + + // when + implicit val kafkaConsumerTimeout: Duration = Duration.ofMillis(0L) + val exception = the[Exception] thrownBy KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets) + + // then + exception.getMessage should include ("Requested consumption") } private def createTopic(kafkaContainer: KafkaContainer, topicName: String, partitions: Int): Unit = { @@ -108,6 +170,7 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter val producerRecord = new ProducerRecord[String, String](topic, partition, null, record) producer.send(producerRecord) } + producer.flush() } } From 1a5c6ffa9721b92c5be24e27a3aad0289a407fa9 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Tue, 1 Dec 2020 18:15:09 +0100 Subject: [PATCH 16/27] Add tests for kafkautil --- .../DeduplicateKafkaSinkTransformer.scala | 18 +- .../implementation/utils/KafkaUtil.scala | 105 +++---- .../implementation/utils/TestKafkaUtil.scala | 152 +++++++++++ .../utils/TestKafkaUtilDockerTest.scala | 256 ++++++++++++++++-- 4 files changed, 448 insertions(+), 83 deletions(-) create mode 100644 ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtil.scala diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala index 1787b0e3..53930bb9 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala @@ -28,7 +28,7 @@ import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaC import org.apache.logging.log4j.LogManager import org.apache.spark.sql.DataFrame import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog} -import org.apache.spark.sql.functions.{col, lit, struct, not} +import org.apache.spark.sql.functions.{col, lit, not, struct} import za.co.absa.hyperdrive.ingestor.api.transformer.{StreamTransformer, StreamTransformerFactory} import za.co.absa.hyperdrive.ingestor.api.utils.ConfigUtils.{getOrThrow, getPropertySubset, getSeqOrThrow} import za.co.absa.hyperdrive.ingestor.api.utils.StreamWriterUtil @@ -71,18 +71,18 @@ private[transformer] class DeduplicateKafkaSinkTransformer( logger.info("Deduplicate rows after retry") implicit val kafkaConsumerTimeout: Duration = Duration.ofSeconds(5L) // TODO: Make it configurable val sourceConsumer = createConsumer(readerBrokers, readerExtraOptions, readerSchemaRegistryUrl) - KafkaUtil.seekToLatestCommittedOffsets(sourceConsumer, readerTopic, offsetLog, commitLog) - val sourceTopicPartitions = KafkaUtil.getTopicPartitionsFromLatestOffset(offsetLog) -// TODO: consumeAndClose into the util methods? - val sourceRecords = sourceTopicPartitions.map(stp => consumeAndClose(sourceConsumer, - (consumer: KafkaConsumer[GenericRecord, GenericRecord]) => KafkaUtil.getMessagesAtLeastToOffset(consumer, stp))).getOrElse(Seq()) + val latestCommittedOffsets = KafkaUtil.getLatestCommittedOffset(offsetLog, commitLog) + KafkaUtil.seekToOffsetsOrBeginning(sourceConsumer, readerTopic, latestCommittedOffsets) + + val latestOffsetsOpt = KafkaUtil.getLatestOffset(offsetLog) + val sourceRecords = latestOffsetsOpt.map(latestOffset => consumeAndClose(sourceConsumer, + consumer => KafkaUtil.getMessagesAtLeastToOffset(consumer, latestOffset))).getOrElse(Seq()) val sourceIds = sourceRecords.map(extractIdFieldsFromRecord(_, sourceIdColumnNames)) val sinkConsumer = createConsumer(writerBrokers, writerExtraOptions, writerSchemaRegistryUrl) val sinkTopicPartitions = KafkaUtil.getTopicPartitions(sinkConsumer, writerTopic) - val latestSinkRecords = consumeAndClose(sinkConsumer, - (consumer: KafkaConsumer[GenericRecord, GenericRecord]) => sinkTopicPartitions.map { - topicPartition => KafkaUtil.getAtLeastNLatestRecords(consumer, topicPartition, sourceRecords.size) + val latestSinkRecords = consumeAndClose(sinkConsumer, consumer => sinkTopicPartitions.map { + topicPartition => KafkaUtil.getAtLeastNLatestRecordsFromPartition(consumer, topicPartition, sourceRecords.size) }) val publishedIds = latestSinkRecords.flatten.map(extractIdFieldsFromRecord(_, destinationIdColumnNames)) diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala index 673e3d90..6a614615 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala @@ -16,19 +16,21 @@ package za.co.absa.hyperdrive.ingestor.implementation.utils import java.time.Duration -import java.util -import java.util.Collections -import org.apache.kafka.clients.consumer.{ConsumerRebalanceListener, ConsumerRecord, KafkaConsumer} +import org.apache.kafka.clients.consumer.{ConsumerRecord, KafkaConsumer} import org.apache.kafka.common.TopicPartition -import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog} +import org.apache.logging.log4j.LogManager +import org.apache.spark.sql.execution.streaming.{CommitLog, Offset, OffsetSeqLog} import za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka.kafka010.KafkaSourceOffset import scala.collection.JavaConverters._ +import scala.collection.mutable object KafkaUtil { + private val logger = LogManager.getLogger - def getAtLeastNLatestRecords[K, V](consumer: KafkaConsumer[K, V], topicPartition: TopicPartition, numberOfRecords: Int)(implicit kafkaConsumerTimeout: Duration): Seq[ConsumerRecord[K, V]] = { + def getAtLeastNLatestRecordsFromPartition[K, V](consumer: KafkaConsumer[K, V], topicPartition: TopicPartition, numberOfRecords: Int) + (implicit kafkaConsumerTimeout: Duration): Seq[ConsumerRecord[K, V]] = { consumer.assign(Seq(topicPartition).asJava) val endOffsets = consumer.endOffsets(Seq(topicPartition).asJava).asScala if (endOffsets.size != 1) { @@ -50,6 +52,7 @@ object KafkaUtil { def getMessagesAtLeastToOffset[K, V](consumer: KafkaConsumer[K, V], toOffsets: Map[TopicPartition, Long]) (implicit kafkaConsumerTimeout: Duration): Seq[ConsumerRecord[K, V]] = { + consumer.assign(toOffsets.keySet.asJava) val endOffsets = consumer.endOffsets(toOffsets.keys.toSeq.asJava).asScala endOffsets.foreach { case (topicPartition, offset) => val toOffset = toOffsets(topicPartition) @@ -59,12 +62,19 @@ object KafkaUtil { } } - val records = consumer.poll(kafkaConsumerTimeout).asScala.toSeq + import scala.util.control.Breaks._ + var records: Seq[ConsumerRecord[K, V]] = mutable.Seq() + breakable { + while (true) { + val newRecords = consumer.poll(kafkaConsumerTimeout).asScala.toSeq + records ++= newRecords + if (newRecords.isEmpty || offsetsHaveBeenReached(consumer, toOffsets)) { + break() + } + } + } toOffsets.foreach { case (tp, toOffset) => - if (!consumer.assignment().contains(tp)) { - throw new IllegalStateException(s"Consumer is unexpectedly not assigned to $tp. Consider increasing the consumer timeout") - } val offsetAfterPoll = consumer.position(tp) if (offsetAfterPoll < toOffset) { throw new IllegalStateException(s"Expected to reach offset $toOffset on $tp, but only reached $offsetAfterPoll." + @@ -75,61 +85,52 @@ object KafkaUtil { records } - /** - * Determines the latest committed offsets by inspecting structured streaming's offset log and commit log. - * If no committed offsets are available, seeks to beginning. - */ - def seekToLatestCommittedOffsets[K, V](consumer: KafkaConsumer[K, V], topic: String, offsetLog: OffsetSeqLog, commitLog: CommitLog): Unit = { - val sourceTopicPartitionOffsetsOpt = getTopicPartitionsFromLatestCommit(offsetLog, commitLog) - consumer.subscribe(Collections.singletonList(topic), new ConsumerRebalanceListener { - override def onPartitionsRevoked(partitions: util.Collection[TopicPartition]): Unit = {} - - override def onPartitionsAssigned(partitions: util.Collection[TopicPartition]): Unit = { - sourceTopicPartitionOffsetsOpt match { - case Some(topicPartitionOffsets) => topicPartitionOffsets.foreach { - case (topicPartition, offset) => consumer.seek(topicPartition, offset) - } - case None => - val partitions = getTopicPartitions(consumer, topic) - consumer.seekToBeginning(partitions.asJava) - } + private def offsetsHaveBeenReached[K, V](consumer: KafkaConsumer[K, V], toOffsets: Map[TopicPartition, Long]) = { + toOffsets.forall { case (tp, toOffset) => + val position = consumer.position(tp) + logger.info(s"Reached position $position on topic partition $tp. Target offset is $toOffset") + position >= toOffset + } + } + + def seekToOffsetsOrBeginning[K, V](consumer: KafkaConsumer[K, V], topic: String, offsetsOpt: Option[Map[TopicPartition, Long]]): Unit = { + val partitions = getTopicPartitions(consumer, topic) + consumer.assign(partitions.asJava) + offsetsOpt match { + case Some(topicPartitionOffsets) => topicPartitionOffsets.foreach { + case (topicPartition, offset) => consumer.seek(topicPartition, offset) } - }) + case None => + consumer.seekToBeginning(partitions.asJava) + } } def getTopicPartitions[K, V](consumer: KafkaConsumer[K, V], topic: String): Seq[TopicPartition] = { consumer.partitionsFor(topic).asScala.map(p => new TopicPartition(p.topic(), p.partition())) } - private def getTopicPartitionsFromLatestCommit(offsetLog: OffsetSeqLog, commitLog: CommitLog): Option[Map[TopicPartition, Long]] = { + def getLatestOffset(offsetLog: OffsetSeqLog): Option[Map[TopicPartition, Long]] = { + val offsetSeqOpt = offsetLog.getLatest().map(_._2.offsets) + offsetSeqOpt.flatMap(parseOffsetSeq) + } + + def getLatestCommittedOffset(offsetLog: OffsetSeqLog, commitLog: CommitLog): Option[Map[TopicPartition, Long]] = { val offsetSeqOpt = commitLog.getLatest().map(_._1) - .flatMap(batchId => offsetLog.get(batchId)) + .map(batchId => offsetLog.get(batchId) + .getOrElse(throw new IllegalStateException(s"No offset found for committed batchId ${batchId}"))) .map(offsetLog => offsetLog.offsets) - offsetSeqOpt.map(offsetSeq => - if (offsetSeq.size == 1) { - if (offsetSeq.head.isDefined) { - KafkaSourceOffset.getPartitionOffsets(offsetSeq.head.get) - } else { - throw new IllegalStateException("Offset must be defined, got None") - } - } else { - throw new IllegalStateException(s"Cannot support more than 1 source, got ${offsetSeqOpt.toString}") - } - ) + offsetSeqOpt.flatMap(parseOffsetSeq) } - def getTopicPartitionsFromLatestOffset(offsetLog: OffsetSeqLog): Option[Map[TopicPartition, Long]] = { - val offsetSeqOpt = offsetLog.getLatest().map(_._2.offsets) - offsetSeqOpt.map(offsetSeq => - if (offsetSeq.size == 1) { - if (offsetSeq.head.isDefined) { - KafkaSourceOffset.getPartitionOffsets(offsetSeq.head.get) - } else { - throw new IllegalStateException("Offset must be defined, got None") - } + private def parseOffsetSeq(offsetSeq: Seq[Option[Offset]]) = { + if (offsetSeq.size == 1) { + if (offsetSeq.head.isDefined) { + Some(KafkaSourceOffset.getPartitionOffsets(offsetSeq.head.get)) } else { - throw new IllegalStateException(s"Cannot support more than 1 source, got ${offsetSeqOpt.toString}") + None } - ) + } else { + throw new IllegalStateException(s"Cannot support more than 1 source, got ${offsetSeq.toString}") + } } } diff --git a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtil.scala b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtil.scala new file mode 100644 index 00000000..71d14375 --- /dev/null +++ b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtil.scala @@ -0,0 +1,152 @@ +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed 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 za.co.absa.hyperdrive.ingestor.implementation.utils + +import org.apache.kafka.common.TopicPartition +import org.apache.spark.sql.execution.streaming._ +import org.scalatest.{BeforeAndAfter, FlatSpec, Matchers} +import za.co.absa.commons.io.TempDirectory +import za.co.absa.commons.spark.SparkTestBase +import za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka.kafka010.KafkaSourceOffset + +class TestKafkaUtil extends FlatSpec with Matchers with BeforeAndAfter with SparkTestBase { + private var baseDir: TempDirectory = _ + + before { + baseDir = TempDirectory("test-dir").deleteOnExit() + } + + after{ + baseDir.delete() + } + + "getLatestOffset" should "return the latest offsets" in { + // given + val offset0 = OffsetSeq.fill(KafkaSourceOffset(("t", 0, 10L), ("t", 1, 110L))) + val offset1 = OffsetSeq.fill(KafkaSourceOffset(("t", 0, 25L), ("t", 1, 125L))) + val offset2 = OffsetSeq.fill(KafkaSourceOffset(("t", 0, 42L), ("t", 1, 142L))) + + val offsetSeqLog = new OffsetSeqLog(spark, baseDir.path.toString) + offsetSeqLog.add(0, offset0) + offsetSeqLog.add(1, offset1) + offsetSeqLog.add(2, offset2) + + // when + val latestOffset = KafkaUtil.getLatestOffset(offsetSeqLog) + + // then + latestOffset.get should contain theSameElementsAs Map( + new TopicPartition("t", 0) -> 42L, + new TopicPartition("t", 1) -> 142L + ) + } + + it should "return None if there are no offsets" in { + val offsetSeqLog = new OffsetSeqLog(spark, baseDir.path.toString) + + val latestOffset = KafkaUtil.getLatestOffset(offsetSeqLog) + + latestOffset shouldBe None + } + + it should "return None if the offset is not defined" in { + // given + val offset = OffsetSeq.fill(null: Offset) + val offsetSeqLog = new OffsetSeqLog(spark, baseDir.path.toString) + offsetSeqLog.add(0, offset) + + // when + val result = KafkaUtil.getLatestOffset(offsetSeqLog) + + // then + result shouldBe None + } + + it should "throw an exception if the offsets contain multiple sources" in { + // given + val offset = OffsetSeq.fill(KafkaSourceOffset(("t", 0, 10L)), KafkaSourceOffset(("t2", 1, 110L))) + val offsetSeqLog = new OffsetSeqLog(spark, baseDir.path.toString) + offsetSeqLog.add(0, offset) + + // when + val exception = the[Exception] thrownBy KafkaUtil.getLatestOffset(offsetSeqLog) + + // then + exception.getMessage should include("Cannot support more than 1 source") + } + + "getLatestCommittedOffset" should "return the latest committed offset" in { + // given + val commitLog = new CommitLog(spark, s"${baseDir.path.toString}/commits") + commitLog.add(0, CommitMetadata()) + commitLog.add(1, CommitMetadata()) + + val offset0 = OffsetSeq.fill(KafkaSourceOffset(("t", 0, 10L), ("t", 1, 110L))) + val offset1 = OffsetSeq.fill(KafkaSourceOffset(("t", 0, 25L), ("t", 1, 125L))) + val offset2 = OffsetSeq.fill(KafkaSourceOffset(("t", 0, 42L), ("t", 1, 142L))) + + val offsetSeqLog = new OffsetSeqLog(spark, s"${baseDir.path.toString}/offsets") + offsetSeqLog.add(0, offset0) + offsetSeqLog.add(1, offset1) + offsetSeqLog.add(2, offset2) + + // when + val actualOffset = KafkaUtil.getLatestCommittedOffset(offsetSeqLog, commitLog) + + // then + actualOffset.get should contain theSameElementsAs Map( + new TopicPartition("t", 0) -> 25L, + new TopicPartition("t", 1) -> 125L + ) + } + + it should "return None if there is no commit" in { + val commitLog = new CommitLog(spark, s"${baseDir.path.toString}/commits") + + val offset0 = OffsetSeq.fill(KafkaSourceOffset(("t", 0, 10L), ("t", 1, 110L))) + val offset1 = OffsetSeq.fill(KafkaSourceOffset(("t", 0, 25L), ("t", 1, 125L))) + val offset2 = OffsetSeq.fill(KafkaSourceOffset(("t", 0, 42L), ("t", 1, 142L))) + + val offsetSeqLog = new OffsetSeqLog(spark, s"${baseDir.path.toString}/offsets") + offsetSeqLog.add(0, offset0) + offsetSeqLog.add(1, offset1) + offsetSeqLog.add(2, offset2) + + // when + val actualOffset = KafkaUtil.getLatestCommittedOffset(offsetSeqLog, commitLog) + + // then + actualOffset shouldBe None + } + + it should "return throw an exception if there is no offset corresponding to the commit" in { + // given + val commitLog = new CommitLog(spark, s"${baseDir.path.toString}/commits") + commitLog.add(0, CommitMetadata()) + commitLog.add(1, CommitMetadata()) + + val offset0 = OffsetSeq.fill(KafkaSourceOffset(("t", 0, 10L), ("t", 1, 110L))) + + val offsetSeqLog = new OffsetSeqLog(spark, s"${baseDir.path.toString}/offsets") + offsetSeqLog.add(0, offset0) + + // when + val result = the[Exception] thrownBy KafkaUtil.getLatestCommittedOffset(offsetSeqLog, commitLog) + + // then + result.getMessage should include ("batchId 1") + } +} diff --git a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala index 5fa7ce88..583f0884 100644 --- a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala +++ b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala @@ -21,17 +21,21 @@ import java.util.UUID.randomUUID import java.util.{Collections, Properties} import org.apache.kafka.clients.admin.{AdminClient, AdminClientConfig, NewTopic} -import org.apache.kafka.clients.consumer.KafkaConsumer +import org.apache.kafka.clients.consumer.{ConsumerRecord, KafkaConsumer} import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} +import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializer} -import org.scalatest.{BeforeAndAfter, FlatSpec, Matchers} +import org.scalatest.{AppendedClues, BeforeAndAfter, FlatSpec, Matchers} import org.testcontainers.containers.KafkaContainer import scala.collection.JavaConverters._ -class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter { +import scala.collection.mutable +class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter with AppendedClues { private val confluentPlatformVersion = "5.3.4" // should be same as kafka.avro.serializer.version property in pom file private val kafka = new KafkaContainer(confluentPlatformVersion) + private val kafkaSufficientTimeout = Duration.ofSeconds(5L) + private val kafkaInsufficientTimeout = Duration.ofMillis(1L) before{ kafka.start() @@ -43,7 +47,7 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter "getAllAvailableMessages" should "get all available messages" in { // given - val topic = "get-all-available-messages-topic" + val topic = "test-topic" val partitions = 3 createTopic(kafka, topic, partitions) val producer = createProducer(kafka) @@ -51,12 +55,11 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter produceData(producer, messages, topic, partitions) val consumer = createConsumer(kafka) - consumer.subscribe(Collections.singletonList(topic)) val topicPartitions = KafkaUtil.getTopicPartitions(consumer, topic) val offsets = consumer.endOffsets(topicPartitions.asJava).asScala.toMap.mapValues(_.asInstanceOf[Long]) // when - implicit val kafkaConsumerTimeout: Duration = Duration.ofMillis(1500L) + implicit val kafkaConsumerTimeout: Duration = kafkaSufficientTimeout val records = KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets) // then @@ -64,31 +67,88 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter actualMessages should contain theSameElementsAs messages } - it should "throw an exception if partitions were not assigned" in { + it should "get all available messages, even if polling is required multiple times" in { // given - val topic = "get-all-available-messages-topic" + val topic = "test-topic" + val partitions = 3 + createTopic(kafka, topic, partitions, Map("segment.ms" -> "100")) + val producer = createProducer(kafka) + val messages = (1 to 100).map(i => s"message_${i}") + produceData(producer, messages, topic, partitions) + + val consumer = createConsumer(kafka) + val topicPartitions = KafkaUtil.getTopicPartitions(consumer, topic) + val offsets = consumer.endOffsets(topicPartitions.asJava).asScala.toMap.mapValues(_.asInstanceOf[Long]) + + // when + implicit val kafkaConsumerTimeout: Duration = kafkaSufficientTimeout + val records = KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets) + + // then + val actualMessages = records.map(_.value()).toList.sorted + actualMessages should contain theSameElementsAs messages + } + + it should "stop polling when the desired end offset has been reached and not run infinitely" in { + // given + val topic = "test-topic" val partitions = 3 createTopic(kafka, topic, partitions) val producer = createProducer(kafka) val messages = (1 to 100).map(i => s"message_${i}") produceData(producer, messages, topic, partitions) + val infiniteProducerThread = new Thread { + override def run { + var i = 0 + while (true) { + val partition = i % partitions + val producerRecord = new ProducerRecord[String, String](topic, partition, null, s"message_${i}") + producer.send(producerRecord) + i += 1 + if (i % 100 == 0) { + producer.flush() + } + } + } + } + val consumer = createConsumer(kafka) - consumer.subscribe(Collections.singletonList(topic)) val topicPartitions = KafkaUtil.getTopicPartitions(consumer, topic) val offsets = consumer.endOffsets(topicPartitions.asJava).asScala.toMap.mapValues(_.asInstanceOf[Long]) + infiniteProducerThread.start() // when - implicit val kafkaConsumerTimeout: Duration = Duration.ofMillis(1L) - val exception = the[Exception] thrownBy KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets) + implicit val kafkaConsumerTimeout: Duration = kafkaSufficientTimeout + val records = KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets) + + // then + val actualMessages = records.map(_.value()).toList.sorted + actualMessages should contain allElementsOf messages + + // cleanup + infiniteProducerThread.interrupt() + } + + it should "throw an exception if consumer is already subscribed" in { + // given + val topic = "test-topic" + createTopic(kafka, topic, 1) + + val consumer = createConsumer(kafka) + consumer.subscribe(Collections.singletonList(topic)) + + // when + implicit val kafkaConsumerTimeout: Duration = kafkaInsufficientTimeout + val exception = the[Exception] thrownBy KafkaUtil.getMessagesAtLeastToOffset(consumer, Map(new TopicPartition(topic, 0) -> 0)) // then - exception.getMessage should include ("Consumer is unexpectedly not assigned") + exception.getMessage should include ("Subscription to topics, partitions and pattern are mutually exclusive") } - it should "throw an exception if not all messages could be consumed" in { + it should "throw an exception if not all messages could be consumed (because the timeout is too short)" in { // given - val topic = "get-all-available-messages-topic" + val topic = "test-topic" val partitions = 3 createTopic(kafka, topic, partitions) val producer = createProducer(kafka) @@ -102,17 +162,16 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter val offsets = consumer.endOffsets(topicPartitions.asJava).asScala.toMap.mapValues(_.asInstanceOf[Long]) // when - implicit val kafkaConsumerTimeout: Duration = Duration.ofMillis(1L) + implicit val kafkaConsumerTimeout: Duration = kafkaInsufficientTimeout val exception = the[Exception] thrownBy KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets) // then exception.getMessage should include ("Not all expected messages were consumed") } - it should "throw an exception if requested offsets are not available" in { // given - val topic = "get-all-available-messages-topic" + val topic = "test-topic" val partitions = 3 createTopic(kafka, topic, partitions) val producer = createProducer(kafka) @@ -120,26 +179,170 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter produceData(producer, messages, topic, partitions) val consumer = createConsumer(kafka) - consumer.subscribe(Collections.singletonList(topic)) val topicPartitions = KafkaUtil.getTopicPartitions(consumer, topic) val offsets = consumer.endOffsets(topicPartitions.asJava).asScala.toMap.mapValues(_ * 2L) // when - implicit val kafkaConsumerTimeout: Duration = Duration.ofMillis(0L) + implicit val kafkaConsumerTimeout: Duration = kafkaInsufficientTimeout val exception = the[Exception] thrownBy KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets) // then exception.getMessage should include ("Requested consumption") } - private def createTopic(kafkaContainer: KafkaContainer, topicName: String, partitions: Int): Unit = { + "getAtLeastNLatestRecords" should "get at least the n latest records" in { + val topicName = "test-topic" + createTopic(kafka, topicName, 1, Map( + "cleanup.policy" -> "compact", + "delete.retention.ms" -> "100", + "segment.ms" -> "100", + "min.cleanable.dirty.ratio" -> "0.01" + )) + + val producer = createProducer(kafka) + val messages = (1 to 100).map(i => { + val key = if (i % 2 == 0) 1000 + i else 1 + (key.toString, s"msg_${i}") + }) + produceData2(producer, messages, topicName) + + val waitForCompactionMillis = 20000L + Thread.sleep(waitForCompactionMillis) + + val testConsumer = createConsumer(kafka) + testConsumer.subscribe(Collections.singletonList(topicName)) + import scala.util.control.Breaks._ + var records: Seq[ConsumerRecord[String, String]] = mutable.Seq() + breakable { + while (true) { + val newRecords = testConsumer.poll(kafkaSufficientTimeout).asScala.toSeq + records ++= newRecords + if (newRecords.isEmpty) { + break() + } + } + } + + withClue(){ + records.size shouldBe messages.map(_._1).distinct.size + } withClue(s"This is likely an artifact of the test itself. You may want to increase waitForCompactionMillis." + + s" The current value is $waitForCompactionMillis") + + val consumer = createConsumer(kafka) + implicit val kafkaConsumerTimeout: Duration = kafkaSufficientTimeout + val actualRecords = KafkaUtil.getAtLeastNLatestRecordsFromPartition(consumer, new TopicPartition(topicName, 0), 10) + val values = actualRecords.map(_.value()) + + values.size should be >= 10 + values should contain allElementsOf Seq("msg_100", "msg_99", "msg_98", "msg_96", "msg_94", "msg_92", "msg_90", "msg_88", "msg_86", "msg_84") + } + + it should "be able to reuse a consumer" in { + // given + val topic = "test-topic" + val partitions = 3 + createTopic(kafka, topic, partitions) + val producer = createProducer(kafka) + val messages = (1 to 100).map(i => s"message_${i}") + produceData(producer, messages, topic, partitions) + + val consumer = createConsumer(kafka) + val topicPartitions = KafkaUtil.getTopicPartitions(consumer, topic) + + // when + implicit val kafkaConsumerTimeout: Duration = kafkaSufficientTimeout + val records = topicPartitions.flatMap(tp => KafkaUtil.getAtLeastNLatestRecordsFromPartition(consumer, tp, 1000)) + + // then + val actualMessages = records.map(_.value()).toList.sorted + actualMessages should contain theSameElementsAs messages + } + + it should "throw an exception if the timeout is too short" in { + val topicName = "test-topic" + createTopic(kafka, topicName, 1) + + val producer = createProducer(kafka) + val messages = (1 to 100).map(i => s"message_${i}") + + produceData(producer, messages, topicName, 1) + + val consumer = createConsumer(kafka) + implicit val kafkaConsumerTimeout: Duration = kafkaInsufficientTimeout + val result = the[Exception] thrownBy KafkaUtil.getAtLeastNLatestRecordsFromPartition(consumer, new TopicPartition(topicName, 0), 10) + result.getMessage should include("increasing the consumer timeout") + } + + "getTopicPartitions" should "return the partitions" in { + val topicName = "test-topic" + createTopic(kafka, topicName, 10) + val consumer = createConsumer(kafka) + + val topicPartitions = KafkaUtil.getTopicPartitions(consumer, topicName) + + val expectedPartitions = (0 until 10).map(i => new TopicPartition(topicName, i)) + topicPartitions should contain theSameElementsAs expectedPartitions + } + + "seekToOffsetsOrBeginning" should "seek to the provided offsets" in { + // given + val topicName = "test-topic" + createTopic(kafka, topicName, 3) + val consumer = createConsumer(kafka) + + val producer = createProducer(kafka) + val messages = (1 to 100).map(i => s"message_${i}") + produceData(producer, messages, topicName, 3) + + val tp0 = new TopicPartition(topicName, 0) + val tp1 = new TopicPartition(topicName, 1) + val tp2 = new TopicPartition(topicName, 2) + val offsets = Map( + tp0 -> 10L, + tp1 -> 15L, + tp2 -> 20L + ) + + // when + KafkaUtil.seekToOffsetsOrBeginning(consumer, topicName, Some(offsets)) + + // then + consumer.position(tp0) shouldBe 10L + consumer.position(tp1) shouldBe 15L + consumer.position(tp2) shouldBe 20L + } + + it should "seek to the beginning if no offsets are given" in { + // given + val topicName = "test-topic" + createTopic(kafka, topicName, 3) + val consumer = createConsumer(kafka) + + val producer = createProducer(kafka) + val messages = (1 to 100).map(i => s"message_${i}") + produceData(producer, messages, topicName, 3) + + val tp0 = new TopicPartition(topicName, 0) + val tp1 = new TopicPartition(topicName, 1) + val tp2 = new TopicPartition(topicName, 2) + + // when + KafkaUtil.seekToOffsetsOrBeginning(consumer, topicName, None) + + // then + consumer.position(tp0) shouldBe 0L + consumer.position(tp1) shouldBe 0L + consumer.position(tp2) shouldBe 0L + } + + private def createTopic(kafkaContainer: KafkaContainer, topicName: String, partitions: Int, extraConfig: Map[String, String] = Map()): Unit = { val config = new Properties() config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaContainer.getBootstrapServers) val localKafkaAdmin = AdminClient.create(config) val replication = 1.toShort - val topic = new NewTopic(topicName, partitions, replication) + val topic = new NewTopic(topicName, partitions, replication).configs(extraConfig.asJava) val topicCreationFut = localKafkaAdmin.createTopics(util.Arrays.asList(topic)).all() - while(!topicCreationFut.isDone) {} + topicCreationFut.get() } def createProducer(kafkaContainer: KafkaContainer): KafkaProducer[String, String] = { @@ -160,6 +363,7 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaContainer.getBootstrapServers) props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, classOf[StringDeserializer].getCanonicalName) props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[StringDeserializer].getCanonicalName) + props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") new KafkaConsumer[String, String](props) } @@ -173,6 +377,14 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter producer.flush() } + private def produceData2(producer: KafkaProducer[String, String], records: Seq[(String, String)], topic: String): Unit = { + records.foreach { record => + val producerRecord = new ProducerRecord[String, String](topic, record._1, record._2) + producer.send(producerRecord) + Thread.sleep(100L) + } + producer.flush() + } } From 680e08570dac2ff45e6e20d75bb698510afb6091 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Fri, 4 Dec 2020 15:02:08 +0100 Subject: [PATCH 17/27] Add tests to AvroUtil --- .../DeduplicateKafkaSinkTransformer.scala | 10 +- .../kafka/kafka010/KafkaSourceOffset.scala | 2 + .../implementation/utils/AvroUtil.scala | 43 +++++--- .../implementation/utils/KafkaUtil.scala | 2 +- .../implementation/utils/TestAvroUtil.scala | 101 +++++++++++++++--- 5 files changed, 119 insertions(+), 39 deletions(-) diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala index 53930bb9..c81e87f9 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala @@ -93,12 +93,10 @@ private[transformer] class DeduplicateKafkaSinkTransformer( } private def extractIdFieldsFromRecord(record: ConsumerRecord[GenericRecord, GenericRecord], idColumnNames: Seq[String]): Seq[Any] = { - try { - AvroUtil.getIdColumnsFromRecord(record, idColumnNames) - } catch { - case throwable: Throwable => logger.error(s"Could not get $idColumnNames from record, schema is ${record.value().getSchema}", throwable) - throw throwable - } + idColumnNames.map(idColumnName => + AvroUtil.getFromConsumerRecord(record, idColumnName) + .getOrElse(throw new IllegalArgumentException(s"Could not find value for field $idColumnName")) + ) } private def consumeAndClose[T](consumer: KafkaConsumer[GenericRecord, GenericRecord], consume: KafkaConsumer[GenericRecord, GenericRecord] => T) = { diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/kafka010/KafkaSourceOffset.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/kafka010/KafkaSourceOffset.scala index 22b76d2f..7c69283f 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/kafka010/KafkaSourceOffset.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/kafka010/KafkaSourceOffset.scala @@ -8,6 +8,8 @@ import org.apache.spark.sql.sources.v2.reader.streaming.PartitionOffset * An [[Offset]] for the [[KafkaSource]]. This one tracks all partitions of subscribed topics and * their offsets. */ +// TODO: Move this to a spark package so the namespace can be reused + case class KafkaSourceOffset(partitionToOffsets: Map[TopicPartition, Long]) extends Offset { override val json = JsonUtils.partitionOffsets(partitionToOffsets) diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/AvroUtil.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/AvroUtil.scala index d2dcbdb3..23d4934e 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/AvroUtil.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/AvroUtil.scala @@ -24,29 +24,40 @@ import scala.annotation.tailrec private[hyperdrive] object AvroUtil { - def getIdColumnsFromRecord(record: ConsumerRecord[GenericRecord, GenericRecord], idColumnNames: Seq[String]): Seq[Any] = { - idColumnNames.map { - case "topic" => record.topic() - case "offset" => record.offset() - case "partition" => record.partition() - case "timestamp" => record.timestamp() - case "timestampType" => record.timestampType() - case "serializedKeySize" => record.serializedKeySize() - case "serializedValueSize" => record.serializedValueSize() - case "headers" => record.headers() - case keyColumn if keyColumn.startsWith("key.") => getRecursively(record.value(), + def getFromConsumerRecord(record: ConsumerRecord[GenericRecord, GenericRecord], fieldName: String): Option[Any] = { + val fieldValue = fieldName match { + case "topic" => Option(record.topic()) + case "offset" => Option(record.offset()) + case "partition" => Option(record.partition()) + case "timestamp" => Option(record.timestamp()) + case "timestampType" => Option(record.timestampType()) + case "serializedKeySize" => Option(record.serializedKeySize()) + case "serializedValueSize" => Option(record.serializedValueSize()) + case "headers" => Option(record.headers()) + case keyColumn if keyColumn.startsWith("key.") => getFromGenericRecordNullSafe(record.key(), UnresolvedAttribute.parseAttributeName(keyColumn.stripPrefix("key.")).toList) - case valueColumn if valueColumn.startsWith("value.") => getRecursively(record.value(), + case valueColumn if valueColumn.startsWith("value.") => getFromGenericRecordNullSafe(record.value(), UnresolvedAttribute.parseAttributeName(valueColumn.stripPrefix("value.")).toList) - }.map { + case _ => None + } + + fieldValue.map { case utf8: Utf8 => utf8.toString case v => v } } + private def getFromGenericRecordNullSafe(record: GenericRecord, keys: Seq[String]) = + Option(record).flatMap(getFromGenericRecord(_, keys)) + @tailrec - def getRecursively(record: GenericRecord, keys: List[String]): Any = keys match { - case key :: Nil => record.get(key) - case head :: tail => getRecursively(record.get(head).asInstanceOf[GenericRecord], tail) + private def getFromGenericRecord(record: GenericRecord, keys: Seq[String]): Option[Any] = keys match { + case key :: Nil => Option(record.get(key)) + case head :: tail => + val value = record.get(head) + value match { + case genericRecord: GenericRecord => getFromGenericRecord(genericRecord, tail) + case _ => None + } } } diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala index 6a614615..1de786b9 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala @@ -26,7 +26,7 @@ import za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kaf import scala.collection.JavaConverters._ import scala.collection.mutable -object KafkaUtil { +private[hyperdrive] object KafkaUtil { private val logger = LogManager.getLogger def getAtLeastNLatestRecordsFromPartition[K, V](consumer: KafkaConsumer[K, V], topicPartition: TopicPartition, numberOfRecords: Int) diff --git a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestAvroUtil.scala b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestAvroUtil.scala index 33fdf78a..24666d18 100644 --- a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestAvroUtil.scala +++ b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestAvroUtil.scala @@ -16,35 +16,104 @@ package za.co.absa.hyperdrive.ingestor.implementation.utils import org.apache.avro.Schema.Parser -import org.apache.avro.generic.{GenericData, GenericRecordBuilder} +import org.apache.avro.generic.{GenericData, GenericRecord, GenericRecordBuilder} +import org.apache.avro.util.Utf8 +import org.apache.kafka.clients.consumer.ConsumerRecord import org.scalatest.{BeforeAndAfter, FlatSpec, Matchers} class TestAvroUtil extends FlatSpec with Matchers with BeforeAndAfter { - "getRecursively" should "get recursively" in { - // given - val rootSchemaString = raw""" + + private val valueSchemaString = raw""" {"type": "record", "name": "schemaName", "fields": [ {"type": "int", "name": "record_id"}, {"type": ["null", "schemaName"], "name": "child_record", "nullable": true} ]}""" - val parser = new Parser() - val schema = parser.parse(rootSchemaString) + private val keySchemaString = raw""" + {"type": "record", "name": "keySchema", "fields": [ + {"type": "string", "name": "key"} + ]}""" - val childRecord2 = new GenericData.Record(schema) - childRecord2.put("record_id", 3) + "getIdColumnsFromRecord" should "get the specified fields from the record" in { + // given + val parser = new Parser() + val valueSchema = parser.parse(valueSchemaString) + val childRecord2 = new GenericData.Record(valueSchema) + childRecord2.put("record_id", 13) childRecord2.put("child_record", null) - val childRecord1 = new GenericData.Record(schema) - childRecord1.put("record_id", 2) + val childRecord1 = new GenericData.Record(valueSchema) + childRecord1.put("record_id", 12) childRecord1.put("child_record", childRecord2) - val rootRecord = new GenericData.Record(schema) - rootRecord.put("record_id", 1) - rootRecord.put("child_record", childRecord1) + val valueRecord = new GenericData.Record(valueSchema) + valueRecord.put("record_id", 11) + valueRecord.put("child_record", childRecord1) + + val keySchema = parser.parse(keySchemaString) + val keyRecord = new GenericData.Record(keySchema) + keyRecord.put("key", new Utf8("abcdef")) + + val consumerRecord: ConsumerRecord[GenericRecord, GenericRecord] = + new ConsumerRecord("topicName", 0, 42, keyRecord, valueRecord) + + // when, then + AvroUtil.getFromConsumerRecord(consumerRecord, "topic") shouldBe Some("topicName") + AvroUtil.getFromConsumerRecord(consumerRecord, "offset") shouldBe Some(42) + AvroUtil.getFromConsumerRecord(consumerRecord, "partition") shouldBe Some(0) + AvroUtil.getFromConsumerRecord(consumerRecord, "key.key") shouldBe Some("abcdef") + AvroUtil.getFromConsumerRecord(consumerRecord, "value.record_id") shouldBe Some(11) + AvroUtil.getFromConsumerRecord(consumerRecord, "value.child_record.record_id") shouldBe Some(12) + AvroUtil.getFromConsumerRecord(consumerRecord, "value.child_record.child_record.record_id") shouldBe Some(13) + } + + it should "return None if a record is not nested as expected" in { + // given + val parser = new Parser() + val valueSchema = parser.parse(valueSchemaString) + val valueRecord = new GenericData.Record(valueSchema) + valueRecord.put("record_id", 11) + valueRecord.put("child_record", null) + + val consumerRecord: ConsumerRecord[GenericRecord, GenericRecord] = + new ConsumerRecord("topicName", 0, 42, null, valueRecord) + + // when, then + AvroUtil.getFromConsumerRecord(consumerRecord, "value.child_record.child_record.record_id") shouldBe None + } + + it should "return None if a field does not exist" in { + // given + val parser = new Parser() + val valueSchema = parser.parse(valueSchemaString) + val valueRecord = new GenericData.Record(valueSchema) + valueRecord.put("record_id", 11) + valueRecord.put("child_record", null) + + val keySchema = parser.parse(keySchemaString) + val keyRecord = new GenericData.Record(keySchema) + keyRecord.put("key", new Utf8("abcdef")) + + val consumerRecord: ConsumerRecord[GenericRecord, GenericRecord] = + new ConsumerRecord("topicName", 0, 42, keyRecord, valueRecord) + + // when, then + AvroUtil.getFromConsumerRecord(consumerRecord, "key.some_nonexistent_field") shouldBe None + } + + it should "return None if a field does not exist on the consumer record" in { + // given + val consumerRecord: ConsumerRecord[GenericRecord, GenericRecord] = + new ConsumerRecord("topicName", 0, 42, null, null) // when, then - AvroUtil.getRecursively(rootRecord, List("record_id")).asInstanceOf[Int] shouldBe 1 - AvroUtil.getRecursively(rootRecord, List("child_record", "record_id")).asInstanceOf[Int] shouldBe 2 - AvroUtil.getRecursively(rootRecord, List("child_record", "child_record", "record_id")).asInstanceOf[Int] shouldBe 3 + AvroUtil.getFromConsumerRecord(consumerRecord, "some_nonexistent_field") shouldBe None } + it should "return None if a field on the record is requested, but the record is null" in { + // given + val consumerRecord: ConsumerRecord[GenericRecord, GenericRecord] = + new ConsumerRecord("topicName", 0, 42, null, null) + + // when, then + AvroUtil.getFromConsumerRecord(consumerRecord, "key.some_nonexistent_field") shouldBe None + } } From 39f854ab9c7cfcbc6ee140efbbd53871dd997103 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Fri, 4 Dec 2020 15:32:47 +0100 Subject: [PATCH 18/27] Replace copy pasted package private code with proxy object in same package --- .../sql/kafka010/KafkaSourceOffsetProxy.scala | 29 +++++ .../DeduplicateKafkaSinkTransformer.scala | 10 +- ...licateKafkaSinkTransformerAttributes.scala | 6 +- .../kafka/kafka010/JsonUtils.scala | 102 ------------------ .../kafka/kafka010/KafkaSourceOffset.scala | 48 --------- .../implementation/utils/KafkaUtil.scala | 4 +- .../implementation/utils/TestKafkaUtil.scala | 24 ++--- 7 files changed, 54 insertions(+), 169 deletions(-) create mode 100644 ingestor-default/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetProxy.scala delete mode 100644 ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/kafka010/JsonUtils.scala delete mode 100644 ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/kafka010/KafkaSourceOffset.scala diff --git a/ingestor-default/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetProxy.scala b/ingestor-default/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetProxy.scala new file mode 100644 index 00000000..66324c5b --- /dev/null +++ b/ingestor-default/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetProxy.scala @@ -0,0 +1,29 @@ +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed 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.spark.sql.kafka010 + +import org.apache.kafka.common.TopicPartition +import org.apache.spark.sql.execution.streaming.Offset + +object KafkaSourceOffsetProxy { + def getPartitionOffsets(offset: Offset): Map[TopicPartition, Long] = { + KafkaSourceOffset.getPartitionOffsets(offset) + } + + def apply(offsetTuples: (String, Int, Long)*): KafkaSourceOffset = { + KafkaSourceOffset.apply(offsetTuples:_*) + } +} diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala index c81e87f9..adaeabb1 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala @@ -49,7 +49,8 @@ private[transformer] class DeduplicateKafkaSinkTransformer( val writerExtraOptions: Map[String, String], val checkpointLocation: String, val sourceIdColumnNames: Seq[String], - val destinationIdColumnNames: Seq[String] + val destinationIdColumnNames: Seq[String], + val kafkaConsumerTimeout: Duration ) extends StreamTransformer { private val logger = LogManager.getLogger @@ -69,7 +70,7 @@ private[transformer] class DeduplicateKafkaSinkTransformer( private def deduplicateDataFrame(dataFrame: DataFrame, offsetLog: OffsetSeqLog, commitLog: CommitLog) = { logger.info("Deduplicate rows after retry") - implicit val kafkaConsumerTimeout: Duration = Duration.ofSeconds(5L) // TODO: Make it configurable + implicit val kafkaConsumerTimeoutImpl: Duration = kafkaConsumerTimeout val sourceConsumer = createConsumer(readerBrokers, readerExtraOptions, readerSchemaRegistryUrl) val latestCommittedOffsets = KafkaUtil.getLatestCommittedOffset(offsetLog, commitLog) KafkaUtil.seekToOffsetsOrBeginning(sourceConsumer, readerTopic, latestCommittedOffsets) @@ -127,6 +128,7 @@ private[transformer] class DeduplicateKafkaSinkTransformer( object DeduplicateKafkaSinkTransformer extends StreamTransformerFactory with DeduplicateKafkaSinkTransformerAttributes { + private val DefaultKafkaConsumerTimeoutSeconds = 120L override def apply(config: Configuration): StreamTransformer = { // TODO: How to get schemaRegistryUrl from transformer config? val readerSchemaRegistryUrl = getOrThrow(schemaRegistryUrl, config) @@ -148,9 +150,11 @@ object DeduplicateKafkaSinkTransformer extends StreamTransformerFactory with Ded s"${sourceIdColumns.size} != ${destinationIdColumns.size}.") } + val kafkaConsumerTimeout = Duration.ofSeconds(config.getLong(KafkaConsumerTimeout, DefaultKafkaConsumerTimeoutSeconds)) + new DeduplicateKafkaSinkTransformer(readerSchemaRegistryUrl, readerTopic, readerBrokers, readerExtraOptions, writerSchemaRegistryUrl, writerTopic, writerBrokers, writerExtraOptions, - checkpointLocation, sourceIdColumns, destinationIdColumns) + checkpointLocation, sourceIdColumns, destinationIdColumns, kafkaConsumerTimeout) } override def getMappingFromRetainedGlobalConfigToLocalConfig(globalConfig: Configuration): Map[String, String] = { diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformerAttributes.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformerAttributes.scala index e941a053..cd530d6a 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformerAttributes.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformerAttributes.scala @@ -22,13 +22,15 @@ trait DeduplicateKafkaSinkTransformerAttributes extends HasComponentAttributes { val SourceIdColumns = "source.id.columns" val DestinationIdColumns = "destination.id.columns" val schemaRegistryUrl = "schema.registry.url" + val KafkaConsumerTimeout = "kafka.consumer.timeout" override def getName: String = "Deduplicate Kafka Sink Transformer" override def getDescription: String = "" override def getProperties: Map[String, PropertyMetadata] = Map( - SourceIdColumns -> PropertyMetadata("Id column", Some(""), required = true), - DestinationIdColumns -> PropertyMetadata("Id column", Some(""), required = true) + SourceIdColumns -> PropertyMetadata("Source Id columns", Some("Comma separated list of columns that represent the id"), required = true), + DestinationIdColumns -> PropertyMetadata("Destination Id columns", Some("Comma separated list of columns that represent the id"), required = true), + KafkaConsumerTimeout -> PropertyMetadata("Kafka consumer timeout in seconds", None, required = false) ) } diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/kafka010/JsonUtils.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/kafka010/JsonUtils.scala deleted file mode 100644 index dc9dd69e..00000000 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/kafka010/JsonUtils.scala +++ /dev/null @@ -1,102 +0,0 @@ - -package za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka.kafka010 - -import scala.collection.mutable.HashMap -import scala.util.control.NonFatal - -import org.apache.kafka.common.TopicPartition -import org.json4s.NoTypeHints -import org.json4s.jackson.Serialization - -/** - * Utilities for converting Kafka related objects to and from json. - */ -private[kafka] object JsonUtils { - private implicit val formats = Serialization.formats(NoTypeHints) - - /** - * Read TopicPartitions from json string - */ - def partitions(str: String): Array[TopicPartition] = { - try { - Serialization.read[Map[String, Seq[Int]]](str).flatMap { case (topic, parts) => - parts.map { part => - new TopicPartition(topic, part) - } - }.toArray - } catch { - case NonFatal(x) => - throw new IllegalArgumentException( - s"""Expected e.g. {"topicA":[0,1],"topicB":[0,1]}, got $str""") - } - } - - /** - * Write TopicPartitions as json string - */ - def partitions(partitions: Iterable[TopicPartition]): String = { - val result = new HashMap[String, List[Int]] - partitions.foreach { tp => - val parts: List[Int] = result.getOrElse(tp.topic, Nil) - result += tp.topic -> (tp.partition::parts) - } - Serialization.write(result) - } - - /** - * Read per-TopicPartition offsets from json string - */ - def partitionOffsets(str: String): Map[TopicPartition, Long] = { - try { - Serialization.read[Map[String, Map[Int, Long]]](str).flatMap { case (topic, partOffsets) => - partOffsets.map { case (part, offset) => - new TopicPartition(topic, part) -> offset - } - } - } catch { - case NonFatal(x) => - throw new IllegalArgumentException( - s"""Expected e.g. {"topicA":{"0":23,"1":-1},"topicB":{"0":-2}}, got $str""") - } - } - - def partitionTimestamps(str: String): Map[TopicPartition, Long] = { - try { - Serialization.read[Map[String, Map[Int, Long]]](str).flatMap { case (topic, partTimestamps) => - partTimestamps.map { case (part, timestamp) => - new TopicPartition(topic, part) -> timestamp - } - } - } catch { - case NonFatal(x) => - throw new IllegalArgumentException( - s"""Expected e.g. {"topicA": {"0": 123456789, "1": 123456789}, - |"topicB": {"0": 123456789, "1": 123456789}}, got $str""".stripMargin) - } - } - - /** - * Write per-TopicPartition offsets as json string - */ - def partitionOffsets(partitionOffsets: Map[TopicPartition, Long]): String = { - val result = new HashMap[String, HashMap[Int, Long]]() - implicit val order = new Ordering[TopicPartition] { - override def compare(x: TopicPartition, y: TopicPartition): Int = { - Ordering.Tuple2[String, Int].compare((x.topic, x.partition), (y.topic, y.partition)) - } - } - val partitions = partitionOffsets.keySet.toSeq.sorted // sort for more determinism - partitions.foreach { tp => - val off = partitionOffsets(tp) - val parts = result.getOrElse(tp.topic, new HashMap[Int, Long]) - parts += tp.partition -> off - result += tp.topic -> parts - } - Serialization.write(result) - } - - def partitionTimestamps(topicTimestamps: Map[TopicPartition, Long]): String = { - // For now it's same as partitionOffsets - partitionOffsets(topicTimestamps) - } -} diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/kafka010/KafkaSourceOffset.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/kafka010/KafkaSourceOffset.scala deleted file mode 100644 index 7c69283f..00000000 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/kafka010/KafkaSourceOffset.scala +++ /dev/null @@ -1,48 +0,0 @@ - -package za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka.kafka010 - -import org.apache.kafka.common.TopicPartition -import org.apache.spark.sql.execution.streaming.{Offset, SerializedOffset} -import org.apache.spark.sql.sources.v2.reader.streaming.PartitionOffset -/** - * An [[Offset]] for the [[KafkaSource]]. This one tracks all partitions of subscribed topics and - * their offsets. - */ -// TODO: Move this to a spark package so the namespace can be reused - -case class KafkaSourceOffset(partitionToOffsets: Map[TopicPartition, Long]) extends Offset { - - override val json = JsonUtils.partitionOffsets(partitionToOffsets) -} - -private[hyperdrive] -case class KafkaSourcePartitionOffset(topicPartition: TopicPartition, partitionOffset: Long) - extends PartitionOffset - -/** Companion object of the [[KafkaSourceOffset]] */ -private[hyperdrive] object KafkaSourceOffset { - - def getPartitionOffsets(offset: Offset): Map[TopicPartition, Long] = { - offset match { - case o: KafkaSourceOffset => o.partitionToOffsets - case so: SerializedOffset => KafkaSourceOffset(so).partitionToOffsets - case _ => - throw new IllegalArgumentException( - s"Invalid conversion from offset of ${offset.getClass} to KafkaSourceOffset") - } - } - - /** - * Returns [[KafkaSourceOffset]] from a variable sequence of (topic, partitionId, offset) - * tuples. - */ - def apply(offsetTuples: (String, Int, Long)*): KafkaSourceOffset = { - KafkaSourceOffset(offsetTuples.map { case(t, p, o) => (new TopicPartition(t, p), o) }.toMap) - } - - /** - * Returns [[KafkaSourceOffset]] from a JSON [[SerializedOffset]] - */ - def apply(offset: SerializedOffset): KafkaSourceOffset = - KafkaSourceOffset(JsonUtils.partitionOffsets(offset.json)) -} diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala index 1de786b9..83da227b 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala @@ -21,7 +21,7 @@ import org.apache.kafka.clients.consumer.{ConsumerRecord, KafkaConsumer} import org.apache.kafka.common.TopicPartition import org.apache.logging.log4j.LogManager import org.apache.spark.sql.execution.streaming.{CommitLog, Offset, OffsetSeqLog} -import za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka.kafka010.KafkaSourceOffset +import org.apache.spark.sql.kafka010.KafkaSourceOffsetProxy import scala.collection.JavaConverters._ import scala.collection.mutable @@ -125,7 +125,7 @@ private[hyperdrive] object KafkaUtil { private def parseOffsetSeq(offsetSeq: Seq[Option[Offset]]) = { if (offsetSeq.size == 1) { if (offsetSeq.head.isDefined) { - Some(KafkaSourceOffset.getPartitionOffsets(offsetSeq.head.get)) + Some(KafkaSourceOffsetProxy.getPartitionOffsets(offsetSeq.head.get)) } else { None } diff --git a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtil.scala b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtil.scala index 71d14375..320f85ae 100644 --- a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtil.scala +++ b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtil.scala @@ -17,10 +17,10 @@ package za.co.absa.hyperdrive.ingestor.implementation.utils import org.apache.kafka.common.TopicPartition import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.kafka010.KafkaSourceOffsetProxy import org.scalatest.{BeforeAndAfter, FlatSpec, Matchers} import za.co.absa.commons.io.TempDirectory import za.co.absa.commons.spark.SparkTestBase -import za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka.kafka010.KafkaSourceOffset class TestKafkaUtil extends FlatSpec with Matchers with BeforeAndAfter with SparkTestBase { private var baseDir: TempDirectory = _ @@ -35,9 +35,9 @@ class TestKafkaUtil extends FlatSpec with Matchers with BeforeAndAfter with Spar "getLatestOffset" should "return the latest offsets" in { // given - val offset0 = OffsetSeq.fill(KafkaSourceOffset(("t", 0, 10L), ("t", 1, 110L))) - val offset1 = OffsetSeq.fill(KafkaSourceOffset(("t", 0, 25L), ("t", 1, 125L))) - val offset2 = OffsetSeq.fill(KafkaSourceOffset(("t", 0, 42L), ("t", 1, 142L))) + val offset0 = OffsetSeq.fill(KafkaSourceOffsetProxy(("t", 0, 10L), ("t", 1, 110L))) + val offset1 = OffsetSeq.fill(KafkaSourceOffsetProxy(("t", 0, 25L), ("t", 1, 125L))) + val offset2 = OffsetSeq.fill(KafkaSourceOffsetProxy(("t", 0, 42L), ("t", 1, 142L))) val offsetSeqLog = new OffsetSeqLog(spark, baseDir.path.toString) offsetSeqLog.add(0, offset0) @@ -77,7 +77,7 @@ class TestKafkaUtil extends FlatSpec with Matchers with BeforeAndAfter with Spar it should "throw an exception if the offsets contain multiple sources" in { // given - val offset = OffsetSeq.fill(KafkaSourceOffset(("t", 0, 10L)), KafkaSourceOffset(("t2", 1, 110L))) + val offset = OffsetSeq.fill(KafkaSourceOffsetProxy(("t", 0, 10L)), KafkaSourceOffsetProxy(("t2", 1, 110L))) val offsetSeqLog = new OffsetSeqLog(spark, baseDir.path.toString) offsetSeqLog.add(0, offset) @@ -94,9 +94,9 @@ class TestKafkaUtil extends FlatSpec with Matchers with BeforeAndAfter with Spar commitLog.add(0, CommitMetadata()) commitLog.add(1, CommitMetadata()) - val offset0 = OffsetSeq.fill(KafkaSourceOffset(("t", 0, 10L), ("t", 1, 110L))) - val offset1 = OffsetSeq.fill(KafkaSourceOffset(("t", 0, 25L), ("t", 1, 125L))) - val offset2 = OffsetSeq.fill(KafkaSourceOffset(("t", 0, 42L), ("t", 1, 142L))) + val offset0 = OffsetSeq.fill(KafkaSourceOffsetProxy(("t", 0, 10L), ("t", 1, 110L))) + val offset1 = OffsetSeq.fill(KafkaSourceOffsetProxy(("t", 0, 25L), ("t", 1, 125L))) + val offset2 = OffsetSeq.fill(KafkaSourceOffsetProxy(("t", 0, 42L), ("t", 1, 142L))) val offsetSeqLog = new OffsetSeqLog(spark, s"${baseDir.path.toString}/offsets") offsetSeqLog.add(0, offset0) @@ -116,9 +116,9 @@ class TestKafkaUtil extends FlatSpec with Matchers with BeforeAndAfter with Spar it should "return None if there is no commit" in { val commitLog = new CommitLog(spark, s"${baseDir.path.toString}/commits") - val offset0 = OffsetSeq.fill(KafkaSourceOffset(("t", 0, 10L), ("t", 1, 110L))) - val offset1 = OffsetSeq.fill(KafkaSourceOffset(("t", 0, 25L), ("t", 1, 125L))) - val offset2 = OffsetSeq.fill(KafkaSourceOffset(("t", 0, 42L), ("t", 1, 142L))) + val offset0 = OffsetSeq.fill(KafkaSourceOffsetProxy(("t", 0, 10L), ("t", 1, 110L))) + val offset1 = OffsetSeq.fill(KafkaSourceOffsetProxy(("t", 0, 25L), ("t", 1, 125L))) + val offset2 = OffsetSeq.fill(KafkaSourceOffsetProxy(("t", 0, 42L), ("t", 1, 142L))) val offsetSeqLog = new OffsetSeqLog(spark, s"${baseDir.path.toString}/offsets") offsetSeqLog.add(0, offset0) @@ -138,7 +138,7 @@ class TestKafkaUtil extends FlatSpec with Matchers with BeforeAndAfter with Spar commitLog.add(0, CommitMetadata()) commitLog.add(1, CommitMetadata()) - val offset0 = OffsetSeq.fill(KafkaSourceOffset(("t", 0, 10L), ("t", 1, 110L))) + val offset0 = OffsetSeq.fill(KafkaSourceOffsetProxy(("t", 0, 10L), ("t", 1, 110L))) val offsetSeqLog = new OffsetSeqLog(spark, s"${baseDir.path.toString}/offsets") offsetSeqLog.add(0, offset0) From f6c6f545bd5069e09d0a5d63df47a6f599d06d3b Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Fri, 4 Dec 2020 16:47:29 +0100 Subject: [PATCH 19/27] Get schema registry url from decoder and encoder config --- .../StreamTransformerFactory.scala | 6 +++ .../ingestor/api/utils/ConfigUtils.scala | 16 ++++++- .../api/utils/DummyStreamTransformer.scala | 23 ++++++++++ .../ingestor/api/utils/TestConfigUtils.scala | 18 ++++++++ ...fkaDeduplicationAfterRetryDockerTest.scala | 1 - .../DeduplicateKafkaSinkTransformer.scala | 31 +++++++++++-- ...licateKafkaSinkTransformerAttributes.scala | 1 - .../StreamTransformerAbstractFactory.scala | 18 ++++---- ...TestStreamTransformerAbstractFactory.scala | 44 +++++++++---------- 9 files changed, 118 insertions(+), 40 deletions(-) create mode 100644 api/src/test/scala/za/co/absa/hyperdrive/ingestor/api/utils/DummyStreamTransformer.scala diff --git a/api/src/main/scala/za/co/absa/hyperdrive/ingestor/api/transformer/StreamTransformerFactory.scala b/api/src/main/scala/za/co/absa/hyperdrive/ingestor/api/transformer/StreamTransformerFactory.scala index 8b512cf7..6c049cc2 100644 --- a/api/src/main/scala/za/co/absa/hyperdrive/ingestor/api/transformer/StreamTransformerFactory.scala +++ b/api/src/main/scala/za/co/absa/hyperdrive/ingestor/api/transformer/StreamTransformerFactory.scala @@ -31,3 +31,9 @@ trait StreamTransformerFactory extends ComponentFactory[StreamTransformer] { */ def getMappingFromRetainedGlobalConfigToLocalConfig(globalConfig: Configuration): Map[String, String] = Map() } + +object StreamTransformerFactory { + val IdsKeyPrefix = "component.transformer.id" + val ClassKeyPrefix = "component.transformer.class" + val TransformerKeyPrefix = "transformer" +} diff --git a/api/src/main/scala/za/co/absa/hyperdrive/ingestor/api/utils/ConfigUtils.scala b/api/src/main/scala/za/co/absa/hyperdrive/ingestor/api/utils/ConfigUtils.scala index e9e1e418..ad425dd5 100644 --- a/api/src/main/scala/za/co/absa/hyperdrive/ingestor/api/utils/ConfigUtils.scala +++ b/api/src/main/scala/za/co/absa/hyperdrive/ingestor/api/utils/ConfigUtils.scala @@ -15,8 +15,10 @@ package za.co.absa.hyperdrive.ingestor.api.utils -import org.apache.commons.configuration2.Configuration +import org.apache.commons.configuration2.{Configuration, ConfigurationConverter} +import za.co.absa.hyperdrive.ingestor.api.transformer.{StreamTransformer, StreamTransformerFactory} +import scala.collection.mutable import scala.util.{Failure, Success, Try} object ConfigUtils { @@ -98,4 +100,16 @@ object ConfigUtils { Success(target) } } + + def getTransformerPrefix[T <: StreamTransformer](config: Configuration, transformerClass: Class[T]): Option[String] = { + import scala.collection.JavaConverters._ + val className = transformerClass.getCanonicalName + val transformerPrefixConfig = config.subset(StreamTransformerFactory.ClassKeyPrefix) + val transformerPrefixMap: mutable.Map[AnyRef, AnyRef] = ConfigurationConverter.getMap(transformerPrefixConfig).asScala + transformerPrefixMap.find { + case (_: String, value: String) => value == className + }.map { + case (key: String, _) => key + } + } } diff --git a/api/src/test/scala/za/co/absa/hyperdrive/ingestor/api/utils/DummyStreamTransformer.scala b/api/src/test/scala/za/co/absa/hyperdrive/ingestor/api/utils/DummyStreamTransformer.scala new file mode 100644 index 00000000..979d2dbc --- /dev/null +++ b/api/src/test/scala/za/co/absa/hyperdrive/ingestor/api/utils/DummyStreamTransformer.scala @@ -0,0 +1,23 @@ +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed 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 za.co.absa.hyperdrive.ingestor.api.utils + +import org.apache.spark.sql.DataFrame +import za.co.absa.hyperdrive.ingestor.api.transformer.StreamTransformer + +class DummyStreamTransformer extends StreamTransformer { + override def transform(streamData: DataFrame): DataFrame = ??? +} diff --git a/api/src/test/scala/za/co/absa/hyperdrive/ingestor/api/utils/TestConfigUtils.scala b/api/src/test/scala/za/co/absa/hyperdrive/ingestor/api/utils/TestConfigUtils.scala index 86287393..8bf44dd2 100644 --- a/api/src/test/scala/za/co/absa/hyperdrive/ingestor/api/utils/TestConfigUtils.scala +++ b/api/src/test/scala/za/co/absa/hyperdrive/ingestor/api/utils/TestConfigUtils.scala @@ -19,6 +19,7 @@ import org.apache.commons.configuration2.convert.DefaultListDelimiterHandler import org.apache.commons.configuration2.{BaseConfiguration, Configuration} import org.scalatest.mockito.MockitoSugar import org.scalatest.{FlatSpec, Matchers} +import za.co.absa.hyperdrive.ingestor.api.transformer.StreamTransformerFactory class TestConfigUtils extends FlatSpec with Matchers with MockitoSugar { @@ -322,4 +323,21 @@ class TestConfigUtils extends FlatSpec with Matchers with MockitoSugar { val ex4 = the[Exception] thrownBy ConfigUtils.getOptionalBoolean("key4", config) ex4.getMessage should include("key4") } + + "getTransformerPrefix" should "get the prefix of a transformer class" in { + val config = new BaseConfiguration + config.addProperty(s"${StreamTransformerFactory.ClassKeyPrefix}.[dummy-transformer]", classOf[DummyStreamTransformer].getCanonicalName) + + val prefix = ConfigUtils.getTransformerPrefix(config, classOf[DummyStreamTransformer]) + + prefix shouldBe Some("[dummy-transformer]") + } + + it should "return None if the transformer class is not registered in the config" in { + val config = new BaseConfiguration + + val prefix = ConfigUtils.getTransformerPrefix(config, classOf[DummyStreamTransformer]) + + prefix shouldBe None + } } diff --git a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala index 8fdeb51d..dfa0d131 100644 --- a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala +++ b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala @@ -167,7 +167,6 @@ class KafkaToKafkaDeduplicationAfterRetryDockerTest extends FlatSpec with Matche // comma separated list of columns to select "transformer.[kafka.deduplicator].source.id.columns" -> "offset,partition", "transformer.[kafka.deduplicator].destination.id.columns" -> "value.hyperdrive_id.source_offset, value.hyperdrive_id.source_partition", - "transformer.[kafka.deduplicator].schema.registry.url" -> "${transformer.[avro.decoder].schema.registry.url}", // Avro Encoder (ABRiS) settings "transformer.[avro.encoder].schema.registry.url" -> "${transformer.[avro.decoder].schema.registry.url}", diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala index adaeabb1..a4ddef09 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala @@ -18,6 +18,8 @@ package za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.ka import java.time.Duration import java.util.{Properties, UUID} +import za.co.absa.hyperdrive.ingestor.api.utils.ConfigUtils +import za.co.absa.hyperdrive.ingestor.implementation.transformer.avro.confluent.{ConfluentAvroDecodingTransformer, ConfluentAvroEncodingTransformer} import za.co.absa.hyperdrive.ingestor.implementation.utils.KafkaUtil //import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig @@ -129,14 +131,17 @@ private[transformer] class DeduplicateKafkaSinkTransformer( object DeduplicateKafkaSinkTransformer extends StreamTransformerFactory with DeduplicateKafkaSinkTransformerAttributes { private val DefaultKafkaConsumerTimeoutSeconds = 120L + + private val readerSchemaRegistryUrlKey = s"readerSchemaRegistryUrl${UUID.randomUUID().toString}" + private val writerSchemaRegistryUrlKey = s"writerSchemaRegistryUrl${UUID.randomUUID().toString}" + override def apply(config: Configuration): StreamTransformer = { -// TODO: How to get schemaRegistryUrl from transformer config? - val readerSchemaRegistryUrl = getOrThrow(schemaRegistryUrl, config) + val readerSchemaRegistryUrl = getOrThrow(readerSchemaRegistryUrlKey, config) val readerTopic = getOrThrow(KafkaStreamReader.KEY_TOPIC, config) val readerBrokers = getOrThrow(KafkaStreamReader.KEY_BROKERS, config) val readerExtraOptions = KafkaStreamReader.getExtraConfigurationPrefix.map(getPropertySubset(config, _)).getOrElse(Map()) - val writerSchemaRegistryUrl = getOrThrow(schemaRegistryUrl, config) + val writerSchemaRegistryUrl = getOrThrow(writerSchemaRegistryUrlKey, config) val writerTopic = getOrThrow(KafkaStreamWriter.KEY_TOPIC, config) val writerBrokers = getOrThrow(KafkaStreamWriter.KEY_BROKERS, config) val writerExtraOptions = KafkaStreamWriter.getExtraConfigurationPrefix.map(getPropertySubset(config, _)).getOrElse(Map()) @@ -171,8 +176,26 @@ object DeduplicateKafkaSinkTransformer extends StreamTransformerFactory with Ded KafkaStreamWriter.KEY_BROKERS, StreamWriterCommonAttributes.keyCheckpointBaseLocation ) - keys.map(e => e -> e).toMap + val oneToOneMappings = keys.map(e => e -> e).toMap + + val readerSchemaRegistryUrlGlobalKey = getSchemaRegistryUrlKey(globalConfig, classOf[ConfluentAvroDecodingTransformer], + ConfluentAvroDecodingTransformer.KEY_SCHEMA_REGISTRY_URL) + val writerSchemaRegistryUrlGlobalKey = getSchemaRegistryUrlKey(globalConfig, classOf[ConfluentAvroEncodingTransformer], + ConfluentAvroEncodingTransformer.KEY_SCHEMA_REGISTRY_URL) + + oneToOneMappings ++ Map( + readerSchemaRegistryUrlGlobalKey -> readerSchemaRegistryUrlKey, + writerSchemaRegistryUrlGlobalKey -> writerSchemaRegistryUrlKey + ) } + + private def getSchemaRegistryUrlKey[T <: StreamTransformer](config: Configuration, transformerClass: Class[T], transformerKey: String) = { + val prefix = ConfigUtils.getTransformerPrefix(config, transformerClass).getOrElse(throw new IllegalArgumentException( + s"Could not find transformer configuration for ${transformerClass.getCanonicalName}, but it is required")) + + s"${StreamTransformerFactory.TransformerKeyPrefix}.${prefix}.${transformerKey}" + } + } diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformerAttributes.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformerAttributes.scala index cd530d6a..b655e4a3 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformerAttributes.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformerAttributes.scala @@ -21,7 +21,6 @@ trait DeduplicateKafkaSinkTransformerAttributes extends HasComponentAttributes { val SourceIdColumns = "source.id.columns" val DestinationIdColumns = "destination.id.columns" - val schemaRegistryUrl = "schema.registry.url" val KafkaConsumerTimeout = "kafka.consumer.timeout" override def getName: String = "Deduplicate Kafka Sink Transformer" diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/factories/StreamTransformerAbstractFactory.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/factories/StreamTransformerAbstractFactory.scala index 022d39e0..9135d79b 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/factories/StreamTransformerAbstractFactory.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/factories/StreamTransformerAbstractFactory.scala @@ -21,6 +21,7 @@ import org.apache.logging.log4j.LogManager import scala.collection.JavaConverters._ import scala.util.{Failure, Success, Try} import za.co.absa.hyperdrive.ingestor.api.transformer.{StreamTransformer, StreamTransformerFactory} +import za.co.absa.hyperdrive.ingestor.api.transformer.StreamTransformerFactory._ import za.co.absa.hyperdrive.ingestor.api.utils.ConfigUtils import za.co.absa.hyperdrive.shared.utils.ClassLoaderUtils @@ -33,9 +34,6 @@ import za.co.absa.hyperdrive.shared.utils.ClassLoaderUtils object StreamTransformerAbstractFactory { private val logger = LogManager.getLogger - val idsKeyPrefix = "component.transformer.id" - val classKeyPrefix = "component.transformer.class" - val transformerKeyPrefix = "transformer" /** * For each transformer, the configuration is assumed to contain property keys according to the following example @@ -52,16 +50,16 @@ object StreamTransformerAbstractFactory { validateConfiguration(config) - val orderedTransformerIds = config.getKeys(idsKeyPrefix).asScala.toList - .map(key => key.replace(s"$idsKeyPrefix.", "").toInt -> config.getString(key)) + val orderedTransformerIds = config.getKeys(IdsKeyPrefix).asScala.toList + .map(key => key.replace(s"$IdsKeyPrefix.", "").toInt -> config.getString(key)) .sortBy { case (order, _) => order } .map { case (_, id) => id } - val transformerClassNames = orderedTransformerIds.map(id => id -> config.getString(s"$classKeyPrefix.$id")) + val transformerClassNames = orderedTransformerIds.map(id => id -> config.getString(s"$ClassKeyPrefix.$id")) transformerClassNames .map { case (id, className) => id -> ClassLoaderUtils.loadSingletonClassOfType[StreamTransformerFactory](className) } - .map { case (id, factory) => factory -> ConfigUtils.copyAndMapConfig(config, config.subset(s"$transformerKeyPrefix.$id"), factory.getMappingFromRetainedGlobalConfigToLocalConfig(config)) } + .map { case (id, factory) => factory -> ConfigUtils.copyAndMapConfig(config, config.subset(s"$TransformerKeyPrefix.$id"), factory.getMappingFromRetainedGlobalConfigToLocalConfig(config)) } .map { case (factory, configTry) => configTry match { case Failure(exception) => throw exception case Success(value) => factory -> value @@ -71,10 +69,10 @@ object StreamTransformerAbstractFactory { } private def validateConfiguration(config: Configuration): Unit = { - val keys = config.getKeys(idsKeyPrefix).asScala.toList + val keys = config.getKeys(IdsKeyPrefix).asScala.toList val invalidTransformerKeys = keys - .map(key => key -> key.replace(s"$idsKeyPrefix.", "")) + .map(key => key -> key.replace(s"$IdsKeyPrefix.", "")) .map { case (key, order) => key -> Try(order.toInt) } .filter { case (_, orderAsInt) => orderAsInt.isFailure } .map { case (key, _) => key } @@ -88,7 +86,7 @@ object StreamTransformerAbstractFactory { } val missingClassKeys = transformerIds - .map(id => s"$classKeyPrefix.$id") + .map(id => s"$ClassKeyPrefix.$id") .filter(classKey => !config.containsKey(classKey)) if (missingClassKeys.nonEmpty) { diff --git a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/factories/TestStreamTransformerAbstractFactory.scala b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/factories/TestStreamTransformerAbstractFactory.scala index 96b28201..10fd192b 100644 --- a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/factories/TestStreamTransformerAbstractFactory.scala +++ b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/factories/TestStreamTransformerAbstractFactory.scala @@ -20,25 +20,23 @@ import org.scalatest.mockito.MockitoSugar import org.scalatest.{BeforeAndAfterEach, FlatSpec, Matchers} import za.co.absa.hyperdrive.ingestor.implementation.transformer.column.selection.ColumnSelectorStreamTransformer import za.co.absa.hyperdrive.ingestor.implementation.transformer.factories.DummyStreamTransformer._ -import za.co.absa.hyperdrive.ingestor.implementation.transformer.factories.StreamTransformerAbstractFactory.{classKeyPrefix, idsKeyPrefix} - +import za.co.absa.hyperdrive.ingestor.api.transformer.StreamTransformerFactory._ class TestStreamTransformerAbstractFactory extends FlatSpec with BeforeAndAfterEach with MockitoSugar with Matchers { behavior of StreamTransformerAbstractFactory.getClass.getSimpleName it should "create transformer instances in the correct order" in { - import StreamTransformerAbstractFactory._ val config = getBaseConfiguration - config.addProperty(s"${idsKeyPrefix}.1", "dummy.transformer.A") - config.addProperty(s"${classKeyPrefix}.dummy.transformer.A", DummyStreamTransformer.getClass.getName) - config.addProperty(s"${transformerKeyPrefix}.dummy.transformer.A.$DummyProperty1Name", "value1") - config.addProperty(s"${transformerKeyPrefix}.dummy.transformer.A.$DummyProperty2Name", "100") + config.addProperty(s"${IdsKeyPrefix}.1", "dummy.transformer.A") + config.addProperty(s"${ClassKeyPrefix}.dummy.transformer.A", DummyStreamTransformer.getClass.getName) + config.addProperty(s"${TransformerKeyPrefix}.dummy.transformer.A.$DummyProperty1Name", "value1") + config.addProperty(s"${TransformerKeyPrefix}.dummy.transformer.A.$DummyProperty2Name", "100") - config.addProperty(s"${idsKeyPrefix}.2", "dummy.transformer.B") - config.addProperty(s"${classKeyPrefix}.dummy.transformer.B", DummyStreamTransformer.getClass.getName) - config.addProperty(s"${transformerKeyPrefix}.dummy.transformer.B.$DummyProperty1Name", "value2") - config.addProperty(s"${transformerKeyPrefix}.dummy.transformer.B.$DummyProperty2Name", "200") + config.addProperty(s"${IdsKeyPrefix}.2", "dummy.transformer.B") + config.addProperty(s"${ClassKeyPrefix}.dummy.transformer.B", DummyStreamTransformer.getClass.getName) + config.addProperty(s"${TransformerKeyPrefix}.dummy.transformer.B.$DummyProperty1Name", "value2") + config.addProperty(s"${TransformerKeyPrefix}.dummy.transformer.B.$DummyProperty2Name", "200") val transformers = StreamTransformerAbstractFactory.build(config) transformers should have size 2 @@ -59,11 +57,11 @@ class TestStreamTransformerAbstractFactory extends FlatSpec with BeforeAndAfterE it should "support negative orders" in { import StreamTransformerAbstractFactory._ val config = getBaseConfiguration - config.addProperty(s"${idsKeyPrefix}.2", "[column.transformer]") - config.addProperty(s"${classKeyPrefix}.[column.transformer]", ColumnSelectorStreamTransformer.getClass.getName) + config.addProperty(s"${IdsKeyPrefix}.2", "[column.transformer]") + config.addProperty(s"${ClassKeyPrefix}.[column.transformer]", ColumnSelectorStreamTransformer.getClass.getName) - config.addProperty(s"${idsKeyPrefix}.-1", "dummy.transformer.A") - config.addProperty(s"${classKeyPrefix}.dummy.transformer.A", DummyStreamTransformer.getClass.getName) + config.addProperty(s"${IdsKeyPrefix}.-1", "dummy.transformer.A") + config.addProperty(s"${ClassKeyPrefix}.dummy.transformer.A", DummyStreamTransformer.getClass.getName) val transformers = StreamTransformerAbstractFactory.build(config) transformers should have size 2 @@ -79,8 +77,8 @@ class TestStreamTransformerAbstractFactory extends FlatSpec with BeforeAndAfterE it should "throw if transformer ids are not unique" in { val config = getBaseConfiguration - config.addProperty(s"${idsKeyPrefix}.1", "dummy.transformer.A") - config.addProperty(s"${idsKeyPrefix}.2", "dummy.transformer.A") + config.addProperty(s"${IdsKeyPrefix}.1", "dummy.transformer.A") + config.addProperty(s"${IdsKeyPrefix}.2", "dummy.transformer.A") val throwable = intercept[IllegalArgumentException](StreamTransformerAbstractFactory.build(config)) throwable.getMessage should include(s"dummy.transformer.A") @@ -88,25 +86,25 @@ class TestStreamTransformerAbstractFactory extends FlatSpec with BeforeAndAfterE it should "throw if transformer id is non-numeric" in { val config = getBaseConfiguration - config.addProperty(s"${idsKeyPrefix}.First", "dummy.transformer.A") + config.addProperty(s"${IdsKeyPrefix}.First", "dummy.transformer.A") val throwable = intercept[IllegalArgumentException](StreamTransformerAbstractFactory.build(config)) - throwable.getMessage should include(s"${idsKeyPrefix}.First") + throwable.getMessage should include(s"${IdsKeyPrefix}.First") } it should "throw if no class name is associated to the transformer id" in { val config = getBaseConfiguration - config.addProperty(s"${idsKeyPrefix}.1", "dummy.transformer.A") + config.addProperty(s"${IdsKeyPrefix}.1", "dummy.transformer.A") val throwable = intercept[IllegalArgumentException](StreamTransformerAbstractFactory.build(config)) - throwable.getMessage should include(s"${classKeyPrefix}.dummy.transformer.A") + throwable.getMessage should include(s"${ClassKeyPrefix}.dummy.transformer.A") } it should "throw if data transformer parameter is invalid" in { val invalidFactoryName = "an-invalid-factory-name" val config = getBaseConfiguration - config.addProperty(s"${idsKeyPrefix}.1", "dummy.transformer.A") - config.addProperty(s"${classKeyPrefix}.dummy.transformer.A", invalidFactoryName) + config.addProperty(s"${IdsKeyPrefix}.1", "dummy.transformer.A") + config.addProperty(s"${ClassKeyPrefix}.dummy.transformer.A", invalidFactoryName) val throwable = intercept[IllegalArgumentException](StreamTransformerAbstractFactory.build(config)) assert(throwable.getMessage.contains(invalidFactoryName)) From cc373d3bf77db7fe02f97f8b0a3362b1bd7d494c Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Fri, 4 Dec 2020 19:25:36 +0100 Subject: [PATCH 20/27] Add test for transformer object --- .../ingestor/api/utils/ConfigUtils.scala | 2 +- .../DeduplicateKafkaSinkTransformer.scala | 17 +-- ...eduplicateKafkaSinkTransformerObject.scala | 141 ++++++++++++++++++ 3 files changed, 150 insertions(+), 10 deletions(-) create mode 100644 ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/TestDeduplicateKafkaSinkTransformerObject.scala diff --git a/api/src/main/scala/za/co/absa/hyperdrive/ingestor/api/utils/ConfigUtils.scala b/api/src/main/scala/za/co/absa/hyperdrive/ingestor/api/utils/ConfigUtils.scala index ad425dd5..2353a359 100644 --- a/api/src/main/scala/za/co/absa/hyperdrive/ingestor/api/utils/ConfigUtils.scala +++ b/api/src/main/scala/za/co/absa/hyperdrive/ingestor/api/utils/ConfigUtils.scala @@ -105,7 +105,7 @@ object ConfigUtils { import scala.collection.JavaConverters._ val className = transformerClass.getCanonicalName val transformerPrefixConfig = config.subset(StreamTransformerFactory.ClassKeyPrefix) - val transformerPrefixMap: mutable.Map[AnyRef, AnyRef] = ConfigurationConverter.getMap(transformerPrefixConfig).asScala + val transformerPrefixMap = ConfigurationConverter.getMap(transformerPrefixConfig).asScala transformerPrefixMap.find { case (_: String, value: String) => value == className }.map { diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala index a4ddef09..6b198d4b 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala @@ -22,7 +22,6 @@ import za.co.absa.hyperdrive.ingestor.api.utils.ConfigUtils import za.co.absa.hyperdrive.ingestor.implementation.transformer.avro.confluent.{ConfluentAvroDecodingTransformer, ConfluentAvroEncodingTransformer} import za.co.absa.hyperdrive.ingestor.implementation.utils.KafkaUtil -//import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig import org.apache.avro.generic.GenericRecord import org.apache.commons.configuration2.Configuration import org.apache.hadoop.fs.Path @@ -41,14 +40,14 @@ import za.co.absa.hyperdrive.ingestor.implementation.writer.kafka.KafkaStreamWri private[transformer] class DeduplicateKafkaSinkTransformer( - val readerSchemaRegistryUrl: String, val readerTopic: String, val readerBrokers: String, val readerExtraOptions: Map[String, String], - val writerSchemaRegistryUrl: String, + val readerSchemaRegistryUrl: String, val writerTopic: String, val writerBrokers: String, val writerExtraOptions: Map[String, String], + val writerSchemaRegistryUrl: String, val checkpointLocation: String, val sourceIdColumnNames: Seq[String], val destinationIdColumnNames: Seq[String], @@ -132,19 +131,19 @@ object DeduplicateKafkaSinkTransformer extends StreamTransformerFactory with Ded private val DefaultKafkaConsumerTimeoutSeconds = 120L - private val readerSchemaRegistryUrlKey = s"readerSchemaRegistryUrl${UUID.randomUUID().toString}" - private val writerSchemaRegistryUrlKey = s"writerSchemaRegistryUrl${UUID.randomUUID().toString}" + private val readerSchemaRegistryUrlKey = "deduplicateKafkaSinkTransformer.readerSchemaRegistryUrl" + private val writerSchemaRegistryUrlKey = "deduplicateKafkaSinkTransformer.writerSchemaRegistryUrl" override def apply(config: Configuration): StreamTransformer = { - val readerSchemaRegistryUrl = getOrThrow(readerSchemaRegistryUrlKey, config) val readerTopic = getOrThrow(KafkaStreamReader.KEY_TOPIC, config) val readerBrokers = getOrThrow(KafkaStreamReader.KEY_BROKERS, config) val readerExtraOptions = KafkaStreamReader.getExtraConfigurationPrefix.map(getPropertySubset(config, _)).getOrElse(Map()) + val readerSchemaRegistryUrl = getOrThrow(readerSchemaRegistryUrlKey, config) - val writerSchemaRegistryUrl = getOrThrow(writerSchemaRegistryUrlKey, config) val writerTopic = getOrThrow(KafkaStreamWriter.KEY_TOPIC, config) val writerBrokers = getOrThrow(KafkaStreamWriter.KEY_BROKERS, config) val writerExtraOptions = KafkaStreamWriter.getExtraConfigurationPrefix.map(getPropertySubset(config, _)).getOrElse(Map()) + val writerSchemaRegistryUrl = getOrThrow(writerSchemaRegistryUrlKey, config) val checkpointLocation = StreamWriterUtil.getCheckpointLocation(config) @@ -157,8 +156,8 @@ object DeduplicateKafkaSinkTransformer extends StreamTransformerFactory with Ded val kafkaConsumerTimeout = Duration.ofSeconds(config.getLong(KafkaConsumerTimeout, DefaultKafkaConsumerTimeoutSeconds)) - new DeduplicateKafkaSinkTransformer(readerSchemaRegistryUrl, readerTopic, readerBrokers, readerExtraOptions, - writerSchemaRegistryUrl, writerTopic, writerBrokers, writerExtraOptions, + new DeduplicateKafkaSinkTransformer(readerTopic, readerBrokers, readerExtraOptions, readerSchemaRegistryUrl, + writerTopic, writerBrokers, writerExtraOptions, writerSchemaRegistryUrl, checkpointLocation, sourceIdColumns, destinationIdColumns, kafkaConsumerTimeout) } diff --git a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/TestDeduplicateKafkaSinkTransformerObject.scala b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/TestDeduplicateKafkaSinkTransformerObject.scala new file mode 100644 index 00000000..f921fb3b --- /dev/null +++ b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/TestDeduplicateKafkaSinkTransformerObject.scala @@ -0,0 +1,141 @@ +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed 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 za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka + +import java.time.Duration + +import org.apache.commons.configuration2.DynamicCombinedConfiguration +import org.apache.commons.configuration2.convert.DefaultListDelimiterHandler +import org.scalatest.{FlatSpec, Matchers} +import za.co.absa.hyperdrive.ingestor.api.writer.StreamWriterCommonAttributes +import za.co.absa.hyperdrive.ingestor.implementation.reader.kafka.KafkaStreamReader +import za.co.absa.hyperdrive.ingestor.implementation.transformer.avro.confluent.{ConfluentAvroDecodingTransformer, ConfluentAvroEncodingTransformer} +import za.co.absa.hyperdrive.ingestor.implementation.writer.kafka.KafkaStreamWriter + +class TestDeduplicateKafkaSinkTransformerObject extends FlatSpec with Matchers { + behavior of DeduplicateKafkaSinkTransformer.getClass.getSimpleName + + "apply" should "create a DeduplicateKafkaSinkTransformer" in { + // given + val config = getLocalConfig() + + // when + val transformer = DeduplicateKafkaSinkTransformer(config).asInstanceOf[DeduplicateKafkaSinkTransformer] + + // then + transformer.readerTopic shouldBe "readerTopic" + transformer.readerBrokers shouldBe "http://readerBrokers:9092" + transformer.readerExtraOptions should contain theSameElementsAs Map( + "kafka.security.protocol" -> "SASL_PLAINTEXT", + "failOnDataLoss" -> "false" + ) + transformer.readerSchemaRegistryUrl shouldBe "http://sourceRegistry:8081" + transformer.writerTopic shouldBe "writerTopic" + transformer.writerBrokers shouldBe "http://writerBrokers:9092" + transformer.writerExtraOptions shouldBe Map( + "kafka.sasl.mechanism" -> "GSSAPI" + ) + transformer.writerSchemaRegistryUrl shouldBe "http://writerRegistry:8081" + + transformer.checkpointLocation shouldBe "/tmp/checkpoint" + transformer.sourceIdColumnNames should contain theSameElementsInOrderAs Seq("offset", "partition") + transformer.destinationIdColumnNames should contain theSameElementsInOrderAs Seq("value.hyperdrive_id.source_offset", "value.hyperdrive_id.source_partition") + transformer.kafkaConsumerTimeout shouldBe Duration.ofSeconds(5L) + } + + it should "throw an exception if source id columns and destination id columns have different size" in { + val config = getLocalConfig() + config.setProperty(DeduplicateKafkaSinkTransformer.DestinationIdColumns, "value.hyperdrive_id") + + val ex = the[IllegalArgumentException] thrownBy DeduplicateKafkaSinkTransformer(config) + + ex.getMessage should include ("The size of source id column names doesn't match") + } + + it should "use the default value for kafka consumer timeout if not provided" in { + // given + val config = getLocalConfig() + config.clearProperty(DeduplicateKafkaSinkTransformer.KafkaConsumerTimeout) + + // when + val transformer = DeduplicateKafkaSinkTransformer(config).asInstanceOf[DeduplicateKafkaSinkTransformer] + + // then + transformer.kafkaConsumerTimeout shouldBe Duration.ofSeconds(120L) + } + + "getMappingFromRetainedGlobalConfigToLocalConfig" should "return the local config mapping" in { + // given + val config = getEmptyConfiguration + config.addProperty("reader.option.kafka.option1", "value1") + config.addProperty("reader.option.kafka.option2", "value2") + config.addProperty("component.transformer.id.0", "decoder") + config.addProperty("component.transformer.class.decoder", classOf[ConfluentAvroDecodingTransformer].getCanonicalName) + config.addProperty(s"transformer.decoder.${ConfluentAvroDecodingTransformer.KEY_SCHEMA_REGISTRY_URL}", "http://sourceRegistry:8081") + + config.addProperty("component.transformer.id.1", "encoder") + config.addProperty("component.transformer.class.encoder", classOf[ConfluentAvroEncodingTransformer].getCanonicalName) + config.addProperty(s"transformer.encoder.${ConfluentAvroEncodingTransformer.KEY_SCHEMA_REGISTRY_URL}", "http://writerRegistry:8081") + + config.addProperty("writer.kafka.option.option3", "value3") + // when + val mapping = DeduplicateKafkaSinkTransformer.getMappingFromRetainedGlobalConfigToLocalConfig(config) + + // then + mapping should contain theSameElementsAs Map( + "reader.option.kafka.option1" -> "reader.option.kafka.option1", + "reader.option.kafka.option2" -> "reader.option.kafka.option2", + "writer.kafka.option.option3" -> "writer.kafka.option.option3", + s"transformer.decoder.${ConfluentAvroDecodingTransformer.KEY_SCHEMA_REGISTRY_URL}" -> "deduplicateKafkaSinkTransformer.readerSchemaRegistryUrl", + s"transformer.encoder.${ConfluentAvroEncodingTransformer.KEY_SCHEMA_REGISTRY_URL}" -> "deduplicateKafkaSinkTransformer.writerSchemaRegistryUrl", + KafkaStreamReader.KEY_TOPIC -> KafkaStreamReader.KEY_TOPIC, + KafkaStreamReader.KEY_BROKERS -> KafkaStreamReader.KEY_BROKERS, + KafkaStreamWriter.KEY_TOPIC -> KafkaStreamWriter.KEY_TOPIC, + KafkaStreamWriter.KEY_BROKERS -> KafkaStreamWriter.KEY_BROKERS, + StreamWriterCommonAttributes.keyCheckpointBaseLocation -> StreamWriterCommonAttributes.keyCheckpointBaseLocation + ) + } + + private def getLocalConfig() = { + val config = getEmptyConfiguration + config.addProperty(KafkaStreamReader.KEY_TOPIC, "readerTopic") + config.addProperty(KafkaStreamReader.KEY_BROKERS, "http://readerBrokers:9092") + config.addProperty("reader.option.kafka.security.protocol", "SASL_PLAINTEXT") + config.addProperty("reader.option.failOnDataLoss", false) + config.addProperty("deduplicateKafkaSinkTransformer.readerSchemaRegistryUrl", "http://sourceRegistry:8081") + + config.addProperty(KafkaStreamWriter.KEY_TOPIC, "writerTopic") + config.addProperty(KafkaStreamWriter.KEY_BROKERS, "http://writerBrokers:9092") + config.addProperty("writer.kafka.option.kafka.sasl.mechanism", "GSSAPI") + config.addProperty("component.transformer.class.encoder", classOf[ConfluentAvroEncodingTransformer].getCanonicalName) + config.addProperty("deduplicateKafkaSinkTransformer.writerSchemaRegistryUrl", "http://writerRegistry:8081") + + config.addProperty(StreamWriterCommonAttributes.keyCheckpointBaseLocation, "/tmp/checkpoint") + + config.addProperty(DeduplicateKafkaSinkTransformer.SourceIdColumns, "offset, partition") + config.addProperty(DeduplicateKafkaSinkTransformer.DestinationIdColumns, "value.hyperdrive_id.source_offset, value.hyperdrive_id.source_partition") + config.addProperty(DeduplicateKafkaSinkTransformer.KafkaConsumerTimeout, 5) + + config + } + + private def getEmptyConfiguration = { + val config = new DynamicCombinedConfiguration() + config.setListDelimiterHandler(new DefaultListDelimiterHandler(',')) + config + } + +} From 1ae61710009414fe200a91e0f2f7d2852f3cda0a Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Mon, 7 Dec 2020 10:27:53 +0100 Subject: [PATCH 21/27] Add some tests, docs --- README.md | 42 ++++++++++++++++++ ...licateKafkaSinkTransformerAttributes.scala | 3 +- .../implementation/utils/AvroUtil.scala | 1 - ...eduplicateKafkaSinkTransformerObject.scala | 44 ++++++++++++++++++- 4 files changed, 86 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index c3a8b3f6..39431484 100644 --- a/README.md +++ b/README.md @@ -230,6 +230,48 @@ will produce the following schema ``` + +##### DeduplicateKafkaSinkTransformer +`DeduplicateKafkaSinkTransformer` deduplicates records in a query from a Kafka source to a Kafka destination in a rerun after a failure. +Records are identified across source and destination topic by a user-defined id, which may be a composite id and may include consumer record +properties such as offset, partition, but also fields from the key or value schema. +Deduplication is needed because the Kafka-destination provides only a at-least-once guarantee. Deduplication works by getting the ids +from the last partial run in the destination topic and excluding them in the query. + +Note that there must be only one source and one destination topic, and there must be only one writer writing to the destination topic, and +no records must have been written to the destination topic after the partial run. Otherwise, records may still be duplicated. + +To use this transformer, `KafkaStreamReader`, `ConfluentAvroDecodingTransformer`, `ConfluentAvroEncodingTransformer` and `KafkaStreamWriter` +must be configured as well. + +Note that usage of the star-operator `*` within column names is not supported and may lead to unexpected behaviour. + +To add the transformer to the pipeline use this class name: +``` +component.transformer.class.{transformer-id} = za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka.DeduplicateKafkaSinkTransformer +``` + +| Property Name | Required | Description | +| :--- | :---: | :--- | +| `transformer.{transformer-id}.source.id.columns` | Yes | A comma-separated list of consumer record properties that define the composite id. For example, `offset, partition` or `key.some_user_id`. | +| `transformer.{transformer-id}.destination.id.columns` | Yes | A comma-separated list of consumer record properties that define the composite id. For example, `value.src_offset, value.src_partition` or `key.some_user_id`. | +| `transformer.{transformer-id}.kafka.consumer.timeout` | No | Kafka consumer timeout in seconds. The default value is 120s. | + +The following fields can be selected on the consumer record + +- `topic` +- `offset` +- `partition` +- `timestamp` +- `timestampType` +- `serializedKeySize` +- `serializedValueSize` +- `key` +- `value` + +In case of `key` and `value`, the fields of their schemas can be specified by adding a dot, e.g. +`key.some_nested_record.some_id` or likewise `value.some_nested_record.some_id` + See [Pipeline settings](#pipeline-settings) for details about `{transformer-id}`. ##### ParquetStreamWriter | Property Name | Required | Description | diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformerAttributes.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformerAttributes.scala index b655e4a3..efb05361 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformerAttributes.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformerAttributes.scala @@ -25,7 +25,8 @@ trait DeduplicateKafkaSinkTransformerAttributes extends HasComponentAttributes { override def getName: String = "Deduplicate Kafka Sink Transformer" - override def getDescription: String = "" + override def getDescription: String = "This transformer deduplicates records in a Kafka-to-Kafka query in a rerun after a failure." + + "It is assumed that only one query writes to the sink and no records have been written to the sink since the failure." override def getProperties: Map[String, PropertyMetadata] = Map( SourceIdColumns -> PropertyMetadata("Source Id columns", Some("Comma separated list of columns that represent the id"), required = true), diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/AvroUtil.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/AvroUtil.scala index 23d4934e..e7453b17 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/AvroUtil.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/AvroUtil.scala @@ -33,7 +33,6 @@ private[hyperdrive] object AvroUtil { case "timestampType" => Option(record.timestampType()) case "serializedKeySize" => Option(record.serializedKeySize()) case "serializedValueSize" => Option(record.serializedValueSize()) - case "headers" => Option(record.headers()) case keyColumn if keyColumn.startsWith("key.") => getFromGenericRecordNullSafe(record.key(), UnresolvedAttribute.parseAttributeName(keyColumn.stripPrefix("key.")).toList) case valueColumn if valueColumn.startsWith("value.") => getFromGenericRecordNullSafe(record.value(), diff --git a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/TestDeduplicateKafkaSinkTransformerObject.scala b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/TestDeduplicateKafkaSinkTransformerObject.scala index f921fb3b..daeb8dee 100644 --- a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/TestDeduplicateKafkaSinkTransformerObject.scala +++ b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/TestDeduplicateKafkaSinkTransformerObject.scala @@ -28,6 +28,9 @@ import za.co.absa.hyperdrive.ingestor.implementation.writer.kafka.KafkaStreamWri class TestDeduplicateKafkaSinkTransformerObject extends FlatSpec with Matchers { behavior of DeduplicateKafkaSinkTransformer.getClass.getSimpleName + private val readerSchemaRegistryUrlKey = "deduplicateKafkaSinkTransformer.readerSchemaRegistryUrl" // copied from DeduplicateKafkaSinkTransformer + private val writerSchemaRegistryUrlKey = "deduplicateKafkaSinkTransformer.writerSchemaRegistryUrl" // copied from DeduplicateKafkaSinkTransformer + "apply" should "create a DeduplicateKafkaSinkTransformer" in { // given val config = getLocalConfig() @@ -77,6 +80,43 @@ class TestDeduplicateKafkaSinkTransformerObject extends FlatSpec with Matchers { transformer.kafkaConsumerTimeout shouldBe Duration.ofSeconds(120L) } + it should "throw an exception if the kafka reader config is missing" in { + val config = getLocalConfig() + config.clearProperty(KafkaStreamReader.KEY_TOPIC) + + val exception = the[Exception] thrownBy DeduplicateKafkaSinkTransformer(config) + + exception.getMessage should include(KafkaStreamReader.KEY_TOPIC) + } + + it should "throw an exception if the kafka writer config is missing" in { + val config = getLocalConfig() + config.clearProperty(KafkaStreamWriter.KEY_TOPIC) + + val exception = the[Exception] thrownBy DeduplicateKafkaSinkTransformer(config) + + exception.getMessage should include(KafkaStreamWriter.KEY_TOPIC) + } + + it should "throw an exception if the reader schema registry config is missing" in { + val config = getLocalConfig() + config.clearProperty(readerSchemaRegistryUrlKey) + + val exception = the[Exception] thrownBy DeduplicateKafkaSinkTransformer(config) + + exception.getMessage should include(readerSchemaRegistryUrlKey) + } + + it should "throw an exception if the writer schema registry config is missing" in { + val config = getLocalConfig() + config.clearProperty(writerSchemaRegistryUrlKey) + + val exception = the[Exception] thrownBy DeduplicateKafkaSinkTransformer(config) + + exception.getMessage should include(writerSchemaRegistryUrlKey) + } + + "getMappingFromRetainedGlobalConfigToLocalConfig" should "return the local config mapping" in { // given val config = getEmptyConfiguration @@ -115,13 +155,13 @@ class TestDeduplicateKafkaSinkTransformerObject extends FlatSpec with Matchers { config.addProperty(KafkaStreamReader.KEY_BROKERS, "http://readerBrokers:9092") config.addProperty("reader.option.kafka.security.protocol", "SASL_PLAINTEXT") config.addProperty("reader.option.failOnDataLoss", false) - config.addProperty("deduplicateKafkaSinkTransformer.readerSchemaRegistryUrl", "http://sourceRegistry:8081") + config.addProperty(readerSchemaRegistryUrlKey, "http://sourceRegistry:8081") config.addProperty(KafkaStreamWriter.KEY_TOPIC, "writerTopic") config.addProperty(KafkaStreamWriter.KEY_BROKERS, "http://writerBrokers:9092") config.addProperty("writer.kafka.option.kafka.sasl.mechanism", "GSSAPI") config.addProperty("component.transformer.class.encoder", classOf[ConfluentAvroEncodingTransformer].getCanonicalName) - config.addProperty("deduplicateKafkaSinkTransformer.writerSchemaRegistryUrl", "http://writerRegistry:8081") + config.addProperty(writerSchemaRegistryUrlKey, "http://writerRegistry:8081") config.addProperty(StreamWriterCommonAttributes.keyCheckpointBaseLocation, "/tmp/checkpoint") From 938af6f7dca2eb05f55652c2a3ee30ad9d046107 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Mon, 7 Dec 2020 11:05:28 +0100 Subject: [PATCH 22/27] Fix test and sonar issues --- ...fkaDeduplicationAfterRetryDockerTest.scala | 17 ++++-- ...eduplicateKafkaSinkTransformerObject.scala | 15 +++-- ...TestStreamTransformerAbstractFactory.scala | 40 ++++++------- .../utils/TestKafkaUtilDockerTest.scala | 57 ++++++++----------- 4 files changed, 64 insertions(+), 65 deletions(-) diff --git a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala index dfa0d131..9561900a 100644 --- a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala +++ b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala @@ -31,6 +31,7 @@ import za.co.absa.abris.avro.read.confluent.SchemaManagerFactory import za.co.absa.commons.io.TempDirectory import za.co.absa.commons.spark.SparkTestBase import za.co.absa.abris.avro.registry.SchemaSubject +import za.co.absa.hyperdrive.ingestor.implementation.utils.KafkaUtil import za.co.absa.hyperdrive.shared.exceptions.IngestionException /** @@ -188,8 +189,7 @@ class KafkaToKafkaDeduplicationAfterRetryDockerTest extends FlatSpec with Matche case _: IngestionException => exceptionWasThrown = true val consumer = createConsumer(kafkaSchemaRegistryWrapper) - consumer.subscribe(Collections.singletonList(destinationTopic)) - val records = consumer.poll(Duration.ofMillis(1000L)).asScala.toList + val records = getAllMessages(consumer, destinationTopic) records.size shouldBe recordsV1.size val retryConfig = driverConfig ++ Map( "transformer.[avro.decoder].value.schema.id" -> s"$schemaV2Id", @@ -205,8 +205,7 @@ class KafkaToKafkaDeduplicationAfterRetryDockerTest extends FlatSpec with Matche fs.exists(new Path(s"$checkpointDir/$sourceTopic")) shouldBe true val consumer = createConsumer(kafkaSchemaRegistryWrapper) - consumer.subscribe(Collections.singletonList(destinationTopic)) - val records = consumer.poll(Duration.ofMillis(1000L)).asScala.toList + val records = getAllMessages(consumer, destinationTopic) val valueFieldNames = records.head.value().getSchema.getFields.asScala.map(_.name()) valueFieldNames should contain theSameElementsAs List("record_id", "value_field", "hyperdrive_id") @@ -219,7 +218,7 @@ class KafkaToKafkaDeduplicationAfterRetryDockerTest extends FlatSpec with Matche SchemaManagerFactory.resetSRClientInstance() } - def createProducer(kafkaSchemaRegistryWrapper: KafkaSchemaRegistryWrapper): KafkaProducer[GenericRecord, GenericRecord] = { + private def createProducer(kafkaSchemaRegistryWrapper: KafkaSchemaRegistryWrapper): KafkaProducer[GenericRecord, GenericRecord] = { val props = new Properties() props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaSchemaRegistryWrapper.kafka.getBootstrapServers) props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "io.confluent.kafka.serializers.KafkaAvroSerializer") @@ -229,7 +228,7 @@ class KafkaToKafkaDeduplicationAfterRetryDockerTest extends FlatSpec with Matche kafkaSchemaRegistryWrapper.createProducer(props) } - def createConsumer(kafkaSchemaRegistryWrapper: KafkaSchemaRegistryWrapper): KafkaConsumer[GenericRecord, GenericRecord] = { + private def createConsumer(kafkaSchemaRegistryWrapper: KafkaSchemaRegistryWrapper): KafkaConsumer[GenericRecord, GenericRecord] = { import org.apache.kafka.clients.consumer.ConsumerConfig val props = new Properties() props.put(ConsumerConfig.GROUP_ID_CONFIG, randomUUID.toString) @@ -240,4 +239,10 @@ class KafkaToKafkaDeduplicationAfterRetryDockerTest extends FlatSpec with Matche kafkaSchemaRegistryWrapper.createConsumer(props) } + private def getAllMessages[K, V](consumer: KafkaConsumer[K, V], topic: String) = { + val topicPartitions = KafkaUtil.getTopicPartitions(consumer, topic) + val offsets = consumer.endOffsets(topicPartitions.asJava) + implicit val kafkaConsumerTimeout: Duration = Duration.ofSeconds(10L) + KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets.asScala.mapValues(_ + 0L).toMap) + } } diff --git a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/TestDeduplicateKafkaSinkTransformerObject.scala b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/TestDeduplicateKafkaSinkTransformerObject.scala index daeb8dee..24f93fe9 100644 --- a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/TestDeduplicateKafkaSinkTransformerObject.scala +++ b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/TestDeduplicateKafkaSinkTransformerObject.scala @@ -31,6 +31,9 @@ class TestDeduplicateKafkaSinkTransformerObject extends FlatSpec with Matchers { private val readerSchemaRegistryUrlKey = "deduplicateKafkaSinkTransformer.readerSchemaRegistryUrl" // copied from DeduplicateKafkaSinkTransformer private val writerSchemaRegistryUrlKey = "deduplicateKafkaSinkTransformer.writerSchemaRegistryUrl" // copied from DeduplicateKafkaSinkTransformer + private val dummySourceRegistry = "http://sourceRegistry:8081" + private val dummyDestinationRegistry = "http://destinationRegistry:8081" + "apply" should "create a DeduplicateKafkaSinkTransformer" in { // given val config = getLocalConfig() @@ -45,13 +48,13 @@ class TestDeduplicateKafkaSinkTransformerObject extends FlatSpec with Matchers { "kafka.security.protocol" -> "SASL_PLAINTEXT", "failOnDataLoss" -> "false" ) - transformer.readerSchemaRegistryUrl shouldBe "http://sourceRegistry:8081" + transformer.readerSchemaRegistryUrl shouldBe dummySourceRegistry transformer.writerTopic shouldBe "writerTopic" transformer.writerBrokers shouldBe "http://writerBrokers:9092" transformer.writerExtraOptions shouldBe Map( "kafka.sasl.mechanism" -> "GSSAPI" ) - transformer.writerSchemaRegistryUrl shouldBe "http://writerRegistry:8081" + transformer.writerSchemaRegistryUrl shouldBe dummyDestinationRegistry transformer.checkpointLocation shouldBe "/tmp/checkpoint" transformer.sourceIdColumnNames should contain theSameElementsInOrderAs Seq("offset", "partition") @@ -124,11 +127,11 @@ class TestDeduplicateKafkaSinkTransformerObject extends FlatSpec with Matchers { config.addProperty("reader.option.kafka.option2", "value2") config.addProperty("component.transformer.id.0", "decoder") config.addProperty("component.transformer.class.decoder", classOf[ConfluentAvroDecodingTransformer].getCanonicalName) - config.addProperty(s"transformer.decoder.${ConfluentAvroDecodingTransformer.KEY_SCHEMA_REGISTRY_URL}", "http://sourceRegistry:8081") + config.addProperty(s"transformer.decoder.${ConfluentAvroDecodingTransformer.KEY_SCHEMA_REGISTRY_URL}", dummySourceRegistry) config.addProperty("component.transformer.id.1", "encoder") config.addProperty("component.transformer.class.encoder", classOf[ConfluentAvroEncodingTransformer].getCanonicalName) - config.addProperty(s"transformer.encoder.${ConfluentAvroEncodingTransformer.KEY_SCHEMA_REGISTRY_URL}", "http://writerRegistry:8081") + config.addProperty(s"transformer.encoder.${ConfluentAvroEncodingTransformer.KEY_SCHEMA_REGISTRY_URL}", dummyDestinationRegistry) config.addProperty("writer.kafka.option.option3", "value3") // when @@ -155,13 +158,13 @@ class TestDeduplicateKafkaSinkTransformerObject extends FlatSpec with Matchers { config.addProperty(KafkaStreamReader.KEY_BROKERS, "http://readerBrokers:9092") config.addProperty("reader.option.kafka.security.protocol", "SASL_PLAINTEXT") config.addProperty("reader.option.failOnDataLoss", false) - config.addProperty(readerSchemaRegistryUrlKey, "http://sourceRegistry:8081") + config.addProperty(readerSchemaRegistryUrlKey, dummySourceRegistry) config.addProperty(KafkaStreamWriter.KEY_TOPIC, "writerTopic") config.addProperty(KafkaStreamWriter.KEY_BROKERS, "http://writerBrokers:9092") config.addProperty("writer.kafka.option.kafka.sasl.mechanism", "GSSAPI") config.addProperty("component.transformer.class.encoder", classOf[ConfluentAvroEncodingTransformer].getCanonicalName) - config.addProperty(writerSchemaRegistryUrlKey, "http://writerRegistry:8081") + config.addProperty(writerSchemaRegistryUrlKey, dummyDestinationRegistry) config.addProperty(StreamWriterCommonAttributes.keyCheckpointBaseLocation, "/tmp/checkpoint") diff --git a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/factories/TestStreamTransformerAbstractFactory.scala b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/factories/TestStreamTransformerAbstractFactory.scala index 10fd192b..7a9e05a2 100644 --- a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/factories/TestStreamTransformerAbstractFactory.scala +++ b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/factories/TestStreamTransformerAbstractFactory.scala @@ -26,17 +26,20 @@ class TestStreamTransformerAbstractFactory extends FlatSpec with BeforeAndAfterE behavior of StreamTransformerAbstractFactory.getClass.getSimpleName + private val dummyTransformerA = "dummy.transformer.A" + private val dummyTransformerB = "dummy.transformer.B" + it should "create transformer instances in the correct order" in { val config = getBaseConfiguration - config.addProperty(s"${IdsKeyPrefix}.1", "dummy.transformer.A") - config.addProperty(s"${ClassKeyPrefix}.dummy.transformer.A", DummyStreamTransformer.getClass.getName) - config.addProperty(s"${TransformerKeyPrefix}.dummy.transformer.A.$DummyProperty1Name", "value1") - config.addProperty(s"${TransformerKeyPrefix}.dummy.transformer.A.$DummyProperty2Name", "100") + config.addProperty(s"${IdsKeyPrefix}.1", dummyTransformerA) + config.addProperty(s"${ClassKeyPrefix}.$dummyTransformerA", DummyStreamTransformer.getClass.getName) + config.addProperty(s"${TransformerKeyPrefix}.$dummyTransformerA.$DummyProperty1Name", "value1") + config.addProperty(s"${TransformerKeyPrefix}.$dummyTransformerA.$DummyProperty2Name", "100") - config.addProperty(s"${IdsKeyPrefix}.2", "dummy.transformer.B") - config.addProperty(s"${ClassKeyPrefix}.dummy.transformer.B", DummyStreamTransformer.getClass.getName) - config.addProperty(s"${TransformerKeyPrefix}.dummy.transformer.B.$DummyProperty1Name", "value2") - config.addProperty(s"${TransformerKeyPrefix}.dummy.transformer.B.$DummyProperty2Name", "200") + config.addProperty(s"${IdsKeyPrefix}.2", dummyTransformerB) + config.addProperty(s"${ClassKeyPrefix}.$dummyTransformerB", DummyStreamTransformer.getClass.getName) + config.addProperty(s"${TransformerKeyPrefix}.$dummyTransformerB.$DummyProperty1Name", "value2") + config.addProperty(s"${TransformerKeyPrefix}.$dummyTransformerB.$DummyProperty2Name", "200") val transformers = StreamTransformerAbstractFactory.build(config) transformers should have size 2 @@ -55,13 +58,12 @@ class TestStreamTransformerAbstractFactory extends FlatSpec with BeforeAndAfterE } it should "support negative orders" in { - import StreamTransformerAbstractFactory._ val config = getBaseConfiguration config.addProperty(s"${IdsKeyPrefix}.2", "[column.transformer]") config.addProperty(s"${ClassKeyPrefix}.[column.transformer]", ColumnSelectorStreamTransformer.getClass.getName) - config.addProperty(s"${IdsKeyPrefix}.-1", "dummy.transformer.A") - config.addProperty(s"${ClassKeyPrefix}.dummy.transformer.A", DummyStreamTransformer.getClass.getName) + config.addProperty(s"${IdsKeyPrefix}.-1", dummyTransformerA) + config.addProperty(s"${ClassKeyPrefix}.$dummyTransformerA", DummyStreamTransformer.getClass.getName) val transformers = StreamTransformerAbstractFactory.build(config) transformers should have size 2 @@ -77,16 +79,16 @@ class TestStreamTransformerAbstractFactory extends FlatSpec with BeforeAndAfterE it should "throw if transformer ids are not unique" in { val config = getBaseConfiguration - config.addProperty(s"${IdsKeyPrefix}.1", "dummy.transformer.A") - config.addProperty(s"${IdsKeyPrefix}.2", "dummy.transformer.A") + config.addProperty(s"${IdsKeyPrefix}.1", dummyTransformerA) + config.addProperty(s"${IdsKeyPrefix}.2", dummyTransformerA) val throwable = intercept[IllegalArgumentException](StreamTransformerAbstractFactory.build(config)) - throwable.getMessage should include(s"dummy.transformer.A") + throwable.getMessage should include(dummyTransformerA) } it should "throw if transformer id is non-numeric" in { val config = getBaseConfiguration - config.addProperty(s"${IdsKeyPrefix}.First", "dummy.transformer.A") + config.addProperty(s"${IdsKeyPrefix}.First", dummyTransformerA) val throwable = intercept[IllegalArgumentException](StreamTransformerAbstractFactory.build(config)) throwable.getMessage should include(s"${IdsKeyPrefix}.First") @@ -94,17 +96,17 @@ class TestStreamTransformerAbstractFactory extends FlatSpec with BeforeAndAfterE it should "throw if no class name is associated to the transformer id" in { val config = getBaseConfiguration - config.addProperty(s"${IdsKeyPrefix}.1", "dummy.transformer.A") + config.addProperty(s"${IdsKeyPrefix}.1", dummyTransformerA) val throwable = intercept[IllegalArgumentException](StreamTransformerAbstractFactory.build(config)) - throwable.getMessage should include(s"${ClassKeyPrefix}.dummy.transformer.A") + throwable.getMessage should include(s"${ClassKeyPrefix}.$dummyTransformerA") } it should "throw if data transformer parameter is invalid" in { val invalidFactoryName = "an-invalid-factory-name" val config = getBaseConfiguration - config.addProperty(s"${IdsKeyPrefix}.1", "dummy.transformer.A") - config.addProperty(s"${ClassKeyPrefix}.dummy.transformer.A", invalidFactoryName) + config.addProperty(s"${IdsKeyPrefix}.1", dummyTransformerA) + config.addProperty(s"${ClassKeyPrefix}.$dummyTransformerA", invalidFactoryName) val throwable = intercept[IllegalArgumentException](StreamTransformerAbstractFactory.build(config)) assert(throwable.getMessage.contains(invalidFactoryName)) diff --git a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala index 583f0884..668a027c 100644 --- a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala +++ b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala @@ -36,6 +36,7 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter private val kafka = new KafkaContainer(confluentPlatformVersion) private val kafkaSufficientTimeout = Duration.ofSeconds(5L) private val kafkaInsufficientTimeout = Duration.ofMillis(1L) + private val topic = "test-topic" before{ kafka.start() @@ -47,7 +48,6 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter "getAllAvailableMessages" should "get all available messages" in { // given - val topic = "test-topic" val partitions = 3 createTopic(kafka, topic, partitions) val producer = createProducer(kafka) @@ -69,7 +69,6 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter it should "get all available messages, even if polling is required multiple times" in { // given - val topic = "test-topic" val partitions = 3 createTopic(kafka, topic, partitions, Map("segment.ms" -> "100")) val producer = createProducer(kafka) @@ -91,7 +90,6 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter it should "stop polling when the desired end offset has been reached and not run infinitely" in { // given - val topic = "test-topic" val partitions = 3 createTopic(kafka, topic, partitions) val producer = createProducer(kafka) @@ -132,7 +130,6 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter it should "throw an exception if consumer is already subscribed" in { // given - val topic = "test-topic" createTopic(kafka, topic, 1) val consumer = createConsumer(kafka) @@ -148,7 +145,6 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter it should "throw an exception if not all messages could be consumed (because the timeout is too short)" in { // given - val topic = "test-topic" val partitions = 3 createTopic(kafka, topic, partitions) val producer = createProducer(kafka) @@ -171,7 +167,6 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter it should "throw an exception if requested offsets are not available" in { // given - val topic = "test-topic" val partitions = 3 createTopic(kafka, topic, partitions) val producer = createProducer(kafka) @@ -191,8 +186,7 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter } "getAtLeastNLatestRecords" should "get at least the n latest records" in { - val topicName = "test-topic" - createTopic(kafka, topicName, 1, Map( + createTopic(kafka, topic, 1, Map( "cleanup.policy" -> "compact", "delete.retention.ms" -> "100", "segment.ms" -> "100", @@ -204,13 +198,13 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter val key = if (i % 2 == 0) 1000 + i else 1 (key.toString, s"msg_${i}") }) - produceData2(producer, messages, topicName) + produceData2(producer, messages, topic) val waitForCompactionMillis = 20000L Thread.sleep(waitForCompactionMillis) val testConsumer = createConsumer(kafka) - testConsumer.subscribe(Collections.singletonList(topicName)) + testConsumer.subscribe(Collections.singletonList(topic)) import scala.util.control.Breaks._ var records: Seq[ConsumerRecord[String, String]] = mutable.Seq() breakable { @@ -230,7 +224,7 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter val consumer = createConsumer(kafka) implicit val kafkaConsumerTimeout: Duration = kafkaSufficientTimeout - val actualRecords = KafkaUtil.getAtLeastNLatestRecordsFromPartition(consumer, new TopicPartition(topicName, 0), 10) + val actualRecords = KafkaUtil.getAtLeastNLatestRecordsFromPartition(consumer, new TopicPartition(topic, 0), 10) val values = actualRecords.map(_.value()) values.size should be >= 10 @@ -239,7 +233,6 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter it should "be able to reuse a consumer" in { // given - val topic = "test-topic" val partitions = 3 createTopic(kafka, topic, partitions) val producer = createProducer(kafka) @@ -259,44 +252,41 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter } it should "throw an exception if the timeout is too short" in { - val topicName = "test-topic" - createTopic(kafka, topicName, 1) + createTopic(kafka, topic, 1) val producer = createProducer(kafka) val messages = (1 to 100).map(i => s"message_${i}") - produceData(producer, messages, topicName, 1) + produceData(producer, messages, topic, 1) val consumer = createConsumer(kafka) implicit val kafkaConsumerTimeout: Duration = kafkaInsufficientTimeout - val result = the[Exception] thrownBy KafkaUtil.getAtLeastNLatestRecordsFromPartition(consumer, new TopicPartition(topicName, 0), 10) + val result = the[Exception] thrownBy KafkaUtil.getAtLeastNLatestRecordsFromPartition(consumer, new TopicPartition(topic, 0), 10) result.getMessage should include("increasing the consumer timeout") } "getTopicPartitions" should "return the partitions" in { - val topicName = "test-topic" - createTopic(kafka, topicName, 10) + createTopic(kafka, topic, 10) val consumer = createConsumer(kafka) - val topicPartitions = KafkaUtil.getTopicPartitions(consumer, topicName) + val topicPartitions = KafkaUtil.getTopicPartitions(consumer, topic) - val expectedPartitions = (0 until 10).map(i => new TopicPartition(topicName, i)) + val expectedPartitions = (0 until 10).map(i => new TopicPartition(topic, i)) topicPartitions should contain theSameElementsAs expectedPartitions } "seekToOffsetsOrBeginning" should "seek to the provided offsets" in { // given - val topicName = "test-topic" - createTopic(kafka, topicName, 3) + createTopic(kafka, topic, 3) val consumer = createConsumer(kafka) val producer = createProducer(kafka) val messages = (1 to 100).map(i => s"message_${i}") - produceData(producer, messages, topicName, 3) + produceData(producer, messages, topic, 3) - val tp0 = new TopicPartition(topicName, 0) - val tp1 = new TopicPartition(topicName, 1) - val tp2 = new TopicPartition(topicName, 2) + val tp0 = new TopicPartition(topic, 0) + val tp1 = new TopicPartition(topic, 1) + val tp2 = new TopicPartition(topic, 2) val offsets = Map( tp0 -> 10L, tp1 -> 15L, @@ -304,7 +294,7 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter ) // when - KafkaUtil.seekToOffsetsOrBeginning(consumer, topicName, Some(offsets)) + KafkaUtil.seekToOffsetsOrBeginning(consumer, topic, Some(offsets)) // then consumer.position(tp0) shouldBe 10L @@ -314,20 +304,19 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter it should "seek to the beginning if no offsets are given" in { // given - val topicName = "test-topic" - createTopic(kafka, topicName, 3) + createTopic(kafka, topic, 3) val consumer = createConsumer(kafka) val producer = createProducer(kafka) val messages = (1 to 100).map(i => s"message_${i}") - produceData(producer, messages, topicName, 3) + produceData(producer, messages, topic, 3) - val tp0 = new TopicPartition(topicName, 0) - val tp1 = new TopicPartition(topicName, 1) - val tp2 = new TopicPartition(topicName, 2) + val tp0 = new TopicPartition(topic, 0) + val tp1 = new TopicPartition(topic, 1) + val tp2 = new TopicPartition(topic, 2) // when - KafkaUtil.seekToOffsetsOrBeginning(consumer, topicName, None) + KafkaUtil.seekToOffsetsOrBeginning(consumer, topic, None) // then consumer.position(tp0) shouldBe 0L From b87bd168d159e87390ae411e3e2abd4cad8fc6dd Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Mon, 7 Dec 2020 11:39:31 +0100 Subject: [PATCH 23/27] Fix test, total order not guaranteed --- .../drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala index 9561900a..9fd9d19d 100644 --- a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala +++ b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala @@ -189,8 +189,6 @@ class KafkaToKafkaDeduplicationAfterRetryDockerTest extends FlatSpec with Matche case _: IngestionException => exceptionWasThrown = true val consumer = createConsumer(kafkaSchemaRegistryWrapper) - val records = getAllMessages(consumer, destinationTopic) - records.size shouldBe recordsV1.size val retryConfig = driverConfig ++ Map( "transformer.[avro.decoder].value.schema.id" -> s"$schemaV2Id", "writer.common.trigger.type" -> "Once", From 65373ac110ce0422dfa8e2669ae9517862809d82 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Tue, 15 Dec 2020 16:06:54 +0100 Subject: [PATCH 24/27] wip --- ...fkaDeduplicationAfterRetryDockerTest.scala | 1 + .../implementation/utils/KafkaUtil.scala | 29 ++++++++++--------- .../utils/TestKafkaUtilDockerTest.scala | 3 +- 3 files changed, 19 insertions(+), 14 deletions(-) diff --git a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala index 9fd9d19d..d206fd43 100644 --- a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala +++ b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala @@ -96,6 +96,7 @@ class KafkaToKafkaDeduplicationAfterRetryDockerTest extends FlatSpec with Matche while(!topicCreationFut.isDone) {} } + // TODO: Add failing test without DeduplicateTransformer it should "execute the whole kafka-to-kafka pipeline" in { // given val kafkaSchemaRegistryWrapper = new KafkaSchemaRegistryWrapper diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala index 83da227b..0c9a55cd 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala @@ -29,22 +29,25 @@ import scala.collection.mutable private[hyperdrive] object KafkaUtil { private val logger = LogManager.getLogger - def getAtLeastNLatestRecordsFromPartition[K, V](consumer: KafkaConsumer[K, V], topicPartition: TopicPartition, numberOfRecords: Int) + def getAtLeastNLatestRecordsFromPartition[K, V](consumer: KafkaConsumer[K, V], numberOfRecordsPerPartition: Map[TopicPartition, Long]) (implicit kafkaConsumerTimeout: Duration): Seq[ConsumerRecord[K, V]] = { - consumer.assign(Seq(topicPartition).asJava) - val endOffsets = consumer.endOffsets(Seq(topicPartition).asJava).asScala - if (endOffsets.size != 1) { - throw new IllegalStateException(s"Expected exactly 1 end offset, got ${endOffsets}") - } - val partition = endOffsets.keys.head - val offset = endOffsets.values.head + val topicPartitions = numberOfRecordsPerPartition.keySet + consumer.assign(topicPartitions.asJava) + val endOffsets = consumer.endOffsets(topicPartitions.asJava).asScala var records: Seq[ConsumerRecord[K, V]] = Seq() - var offsetLowerBound = offset - while (records.size < numberOfRecords && offsetLowerBound != 0) { - offsetLowerBound = Math.max(0, offsetLowerBound - numberOfRecords) - consumer.seek(partition, offsetLowerBound) - records = getMessagesAtLeastToOffset(consumer, Map(topicPartition -> offset)) + var recordSizesPerPartition: Map[TopicPartition, Long] = Map() + var offsetLowerBoundPerPartition = endOffsets.mapValues(_ + 0L) + while (topicPartitions.exists(p => recordSizesPerPartition(p) < numberOfRecordsPerPartition(p) && offsetLowerBoundPerPartition(p) != 0)) { + offsetLowerBoundPerPartition = offsetLowerBoundPerPartition.map { + case (p, offsetLowerBound) => p -> Math.max(0, offsetLowerBound - numberOfRecordsPerPartition(p)) + } + offsetLowerBoundPerPartition.foreach { + case (p, offsetLowerBound) => consumer.seek(p, offsetLowerBound) + } + + records = getMessagesAtLeastToOffset(consumer, endOffsets.mapValues(_ + 0L).toMap) + // TODO: Fill recordSizesPerPartition } records diff --git a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala index 668a027c..458fc185 100644 --- a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala +++ b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala @@ -46,7 +46,7 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter kafka.stop() } - "getAllAvailableMessages" should "get all available messages" in { + "getMessagesAtLeastToOffset" should "get all available messages" in { // given val partitions = 3 createTopic(kafka, topic, partitions) @@ -70,6 +70,7 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter it should "get all available messages, even if polling is required multiple times" in { // given val partitions = 3 + // TODO: Use max.poll.records createTopic(kafka, topic, partitions, Map("segment.ms" -> "100")) val producer = createProducer(kafka) val messages = (1 to 100).map(i => s"message_${i}") From bcf06af842d6cceb95aa21177a6a8b4b571538db Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Wed, 16 Dec 2020 12:20:38 +0100 Subject: [PATCH 25/27] works now --- .../DeduplicateKafkaSinkTransformer.scala | 8 +-- .../implementation/utils/KafkaUtil.scala | 14 ++--- .../utils/TestKafkaUtilDockerTest.scala | 54 ++++++++++--------- 3 files changed, 41 insertions(+), 35 deletions(-) diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala index 6b198d4b..277fc60f 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala @@ -83,10 +83,10 @@ private[transformer] class DeduplicateKafkaSinkTransformer( val sinkConsumer = createConsumer(writerBrokers, writerExtraOptions, writerSchemaRegistryUrl) val sinkTopicPartitions = KafkaUtil.getTopicPartitions(sinkConsumer, writerTopic) - val latestSinkRecords = consumeAndClose(sinkConsumer, consumer => sinkTopicPartitions.map { - topicPartition => KafkaUtil.getAtLeastNLatestRecordsFromPartition(consumer, topicPartition, sourceRecords.size) - }) - val publishedIds = latestSinkRecords.flatten.map(extractIdFieldsFromRecord(_, destinationIdColumnNames)) + val recordsPerPartition = sinkTopicPartitions.map(p => p -> sourceRecords.size.toLong).toMap + val latestSinkRecords = consumeAndClose(sinkConsumer, consumer => + KafkaUtil.getAtLeastNLatestRecordsFromPartition(consumer, recordsPerPartition)) + val publishedIds = latestSinkRecords.map(extractIdFieldsFromRecord(_, destinationIdColumnNames)) val duplicatedIds = sourceIds.intersect(publishedIds) val duplicatedIdsLit = duplicatedIds.map(duplicatedId => struct(duplicatedId.map(lit): _*)) diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala index 0c9a55cd..293bbeaa 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala @@ -31,12 +31,12 @@ private[hyperdrive] object KafkaUtil { def getAtLeastNLatestRecordsFromPartition[K, V](consumer: KafkaConsumer[K, V], numberOfRecordsPerPartition: Map[TopicPartition, Long]) (implicit kafkaConsumerTimeout: Duration): Seq[ConsumerRecord[K, V]] = { - val topicPartitions = numberOfRecordsPerPartition.keySet - consumer.assign(topicPartitions.asJava) - val endOffsets = consumer.endOffsets(topicPartitions.asJava).asScala + consumer.assign(numberOfRecordsPerPartition.keySet.asJava) + val endOffsets = consumer.endOffsets(numberOfRecordsPerPartition.keySet.asJava).asScala + val topicPartitions = endOffsets.keySet var records: Seq[ConsumerRecord[K, V]] = Seq() - var recordSizesPerPartition: Map[TopicPartition, Long] = Map() + var recordSizesPerPartition: Map[TopicPartition, Long] = topicPartitions.map(p => p -> 0L).toMap var offsetLowerBoundPerPartition = endOffsets.mapValues(_ + 0L) while (topicPartitions.exists(p => recordSizesPerPartition(p) < numberOfRecordsPerPartition(p) && offsetLowerBoundPerPartition(p) != 0)) { offsetLowerBoundPerPartition = offsetLowerBoundPerPartition.map { @@ -47,14 +47,16 @@ private[hyperdrive] object KafkaUtil { } records = getMessagesAtLeastToOffset(consumer, endOffsets.mapValues(_ + 0L).toMap) - // TODO: Fill recordSizesPerPartition + recordSizesPerPartition = records + .groupBy(r => new TopicPartition(r.topic(), r.partition())) + .mapValues(records => records.size) } records } def getMessagesAtLeastToOffset[K, V](consumer: KafkaConsumer[K, V], toOffsets: Map[TopicPartition, Long]) - (implicit kafkaConsumerTimeout: Duration): Seq[ConsumerRecord[K, V]] = { + (implicit kafkaConsumerTimeout: Duration): Seq[ConsumerRecord[K, V]] = { consumer.assign(toOffsets.keySet.asJava) val endOffsets = consumer.endOffsets(toOffsets.keys.toSeq.asJava).asScala endOffsets.foreach { case (topicPartition, offset) => diff --git a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala index 458fc185..2dc50f87 100644 --- a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala +++ b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala @@ -186,20 +186,22 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter exception.getMessage should include ("Requested consumption") } - "getAtLeastNLatestRecords" should "get at least the n latest records" in { - createTopic(kafka, topic, 1, Map( + "getAtLeastNLatestRecords" should "get at least the n latest records if there are gaps in the offsets" in { + val messageCreationTimeout = 100L + val partitions = 3 + createTopic(kafka, topic, partitions, Map( "cleanup.policy" -> "compact", "delete.retention.ms" -> "100", - "segment.ms" -> "100", + "segment.ms" -> s"$messageCreationTimeout", "min.cleanable.dirty.ratio" -> "0.01" )) val producer = createProducer(kafka) - val messages = (1 to 100).map(i => { - val key = if (i % 2 == 0) 1000 + i else 1 - (key.toString, s"msg_${i}") + val messages = (1 to 103).map(i => { + val key = if (i % 2 == 0 || i > 100) 1000 + i else 1 + (key.toString, s"msg_${i}", i % partitions) }) - produceData2(producer, messages, topic) + produceData(producer, messages, topic, Some(messageCreationTimeout)) val waitForCompactionMillis = 20000L Thread.sleep(waitForCompactionMillis) @@ -219,20 +221,22 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter } withClue(){ - records.size shouldBe messages.map(_._1).distinct.size + records.size shouldBe messages.map(r => (r._1, r._3)).distinct.size } withClue(s"This is likely an artifact of the test itself. You may want to increase waitForCompactionMillis." + s" The current value is $waitForCompactionMillis") val consumer = createConsumer(kafka) implicit val kafkaConsumerTimeout: Duration = kafkaSufficientTimeout - val actualRecords = KafkaUtil.getAtLeastNLatestRecordsFromPartition(consumer, new TopicPartition(topic, 0), 10) + val recordsPerPartition = (0 to partitions).map(p => new TopicPartition(topic, p) -> 4L).toMap + val actualRecords = KafkaUtil.getAtLeastNLatestRecordsFromPartition(consumer, recordsPerPartition) val values = actualRecords.map(_.value()) - values.size should be >= 10 - values should contain allElementsOf Seq("msg_100", "msg_99", "msg_98", "msg_96", "msg_94", "msg_92", "msg_90", "msg_88", "msg_86", "msg_84") + values.size should be >= 12 + values should contain allElementsOf Seq("msg_103", "msg_102", "msg_101", "msg_100", "msg_97", "msg_95", "msg_94", + "msg_92", "msg_90", "msg_88", "msg_86", "msg_84") } - it should "be able to reuse a consumer" in { + it should "get from multiple topics simultaneously" in { // given val partitions = 3 createTopic(kafka, topic, partitions) @@ -245,7 +249,8 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter // when implicit val kafkaConsumerTimeout: Duration = kafkaSufficientTimeout - val records = topicPartitions.flatMap(tp => KafkaUtil.getAtLeastNLatestRecordsFromPartition(consumer, tp, 1000)) + val recordsPerPartition = topicPartitions.map(t => t -> 1000L).toMap + val records = KafkaUtil.getAtLeastNLatestRecordsFromPartition(consumer, recordsPerPartition) // then val actualMessages = records.map(_.value()).toList.sorted @@ -262,7 +267,7 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter val consumer = createConsumer(kafka) implicit val kafkaConsumerTimeout: Duration = kafkaInsufficientTimeout - val result = the[Exception] thrownBy KafkaUtil.getAtLeastNLatestRecordsFromPartition(consumer, new TopicPartition(topic, 0), 10) + val result = the[Exception] thrownBy KafkaUtil.getAtLeastNLatestRecordsFromPartition(consumer, Map(new TopicPartition(topic, 0) -> 10)) result.getMessage should include("increasing the consumer timeout") } @@ -357,21 +362,20 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter new KafkaConsumer[String, String](props) } - private def produceData(producer: KafkaProducer[String, String], records: Seq[String], topic: String, partitions: Int): Unit = { - records.zipWithIndex.foreach { - case (record, i) => - val partition = i % partitions - val producerRecord = new ProducerRecord[String, String](topic, partition, null, record) - producer.send(producerRecord) + private def produceData(producer: KafkaProducer[String, String], valueRecords: Seq[String], topic: String, partitions: Int): Unit = { + val records = valueRecords.zipWithIndex.map { + case (value, i) => (null, value, i % partitions) } - producer.flush() + produceData(producer, records, topic) } - private def produceData2(producer: KafkaProducer[String, String], records: Seq[(String, String)], topic: String): Unit = { - records.foreach { record => - val producerRecord = new ProducerRecord[String, String](topic, record._1, record._2) + private def produceData(producer: KafkaProducer[String, String], records: Seq[(String, String, Int)], topic: String, + timeout: Option[Long] = None): Unit = { + records.foreach { + record => + val producerRecord = new ProducerRecord[String, String](topic, record._3, record._1, record._2) producer.send(producerRecord) - Thread.sleep(100L) + timeout.foreach(Thread.sleep) } producer.flush() } From 874b474f7b09f721f7e14aca9470d0484261e600 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Wed, 16 Dec 2020 16:20:36 +0100 Subject: [PATCH 26/27] Add test with duplicates, use max.poll.records --- ...fkaDeduplicationAfterRetryDockerTest.scala | 131 +++++++++++------- .../implementation/utils/KafkaUtil.scala | 38 ++--- .../utils/TestKafkaUtilDockerTest.scala | 28 +--- 3 files changed, 106 insertions(+), 91 deletions(-) diff --git a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala index d206fd43..631eca0d 100644 --- a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala +++ b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala @@ -53,55 +53,63 @@ import za.co.absa.hyperdrive.shared.exceptions.IngestionException * and transformer.[kafka.deduplicator].destination.id.columns) * * Finally, the destination topic is expected to contain all messages from the source topic - * exactly once. Without the deduplication transformer, the 41st-50th messages would be duplicated. + * exactly once, thanks to the deduplication transformer (see test case 1). + * Without the deduplication transformer, the 41st-50th messages are duplicated (see test case 2). */ class KafkaToKafkaDeduplicationAfterRetryDockerTest extends FlatSpec with Matchers with SparkTestBase with BeforeAndAfter { import scala.collection.JavaConverters._ private val fs = FileSystem.get(spark.sparkContext.hadoopConfiguration) - private val baseDir = TempDirectory("hyperdriveE2eTest").deleteOnExit() - private val baseDirPath = baseDir.path.toUri - private val checkpointDir = s"$baseDirPath/checkpoint" + private var baseDir: TempDirectory = _ behavior of "CommandLineIngestionDriver" - private def schemaV1String(name: String) = - raw"""{"type": "record", "name": "$name", "fields": [ - |{"type": "int", "name": "record_id"}, - |{"type": "string", "name": "value_field", "nullable": false} - |]}""".stripMargin + it should "write exactly-once using the deduplicate transformer" in { + val recordIdsV1 = 0 until 50 + val recordIdsV2 = 50 until 100 + val deduplicatorConfig = Map( + "component.transformer.id.2" -> "[kafka.deduplicator]", + "component.transformer.class.[kafka.deduplicator]" -> "za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka.DeduplicateKafkaSinkTransformer", + "transformer.[kafka.deduplicator].source.id.columns" -> "offset,partition", + "transformer.[kafka.deduplicator].destination.id.columns" -> "value.hyperdrive_id.source_offset, value.hyperdrive_id.source_partition" + ) + val kafkaSchemaRegistryWrapper = new KafkaSchemaRegistryWrapper + val destinationTopic = "deduplication_dest" - private def schemaV2String(name: String) = - raw"""{"type": "record", "name": "$name", "fields": [ - |{"type": "int", "name": "record_id"}, - |{"type": ["null", "string"], "name": "value_field", "nullable": true} - |]}""".stripMargin + executeTestCase(deduplicatorConfig, recordIdsV1, recordIdsV2, kafkaSchemaRegistryWrapper, destinationTopic) - private def sendData(producer: KafkaProducer[GenericRecord, GenericRecord], records: Seq[GenericRecord], topic: String, partitions: Int): Unit = { - records.zipWithIndex.foreach { - case (record, i) => - val partition = i % partitions - val producerRecord = new ProducerRecord[GenericRecord, GenericRecord](topic, partition, null, record) - producer.send(producerRecord) - } + val consumer = createConsumer(kafkaSchemaRegistryWrapper) + val records = getAllMessages(consumer, destinationTopic) + val valueFieldNames = records.head.value().getSchema.getFields.asScala.map(_.name()) + valueFieldNames should contain theSameElementsAs List("record_id", "value_field", "hyperdrive_id") + val actualRecordIds = records.map(_.value().get("record_id")) + actualRecordIds.distinct.size shouldBe actualRecordIds.size + val expectedRecordIds = 0 until recordIdsV1.size + recordIdsV2.size + actualRecordIds should contain theSameElementsAs expectedRecordIds } - private def createTopic(kafkaSchemaRegistryWrapper: KafkaSchemaRegistryWrapper, topicName: String, partitions: Int): Unit = { - val config = new Properties() - config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaSchemaRegistryWrapper.kafka.getBootstrapServers) - val localKafkaAdmin = AdminClient.create(config) - val replication = 1.toShort - val topic = new NewTopic(topicName, partitions, replication) - val topicCreationFut = localKafkaAdmin.createTopics(util.Arrays.asList(topic)).all() - while(!topicCreationFut.isDone) {} + it should "write duplicate entries without the deduplicate transformer" in { + val recordIdsV1 = 0 until 50 + val recordIdsV2 = 50 until 100 + val kafkaSchemaRegistryWrapper = new KafkaSchemaRegistryWrapper + val destinationTopic = "deduplication_dest" + + executeTestCase(Map(), recordIdsV1, recordIdsV2, kafkaSchemaRegistryWrapper, destinationTopic) + + val consumer = createConsumer(kafkaSchemaRegistryWrapper) + val records = getAllMessages(consumer, destinationTopic) + val valueFieldNames = records.head.value().getSchema.getFields.asScala.map(_.name()) + valueFieldNames should contain theSameElementsAs List("record_id", "value_field", "hyperdrive_id") + val actualRecordIds = records.map(_.value().get("record_id")) + actualRecordIds.distinct.size should be < actualRecordIds.size } - // TODO: Add failing test without DeduplicateTransformer - it should "execute the whole kafka-to-kafka pipeline" in { + // scalastyle:off method.length + private def executeTestCase(deduplicatorConfig: Map[String, String], recordIdsV1: Seq[Int], recordIdsV2: Seq[Int], + kafkaSchemaRegistryWrapper: KafkaSchemaRegistryWrapper, destinationTopic: String) = { // given - val kafkaSchemaRegistryWrapper = new KafkaSchemaRegistryWrapper + val checkpointDir = s"${baseDir.path.toUri}/checkpoint" val sourceTopic = "deduplication_src" - val destinationTopic = "deduplication_dest" val sourceTopicPartitions = 5 val destinationTopicPartitions = 3 val schemaManager = SchemaManagerFactory.create(Map("schema.registry.url" -> kafkaSchemaRegistryWrapper.schemaRegistryUrl)) @@ -117,13 +125,13 @@ class KafkaToKafkaDeduplicationAfterRetryDockerTest extends FlatSpec with Matche createTopic(kafkaSchemaRegistryWrapper, sourceTopic, sourceTopicPartitions) createTopic(kafkaSchemaRegistryWrapper, destinationTopic, destinationTopicPartitions) - val recordsV1 = (0 until 50).map(i => { + val recordsV1 = recordIdsV1.map(i => { val valueRecord = new GenericData.Record(schemaV1) valueRecord.put("record_id", i) valueRecord.put("value_field", s"valueHello_$i") valueRecord }) - val recordsV2 = (50 until 100).map(i => { + val recordsV2 = recordIdsV2.map(i => { val valueRecord = new GenericData.Record(schemaV2) valueRecord.put("record_id", i) valueRecord.put("value_field", null) @@ -142,8 +150,6 @@ class KafkaToKafkaDeduplicationAfterRetryDockerTest extends FlatSpec with Matche "component.transformer.class.[column.copy]" -> "za.co.absa.hyperdrive.ingestor.implementation.transformer.column.copy.ColumnCopyStreamTransformer", "component.transformer.id.1" -> "[avro.decoder]", "component.transformer.class.[avro.decoder]" -> "za.co.absa.hyperdrive.ingestor.implementation.transformer.avro.confluent.ConfluentAvroDecodingTransformer", - "component.transformer.id.2" -> "[kafka.deduplicator]", - "component.transformer.class.[kafka.deduplicator]" -> "za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka.DeduplicateKafkaSinkTransformer", "component.transformer.id.3" -> "[avro.encoder]", "component.transformer.class.[avro.encoder]" -> "za.co.absa.hyperdrive.ingestor.implementation.transformer.avro.confluent.ConfluentAvroEncodingTransformer", "component.writer" -> "za.co.absa.hyperdrive.ingestor.implementation.writer.kafka.KafkaStreamWriter", @@ -166,10 +172,6 @@ class KafkaToKafkaDeduplicationAfterRetryDockerTest extends FlatSpec with Matche "transformer.[avro.decoder].value.schema.naming.strategy" -> "topic.name", "transformer.[avro.decoder].keep.columns" -> "hyperdrive_id", - // comma separated list of columns to select - "transformer.[kafka.deduplicator].source.id.columns" -> "offset,partition", - "transformer.[kafka.deduplicator].destination.id.columns" -> "value.hyperdrive_id.source_offset, value.hyperdrive_id.source_partition", - // Avro Encoder (ABRiS) settings "transformer.[avro.encoder].schema.registry.url" -> "${transformer.[avro.decoder].schema.registry.url}", "transformer.[avro.encoder].value.schema.naming.strategy" -> "topic.name", @@ -179,7 +181,7 @@ class KafkaToKafkaDeduplicationAfterRetryDockerTest extends FlatSpec with Matche "writer.common.trigger.type" -> "ProcessingTime", "writer.kafka.topic" -> destinationTopic, "writer.kafka.brokers" -> "${reader.kafka.brokers}" - ) + ) ++ deduplicatorConfig val driverConfigArray = driverConfig.map { case (key, value) => s"$key=$value" }.toArray // when, then @@ -189,7 +191,6 @@ class KafkaToKafkaDeduplicationAfterRetryDockerTest extends FlatSpec with Matche } catch { case _: IngestionException => exceptionWasThrown = true - val consumer = createConsumer(kafkaSchemaRegistryWrapper) val retryConfig = driverConfig ++ Map( "transformer.[avro.decoder].value.schema.id" -> s"$schemaV2Id", "writer.common.trigger.type" -> "Once", @@ -202,19 +203,47 @@ class KafkaToKafkaDeduplicationAfterRetryDockerTest extends FlatSpec with Matche exceptionWasThrown shouldBe true fs.exists(new Path(s"$checkpointDir/$sourceTopic")) shouldBe true + } + // scalastyle:on method.length - val consumer = createConsumer(kafkaSchemaRegistryWrapper) - val records = getAllMessages(consumer, destinationTopic) - - val valueFieldNames = records.head.value().getSchema.getFields.asScala.map(_.name()) - valueFieldNames should contain theSameElementsAs List("record_id", "value_field", "hyperdrive_id") - val actualRecordIds = records.map(_.value().get("record_id")) - val expectedRecordIds = 0 until recordsV1.size + recordsV2.size - actualRecordIds should contain theSameElementsAs expectedRecordIds + before { + baseDir = TempDirectory("hyperdriveE2eTest").deleteOnExit() } after { SchemaManagerFactory.resetSRClientInstance() + baseDir.delete() + } + + private def schemaV1String(name: String) = + raw"""{"type": "record", "name": "$name", "fields": [ + |{"type": "int", "name": "record_id"}, + |{"type": "string", "name": "value_field", "nullable": false} + |]}""".stripMargin + + private def schemaV2String(name: String) = + raw"""{"type": "record", "name": "$name", "fields": [ + |{"type": "int", "name": "record_id"}, + |{"type": ["null", "string"], "name": "value_field", "nullable": true} + |]}""".stripMargin + + private def sendData(producer: KafkaProducer[GenericRecord, GenericRecord], records: Seq[GenericRecord], topic: String, partitions: Int): Unit = { + records.zipWithIndex.foreach { + case (record, i) => + val partition = i % partitions + val producerRecord = new ProducerRecord[GenericRecord, GenericRecord](topic, partition, null, record) + producer.send(producerRecord) + } + } + + private def createTopic(kafkaSchemaRegistryWrapper: KafkaSchemaRegistryWrapper, topicName: String, partitions: Int): Unit = { + val config = new Properties() + config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaSchemaRegistryWrapper.kafka.getBootstrapServers) + val localKafkaAdmin = AdminClient.create(config) + val replication = 1.toShort + val topic = new NewTopic(topicName, partitions, replication) + val topicCreationFut = localKafkaAdmin.createTopics(util.Arrays.asList(topic)).all() + while(!topicCreationFut.isDone) {} } private def createProducer(kafkaSchemaRegistryWrapper: KafkaSchemaRegistryWrapper): KafkaProducer[GenericRecord, GenericRecord] = { diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala index 293bbeaa..c0a02618 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala @@ -29,27 +29,33 @@ import scala.collection.mutable private[hyperdrive] object KafkaUtil { private val logger = LogManager.getLogger - def getAtLeastNLatestRecordsFromPartition[K, V](consumer: KafkaConsumer[K, V], numberOfRecordsPerPartition: Map[TopicPartition, Long]) + def getAtLeastNLatestRecordsFromPartition[K, V](consumer: KafkaConsumer[K, V], numberOfRecords: Map[TopicPartition, Long]) (implicit kafkaConsumerTimeout: Duration): Seq[ConsumerRecord[K, V]] = { - consumer.assign(numberOfRecordsPerPartition.keySet.asJava) - val endOffsets = consumer.endOffsets(numberOfRecordsPerPartition.keySet.asJava).asScala + consumer.assign(numberOfRecords.keySet.asJava) + val endOffsets = consumer.endOffsets(numberOfRecords.keySet.asJava).asScala val topicPartitions = endOffsets.keySet var records: Seq[ConsumerRecord[K, V]] = Seq() - var recordSizesPerPartition: Map[TopicPartition, Long] = topicPartitions.map(p => p -> 0L).toMap - var offsetLowerBoundPerPartition = endOffsets.mapValues(_ + 0L) - while (topicPartitions.exists(p => recordSizesPerPartition(p) < numberOfRecordsPerPartition(p) && offsetLowerBoundPerPartition(p) != 0)) { - offsetLowerBoundPerPartition = offsetLowerBoundPerPartition.map { - case (p, offsetLowerBound) => p -> Math.max(0, offsetLowerBound - numberOfRecordsPerPartition(p)) - } - offsetLowerBoundPerPartition.foreach { - case (p, offsetLowerBound) => consumer.seek(p, offsetLowerBound) - } + val offsetLowerBounds = mutable.Map(endOffsets.mapValues(_ + 0L).toSeq: _*) + import scala.util.control.Breaks._ + breakable { + while (true) { + val recordSizes = records + .groupBy(r => new TopicPartition(r.topic(), r.partition())) + .mapValues(records => records.size) + val unfinishedPartitions = topicPartitions.filter(p => recordSizes.getOrElse(p, 0) < numberOfRecords(p) && offsetLowerBounds(p) != 0) + if (unfinishedPartitions.isEmpty) { + break() + } - records = getMessagesAtLeastToOffset(consumer, endOffsets.mapValues(_ + 0L).toMap) - recordSizesPerPartition = records - .groupBy(r => new TopicPartition(r.topic(), r.partition())) - .mapValues(records => records.size) + unfinishedPartitions.foreach { p => + offsetLowerBounds(p) = Math.max(0, offsetLowerBounds(p) - numberOfRecords(p)) + } + offsetLowerBounds.foreach { + case (partition, offset) => consumer.seek(partition, offset) + } + records = getMessagesAtLeastToOffset(consumer, endOffsets.mapValues(_ + 0L).toMap) + } } records diff --git a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala index 2dc50f87..60481457 100644 --- a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala +++ b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala @@ -37,6 +37,7 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter private val kafkaSufficientTimeout = Duration.ofSeconds(5L) private val kafkaInsufficientTimeout = Duration.ofMillis(1L) private val topic = "test-topic" + private val maxPollRecords = 10 before{ kafka.start() @@ -67,28 +68,6 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter actualMessages should contain theSameElementsAs messages } - it should "get all available messages, even if polling is required multiple times" in { - // given - val partitions = 3 - // TODO: Use max.poll.records - createTopic(kafka, topic, partitions, Map("segment.ms" -> "100")) - val producer = createProducer(kafka) - val messages = (1 to 100).map(i => s"message_${i}") - produceData(producer, messages, topic, partitions) - - val consumer = createConsumer(kafka) - val topicPartitions = KafkaUtil.getTopicPartitions(consumer, topic) - val offsets = consumer.endOffsets(topicPartitions.asJava).asScala.toMap.mapValues(_.asInstanceOf[Long]) - - // when - implicit val kafkaConsumerTimeout: Duration = kafkaSufficientTimeout - val records = KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets) - - // then - val actualMessages = records.map(_.value()).toList.sorted - actualMessages should contain theSameElementsAs messages - } - it should "stop polling when the desired end offset has been reached and not run infinitely" in { // given val partitions = 3 @@ -232,8 +211,8 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter val values = actualRecords.map(_.value()) values.size should be >= 12 - values should contain allElementsOf Seq("msg_103", "msg_102", "msg_101", "msg_100", "msg_97", "msg_95", "msg_94", - "msg_92", "msg_90", "msg_88", "msg_86", "msg_84") + values should contain allElementsOf Seq("msg_103", "msg_102", "msg_101", "msg_100", "msg_99", "msg_97", "msg_95", + "msg_94", "msg_92", "msg_90", "msg_84") } it should "get from multiple topics simultaneously" in { @@ -359,6 +338,7 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, classOf[StringDeserializer].getCanonicalName) props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[StringDeserializer].getCanonicalName) props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") + props.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPollRecords.toString) new KafkaConsumer[String, String](props) } From 2ed580b119f39c0fa48f0aa56cbe246f87cefbff Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Thu, 17 Dec 2020 12:02:48 +0100 Subject: [PATCH 27/27] PR fixes --- .../KafkaToKafkaDeduplicationAfterRetryDockerTest.scala | 8 +++----- .../ingestor/implementation/utils/KafkaUtil.scala | 6 +++--- .../implementation/utils/TestKafkaUtilDockerTest.scala | 2 +- 3 files changed, 7 insertions(+), 9 deletions(-) diff --git a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala index 631eca0d..3bb97f31 100644 --- a/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala +++ b/driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala @@ -84,8 +84,7 @@ class KafkaToKafkaDeduplicationAfterRetryDockerTest extends FlatSpec with Matche valueFieldNames should contain theSameElementsAs List("record_id", "value_field", "hyperdrive_id") val actualRecordIds = records.map(_.value().get("record_id")) actualRecordIds.distinct.size shouldBe actualRecordIds.size - val expectedRecordIds = 0 until recordIdsV1.size + recordIdsV2.size - actualRecordIds should contain theSameElementsAs expectedRecordIds + actualRecordIds should contain theSameElementsAs recordIdsV1 ++ recordIdsV2 } it should "write duplicate entries without the deduplicate transformer" in { @@ -242,8 +241,7 @@ class KafkaToKafkaDeduplicationAfterRetryDockerTest extends FlatSpec with Matche val localKafkaAdmin = AdminClient.create(config) val replication = 1.toShort val topic = new NewTopic(topicName, partitions, replication) - val topicCreationFut = localKafkaAdmin.createTopics(util.Arrays.asList(topic)).all() - while(!topicCreationFut.isDone) {} + localKafkaAdmin.createTopics(util.Arrays.asList(topic)).all().get() } private def createProducer(kafkaSchemaRegistryWrapper: KafkaSchemaRegistryWrapper): KafkaProducer[GenericRecord, GenericRecord] = { @@ -271,6 +269,6 @@ class KafkaToKafkaDeduplicationAfterRetryDockerTest extends FlatSpec with Matche val topicPartitions = KafkaUtil.getTopicPartitions(consumer, topic) val offsets = consumer.endOffsets(topicPartitions.asJava) implicit val kafkaConsumerTimeout: Duration = Duration.ofSeconds(10L) - KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets.asScala.mapValues(_ + 0L).toMap) + KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets.asScala.mapValues(Long2long).toMap) } } diff --git a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala index c0a02618..528e6d11 100644 --- a/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala +++ b/ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala @@ -32,11 +32,11 @@ private[hyperdrive] object KafkaUtil { def getAtLeastNLatestRecordsFromPartition[K, V](consumer: KafkaConsumer[K, V], numberOfRecords: Map[TopicPartition, Long]) (implicit kafkaConsumerTimeout: Duration): Seq[ConsumerRecord[K, V]] = { consumer.assign(numberOfRecords.keySet.asJava) - val endOffsets = consumer.endOffsets(numberOfRecords.keySet.asJava).asScala + val endOffsets = consumer.endOffsets(numberOfRecords.keySet.asJava).asScala.mapValues(Long2long) val topicPartitions = endOffsets.keySet var records: Seq[ConsumerRecord[K, V]] = Seq() - val offsetLowerBounds = mutable.Map(endOffsets.mapValues(_ + 0L).toSeq: _*) + val offsetLowerBounds = mutable.Map(endOffsets.toSeq: _*) import scala.util.control.Breaks._ breakable { while (true) { @@ -54,7 +54,7 @@ private[hyperdrive] object KafkaUtil { offsetLowerBounds.foreach { case (partition, offset) => consumer.seek(partition, offset) } - records = getMessagesAtLeastToOffset(consumer, endOffsets.mapValues(_ + 0L).toMap) + records = getMessagesAtLeastToOffset(consumer, endOffsets.toMap) } } diff --git a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala index 60481457..94bf0845 100644 --- a/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala +++ b/ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala @@ -212,7 +212,7 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter values.size should be >= 12 values should contain allElementsOf Seq("msg_103", "msg_102", "msg_101", "msg_100", "msg_99", "msg_97", "msg_95", - "msg_94", "msg_92", "msg_90", "msg_84") + "msg_98", "msg_96", "msg_94", "msg_92", "msg_90") } it should "get from multiple topics simultaneously" in {