Skip to content

Commit 28b8d81

Browse files
Feature/212 improve logging (#213)
* Improve logging * Refactoring
1 parent 9b33301 commit 28b8d81

File tree

2 files changed

+44
-4
lines changed

2 files changed

+44
-4
lines changed

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

Lines changed: 44 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -17,15 +17,14 @@ package za.co.absa.hyperdrive.ingestor.implementation.transformer.deduplicate.ka
1717

1818
import java.time.Duration
1919
import java.util.{Properties, UUID}
20-
2120
import za.co.absa.hyperdrive.ingestor.api.utils.ConfigUtils
2221
import za.co.absa.hyperdrive.ingestor.implementation.transformer.avro.confluent.{ConfluentAvroDecodingTransformer, ConfluentAvroEncodingTransformer}
2322
import za.co.absa.hyperdrive.ingestor.implementation.utils.KafkaUtil
24-
2523
import org.apache.avro.generic.GenericRecord
2624
import org.apache.commons.configuration2.Configuration
2725
import org.apache.hadoop.fs.Path
2826
import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
27+
import org.apache.kafka.common.TopicPartition
2928
import org.apache.logging.log4j.LogManager
3029
import org.apache.spark.sql.DataFrame
3130
import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog}
@@ -74,9 +73,14 @@ private[transformer] class DeduplicateKafkaSinkTransformer(
7473
implicit val kafkaConsumerTimeoutImpl: Duration = kafkaConsumerTimeout
7574
val sourceConsumer = createConsumer(readerBrokers, readerExtraOptions, readerSchemaRegistryUrl)
7675
val latestCommittedOffsets = KafkaUtil.getLatestCommittedOffset(offsetLog, commitLog)
76+
logCommittedOffsets(latestCommittedOffsets)
77+
7778
KafkaUtil.seekToOffsetsOrBeginning(sourceConsumer, readerTopic, latestCommittedOffsets)
79+
logCurrentPositions(sourceConsumer)
7880

7981
val latestOffsetsOpt = KafkaUtil.getLatestOffset(offsetLog)
82+
logOffsets(latestOffsetsOpt)
83+
8084
val sourceRecords = latestOffsetsOpt.map(latestOffset => consumeAndClose(sourceConsumer,
8185
consumer => KafkaUtil.getMessagesAtLeastToOffset(consumer, latestOffset))).getOrElse(Seq())
8286
val sourceIds = sourceRecords.map(extractIdFieldsFromRecord(_, sourceIdColumnNames))
@@ -86,14 +90,51 @@ private[transformer] class DeduplicateKafkaSinkTransformer(
8690
val recordsPerPartition = sinkTopicPartitions.map(p => p -> sourceRecords.size.toLong).toMap
8791
val latestSinkRecords = consumeAndClose(sinkConsumer, consumer =>
8892
KafkaUtil.getAtLeastNLatestRecordsFromPartition(consumer, recordsPerPartition))
89-
val publishedIds = latestSinkRecords.map(extractIdFieldsFromRecord(_, destinationIdColumnNames))
93+
logConsumedSinkRecords(latestSinkRecords)
9094

95+
val publishedIds = latestSinkRecords.map(extractIdFieldsFromRecord(_, destinationIdColumnNames))
9196
val duplicatedIds = sourceIds.intersect(publishedIds)
97+
logDuplicatedIds(duplicatedIds)
9298
val duplicatedIdsLit = duplicatedIds.map(duplicatedId => struct(duplicatedId.map(lit): _*))
9399
val idColumns = sourceIdColumnNames.map(col)
94100
dataFrame.filter(not(struct(idColumns: _*).isInCollection(duplicatedIdsLit)))
95101
}
96102

103+
private def logCommittedOffsets(offsets: Option[Map[TopicPartition, Long]]): Unit =
104+
logger.info(s"Latest committed source offsets by partition for ${readerTopic}: { ${offsetsToString(offsets)} }" )
105+
106+
private def logOffsets(offsets: Option[Map[TopicPartition, Long]]): Unit =
107+
logger.info(s"Latest source offsets by partition for ${readerTopic}: { ${offsetsToString(offsets)} }" )
108+
109+
private def offsetsToString(offsets: Option[Map[TopicPartition, Long]]) = {
110+
offsets.map(_.toSeq
111+
.sortBy{ case (tp, _) => tp.partition()}
112+
.map{ case (tp, offset) => s"${tp.partition()}: $offset"}.reduce(_ + ", " + _)).getOrElse("-")
113+
}
114+
115+
private def logCurrentPositions(consumer: KafkaConsumer[GenericRecord, GenericRecord]): Unit = {
116+
val sourcePartitions = KafkaUtil.getTopicPartitions(consumer, readerTopic)
117+
val currentPositions = sourcePartitions
118+
.sortBy(_.partition())
119+
.map { tp => s"${tp.partition()}: ${consumer.position(tp)}"}.reduce(_ + ", " + _)
120+
logger.info(s"Reset source offsets by partition to { ${currentPositions} }")
121+
}
122+
123+
private def logConsumedSinkRecords(latestSinkRecords: Seq[ConsumerRecord[GenericRecord, GenericRecord]]): Unit = {
124+
val offsetsByPartition = latestSinkRecords.map(r => r.partition() -> r.offset())
125+
.groupBy(_._1)
126+
.mapValues(_.map(_._2))
127+
.toSeq
128+
.sortBy(_._1)
129+
val firstOffsets = offsetsByPartition.map { case (partition, offsets) => s"$partition: ${offsets.take(3)}"}.reduce(_ + ", " + _)
130+
val lastOffsets = offsetsByPartition.map { case (partition, offsets) => s"$partition: ${offsets.takeRight(3)}"}.reduce(_ + ", " + _)
131+
logger.info(s"Consumed ${latestSinkRecords.size} sink records. First three offsets by partition: { ${firstOffsets} }. Last three offsets: { ${lastOffsets} }")
132+
}
133+
134+
private def logDuplicatedIds(duplicatedIds: Seq[Seq[Any]]): Unit = {
135+
logger.info(s"Found ${duplicatedIds.size} duplicated ids. First three: ${duplicatedIds.take(3)}.")
136+
}
137+
97138
private def extractIdFieldsFromRecord(record: ConsumerRecord[GenericRecord, GenericRecord], idColumnNames: Seq[String]): Seq[Any] = {
98139
idColumnNames.map(idColumnName =>
99140
AvroUtil.getFromConsumerRecord(record, idColumnName)

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

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -99,7 +99,6 @@ private[hyperdrive] object KafkaUtil {
9999
private def offsetsHaveBeenReached[K, V](consumer: KafkaConsumer[K, V], toOffsets: Map[TopicPartition, Long]) = {
100100
toOffsets.forall { case (tp, toOffset) =>
101101
val position = consumer.position(tp)
102-
logger.info(s"Reached position $position on topic partition $tp. Target offset is $toOffset")
103102
position >= toOffset
104103
}
105104
}

0 commit comments

Comments
 (0)