Skip to content

Commit eec8181

Browse files
Feature/241 deduplicate save memory 2 (#242)
* works * Fix test * Fix code style * Fix scalastyle
1 parent bab72cc commit eec8181

File tree

5 files changed

+96
-43
lines changed

5 files changed

+96
-43
lines changed

driver/src/test/scala/za/co/absa/hyperdrive/driver/drivers/KafkaToKafkaDeduplicationAfterRetryDockerTest.scala

Lines changed: 23 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -19,18 +19,18 @@ import java.time.Duration
1919
import java.util
2020
import java.util.UUID.randomUUID
2121
import java.util.{Collections, Properties}
22-
2322
import org.apache.avro.Schema.Parser
2423
import org.apache.avro.generic.{GenericData, GenericRecord}
2524
import org.apache.hadoop.fs.{FileSystem, Path}
2625
import org.apache.kafka.clients.admin.{AdminClient, AdminClientConfig, NewTopic}
27-
import org.apache.kafka.clients.consumer.KafkaConsumer
26+
import org.apache.kafka.clients.consumer.{ConsumerRecord, KafkaConsumer}
2827
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord}
2928
import org.scalatest.{BeforeAndAfter, FlatSpec, Matchers}
3029
import za.co.absa.abris.avro.read.confluent.SchemaManagerFactory
3130
import za.co.absa.commons.io.TempDirectory
3231
import za.co.absa.commons.spark.SparkTestBase
3332
import za.co.absa.abris.avro.registry.SchemaSubject
33+
import za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka.PrunedConsumerRecord
3434
import za.co.absa.hyperdrive.ingestor.implementation.utils.KafkaUtil
3535
import za.co.absa.hyperdrive.shared.exceptions.IngestionException
3636

@@ -61,6 +61,12 @@ class KafkaToKafkaDeduplicationAfterRetryDockerTest extends FlatSpec with Matche
6161

6262
private val fs = FileSystem.get(spark.sparkContext.hadoopConfiguration)
6363
private var baseDir: TempDirectory = _
64+
private val pruningFn = (r: ConsumerRecord[GenericRecord, GenericRecord]) => PrunedConsumerRecord(
65+
r.topic(),
66+
r.partition(),
67+
r.offset(),
68+
Seq(r.value().get("record_id"))
69+
)
6470

6571
behavior of "CommandLineIngestionDriver"
6672

@@ -79,10 +85,11 @@ class KafkaToKafkaDeduplicationAfterRetryDockerTest extends FlatSpec with Matche
7985
executeTestCase(deduplicatorConfig, recordIdsV1, recordIdsV2, kafkaSchemaRegistryWrapper, destinationTopic)
8086

8187
val consumer = createConsumer(kafkaSchemaRegistryWrapper)
82-
val records = getAllMessages(consumer, destinationTopic)
83-
val valueFieldNames = records.head.value().getSchema.getFields.asScala.map(_.name())
88+
val valueFieldNames = getValueSchema(consumer, destinationTopic).getFields.asScala.map(_.name())
89+
val consumer2 = createConsumer(kafkaSchemaRegistryWrapper)
90+
val records = getAllMessages(consumer2, destinationTopic, pruningFn)
91+
val actualRecordIds = records.flatMap(_.data.map(_.asInstanceOf[Int]))
8492
valueFieldNames should contain theSameElementsAs List("record_id", "value_field", "hyperdrive_id")
85-
val actualRecordIds = records.map(_.value().get("record_id"))
8693
actualRecordIds.distinct.size shouldBe actualRecordIds.size
8794
actualRecordIds should contain theSameElementsAs recordIdsV1 ++ recordIdsV2
8895
}
@@ -96,10 +103,11 @@ class KafkaToKafkaDeduplicationAfterRetryDockerTest extends FlatSpec with Matche
96103
executeTestCase(Map(), recordIdsV1, recordIdsV2, kafkaSchemaRegistryWrapper, destinationTopic)
97104

98105
val consumer = createConsumer(kafkaSchemaRegistryWrapper)
99-
val records = getAllMessages(consumer, destinationTopic)
100-
val valueFieldNames = records.head.value().getSchema.getFields.asScala.map(_.name())
106+
val valueFieldNames = getValueSchema(consumer, destinationTopic).getFields.asScala.map(_.name())
107+
val consumer2 = createConsumer(kafkaSchemaRegistryWrapper)
108+
val records = getAllMessages(consumer2, destinationTopic, pruningFn)
109+
val actualRecordIds = records.flatMap(_.data)
101110
valueFieldNames should contain theSameElementsAs List("record_id", "value_field", "hyperdrive_id")
102-
val actualRecordIds = records.map(_.value().get("record_id"))
103111
actualRecordIds.distinct.size should be < actualRecordIds.size
104112
}
105113

@@ -265,10 +273,15 @@ class KafkaToKafkaDeduplicationAfterRetryDockerTest extends FlatSpec with Matche
265273
kafkaSchemaRegistryWrapper.createConsumer(props)
266274
}
267275

268-
private def getAllMessages[K, V](consumer: KafkaConsumer[K, V], topic: String) = {
276+
private def getValueSchema(consumer: KafkaConsumer[GenericRecord, GenericRecord], topic: String) = {
277+
consumer.subscribe(Seq(topic).asJava)
278+
consumer.poll(Duration.ofSeconds(10L)).asScala.head.value().getSchema
279+
}
280+
281+
private def getAllMessages[K, V](consumer: KafkaConsumer[K, V], topic: String, pruningFn: ConsumerRecord[K, V] => PrunedConsumerRecord) = {
269282
val topicPartitions = KafkaUtil.getTopicPartitions(consumer, topic)
270283
val offsets = consumer.endOffsets(topicPartitions.asJava)
271284
implicit val kafkaConsumerTimeout: Duration = Duration.ofSeconds(10L)
272-
KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets.asScala.mapValues(Long2long).toMap)
285+
KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets.asScala.mapValues(Long2long).toMap, pruningFn)
273286
}
274287
}

ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/transformer/deduplicate/kafka/DeduplicateKafkaSinkTransformer.scala

Lines changed: 18 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -81,17 +81,17 @@ private[transformer] class DeduplicateKafkaSinkTransformer(
8181
logOffsets(latestOffsetsOpt)
8282

8383
val sourceRecords = latestOffsetsOpt.map(latestOffset => consumeAndClose(sourceConsumer,
84-
consumer => KafkaUtil.getMessagesAtLeastToOffset(consumer, latestOffset))).getOrElse(Seq())
85-
val sourceIds = sourceRecords.map(extractIdFieldsFromRecord(_, sourceIdColumnNames))
84+
consumer => KafkaUtil.getMessagesAtLeastToOffset(consumer, latestOffset, pruneRecord(sourceIdColumnNames)))).getOrElse(Seq())
85+
val sourceIds = sourceRecords.map(_.data)
8686

8787
val sinkConsumer = createConsumer(writerBrokers, writerExtraOptions, encoderSchemaRegistryConfig)
8888
val sinkTopicPartitions = KafkaUtil.getTopicPartitions(sinkConsumer, writerTopic)
8989
val recordsPerPartition = sinkTopicPartitions.map(p => p -> sourceRecords.size.toLong).toMap
9090
val latestSinkRecords = consumeAndClose(sinkConsumer, consumer =>
91-
KafkaUtil.getAtLeastNLatestRecordsFromPartition(consumer, recordsPerPartition))
91+
KafkaUtil.getAtLeastNLatestRecordsFromPartition(consumer, recordsPerPartition, pruneRecord(destinationIdColumnNames)))
9292
logConsumedSinkRecords(latestSinkRecords)
9393

94-
val publishedIds = latestSinkRecords.map(extractIdFieldsFromRecord(_, destinationIdColumnNames))
94+
val publishedIds = latestSinkRecords.map(_.data)
9595
val duplicatedIds = sourceIds.intersect(publishedIds)
9696
logDuplicatedIds(duplicatedIds)
9797
val duplicatedIdsLit = duplicatedIds.map(duplicatedId => struct(duplicatedId.map(lit): _*))
@@ -123,8 +123,8 @@ private[transformer] class DeduplicateKafkaSinkTransformer(
123123
logger.info(s"Reset source offsets by partition to { ${currentPositions} }")
124124
}
125125

126-
private def logConsumedSinkRecords(latestSinkRecords: Seq[ConsumerRecord[GenericRecord, GenericRecord]]): Unit = {
127-
val offsetsByPartition = latestSinkRecords.map(r => r.partition() -> r.offset())
126+
private def logConsumedSinkRecords(latestSinkRecords: Seq[PrunedConsumerRecord]): Unit = {
127+
val offsetsByPartition = latestSinkRecords.map(r => r.partition -> r.offset)
128128
.groupBy(_._1)
129129
.mapValues(_.map(_._2))
130130
.toSeq
@@ -138,11 +138,18 @@ private[transformer] class DeduplicateKafkaSinkTransformer(
138138
logger.info(s"Found ${duplicatedIds.size} duplicated ids. First three: ${duplicatedIds.take(3)}.")
139139
}
140140

141-
private def extractIdFieldsFromRecord(record: ConsumerRecord[GenericRecord, GenericRecord], idColumnNames: Seq[String]): Seq[Any] = {
142-
idColumnNames.map(idColumnName =>
143-
AvroUtil.getFromConsumerRecord(record, idColumnName)
144-
.getOrElse(throw new IllegalArgumentException(s"Could not find value for field $idColumnName"))
145-
)
141+
private def pruneRecord(idColumnNames: Seq[String]): ConsumerRecord[GenericRecord, GenericRecord] => PrunedConsumerRecord = {
142+
record: ConsumerRecord[GenericRecord, GenericRecord] =>
143+
val prunedPayload = idColumnNames.map(idColumnName =>
144+
AvroUtil.getFromConsumerRecord(record, idColumnName)
145+
.getOrElse(throw new IllegalArgumentException(s"Could not find value for field $idColumnName"))
146+
)
147+
PrunedConsumerRecord(
148+
record.topic(),
149+
record.partition(),
150+
record.offset(),
151+
prunedPayload
152+
)
146153
}
147154

148155
private def consumeAndClose[T](consumer: KafkaConsumer[GenericRecord, GenericRecord], consume: KafkaConsumer[GenericRecord, GenericRecord] => T) = {
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,23 @@
1+
/*
2+
* Copyright 2018 ABSA Group Limited
3+
*
4+
* Licensed under the Apache License, Version 2.0 (the "License");
5+
* you may not use this file except in compliance with the License.
6+
* You may obtain a copy of the License at
7+
* http://www.apache.org/licenses/LICENSE-2.0
8+
*
9+
* Unless required by applicable law or agreed to in writing, software
10+
* distributed under the License is distributed on an "AS IS" BASIS,
11+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12+
* See the License for the specific language governing permissions and
13+
* limitations under the License.
14+
*/
15+
16+
package za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka
17+
18+
case class PrunedConsumerRecord(
19+
topic: String,
20+
partition: Int,
21+
offset: Long,
22+
data: Seq[Any]
23+
)

ingestor-default/src/main/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/KafkaUtil.scala

Lines changed: 12 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -22,26 +22,28 @@ import org.apache.logging.log4j.LogManager
2222
import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog}
2323
import org.apache.spark.sql.kafka010.KafkaSourceOffsetProxy
2424
import za.co.absa.hyperdrive.compatibility.provider.CompatibleOffsetProvider
25+
import za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka.PrunedConsumerRecord
2526

2627
import scala.collection.JavaConverters._
2728
import scala.collection.mutable
2829

2930
private[hyperdrive] object KafkaUtil {
3031
private val logger = LogManager.getLogger
3132

32-
def getAtLeastNLatestRecordsFromPartition[K, V](consumer: KafkaConsumer[K, V], numberOfRecords: Map[TopicPartition, Long])
33-
(implicit kafkaConsumerTimeout: Duration): Seq[ConsumerRecord[K, V]] = {
33+
def getAtLeastNLatestRecordsFromPartition[K, V](consumer: KafkaConsumer[K, V], numberOfRecords: Map[TopicPartition, Long],
34+
pruningFn: ConsumerRecord[K, V] => PrunedConsumerRecord)
35+
(implicit kafkaConsumerTimeout: Duration): Seq[PrunedConsumerRecord] = {
3436
consumer.assign(numberOfRecords.keySet.asJava)
3537
val endOffsets = consumer.endOffsets(numberOfRecords.keySet.asJava).asScala.mapValues(Long2long)
3638
val topicPartitions = endOffsets.keySet
3739

38-
var records: Seq[ConsumerRecord[K, V]] = Seq()
40+
var records: Seq[PrunedConsumerRecord] = Seq()
3941
val offsetLowerBounds = mutable.Map(endOffsets.toSeq: _*)
4042
import scala.util.control.Breaks._
4143
breakable {
4244
while (true) {
4345
val recordSizes = records
44-
.groupBy(r => new TopicPartition(r.topic(), r.partition()))
46+
.groupBy(r => new TopicPartition(r.topic, r.partition))
4547
.mapValues(records => records.size)
4648
val unfinishedPartitions = topicPartitions.filter(p => recordSizes.getOrElse(p, 0) < numberOfRecords(p) && offsetLowerBounds(p) != 0)
4749
if (unfinishedPartitions.isEmpty) {
@@ -54,15 +56,16 @@ private[hyperdrive] object KafkaUtil {
5456
offsetLowerBounds.foreach {
5557
case (partition, offset) => consumer.seek(partition, offset)
5658
}
57-
records = getMessagesAtLeastToOffset(consumer, endOffsets.toMap)
59+
records = getMessagesAtLeastToOffset(consumer, endOffsets.toMap, pruningFn)
5860
}
5961
}
6062

6163
records
6264
}
6365

64-
def getMessagesAtLeastToOffset[K, V](consumer: KafkaConsumer[K, V], toOffsets: Map[TopicPartition, Long])
65-
(implicit kafkaConsumerTimeout: Duration): Seq[ConsumerRecord[K, V]] = {
66+
def getMessagesAtLeastToOffset[K, V](consumer: KafkaConsumer[K, V], toOffsets: Map[TopicPartition, Long],
67+
pruningFn: ConsumerRecord[K, V] => PrunedConsumerRecord)
68+
(implicit kafkaConsumerTimeout: Duration): Seq[PrunedConsumerRecord] = {
6669
consumer.assign(toOffsets.keySet.asJava)
6770
val endOffsets = consumer.endOffsets(toOffsets.keys.toSeq.asJava).asScala
6871
endOffsets.foreach { case (topicPartition, offset) =>
@@ -74,11 +77,11 @@ private[hyperdrive] object KafkaUtil {
7477
}
7578

7679
import scala.util.control.Breaks._
77-
var records: Seq[ConsumerRecord[K, V]] = mutable.Seq()
80+
var records: Seq[PrunedConsumerRecord] = mutable.Seq()
7881
breakable {
7982
while (true) {
8083
val newRecords = consumer.poll(kafkaConsumerTimeout).asScala.toSeq
81-
records ++= newRecords
84+
records ++= newRecords.map(pruningFn)
8285
if (newRecords.isEmpty || offsetsHaveBeenReached(consumer, toOffsets)) {
8386
break()
8487
}

ingestor-default/src/test/scala/za/co/absa/hyperdrive/ingestor/implementation/utils/TestKafkaUtilDockerTest.scala

Lines changed: 20 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,6 @@ import java.time.Duration
1919
import java.util
2020
import java.util.UUID.randomUUID
2121
import java.util.{Collections, Properties}
22-
2322
import org.apache.kafka.clients.admin.{AdminClient, AdminClientConfig, NewTopic}
2423
import org.apache.kafka.clients.consumer.{ConsumerRecord, KafkaConsumer}
2524
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord}
@@ -28,6 +27,7 @@ import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializ
2827
import org.scalatest.{AppendedClues, BeforeAndAfter, FlatSpec, Matchers}
2928
import org.testcontainers.containers.KafkaContainer
3029
import org.testcontainers.utility.DockerImageName
30+
import za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.kafka.PrunedConsumerRecord
3131

3232
import scala.collection.JavaConverters._
3333
import scala.collection.mutable
@@ -39,6 +39,12 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter
3939
private val kafkaInsufficientTimeout = Duration.ofMillis(1L)
4040
private val topic = "test-topic"
4141
private val maxPollRecords = 10
42+
private val pruningFn = (r: ConsumerRecord[String, String]) => PrunedConsumerRecord(
43+
r.topic(),
44+
r.partition(),
45+
r.offset(),
46+
Seq(r.value())
47+
)
4248

4349
before{
4450
kafka.start()
@@ -62,10 +68,10 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter
6268

6369
// when
6470
implicit val kafkaConsumerTimeout: Duration = kafkaSufficientTimeout
65-
val records = KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets)
71+
val records = KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets, pruningFn)
6672

6773
// then
68-
val actualMessages = records.map(_.value()).toList.sorted
74+
val actualMessages = records.map(_.data.head.asInstanceOf[String]).toList.sorted
6975
actualMessages should contain theSameElementsAs messages
7076
}
7177

@@ -99,10 +105,10 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter
99105

100106
// when
101107
implicit val kafkaConsumerTimeout: Duration = kafkaSufficientTimeout
102-
val records = KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets)
108+
val records = KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets, pruningFn)
103109

104110
// then
105-
val actualMessages = records.map(_.value()).toList.sorted
111+
val actualMessages = records.map(_.data.head.asInstanceOf[String]).toList.sorted
106112
actualMessages should contain allElementsOf messages
107113

108114
// cleanup
@@ -118,7 +124,7 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter
118124

119125
// when
120126
implicit val kafkaConsumerTimeout: Duration = kafkaInsufficientTimeout
121-
val exception = the[Exception] thrownBy KafkaUtil.getMessagesAtLeastToOffset(consumer, Map(new TopicPartition(topic, 0) -> 0))
127+
val exception = the[Exception] thrownBy KafkaUtil.getMessagesAtLeastToOffset(consumer, Map(new TopicPartition(topic, 0) -> 0), pruningFn)
122128

123129
// then
124130
exception.getMessage should include ("Subscription to topics, partitions and pattern are mutually exclusive")
@@ -140,7 +146,7 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter
140146

141147
// when
142148
implicit val kafkaConsumerTimeout: Duration = kafkaInsufficientTimeout
143-
val exception = the[Exception] thrownBy KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets)
149+
val exception = the[Exception] thrownBy KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets, pruningFn)
144150

145151
// then
146152
exception.getMessage should include ("Not all expected messages were consumed")
@@ -160,7 +166,7 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter
160166

161167
// when
162168
implicit val kafkaConsumerTimeout: Duration = kafkaInsufficientTimeout
163-
val exception = the[Exception] thrownBy KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets)
169+
val exception = the[Exception] thrownBy KafkaUtil.getMessagesAtLeastToOffset(consumer, offsets, pruningFn)
164170

165171
// then
166172
exception.getMessage should include ("Requested consumption")
@@ -209,8 +215,8 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter
209215
implicit val kafkaConsumerTimeout: Duration = kafkaSufficientTimeout
210216
val topicPartitions = KafkaUtil.getTopicPartitions(consumer, topic)
211217
val recordsPerPartition = topicPartitions.map(p => p -> 4L).toMap
212-
val actualRecords = KafkaUtil.getAtLeastNLatestRecordsFromPartition(consumer, recordsPerPartition)
213-
val values = actualRecords.map(_.value())
218+
val actualRecords = KafkaUtil.getAtLeastNLatestRecordsFromPartition(consumer, recordsPerPartition, pruningFn)
219+
val values = actualRecords.map(_.data.head.asInstanceOf[String])
214220

215221
values.size should be >= 12
216222
values should contain allElementsOf Seq("msg_103", "msg_102", "msg_101", "msg_100", "msg_99", "msg_97", "msg_95",
@@ -231,10 +237,10 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter
231237
// when
232238
implicit val kafkaConsumerTimeout: Duration = kafkaSufficientTimeout
233239
val recordsPerPartition = topicPartitions.map(t => t -> 1000L).toMap
234-
val records = KafkaUtil.getAtLeastNLatestRecordsFromPartition(consumer, recordsPerPartition)
240+
val records = KafkaUtil.getAtLeastNLatestRecordsFromPartition(consumer, recordsPerPartition, pruningFn)
235241

236242
// then
237-
val actualMessages = records.map(_.value()).toList.sorted
243+
val actualMessages = records.map(_.data.head.asInstanceOf[String]).toList.sorted
238244
actualMessages should contain theSameElementsAs messages
239245
}
240246

@@ -248,7 +254,8 @@ class TestKafkaUtilDockerTest extends FlatSpec with Matchers with BeforeAndAfter
248254

249255
val consumer = createConsumer(kafka)
250256
implicit val kafkaConsumerTimeout: Duration = kafkaInsufficientTimeout
251-
val result = the[Exception] thrownBy KafkaUtil.getAtLeastNLatestRecordsFromPartition(consumer, Map(new TopicPartition(topic, 0) -> 10))
257+
val result = the[Exception] thrownBy KafkaUtil.getAtLeastNLatestRecordsFromPartition(consumer,
258+
Map(new TopicPartition(topic, 0) -> 10), pruningFn)
252259
result.getMessage should include("increasing the consumer timeout")
253260
}
254261

0 commit comments

Comments
 (0)