Structured Streaming消费Kafka时并不会将Offset提交到Kafka集群。但可以通过以下3种方式间接实现对Kafka Topic Lag的监控。

方式一: Checkpoint

先看下Checkpoint目录的结构:

checkpoint/
├── commits
│   ├── 0
│   ├── 1
│   ├── 2
│   └── 3
├── metadata
├── offsets
│   ├── 0
│   ├── 1
│   ├── 2
│   └── 3
└── sources
    ├── 0
    │   └── 0
    └── 1
        └── 0
  • metadata文件: 记录了Query ID。Query从Checkpoint恢复后,此id不变。内容示例:
{"id":"b33d6d50-fb5e-4569-a3ba-7a1ca5884f14"}
  • soures目录: 记录了每一个Source初始化时的一些信息。假设是Kafka Source,这里记录了该Source初始化时的topic、partition、offset。目录结构: checkpoint/sources/$sourceID/$batchID。内容示例:
// 第一个Source
cat checkpoint/sources/0/0
v1
{"topic_1":{"2":17599,"1":17602,"0":17554},"test_2":{"0":453972}}

// 第二个Source
cat checkpoint/sources/1/0
v1
{"test_3":{"2":34,"1":32,"0":31}}
  • offsets目录: 以batchID为文件名,记录了每个Batch相关的信息。如当前batch处理的offset; 水印配置等。目录结构: checkpoint/offsets/$batchID。内容示例:
// 第二个batch
cat checkpoint/offsets/2
v1
{"batchWatermarkMs":0,"batchTimestampMs":1585488752000,"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"}}
{"topic_1":{"2":17600,"1":17602,"0":17555},"test_2":{"0":453972}}
{"test_3":{"2":34,"1":32,"0":31}}

// 第三个batch
cat checkpoint/offsets/3
v1
{"batchWatermarkMs":0,"batchTimestampMs":1585488757000,"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"}}
{"topic_1":{"2":17600,"1":17602,"0":17555},"test_2":{"0":453973}}
{"test_3":{"2":34,"1":32,"0":32}}
  • commits目录: 记录已成功完成的batch,每完成一个batch,则创建一个以batchID为文件名的文件。目录结构: checkpoint/commits/$batchID。一个batch开始时,会在checkpoint/offsets目录中记录一个batchID文件,当这个batch完成后,会在checkpoint/commits目录中再记录一个batchID文件,表明这个batch已正常处理。
  • state目录: 记录状态。当有状态操作时,如累加聚合、去重、最大最小等场景,这个目录会被用来记录这些状态数据。目录结构:checkpoint/state/xxx.deltacheckpoint/state/xxx.snapshot。新的.snapshot是老的.snapshot.delta合并生成的文件。Structured Streaming会根据配置周期性地生成.snapshot文件用于记录状态。

从Kafka消费数据,当开启Checkpoint后,Structured Streaming会将消费进度记录到Checkpoint目录中。因此,结合commits目录offsets目录可实现对Kafka消费进度的监控。

  1. commits中获取最新已完成的batchID
  2. 根据batchIDoffsets查询这个batch消费的offset。再获取Topic最新的Offset,即可实现对Lag的监控。

注意:

  1. 每一个Kafka Source,在初始化时,都会生成一个唯一的groupId(参考KafkaSourceProvider#createSource方法,val uniqueGroupId = s"spark-kafka-source-${UUID.randomUUID}-${metadataPath.hashCode}"),供内部使用。该groupId在Query运行周期内不变,从Checkpoint恢复后会变化。
  2. 即使用各种方式,将offset提交到了如Kafka集群,从Checkpoint恢复时,默认使用的是Checkpoint里的Offset。
  3. 从Checkpoint恢复后,batchID编号会接着之前的增加。Checkpoint中保存最近100个已成功的batch状态。

方式二: StreamingQuery API

在Structured Streaming中,可以通过StreamingQuery API来管理和监控工作流。简单示例如下:

val query: StreamingQuery =resultTable
    .writeStream
    .format("console")
    .option("truncate","false")
    .outputMode("append")
    .trigger(Trigger.ProcessingTime("2 seconds"))
    .queryName("WordCount")
    .option("checkpointLocation", "/Users/wangpei/data/apps/word_count/checkpoint")
    .start()

while (true){
    println("Query Name: "+query.name)
    println("Query ID: "+query.id)
    println("Query RunID: "+query.runId)
    println("Query IsActive: "+query.isActive)
    println("Query Status: "+query.status)
    println("Query LastProgress: "+query.lastProgress)
    Thread.sleep(10 * 1000)
}

StreamingQuery API含义:

API

含义

备注

query

StreamingQuery实例

一个...start()...对应一个query,即一个StreamingQuery实例

query.name

Query的名称

可通过如...queryName("WordCount")...start()设置

query.id

Query的唯一ID

Query从Checkpoint恢复后,此id不变

query.runId

Query运行时的唯一ID

Query从Checkpoint恢复后,runId会变。但运行后,在Query运行周期内, 此id不变

query.isActive

Query当前是否活跃

query.status

Query当前状态

如当前Query正在做什么事情、是否有新数据要处理、触发器在激活中还是在等待下次被激活

query.explain

打印出Query物理执行计划

query.exception

如果查询被异常终止,则返回异常信息

query.stop

停止正在运行的Query

query.awaitTermination

阻塞主线程

当调用stop方法时停止或遇到异常时停止

query.recentProgress

以数组的形式返回最近几次查询的进度

Query最近几次查询进度的数量由spark.sql.streaming.numRecentProgressUpdates参数控制

query.lastProgress

最近一次查询的进度

借助StreamingQuery API,可从lastProgress中获取最近一次查询的进度,进度中包含了最近一次消费Kafka的Offset,将此Offset提交到kafka集群,然后通过监控平台即可实现监控。

方式三: StreamingQueryListener

StreamingQueryListener,即监听StreamingQuery各种事件的接口,如下:

abstract class StreamingQueryListener {

  import StreamingQueryListener._

  // 查询开始时调用
  def onQueryStarted(event: QueryStartedEvent): Unit

  // 查询过程中状态发生更新时调用
  def onQueryProgress(event: QueryProgressEvent): Unit

  // 查询结束时调用
  def onQueryTerminated(event: QueryTerminatedEvent): Unit
}

在QueryProgressEvent中,我们是可以拿到每个Source消费的Offset的。因此,基于StreamingQueryListener,可以将消费的offset的提交到kafka集群,进而实现对Kafka Lag的监控。

基于StreamingQueryListener向Kafka提交Offset

监控Kafka Lag的关键是能够向Kafka集群提交消费的Offset,以下示例演示了如何通过StreamingQueryListener向Kafka提交Offset。

KafkaOffsetCommiter

package com.bigdata.structured.streaming.monitor

import java.util
import java.util.Properties

import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper}
import com.fasterxml.jackson.module.scala.DefaultScalaModule
import org.apache.kafka.clients.consumer.OffsetAndMetadata
import org.apache.kafka.common.TopicPartition

import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer}
import org.apache.spark.sql.streaming.StreamingQueryListener
import org.apache.spark.sql.streaming.StreamingQueryListener._
import org.slf4j.LoggerFactory

/**
  * Author: Wang Pei
  * Summary:
  *   向Kafka集群提交Offset的Listener
  */


class KafkaOffsetCommiter(brokers: String, group: String) extends StreamingQueryListener {

  val logger = LoggerFactory.getLogger(this.getClass)

  // Kafka配置
  val properties= new Properties()
  properties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokers)
  properties.put(ConsumerConfig.GROUP_ID_CONFIG, group)
  properties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer")
  properties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer")
  val kafkaConsumer = new KafkaConsumer[String, String](properties)

  def onQueryStarted(event: QueryStartedEvent): Unit = {}

  def onQueryTerminated(event: QueryTerminatedEvent): Unit = {}

  // 提交Offset
  def onQueryProgress(event: QueryProgressEvent): Unit = {

    // 遍历所有Source
    event.progress.sources.foreach(source=>{

      val objectMapper = new ObjectMapper()
        .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false)
        .configure(DeserializationFeature.USE_LONG_FOR_INTS, true)
        .registerModule(DefaultScalaModule)

      val endOffset = objectMapper.readValue(source.endOffset,classOf[Map[String, Map[String, Long]]])

      // 遍历Source中的每个Topic
      for((topic,topicEndOffset) <- endOffset){
        val topicPartitionsOffset = new util.HashMap[TopicPartition, OffsetAndMetadata]()

        //遍历Topic中的每个Partition
        for ((partition,offset) <- topicEndOffset) {
          val topicPartition = new TopicPartition(topic, partition.toInt)
          val offsetAndMetadata = new OffsetAndMetadata(offset)
          topicPartitionsOffset.put(topicPartition,offsetAndMetadata)
        }

        logger.warn(s"提交偏移量... Topic: $topic Group: $group Offset: $topicEndOffset")
        kafkaConsumer.commitSync(topicPartitionsOffset)
      }
    })
  }
}

Structured Streaming App

package com.bigdata.structured.streaming.monitor

import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.streaming.{StreamingQuery, Trigger}

/**
  * Author: Wang Pei
  * Summary:
  *   读取Kafka数据
  */
object ReadKafkaApp {
  def main(args: Array[String]): Unit = {

    val kafkaBrokers="kafka01:9092,kafka02:9092,kafka03:9092"
    val kafkaGroup="read_kafka_c2"
    val kafkaTopics1="topic_1,test_2"
    val kafkaTopics2="test_3"
    val checkpointDir="/Users/wangpei/data/apps/read_kafka/checkpoint/"
    val queryName="read_kafka"

    val spark = SparkSession.builder().master("local[3]").appName(this.getClass.getSimpleName.replace("$","")).getOrCreate()
    import spark.implicits._

    // 添加监听器
    val kafkaOffsetCommiter = new KafkaOffsetCommiter(kafkaBrokers,kafkaGroup)
    spark.streams.addListener(kafkaOffsetCommiter)

    // Kafka数据源1
    val inputTable1=spark
      .readStream
      .format("kafka")
      .option("kafka.bootstrap.servers",kafkaBrokers )
      .option("subscribe",kafkaTopics1)
      .load()
      .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
      .as[(String, String)]
      .select($"value")

    // Kafka数据源2
    val inputTable2=spark
      .readStream
      .format("kafka")
      .option("kafka.bootstrap.servers",kafkaBrokers )
      .option("subscribe",kafkaTopics2)
      .load()
      .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
      .as[(String, String)]
      .select($"value")

    // 结果表
    val resultTable = inputTable1.union(inputTable2)

    // 启动Query
    val query: StreamingQuery =resultTable
      .writeStream
      .format("console")
      .option("truncate","false")
      .outputMode("append")
      .trigger(Trigger.ProcessingTime("2 seconds"))
      .queryName(queryName)
      .option("checkpointLocation", checkpointDir)
      .start()

    spark.streams.awaitAnyTermination()

  }
}

查看Kafka Offset

kafka是1.10版本的, 可通过以下命令查看Topic消费者组对应的Offset。

bin/kafka-consumer-offset-checker.sh --zookeeper kafka01:2181  --topic test_3 --group read_kafka_c2
Group           Topic                          Pid Offset          logSize         Lag             Owner
read_kafka_c2   test_3                         0   32              32              0               none
read_kafka_c2   test_3                         1   32              32              0               none
read_kafka_c2   test_3                         2   34              34              0               none

同理,可查看另外两个Topic对应的Group的Offset。