Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

support kafka with security certification #186

Merged
merged 1 commit into from
Dec 6, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -391,15 +391,10 @@ object Configs {
hiveEntryOpt = Option(hiveEntry)
}

var hasKafka = false

val tags = mutable.ListBuffer[TagConfigEntry]()
val tagConfigs = getConfigsOrNone(config, "tags")
if (tagConfigs.isDefined) {
for (tagConfig <- tagConfigs.get.asScala) {
if (hasKafka) {
throw new IllegalArgumentException("Can not define any other configs when kafka exists")
}
if (!tagConfig.hasPath("name") ||
!tagConfig.hasPath("type.source") ||
!tagConfig.hasPath("type.sink")) {
Expand Down Expand Up @@ -436,7 +431,6 @@ object Configs {
val sourceConfig =
dataSourceConfig(sourceCategory, tagConfig, nebulaConfig, variable, paths)
LOG.info(s"Source Config ${sourceConfig}")
hasKafka = sourceCategory == SourceCategory.KAFKA

val sinkCategory = toSinkCategory(tagConfig.getString("type.sink"))
val sinkConfig = dataSinkConfig(sinkCategory, nebulaConfig)
Expand Down Expand Up @@ -494,9 +488,6 @@ object Configs {
val edgeConfigs = getConfigsOrNone(config, "edges")
if (edgeConfigs.isDefined) {
for (edgeConfig <- edgeConfigs.get.asScala) {
if (hasKafka) {
throw new IllegalArgumentException("Can not define any other configs when kafka exists")
}
if (!edgeConfig.hasPath("name") ||
!edgeConfig.hasPath("type.source") ||
!edgeConfig.hasPath("type.sink")) {
Expand All @@ -520,7 +511,6 @@ object Configs {
val sourceConfig =
dataSourceConfig(sourceCategory, edgeConfig, nebulaConfig, variable, paths)
LOG.info(s"Source Config ${sourceConfig}")
hasKafka = sourceCategory == SourceCategory.KAFKA

val sinkCategory = toSinkCategory(edgeConfig.getString("type.sink"))
val sinkConfig = dataSinkConfig(sinkCategory, nebulaConfig)
Expand Down Expand Up @@ -863,12 +853,29 @@ object Configs {
val maxOffsetsPerTrigger =
if (config.hasPath("maxOffsetsPerTrigger")) Some(config.getLong("maxOffsetsPerTrigger"))
else None
KafkaSourceConfigEntry(SourceCategory.KAFKA,
intervalSeconds,
config.getString("service"),
config.getString("topic"),
startingOffsets,
maxOffsetsPerTrigger)

val securityProtocol =
if (config.hasPath("securityProtocol")) Some(config.getString("securityProtocol"))
else None
val mechanism =
if (config.hasPath("mechanism")) Some(config.getString("mechanism")) else None
val kerberos = if (config.hasPath("kerberos")) config.getBoolean("kerberos") else false
val kerberosServiceName =
if (config.hasPath("kerberosServiceName")) config.getString("kerberosServiceName")
else null

KafkaSourceConfigEntry(
SourceCategory.KAFKA,
intervalSeconds,
config.getString("service"),
config.getString("topic"),
startingOffsets,
maxOffsetsPerTrigger,
securityProtocol,
mechanism,
kerberos,
kerberosServiceName
)
case SourceCategory.PULSAR =>
val options =
config.getObject("options").unwrapped.asScala.map(x => x._1 -> x._2.toString).toMap
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -213,9 +213,17 @@ case class KafkaSourceConfigEntry(override val category: SourceCategory.Value,
server: String,
topic: String,
startingOffsets: String,
maxOffsetsPerTrigger: Option[Long] = None)
maxOffsetsPerTrigger: Option[Long] = None,
securityProtocol: Option[String] = None,
mechanism: Option[String] = None,
kerberos: Boolean = false,
kerberosServiceName: String = null)
extends StreamingDataSourceConfigEntry {
require(server.trim.nonEmpty && topic.trim.nonEmpty)
require(server.trim.nonEmpty && topic.trim.nonEmpty, "server or topic cannot be empty")
require(securityProtocol.isEmpty || mechanism.isDefined,
"security protocol is defined, mechanism must be config.")
require(!kerberos || kerberosServiceName.nonEmpty,
"kerberos is true, service name must be config")

override def toString: String = {
s"Kafka source server: ${server} topic:${topic} startingOffsets:${startingOffsets} maxOffsetsPerTrigger:${maxOffsetsPerTrigger}"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,14 @@ class KafkaReader(override val session: SparkSession,
.option("subscribe", kafkaConfig.topic)
.option("startingOffsets", kafkaConfig.startingOffsets)

if (kafkaConfig.securityProtocol.isDefined) {
reader.option("kafka.security.protocol", kafkaConfig.securityProtocol.get)
reader.option("kafka.sasl.mechanism", kafkaConfig.mechanism.get)
}
if (kafkaConfig.kerberos) {
reader.option("kafka.sasl.kerberos.service.name", kafkaConfig.kerberosServiceName)
}

val maxOffsetsPerTrigger = kafkaConfig.maxOffsetsPerTrigger
if (maxOffsetsPerTrigger.isDefined)
reader.option("maxOffsetsPerTrigger", maxOffsetsPerTrigger.get)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,8 @@ import com.vesoft.exchange.common.config.{KafkaSourceConfigEntry, PulsarSourceCo
import org.apache.spark.sql.types.StringType
import org.apache.spark.sql.{DataFrame, SparkSession}

import scala.collection.mutable

/**
* Spark Streaming
*
Expand Down Expand Up @@ -46,6 +48,13 @@ class KafkaReader(override val session: SparkSession,
.option("subscribe", kafkaConfig.topic)
.option("startingOffsets", kafkaConfig.startingOffsets)

if(kafkaConfig.securityProtocol.isDefined){
reader.option("kafka.security.protocol", kafkaConfig.securityProtocol.get)
reader.option("kafka.sasl.mechanism", kafkaConfig.mechanism.get)
}
if(kafkaConfig.kerberos){
reader.option("kafka.sasl.kerberos.service.name", kafkaConfig.kerberosServiceName)
}
val maxOffsetsPerTrigger = kafkaConfig.maxOffsetsPerTrigger
if (maxOffsetsPerTrigger.isDefined)
reader.option("maxOffsetsPerTrigger", maxOffsetsPerTrigger.get)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,14 @@ class KafkaReader(override val session: SparkSession,
.option("subscribe", kafkaConfig.topic)
.option("startingOffsets", kafkaConfig.startingOffsets)

if (kafkaConfig.securityProtocol.isDefined) {
reader.option("kafka.security.protocol", kafkaConfig.securityProtocol.get)
reader.option("kafka.sasl.mechanism", kafkaConfig.mechanism.get)
}
if (kafkaConfig.kerberos) {
reader.option("kafka.sasl.kerberos.service.name", kafkaConfig.kerberosServiceName)
}

val maxOffsetsPerTrigger = kafkaConfig.maxOffsetsPerTrigger
if (maxOffsetsPerTrigger.isDefined)
reader.option("maxOffsetsPerTrigger", maxOffsetsPerTrigger.get)
Expand Down
Loading