/
KafkaContinuousWriter.scala
119 lines (103 loc) · 4.92 KB
/
KafkaContinuousWriter.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.kafka010
import org.apache.kafka.clients.producer.{Callback, ProducerRecord, RecordMetadata}
import scala.collection.JavaConverters._
import org.apache.spark.internal.Logging
import org.apache.spark.sql.{Row, SparkSession}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal, UnsafeProjection}
import org.apache.spark.sql.kafka010.KafkaSourceProvider.{kafkaParamsForProducer, TOPIC_OPTION_KEY}
import org.apache.spark.sql.kafka010.KafkaWriter.validateQuery
import org.apache.spark.sql.sources.v2.streaming.writer.ContinuousWriter
import org.apache.spark.sql.sources.v2.writer._
import org.apache.spark.sql.streaming.OutputMode
import org.apache.spark.sql.types.{BinaryType, StringType, StructType}
/**
* Dummy commit message. The DataSourceV2 framework requires a commit message implementation but we
* don't need to really send one.
*/
case object KafkaWriterCommitMessage extends WriterCommitMessage
/**
* A [[ContinuousWriter]] for Kafka writing. Responsible for generating the writer factory.
* @param topic The topic this writer is responsible for. If None, topic will be inferred from
* a `topic` field in the incoming data.
* @param producerParams Parameters for Kafka producers in each task.
* @param schema The schema of the input data.
*/
class KafkaContinuousWriter(
topic: Option[String], producerParams: Map[String, String], schema: StructType)
extends ContinuousWriter with SupportsWriteInternalRow {
validateQuery(schema.toAttributes, producerParams.toMap[String, Object].asJava, topic)
override def createInternalRowWriterFactory(): KafkaContinuousWriterFactory =
KafkaContinuousWriterFactory(topic, producerParams, schema)
override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {}
override def abort(messages: Array[WriterCommitMessage]): Unit = {}
}
/**
* A [[DataWriterFactory]] for Kafka writing. Will be serialized and sent to executors to generate
* the per-task data writers.
* @param topic The topic that should be written to. If None, topic will be inferred from
* a `topic` field in the incoming data.
* @param producerParams Parameters for Kafka producers in each task.
* @param schema The schema of the input data.
*/
case class KafkaContinuousWriterFactory(
topic: Option[String], producerParams: Map[String, String], schema: StructType)
extends DataWriterFactory[InternalRow] {
override def createDataWriter(partitionId: Int, attemptNumber: Int): DataWriter[InternalRow] = {
new KafkaContinuousDataWriter(topic, producerParams, schema.toAttributes)
}
}
/**
* A [[DataWriter]] for Kafka writing. One data writer will be created in each partition to
* process incoming rows.
*
* @param targetTopic The topic that this data writer is targeting. If None, topic will be inferred
* from a `topic` field in the incoming data.
* @param producerParams Parameters to use for the Kafka producer.
* @param inputSchema The attributes in the input data.
*/
class KafkaContinuousDataWriter(
targetTopic: Option[String], producerParams: Map[String, String], inputSchema: Seq[Attribute])
extends KafkaRowWriter(inputSchema, targetTopic) with DataWriter[InternalRow] {
import scala.collection.JavaConverters._
private lazy val producer = CachedKafkaProducer.getOrCreate(
new java.util.HashMap[String, Object](producerParams.asJava))
def write(row: InternalRow): Unit = {
checkForErrors()
sendRow(row, producer)
}
def commit(): WriterCommitMessage = {
// Send is asynchronous, but we can't commit until all rows are actually in Kafka.
// This requires flushing and then checking that no callbacks produced errors.
// We also check for errors before to fail as soon as possible - the check is cheap.
checkForErrors()
producer.flush()
checkForErrors()
KafkaWriterCommitMessage
}
def abort(): Unit = {}
def close(): Unit = {
checkForErrors()
if (producer != null) {
producer.flush()
checkForErrors()
CachedKafkaProducer.close(new java.util.HashMap[String, Object](producerParams.asJava))
}
}
}