Github user tdas commented on a diff in the pull request:
https://github.com/apache/spark/pull/3798#discussion_r23728286
--- Diff:
external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
---
@@ -144,4 +150,116 @@ object KafkaUtils {
createStream[K, V, U, T](
jssc.ssc, kafkaParams.toMap,
Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel)
}
+
+ /** A batch-oriented interface for consuming from Kafka.
+ * Starting and ending offsets are specified in advance,
+ * so that you can control exactly-once semantics.
+ * @param sc SparkContext object
+ * @param kafkaParams Kafka <a
href="http://kafka.apache.org/documentation.html#configuration">
+ * configuration parameters</a>.
+ * Requires "metadata.broker.list" or "bootstrap.servers" to be set
with Kafka broker(s),
+ * NOT zookeeper servers, specified in host1:port1,host2:port2 form.
+ * @param batch Each OffsetRange in the batch corresponds to a
+ * range of offsets for a given Kafka topic/partition
+ * @param messageHandler function for translating each message into the
desired type
+ */
+ def createRDD[
+ K: ClassTag,
+ V: ClassTag,
+ U <: Decoder[_]: ClassTag,
+ T <: Decoder[_]: ClassTag,
+ R: ClassTag] (
+ sc: SparkContext,
+ kafkaParams: Map[String, String],
+ batch: Array[OffsetRange],
+ messageHandler: MessageAndMetadata[K, V] => R
+ ): RDD[R] = {
+ val parts = batch.zipWithIndex.map { case (o, i) =>
+ new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset,
o.untilOffset, o.host, o.port)
+ }.toArray
+ new KafkaRDD[K, V, U, T, R](sc, kafkaParams, parts, messageHandler)
+ }
+
+ /**
+ * This DOES NOT guarantee that side-effects of an action will see each
message exactly once.
+ * If you need that guarantee, get the offsets from this stream and
store them with your output.
+ * Nor does this store offsets in Kafka / Zookeeper.
+ * If checkpointed, it will store offset ranges in the checkpoint, such
that each message
+ * will be transformed effectively exactly once even after failure,
+ * provided you have sufficient Kafka log retention.
+ *
+ * @param ssc StreamingContext object
+ * @param kafkaParams Kafka <a
href="http://kafka.apache.org/documentation.html#configuration">
+ * configuration parameters</a>.
+ * Requires "metadata.broker.list" or "bootstrap.servers" to be set
with Kafka broker(s),
+ * NOT zookeeper servers, specified in host1:port1,host2:port2 form.
+ * @param messageHandler function for translating each message into the
desired type
+ * @param fromOffsets per-topic/partition Kafka offsets defining the
(inclusive)
+ * starting point of the stream
+ * @param maxRetries maximum number of times in a row to retry getting
leaders' offsets
+ */
+ def createExactlyOnceStream[
+ K: ClassTag,
+ V: ClassTag,
+ U <: Decoder[_]: ClassTag,
+ T <: Decoder[_]: ClassTag,
+ R: ClassTag] (
+ ssc: StreamingContext,
+ kafkaParams: Map[String, String],
+ fromOffsets: Map[TopicAndPartition, Long],
+ messageHandler: MessageAndMetadata[K, V] => R,
+ maxRetries: Int
+ ): InputDStream[R] = {
+ new DeterministicKafkaInputDStream[K, V, U, T, R](
+ ssc, kafkaParams, fromOffsets, messageHandler, maxRetries)
+ }
+
+ /**
+ * This DOES NOT guarantee that side-effects of an action will see each
message exactly once.
+ * If you need that guarantee, get the offsets from this stream and
store them with your output.
+ * Nor does this store offsets in Kafka / Zookeeper.
+ * If checkpointed, it will store offset ranges in the checkpoint, such
that each message
+ * will be transformed effectively exactly once even after failure,
+ * provided you have sufficient Kafka log retention.
+ *
+ * @param ssc StreamingContext object
+ * @param kafkaParams Kafka <a
href="http://kafka.apache.org/documentation.html#configuration">
+ * configuration parameters</a>.
+ * Requires "metadata.broker.list" or "bootstrap.servers" to be set
with Kafka broker(s),
+ * NOT zookeeper servers, specified in host1:port1,host2:port2 form.
+ * If starting without a checkpoint, "auto.offset.reset" may be set to
"largest" or "smallest"
+ * to determine where the stream starts (defaults to "largest")
+ * @param topics names of the topics to consume
+ */
+ def createExactlyOnceStream[
+ K: ClassTag,
+ V: ClassTag,
+ U <: Decoder[_]: ClassTag,
+ T <: Decoder[_]: ClassTag] (
+ ssc: StreamingContext,
+ kafkaParams: Map[String, String],
+ topics: Set[String]
+ ): InputDStream[(K, V)] = {
+ val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key,
mmd.message)
+ val kc = new KafkaCluster(kafkaParams)
+ val reset = kafkaParams.get("auto.offset.reset").map(_.toLowerCase)
+
+ (for {
+ topicPartitions <- kc.getPartitions(topics).right
+ leaderOffsets <- (if (reset == Some("smallest")) {
+ kc.getEarliestLeaderOffsets(topicPartitions)
+ } else {
+ kc.getLatestLeaderOffsets(topicPartitions)
+ }).right
+ } yield {
+ val fromOffsets = leaderOffsets.map { case (tp, lo) =>
+ (tp, lo.offset)
+ }
+ new DeterministicKafkaInputDStream[K, V, U, T, (K, V)](
+ ssc, kafkaParams, fromOffsets, messageHandler, 1)
+ }).fold(
+ errs => throw new Exception(errs.mkString("\n")),
--- End diff --
When will this code throw exception? When it could find the offset? When it
could not contact the leader? Can we surface the underlying exception?
The exception it should finally throw is a SparkException() which can wrap
the underlying exception.
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]