You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by koeninger <gi...@git.apache.org> on 2014/12/25 08:41:39 UTC

[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

GitHub user koeninger opened a pull request:

    https://github.com/apache/spark/pull/3798

    [SPARK-4964] [Streaming] Exactly-once semantics for Kafka

    

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/koeninger/spark-1 kafkaRdd

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/3798.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #3798
    
----
commit 76913e23179228481c98fbba36a54ca32fe20aed
Author: cody koeninger <co...@koeninger.org>
Date:   2014-11-23T03:15:30Z

    Batch oriented kafka rdd, WIP. todo: cluster metadata / finding leader

commit 1d706257ac848d37caeaff0409bf60b080d66e48
Author: cody koeninger <co...@koeninger.org>
Date:   2014-11-23T06:10:56Z

    WIP on kafka cluster

commit 0b94b3363cbc97c5d99e78c42f9be1c08a974fb1
Author: cody koeninger <co...@koeninger.org>
Date:   2014-11-24T14:49:24Z

    use dropWhile rather than filter to trim beginning of fetch response

commit 4dafd1b0d58215cb27218e569cb5bea9d5146815
Author: cody koeninger <co...@koeninger.org>
Date:   2014-11-24T16:45:40Z

    method to get leader offsets, switch rdd bound to being exclusive start, inclusive end to match offsets typically returned from cluster

commit ce91c591569b8ac4e91dd29d013961fe0ee5c316
Author: cody koeninger <co...@koeninger.org>
Date:   2014-11-24T18:07:24Z

    method to get consumer offsets, explicit error handling

commit 7d050bcb0bcacfbd4a7b858cffae809fd2af8e9d
Author: cody koeninger <co...@koeninger.org>
Date:   2014-11-24T22:11:24Z

    methods to set consumer offsets and get topic metadata, switch back to inclusive start / exclusive end to match typical kafka consumer behavior

commit 783b4775c89dbcbde9172d34653eab2718eee494
Author: cody koeninger <co...@koeninger.org>
Date:   2014-11-25T14:29:20Z

    update tests for kafka 8.1.1

commit 29c6b430cc6bf5e2354b397289c4445f4993fc5b
Author: cody koeninger <co...@koeninger.org>
Date:   2014-11-25T15:33:45Z

    cleanup logging

commit 3c2a96af2322754e8c76000b083ec3630a03e8c8
Author: cody koeninger <co...@koeninger.org>
Date:   2014-11-25T20:02:37Z

    fix scalastyle errors

commit 4b078bf1e71745a6bc160c0836b54cc7b0d4171d
Author: cody koeninger <co...@koeninger.org>
Date:   2014-11-25T20:48:32Z

    differentiate between leader and consumer offsets in error message

commit 8d7de4ab5a447a53f65be852702ca90512b2a639
Author: cody koeninger <co...@koeninger.org>
Date:   2014-11-25T23:54:40Z

    make sure leader offsets can be found even for leaders that arent in the seed brokers

commit 979da25f4d48e5ffccf13ba1ff66eb2527ff01f9
Author: cody koeninger <co...@koeninger.org>
Date:   2014-11-26T15:31:38Z

    dont allow empty leader offsets to be returned

commit 38bb727cf31744fa625248c86c2a666920e83c36
Author: cody koeninger <co...@koeninger.org>
Date:   2014-12-03T21:42:25Z

    give easy access to the parameters of a KafkaRDD

commit 326ff3cbda37066ebef7492241276754164d2879
Author: cody koeninger <co...@koeninger.org>
Date:   2014-12-16T21:27:44Z

    add some tests

commit 6bf14f2850f9f40f53b4c1eec373214e1b6d3465
Author: cody koeninger <co...@koeninger.org>
Date:   2014-12-24T17:38:52Z

    first attempt at a Kafka dstream that allows for exactly-once semantics

commit bcca8a4b69f73b48f71b9558adf718b5324ed933
Author: cody koeninger <co...@koeninger.org>
Date:   2014-12-24T20:35:43Z

    Merge branch 'master' of https://github.com/apache/spark into kafkaRdd

commit 37d305320e72de1ee6ffcd42f6a45d331a4d465d
Author: cody koeninger <co...@koeninger.org>
Date:   2014-12-25T04:41:40Z

    make KafkaRDDPartition available to users so offsets can be committed per partition

commit cac63eec4a0bee6b662c4577404622a08904f0cb
Author: cody koeninger <co...@koeninger.org>
Date:   2014-12-25T07:11:58Z

    additional testing, fix fencepost error

----


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23571836
  
    --- Diff: core/src/main/scala/org/apache/spark/rdd/RDD.scala ---
    @@ -788,6 +788,20 @@ abstract class RDD[T: ClassTag](
       }
     
       /**
    +   * Applies a function to each partition of this RDD, while tracking the index
    +   * of the original partition.
    +   */
    +  def foreachPartitionWithIndex(
    --- End diff --
    
    Can't this be done with just mapPartitionsWithContext followed by a foreach? I feel we are adding all kinds of random APIs to RDD that is making it very hard to maintain. If it is completely up to me, I'd even remove a bunch of existing RDD APIs ...


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23973351
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,338 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.util.Random
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    + * Convenience methods for interacting with a Kafka cluster.
    + * @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
    + */
    +private[spark]
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    +  import KafkaCluster.{Err, LeaderOffset}
    +
    +  val seedBrokers: Array[(String, Int)] =
    +    kafkaParams.get("metadata.broker.list")
    +      .orElse(kafkaParams.get("bootstrap.servers"))
    +      .getOrElse(throw new Exception("Must specify metadata.broker.list or bootstrap.servers"))
    +      .split(",").map { hp =>
    +        val hpa = hp.split(":")
    +        (hpa(0), hpa(1).toInt)
    +      }
    +
    +  // ConsumerConfig isn't serializable
    +  @transient private var _config: ConsumerConfig = null
    +
    +  def config: ConsumerConfig = this.synchronized {
    +    if (_config == null) {
    +      _config = KafkaCluster.consumerConfig(kafkaParams)
    +    }
    +    _config
    +  }
    +
    +  def connect(host: String, port: Int): SimpleConsumer =
    +    new SimpleConsumer(host, port, config.socketTimeoutMs,
    +      config.socketReceiveBufferBytes, config.clientId)
    +
    +  def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] =
    +    findLeader(topic, partition).right.map(hp => connect(hp._1, hp._2))
    +
    +  // Metadata api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataAPI
    +  // scalastyle:on
    +
    +  def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, Seq(topic))
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      resp.topicsMetadata.find(_.topic == topic).flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.find(_.partitionId == partition)
    +      }.foreach { pm: PartitionMetadata =>
    +        pm.leader.foreach { leader =>
    +          return Right((leader.host, leader.port))
    +        }
    +      }
    +    }
    +    Left(errs)
    +  }
    +
    +  def findLeaders(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, (String, Int)]] = {
    +    val topics = topicAndPartitions.map(_.topic)
    +    getPartitionMetadata(topics).right.flatMap { tms: Set[TopicMetadata] =>
    +      val result = tms.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.flatMap { pm: PartitionMetadata =>
    +          val tp = TopicAndPartition(tm.topic, pm.partitionId)
    +          if (topicAndPartitions(tp)) {
    +            pm.leader.map { l =>
    +              tp -> (l.host -> l.port)
    +            }
    +          } else {
    +            None
    +          }
    +        }
    +      }.toMap
    +      if (result.keys.size == topicAndPartitions.size) {
    +        Right(result)
    +      } else {
    +        val missing = topicAndPartitions.diff(result.keySet)
    +        val err = new Err
    +        err.append(new Exception(s"Couldn't find leaders for ${missing}"))
    +        Left(err)
    +      }
    +    }
    +  }
    +
    +  def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] =
    +    getPartitionMetadata(topics).right.map { r =>
    +      r.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.map { pm: PartitionMetadata =>
    +          TopicAndPartition(tm.topic, pm.partitionId)
    +        }    
    +      }
    +    }
    +
    +  def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = {
    +    val req = TopicMetadataRequest(
    +      TopicMetadataRequest.CurrentVersion, 0, config.clientId, topics.toSeq)
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      // error codes here indicate missing / just created topic,
    +      // repeating on a different broker wont be useful
    +      return Right(resp.topicsMetadata.toSet)
    +    }
    +    Left(errs)
    +  }
    +
    +  // Leader offset api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI
    +  // scalastyle:on
    +
    +  def getLatestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime)
    +
    +  def getEarliestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime)
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long
    +  ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    +    getLeaderOffsets(topicAndPartitions, before, 1).right.map { r =>
    +      r.map { kv =>
    +        // mapValues isnt serializable, see SI-7005
    +        kv._1 -> kv._2.head
    +      }
    +    }
    +
    +  private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] =
    +    m.groupBy(_._2).map { kv =>
    +      kv._1 -> kv._2.keys.toSeq
    +    }
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long,
    +    maxNumOffsets: Int
    +  ): Either[Err, Map[TopicAndPartition, Seq[LeaderOffset]]] = {
    +    findLeaders(topicAndPartitions).right.flatMap { tpToLeader =>
    +      val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] = flip(tpToLeader)
    +      val leaders = leaderToTp.keys
    +      var result = Map[TopicAndPartition, Seq[LeaderOffset]]()
    +      val errs = new Err
    +      withBrokers(leaders, errs) { consumer =>
    +        val needed: Seq[TopicAndPartition] = leaderToTp((consumer.host, consumer.port))
    +        val req = OffsetRequest(
    +          needed.map { tp: TopicAndPartition =>
    +            tp -> PartitionOffsetRequestInfo(before, maxNumOffsets)
    +          }.toMap
    +        )
    +        val resp = consumer.getOffsetsBefore(req)
    +        val respMap = resp.partitionErrorAndOffsets
    +        needed.foreach { tp: TopicAndPartition =>
    +          respMap.get(tp).foreach { por: PartitionOffsetsResponse =>
    +            if (por.error == ErrorMapping.NoError) {
    +              if (por.offsets.nonEmpty) {
    +                result += tp -> por.offsets.map { off =>
    +                  LeaderOffset(consumer.host, consumer.port, off)
    +                }
    +              } else {
    +                errs.append(new Exception(
    +                  s"Empty offsets for ${tp}, is ${before} before log beginning?"))
    +              }
    +            } else {
    +              errs.append(ErrorMapping.exceptionFor(por.error))
    +            }
    +          }
    +        }
    +        if (result.keys.size == topicAndPartitions.size) {
    +          return Right(result)
    +        }
    +      }
    +      val missing = topicAndPartitions.diff(result.keySet)
    +      errs.append(new Exception(s"Couldn't find leader offsets for ${missing}"))
    +      Left(errs)
    +    }
    +  }
    +
    +  // Consumer offset api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI
    +  // scalastyle:on
    +
    +  def getConsumerOffsets(
    +      groupId: String,
    +      topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] = {
    +    getConsumerOffsetMetadata(groupId, topicAndPartitions).right.map { r =>
    +      r.map { kv =>
    +        kv._1 -> kv._2.offset
    +      }
    +    }
    +  }
    +
    +  def getConsumerOffsetMetadata(
    +      groupId: String,
    +      topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = {
    --- End diff --
    
    Indentation issue.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72784748
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/26706/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24029025
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,174 @@ 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    --- End diff --
    
    It's just confusing though because the byte code reutrn type does not reflect the presence of the trait. I guess it is embbedded in the ScalaSig for the function. In any case, let me talk to TD, becuase this is an issue wrt binary compatiblity.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23746235
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/OffsetRange.scala ---
    @@ -0,0 +1,70 @@
    +/*
    + * 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.rdd.kafka
    +
    +/** Represents a range of offsets from a single Kafka TopicAndPartition */
    +trait OffsetRange {
    --- End diff --
    
    Ok, so there are a couple of different concerns here.
    
    First, the easy one.  Case classes.  KafkaRDDPartition isn't a case class.  The only case class in the entire PR is LeaderOffset, which isn't public and probably doesn't need to be a case class anyway.  No worries.
    
    Second, the question of whether OffsetRange needs to have a host and port.  The issue here is that in order to get a meaningful endpoint for the range, you have to make a remote call to find the kafka leader anyway.  So if you give people a constructor that allows them to specify an ending offset, but don't allow them to specify a preferred leader, you are forcing an interface that requires 2x the number of remote calls.
    
    Third, clients need to not only define offset ranges, they need to obtain offsets from the stream (for those that need them for exactly-once, or zookeeper interop, or whatever).  The idea of the interface is to provide limited access to the offsets without exposing any concrete implementation classes, so that you can change them later if need be.  That allows clients to do
    
        stream.foreachRDD { rdd =>
          rdd.foreachPartitionWithIndex { (i, iter) =>
            val offsetRange = rdd.partitions(i).asInstanceOf[OffsetRange]
    
    or
    
        stream.foreachRDD { rdd =>
          val allOffsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges
    
    without knowing anything at all about KafkaRDD or its partition class (or any concrete classes for that matter).  I'm pretty sure the same cannot be done with your suggestion, because there's nothing public to cast the RDD or the partition to.  I updated the usage examples to show how this works.
    
    https://github.com/koeninger/kafka-exactly-once/commit/d1641718807fc97f46e729e28acaba96ebc94c33
    
    The asInstanceOf is unfortunate, but because of the way DStream is implemented, we cannot say anything at compile time about what the RDDs returned in a DStream are capable of.  By this I mean we can make KafkaUtils.createRDD return a "RDD[R] with HasOffsetRanges" instead of KafkaRDD, but we cannot make a corresponding change to KafkaUtils.createNewStream, because foreachRDD just returns RDD, not a parameterized type.
    



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23616474
  
    --- Diff: core/src/main/scala/org/apache/spark/rdd/RDD.scala ---
    @@ -788,6 +788,20 @@ abstract class RDD[T: ClassTag](
       }
     
       /**
    +   * Applies a function to each partition of this RDD, while tracking the index
    +   * of the original partition.
    +   */
    +  def foreachPartitionWithIndex(
    --- End diff --
    
    Yeah, I was originally doing map followed by empty foreach and thought it looked confusing.
    
    I think it's really a question of what's easier to explain, it's just a syntax sugar issue not a correctness issue, so no problem either way.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-68815205
  
    Great! Keep me posted.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23972523
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,338 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.util.Random
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    + * Convenience methods for interacting with a Kafka cluster.
    + * @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
    + */
    +private[spark]
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    +  import KafkaCluster.{Err, LeaderOffset}
    +
    +  val seedBrokers: Array[(String, Int)] =
    +    kafkaParams.get("metadata.broker.list")
    +      .orElse(kafkaParams.get("bootstrap.servers"))
    +      .getOrElse(throw new Exception("Must specify metadata.broker.list or bootstrap.servers"))
    +      .split(",").map { hp =>
    +        val hpa = hp.split(":")
    +        (hpa(0), hpa(1).toInt)
    +      }
    +
    +  // ConsumerConfig isn't serializable
    +  @transient private var _config: ConsumerConfig = null
    +
    +  def config: ConsumerConfig = this.synchronized {
    +    if (_config == null) {
    +      _config = KafkaCluster.consumerConfig(kafkaParams)
    +    }
    +    _config
    +  }
    +
    +  def connect(host: String, port: Int): SimpleConsumer =
    +    new SimpleConsumer(host, port, config.socketTimeoutMs,
    +      config.socketReceiveBufferBytes, config.clientId)
    +
    +  def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] =
    +    findLeader(topic, partition).right.map(hp => connect(hp._1, hp._2))
    +
    +  // Metadata api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataAPI
    +  // scalastyle:on
    +
    +  def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, Seq(topic))
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      resp.topicsMetadata.find(_.topic == topic).flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.find(_.partitionId == partition)
    +      }.foreach { pm: PartitionMetadata =>
    +        pm.leader.foreach { leader =>
    +          return Right((leader.host, leader.port))
    +        }
    +      }
    +    }
    +    Left(errs)
    +  }
    +
    +  def findLeaders(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, (String, Int)]] = {
    +    val topics = topicAndPartitions.map(_.topic)
    +    getPartitionMetadata(topics).right.flatMap { tms: Set[TopicMetadata] =>
    +      val result = tms.flatMap { tm: TopicMetadata =>
    --- End diff --
    
    Its not obvious what this loop is doing. What is the result? leaders? 
    The loop is also nested quite deep to understand. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22498224
  
    --- Diff: core/src/main/scala/org/apache/spark/rdd/RDD.scala ---
    @@ -788,6 +788,20 @@ abstract class RDD[T: ClassTag](
       }
     
       /**
    +   * Applies a function to each partition of this RDD, while tracking the index
    +   * of the original partition.
    +   */
    +  def foreachPartitionWithIndex(
    --- End diff --
    
    Well, this is not necessary for this particular PR, isnt it? Then no point introducing it in this PR. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22653442
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,313 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    +  * Convenience methods for interacting with a Kafka cluster.
    +  * @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
    +  */
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    +  import KafkaCluster.Err
    +
    +  val seedBrokers: Array[(String, Int)] =
    +    kafkaParams.get("metadata.broker.list")
    +      .orElse(kafkaParams.get("bootstrap.servers"))
    +      .getOrElse(throw new Exception("Must specify metadata.broker.list or bootstrap.servers"))
    +      .split(",").map { hp =>
    +        val hpa = hp.split(":")
    +        (hpa(0), hpa(1).toInt)
    +      }
    +
    +  // ConsumerConfig isn't serializable
    +  @transient private var _config: ConsumerConfig = null
    +
    +  def config: ConsumerConfig = this.synchronized {
    +    if (_config == null) {
    +      _config = KafkaCluster.consumerConfig(kafkaParams)
    +    }
    +    _config
    +  }
    +
    +  def connect(host: String, port: Int): SimpleConsumer =
    +    new SimpleConsumer(host, port, config.socketTimeoutMs,
    +      config.socketReceiveBufferBytes, config.clientId)
    +
    +  def connect(hostAndPort: (String, Int)): SimpleConsumer =
    +    connect(hostAndPort._1, hostAndPort._2)
    +
    +  def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] =
    +    findLeader(topic, partition).right.map(connect)
    +
    +  def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, Seq(topic))
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      resp.topicsMetadata.find(_.topic == topic).flatMap { t =>
    +        t.partitionsMetadata.find(_.partitionId == partition)
    +      }.foreach { partitionMeta =>
    +        partitionMeta.leader.foreach { leader =>
    +          return Right((leader.host, leader.port))
    +        }
    +      }
    +    }
    +    Left(errs)
    +  }
    +
    +  def findLeaders(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, (String, Int)]] = {
    +    getPartitionMetadata(topicAndPartitions.map(_.topic)).right.flatMap { tms =>
    +      val result = tms.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.flatMap { pm =>
    +          val tp = TopicAndPartition(tm.topic, pm.partitionId)
    +          if (topicAndPartitions(tp)) {
    +            pm.leader.map { l =>
    +              tp -> (l.host -> l.port)
    +            }
    +          } else {
    +            None
    +          }
    +        }
    +      }.toMap
    +      if (result.keys.size == topicAndPartitions.size) {
    +        Right(result)
    +      } else {
    +        val missing = topicAndPartitions.diff(result.keys.toSet)
    +        val err = new Err
    +        err.append(new Exception(s"Couldn't find leaders for ${missing}"))
    +        Left(err)
    +      }
    +    }
    +  }
    +
    +  def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] =
    +    getPartitionMetadata(topics).right.map { r =>
    +      r.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.map { pm =>
    +          TopicAndPartition(tm.topic, pm.partitionId)
    +        }    
    +      }
    +    }
    +
    +  def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, topics.toSeq)
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      // error codes here indicate missing / just created topic,
    +      // repeating on a different broker wont be useful
    +      return Right(resp.topicsMetadata.toSet)
    +    }
    +    Left(errs)
    +  }
    +
    +  def getLatestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime)
    +
    +  def getEarliestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime)
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, before, 1).right.map { r =>
    +      r.map { kv =>
    +        // mapValues isnt serializable, see SI-7005
    +        kv._1 -> kv._2.head
    +      }
    +    }
    +
    +  private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] =
    +    m.groupBy(_._2).map { kv =>
    +      kv._1 -> kv._2.keys.toSeq
    +    }
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long,
    +    maxNumOffsets: Int
    +  ): Either[Err, Map[TopicAndPartition, Seq[Long]]] = {
    +    findLeaders(topicAndPartitions).right.flatMap { tpToLeader =>
    +      val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] = flip(tpToLeader)
    +      val leaders = leaderToTp.keys
    +      var result = Map[TopicAndPartition, Seq[Long]]()
    +      val errs = new Err
    +      withBrokers(leaders, errs) { consumer =>
    +        val needed: Seq[TopicAndPartition] = leaderToTp((consumer.host, consumer.port))
    +        val req = OffsetRequest(
    +          needed.map { tp =>
    +            tp -> PartitionOffsetRequestInfo(before, maxNumOffsets)
    +          }.toMap
    +        )
    +        val resp = consumer.getOffsetsBefore(req)
    +        val respMap = resp.partitionErrorAndOffsets
    +        needed.foreach { tp =>
    +          respMap.get(tp).foreach { errAndOffsets =>
    +            if (errAndOffsets.error == ErrorMapping.NoError) {
    +              if (errAndOffsets.offsets.nonEmpty) {
    +                result += tp -> errAndOffsets.offsets
    +              } else {
    +                errs.append(new Exception(
    +                  s"Empty offsets for ${tp}, is ${before} before log beginning?"))
    +              }
    +            } else {
    +              errs.append(ErrorMapping.exceptionFor(errAndOffsets.error))
    +            }
    +          }
    +        }
    +        if (result.keys.size == topicAndPartitions.size) {
    +          return Right(result)
    +        }
    +      }
    +      val missing = topicAndPartitions.diff(result.keys.toSet)
    +      errs.append(new Exception(s"Couldn't find leader offsets for ${missing}"))
    +      Left(errs)
    +    }
    +  }
    +
    +  def getConsumerOffsets(
    +    groupId: String,
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] = {
    +    getConsumerOffsetMetadata(groupId, topicAndPartitions).right.map { r =>
    +      r.map { kv =>
    +        kv._1 -> kv._2.offset
    +      }
    +    }
    +  }
    +
    +  def getConsumerOffsetMetadata(
    +    groupId: String,
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = {
    +    var result = Map[TopicAndPartition, OffsetMetadataAndError]()
    +    val req = OffsetFetchRequest(groupId, topicAndPartitions.toSeq)
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp = consumer.fetchOffsets(req)
    +      val respMap = resp.requestInfo
    +      val needed = topicAndPartitions.diff(result.keys.toSet)
    +      needed.foreach { tp =>
    +        respMap.get(tp).foreach { offsetMeta =>
    +          if (offsetMeta.error == ErrorMapping.NoError) {
    +            result += tp -> offsetMeta
    +          } else {
    +            errs.append(ErrorMapping.exceptionFor(offsetMeta.error))
    +          }
    +        }
    +      }
    +      if (result.keys.size == topicAndPartitions.size) {
    +        return Right(result)
    +      }
    +    }
    +    val missing = topicAndPartitions.diff(result.keys.toSet)
    +    errs.append(new Exception(s"Couldn't find consumer offsets for ${missing}"))
    +    Left(errs)
    +  }
    +
    +  def setConsumerOffsets(
    +    groupId: String,
    +    offsets: Map[TopicAndPartition, Long]
    +  ): Either[Err, Map[TopicAndPartition, Short]] = {
    +    setConsumerOffsetMetadata(groupId, offsets.map { kv =>
    +      kv._1 -> OffsetMetadataAndError(kv._2)
    +    })
    +  }
    +
    +  def setConsumerOffsetMetadata(
    +    groupId: String,
    +    metadata: Map[TopicAndPartition, OffsetMetadataAndError]
    +  ): Either[Err, Map[TopicAndPartition, Short]] = {
    +    var result = Map[TopicAndPartition, Short]()
    +    val req = OffsetCommitRequest(groupId, metadata)
    +    val errs = new Err
    +    val topicAndPartitions = metadata.keys.toSet
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp = consumer.commitOffsets(req)
    +      val respMap = resp.requestInfo
    +      val needed = topicAndPartitions.diff(result.keys.toSet)
    +      needed.foreach { tp =>
    +        respMap.get(tp).foreach { err =>
    +          if (err == ErrorMapping.NoError) {
    +            result += tp -> err
    +          } else {
    +            errs.append(ErrorMapping.exceptionFor(err))
    +          }
    +        }
    +      }
    +      if (result.keys.size == topicAndPartitions.size) {
    +        return Right(result)
    +      }
    +    }
    +    val missing = topicAndPartitions.diff(result.keys.toSet)
    +    errs.append(new Exception(s"Couldn't set offsets for ${missing}"))
    +    Left(errs)
    +  }
    +
    +  private def withBrokers(brokers: Iterable[(String, Int)], errs: Err)
    +    (fn: SimpleConsumer => Any): Unit = {
    +    brokers.foreach { hp =>
    +      var consumer: SimpleConsumer = null
    +      try {
    +        consumer = connect(hp)
    +        fn(consumer)
    +      } catch {
    +        case NonFatal(e) =>
    +          errs.append(e)
    +      } finally {
    +        if (consumer != null) consumer.close()
    +      }
    --- End diff --
    
    I appreciate the feedback, but it's impossible to "stay away from java nulls" in a jvm language, without runtime checks.
    
    Despite propaganda to the contrary, option.map is not a replacement for null checks.
    
    The code you wrote can still throw a null pointer exception (if SimpleConsumer returns null, for instance).
    
    You can hide the null pointer check inside of Option.apply instead of using Some, but at that point I'd rather just be explicit about what is going on.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71782133
  
      [Test build #26206 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26206/consoleFull) for   PR 3798 at commit [`19406cc`](https://github.com/apache/spark/commit/19406cce66672d74bd0b9c1d98cd8486c186f8ee).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by huitseeker <gi...@git.apache.org>.
Github user huitseeker commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22646185
  
    --- Diff: external/kafka/pom.xml ---
    @@ -44,7 +44,7 @@
         <dependency>
           <groupId>org.apache.kafka</groupId>
           <artifactId>kafka_${scala.binary.version}</artifactId>
    -      <version>0.8.0</version>
    --- End diff --
    
    That being said, @helena may soon update this version anyway in https://github.com/apache/spark/pull/3631 IIUC.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23586598
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,313 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    +  * Convenience methods for interacting with a Kafka cluster.
    +  * @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
    +  */
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    +  import KafkaCluster.Err
    +
    +  val seedBrokers: Array[(String, Int)] =
    +    kafkaParams.get("metadata.broker.list")
    +      .orElse(kafkaParams.get("bootstrap.servers"))
    +      .getOrElse(throw new Exception("Must specify metadata.broker.list or bootstrap.servers"))
    +      .split(",").map { hp =>
    +        val hpa = hp.split(":")
    +        (hpa(0), hpa(1).toInt)
    +      }
    +
    +  // ConsumerConfig isn't serializable
    +  @transient private var _config: ConsumerConfig = null
    +
    +  def config: ConsumerConfig = this.synchronized {
    +    if (_config == null) {
    +      _config = KafkaCluster.consumerConfig(kafkaParams)
    +    }
    +    _config
    +  }
    +
    +  def connect(host: String, port: Int): SimpleConsumer =
    +    new SimpleConsumer(host, port, config.socketTimeoutMs,
    +      config.socketReceiveBufferBytes, config.clientId)
    +
    +  def connect(hostAndPort: (String, Int)): SimpleConsumer =
    +    connect(hostAndPort._1, hostAndPort._2)
    +
    +  def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] =
    +    findLeader(topic, partition).right.map(connect)
    +
    +  def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, Seq(topic))
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      resp.topicsMetadata.find(_.topic == topic).flatMap { t =>
    +        t.partitionsMetadata.find(_.partitionId == partition)
    +      }.foreach { partitionMeta =>
    +        partitionMeta.leader.foreach { leader =>
    +          return Right((leader.host, leader.port))
    +        }
    +      }
    +    }
    +    Left(errs)
    +  }
    +
    +  def findLeaders(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, (String, Int)]] = {
    +    getPartitionMetadata(topicAndPartitions.map(_.topic)).right.flatMap { tms =>
    +      val result = tms.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.flatMap { pm =>
    +          val tp = TopicAndPartition(tm.topic, pm.partitionId)
    +          if (topicAndPartitions(tp)) {
    +            pm.leader.map { l =>
    +              tp -> (l.host -> l.port)
    +            }
    +          } else {
    +            None
    +          }
    +        }
    +      }.toMap
    +      if (result.keys.size == topicAndPartitions.size) {
    +        Right(result)
    +      } else {
    +        val missing = topicAndPartitions.diff(result.keys.toSet)
    +        val err = new Err
    +        err.append(new Exception(s"Couldn't find leaders for ${missing}"))
    +        Left(err)
    +      }
    +    }
    +  }
    +
    +  def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] =
    +    getPartitionMetadata(topics).right.map { r =>
    +      r.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.map { pm =>
    +          TopicAndPartition(tm.topic, pm.partitionId)
    +        }    
    +      }
    +    }
    +
    +  def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, topics.toSeq)
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      // error codes here indicate missing / just created topic,
    +      // repeating on a different broker wont be useful
    +      return Right(resp.topicsMetadata.toSet)
    +    }
    +    Left(errs)
    +  }
    +
    +  def getLatestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime)
    +
    +  def getEarliestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime)
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, before, 1).right.map { r =>
    +      r.map { kv =>
    +        // mapValues isnt serializable, see SI-7005
    +        kv._1 -> kv._2.head
    +      }
    +    }
    +
    +  private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] =
    +    m.groupBy(_._2).map { kv =>
    +      kv._1 -> kv._2.keys.toSeq
    +    }
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long,
    +    maxNumOffsets: Int
    +  ): Either[Err, Map[TopicAndPartition, Seq[Long]]] = {
    +    findLeaders(topicAndPartitions).right.flatMap { tpToLeader =>
    +      val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] = flip(tpToLeader)
    +      val leaders = leaderToTp.keys
    +      var result = Map[TopicAndPartition, Seq[Long]]()
    +      val errs = new Err
    +      withBrokers(leaders, errs) { consumer =>
    +        val needed: Seq[TopicAndPartition] = leaderToTp((consumer.host, consumer.port))
    +        val req = OffsetRequest(
    +          needed.map { tp =>
    +            tp -> PartitionOffsetRequestInfo(before, maxNumOffsets)
    +          }.toMap
    +        )
    +        val resp = consumer.getOffsetsBefore(req)
    +        val respMap = resp.partitionErrorAndOffsets
    +        needed.foreach { tp =>
    +          respMap.get(tp).foreach { errAndOffsets =>
    +            if (errAndOffsets.error == ErrorMapping.NoError) {
    +              if (errAndOffsets.offsets.nonEmpty) {
    +                result += tp -> errAndOffsets.offsets
    +              } else {
    +                errs.append(new Exception(
    +                  s"Empty offsets for ${tp}, is ${before} before log beginning?"))
    +              }
    +            } else {
    +              errs.append(ErrorMapping.exceptionFor(errAndOffsets.error))
    +            }
    +          }
    +        }
    +        if (result.keys.size == topicAndPartitions.size) {
    +          return Right(result)
    +        }
    +      }
    +      val missing = topicAndPartitions.diff(result.keys.toSet)
    +      errs.append(new Exception(s"Couldn't find leader offsets for ${missing}"))
    +      Left(errs)
    +    }
    +  }
    +
    +  def getConsumerOffsets(
    +    groupId: String,
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] = {
    +    getConsumerOffsetMetadata(groupId, topicAndPartitions).right.map { r =>
    +      r.map { kv =>
    +        kv._1 -> kv._2.offset
    +      }
    +    }
    +  }
    +
    +  def getConsumerOffsetMetadata(
    +    groupId: String,
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = {
    +    var result = Map[TopicAndPartition, OffsetMetadataAndError]()
    +    val req = OffsetFetchRequest(groupId, topicAndPartitions.toSeq)
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp = consumer.fetchOffsets(req)
    +      val respMap = resp.requestInfo
    +      val needed = topicAndPartitions.diff(result.keys.toSet)
    +      needed.foreach { tp =>
    +        respMap.get(tp).foreach { offsetMeta =>
    +          if (offsetMeta.error == ErrorMapping.NoError) {
    +            result += tp -> offsetMeta
    +          } else {
    +            errs.append(ErrorMapping.exceptionFor(offsetMeta.error))
    +          }
    +        }
    +      }
    +      if (result.keys.size == topicAndPartitions.size) {
    +        return Right(result)
    +      }
    +    }
    +    val missing = topicAndPartitions.diff(result.keys.toSet)
    +    errs.append(new Exception(s"Couldn't find consumer offsets for ${missing}"))
    +    Left(errs)
    +  }
    +
    +  def setConsumerOffsets(
    +    groupId: String,
    +    offsets: Map[TopicAndPartition, Long]
    +  ): Either[Err, Map[TopicAndPartition, Short]] = {
    +    setConsumerOffsetMetadata(groupId, offsets.map { kv =>
    +      kv._1 -> OffsetMetadataAndError(kv._2)
    +    })
    +  }
    +
    +  def setConsumerOffsetMetadata(
    +    groupId: String,
    +    metadata: Map[TopicAndPartition, OffsetMetadataAndError]
    +  ): Either[Err, Map[TopicAndPartition, Short]] = {
    +    var result = Map[TopicAndPartition, Short]()
    +    val req = OffsetCommitRequest(groupId, metadata)
    +    val errs = new Err
    +    val topicAndPartitions = metadata.keys.toSet
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp = consumer.commitOffsets(req)
    +      val respMap = resp.requestInfo
    +      val needed = topicAndPartitions.diff(result.keys.toSet)
    +      needed.foreach { tp =>
    +        respMap.get(tp).foreach { err =>
    +          if (err == ErrorMapping.NoError) {
    +            result += tp -> err
    +          } else {
    +            errs.append(ErrorMapping.exceptionFor(err))
    +          }
    +        }
    +      }
    +      if (result.keys.size == topicAndPartitions.size) {
    +        return Right(result)
    +      }
    +    }
    +    val missing = topicAndPartitions.diff(result.keys.toSet)
    +    errs.append(new Exception(s"Couldn't set offsets for ${missing}"))
    +    Left(errs)
    +  }
    +
    +  private def withBrokers(brokers: Iterable[(String, Int)], errs: Err)
    +    (fn: SimpleConsumer => Any): Unit = {
    +    brokers.foreach { hp =>
    +      var consumer: SimpleConsumer = null
    +      try {
    +        consumer = connect(hp)
    +        fn(consumer)
    +      } catch {
    +        case NonFatal(e) =>
    +          errs.append(e)
    +      } finally {
    +        if (consumer != null) consumer.close()
    +      }
    --- End diff --
    
    the use of null here is fine, and very clear. the pattern matching with the finally actually makes it much harder to understand what is going on.
    
    one nitpick, you need to put curly braces around consumer.close(), i.e.
    ```scala
    if (consumer != null) {
      consumer.close()
    }
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72777123
  
    Ohh I meant createStream --> createDirectStream. I would have preferred something like createReceiverLessStream but thats a mouthful. I think "direct" is something that comes close without being a mouthful. Had not occurred to me until Patrick suggested it.
    
    And the underlying assumptions, I confess are not super concrete. Somethings like binary compatiblity issues (ex, do not use scala traits with implemented methods) are fairly concrete, where as things about API elegance (e.g. rdd.asInstanceOf[KafkaRDD] vs rdd.asInstanceOf[HasOffsetRanges]) are a little fuzzy and opinions vary from person to person. Often what seems intuitive to me is not intuitive to someone else, even within the key committers like Patrick, Michael, Matei, etc. We usually argue about this in design docs, get as many eyeballs as possible, and try to reach a consensus. Its is indeed a bit fuzzy, but its all towards making the API that we *think* will be the best for the developers. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23747942
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/OffsetRange.scala ---
    @@ -0,0 +1,70 @@
    +/*
    + * 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.rdd.kafka
    +
    +/** Represents a range of offsets from a single Kafka TopicAndPartition */
    +trait OffsetRange {
    --- End diff --
    
    Right, I get your point. Though I thought about the usage based on the example, and I think we need to think this a little bit more. From what I understood, you are attaching the offset in every records, and shuffling everything with that offset attached. That is quite a loss of efficiency. Also, accessing the RDD and its partition object from within the mapPartition function is very confusing, and ... does it actually work???? If at all this works, thats not even the recommended RDD operation!
    
    We really need to come up with a better way to expose offsets. Brainstorming a little more on this. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71968759
  
    packaging, makes sense
    
    method name, agreed, named it createNewStream for now
    
    offset range, see my explanation of the interface above.  I think this is
    the last substantive / non-style issue to get worked out
    
    unit tests for the stream / kafkacluster, will see what I can do
    
    On Wed, Jan 28, 2015 at 8:49 PM, Tathagata Das <no...@github.com>
    wrote:
    
    > At a high level these are the high level design issues that are still
    > pending
    >
    >    - Packaging - I think all the classes should be
    >    org.apache.spark.streaming.kafka (even KafkaRDD), because (i) all of them
    >    are published in spark-streaming-kafka artifact, (ii) just importing one
    >    path (o.a.spark.streaming.kafka.KafkaUtils._) is sufficient get all the
    >    relevant classes.
    >    - KafkaUtils method name and scala doc - Lets keep thinking of names,
    >    but lets not stay blocked on this and continue focussing on other issues.
    >    - OffsetRange - I have proposed a design as well. Either we will go
    >    for Java interface or the simple class that I proposed.
    >    - Unit tests - No unit test for the new stream. Also is it possible to
    >    include a few unit tests for KafkaCluster?
    >
    > Other than these, I will start commenting on the code styles, etc very
    > soon.
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798#issuecomment-71960076>.
    >


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24019904
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,174 @@ 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    +    val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message)
    +    val kc = new KafkaCluster(kafkaParams)
    +    val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet
    +    val leaders = kc.findLeaders(topics).fold(
    +      errs => throw new SparkException(errs.mkString("\n")),
    +      ok => ok
    +    )
    +    new KafkaRDD[K, V, U, T, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler)
    +  }
    +
    +  /** 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   * @param leaders Kafka leaders for each offset range in batch
    +   * @param messageHandler function for translating each message into the desired type
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange],
    +      leaders: Array[Leader],
    +      messageHandler: MessageAndMetadata[K, V] => R
    +  ): RDD[R] with HasOffsetRanges = {
    +
    +    val leaderMap = leaders
    +      .map(l => TopicAndPartition(l.topic, l.partition) -> (l.host, l.port))
    +      .toMap
    +    new KafkaRDD[K, V, U, T, R](sc, kafkaParams, offsetRanges, leaderMap, messageHandler)
    +  }
    +
    +  /**
    +   * This stream can guarantee that each message from Kafka is included in transformations
    --- End diff --
    
    I think we want the first sentence of the doc to convey why someone would choose this method.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22498258
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaRDD.scala ---
    @@ -0,0 +1,157 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.reflect.{classTag, ClassTag}
    +
    +import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.NextIterator
    +
    +import java.util.Properties
    +import kafka.api.FetchRequestBuilder
    +import kafka.common.{ErrorMapping, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +import kafka.message.{MessageAndMetadata, MessageAndOffset}
    +import kafka.serializer.Decoder
    +import kafka.utils.VerifiableProperties
    +
    +case class KafkaRDDPartition(
    +  override val index: Int,
    +  topic: String,
    +  partition: Int,
    +  fromOffset: Long,
    +  untilOffset: Long
    +) extends Partition
    +
    +/** A batch-oriented interface for consuming from Kafka.
    +  * Each given Kafka topic/partition corresponds to an RDD partition.
    +  * Starting and ending offsets are specified in advance,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the batch
    +  * @param untilOffsets per-topic/partition Kafka offsets defining the (exclusive)
    +  *  ending point of the batch
    +  * @param messageHandler function for translating each message into the desired type
    +  */
    +class KafkaRDD[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    sc: SparkContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    val untilOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R
    +  ) extends RDD[R](sc, Nil) with Logging {
    +
    +  assert(fromOffsets.keys == untilOffsets.keys,
    +    "Must provide both from and until offsets for each topic/partition")
    +
    +  override def getPartitions: Array[Partition] = fromOffsets.zipWithIndex.map { kvi =>
    +    val ((tp, from), index) = kvi
    +    new KafkaRDDPartition(index, tp.topic, tp.partition, from, untilOffsets(tp))
    +  }.toArray
    +
    +  override def compute(thePart: Partition, context: TaskContext) = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    if (part.fromOffset >= part.untilOffset) {
    +      log.warn("Beginning offset is same or after ending offset " +
    +        s"skipping ${part.topic} ${part.partition}")
    +      Iterator.empty
    +    } else {
    +      new NextIterator[R] {
    +        context.addTaskCompletionListener{ context => closeIfNeeded() }
    +
    +        val kc = new KafkaCluster(kafkaParams)
    +        log.info(s"Computing topic ${part.topic}, partition ${part.partition} " +
    +          s"offsets ${part.fromOffset} -> ${part.untilOffset}")
    +        val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[K]]
    +        val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[V]]
    +        val consumer: SimpleConsumer = kc.connectLeader(part.topic, part.partition).fold(
    +          errs => throw new Exception(
    +            s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " +
    +              errs.mkString("\n")),
    +          consumer => consumer
    +        )
    +        var requestOffset = part.fromOffset
    +        var iter: Iterator[MessageAndOffset] = null
    +
    +        override def close() = consumer.close()
    +
    +        override def getNext: R = {
    +          if (iter == null || !iter.hasNext) {
    +            val req = new FetchRequestBuilder().
    +              addFetch(part.topic, part.partition, requestOffset, kc.config.fetchMessageMaxBytes).
    +              build()
    +            val resp = consumer.fetch(req)
    +            if (resp.hasError) {
    +              val err = resp.errorCode(part.topic, part.partition)
    +              if (err == ErrorMapping.LeaderNotAvailableCode ||
    +                err == ErrorMapping.NotLeaderForPartitionCode) {
    +                log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " +
    +                  s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms")
    +                Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +              }
    +              // Let normal rdd retry sort out reconnect attempts
    +              throw ErrorMapping.exceptionFor(err)
    +            }
    +            iter = resp.messageSet(part.topic, part.partition)
    +              .iterator
    +              .dropWhile(_.offset < requestOffset)
    --- End diff --
    
    Wow! That was not intuitive. Worth mentioning this in the code.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23973547
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala ---
    @@ -0,0 +1,150 @@
    +/*
    + * 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.streaming.kafka
    +
    +
    +import scala.annotation.tailrec
    +import scala.collection.mutable
    +import scala.reflect.{classTag, ClassTag}
    +
    +import kafka.common.TopicAndPartition
    +import kafka.message.MessageAndMetadata
    +import kafka.serializer.Decoder
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset
    +import org.apache.spark.streaming.{StreamingContext, Time}
    +import org.apache.spark.streaming.dstream._
    +
    +/** A stream of {@link org.apache.spark.rdd.kafka.KafkaRDD} where
    +  * each given Kafka topic/partition corresponds to an RDD partition.
    +  * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages
    +  * per second that each '''partition''' will accept.
    +  * Starting offsets are specified in advance,
    +  * and this DStream is not responsible for committing offsets,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the stream
    +  * @param messageHandler function for translating each message into the desired type
    +  * @param maxRetries maximum number of times in a row to retry getting leaders' offsets
    +  */
    +private[streaming]
    +class DeterministicKafkaInputDStream[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    @transient ssc_ : StreamingContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R,
    +    maxRetries: Int
    +) extends InputDStream[R](ssc_) with Logging {
    +
    +  protected[streaming] override val checkpointData =
    +    new DeterministicKafkaInputDStreamCheckpointData
    +
    +  protected val kc = new KafkaCluster(kafkaParams)
    +
    +  protected val maxMessagesPerPartition: Option[Long] = {
    +    val ratePerSec = context.sparkContext.getConf.getInt("spark.streaming.receiver.maxRate", 0)
    +    if (ratePerSec > 0) {
    +      val secsPerBatch = context.graph.batchDuration.milliseconds.toDouble / 1000
    +      Some((secsPerBatch * ratePerSec).toLong)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  protected var currentOffsets = fromOffsets
    +
    +  @tailrec
    +  protected final def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, LeaderOffset] = {
    --- End diff --
    
    Nono, KafkaCluster wont. Method in this DStream class will throw the exception based on the returned `Either[Err, ...]`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23989107
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala ---
    @@ -0,0 +1,69 @@
    +/*
    + * 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.streaming.kafka
    +
    +import kafka.common.TopicAndPartition
    +
    +/** Something that has a collection of OffsetRanges */
    +trait HasOffsetRanges {
    +  def offsetRanges: Array[OffsetRange]
    +}
    +
    +/** Represents a range of offsets from a single Kafka TopicAndPartition */
    +final class OffsetRange private(
    +    /** kafka topic name */
    +    val topic: String,
    +    /** kafka partition id */
    +    val partition: Int,
    +    /** inclusive starting offset */
    +    val fromOffset: Long,
    +    /** exclusive ending offset */
    +    val untilOffset: Long) extends Serializable {
    +  import OffsetRange.OffsetRangeTuple
    +
    +  /** this is to avoid ClassNotFoundException during checkpoint restore */
    +  private[streaming]
    +  def toTuple: OffsetRangeTuple = (topic, partition, fromOffset, untilOffset)
    +}
    +
    +object OffsetRange {
    +  private[spark]
    +  type OffsetRangeTuple = (String, Int, Long, Long)
    --- End diff --
    
    Can you group this at the bottom with the related `apply` method? 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23727391
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaRDD.scala ---
    @@ -0,0 +1,199 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.reflect.{classTag, ClassTag}
    +
    +import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.NextIterator
    +
    +import java.util.Properties
    +import kafka.api.{FetchRequestBuilder, FetchResponse}
    +import kafka.common.{ErrorMapping, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +import kafka.message.{MessageAndMetadata, MessageAndOffset}
    +import kafka.serializer.Decoder
    +import kafka.utils.VerifiableProperties
    +
    +/** A batch-oriented interface for consuming from Kafka.
    +  * Starting and ending offsets are specified in advance,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 KafkaRDDPartition 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
    +  */
    +class KafkaRDD[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    sc: SparkContext,
    +    val kafkaParams: Map[String, String],
    +    val batch: Array[KafkaRDDPartition],
    +    messageHandler: MessageAndMetadata[K, V] => R
    +  ) extends RDD[R](sc, Nil) with Logging {
    +
    +  override def getPartitions: Array[Partition] = batch.asInstanceOf[Array[Partition]]
    +
    +  override def getPreferredLocations(thePart: Partition): Seq[String] = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    // TODO is additional hostname resolution necessary here
    +    Seq(part.host)
    +  }
    +
    +  override def compute(thePart: Partition, context: TaskContext): Iterator[R] = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    if (part.fromOffset >= part.untilOffset) {
    +      log.warn("Beginning offset is same or after ending offset " +
    +        s"skipping ${part.topic} ${part.partition}")
    +      Iterator.empty
    +    } else {
    +      new NextIterator[R] {
    +        context.addTaskCompletionListener{ context => closeIfNeeded() }
    +
    +        log.info(s"Computing topic ${part.topic}, partition ${part.partition} " +
    +          s"offsets ${part.fromOffset} -> ${part.untilOffset}")
    +
    +        val kc = new KafkaCluster(kafkaParams)
    +        val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[K]]
    +        val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[V]]
    +        val consumer = connectLeader
    +        var requestOffset = part.fromOffset
    +        var iter: Iterator[MessageAndOffset] = null
    +
    +        // The idea is to use the provided preferred host, except on task retry atttempts,
    +        // to minimize number of kafka metadata requests
    +        private def connectLeader: SimpleConsumer = {
    +          if (context.attemptNumber > 0) {
    +            kc.connectLeader(part.topic, part.partition).fold(
    +              errs => throw new Exception(
    +                s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " +
    +                  errs.mkString("\n")),
    +              consumer => consumer
    +            )
    +          } else {
    +            kc.connect(part.host, part.port)
    +          }
    +        }
    +
    +        private def handleErr(resp: FetchResponse) {
    +          if (resp.hasError) {
    +            val err = resp.errorCode(part.topic, part.partition)
    +            if (err == ErrorMapping.LeaderNotAvailableCode ||
    +              err == ErrorMapping.NotLeaderForPartitionCode) {
    +              log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " +
    +                s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms")
    +              Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +            }
    +            // Let normal rdd retry sort out reconnect attempts
    +            throw ErrorMapping.exceptionFor(err)
    +          }
    +        }
    +
    +        override def close() = consumer.close()
    +
    +        override def getNext: R = {
    +          if (iter == null || !iter.hasNext) {
    +            val req = new FetchRequestBuilder().
    +              addFetch(part.topic, part.partition, requestOffset, kc.config.fetchMessageMaxBytes).
    +              build()
    +            val resp = consumer.fetch(req)
    +            handleErr(resp)
    +            // kafka may return a batch that starts before the requested offset
    +            iter = resp.messageSet(part.topic, part.partition)
    +              .iterator
    +              .dropWhile(_.offset < requestOffset)
    +          }
    +          if (!iter.hasNext) {
    +            assert(requestOffset == part.untilOffset,
    +              s"ran out of messages before reaching ending offset ${part.untilOffset} " +
    +                s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." +
    +                " This should not happen, and indicates that messages may have been lost")
    +            finished = true
    +            null.asInstanceOf[R]
    +          } else {
    +            val item = iter.next
    +            if (item.offset >= part.untilOffset) {
    +              assert(item.offset == part.untilOffset,
    +                s"got ${item.offset} > ending offset ${part.untilOffset} " +
    +                  s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." +
    +                  " This should not happen, and indicates a message may have been skipped")
    +              finished = true
    +              null.asInstanceOf[R]
    +            } else {
    +              requestOffset = item.nextOffset
    +              messageHandler(new MessageAndMetadata(
    +                part.topic, part.partition, item.message, item.offset, keyDecoder, valueDecoder))
    +            }
    --- End diff --
    
    A good way is to define subfunctions within meaningful names. Then the complex condition logic can broken down and will be easier to read. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23973191
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,338 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.util.Random
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    + * Convenience methods for interacting with a Kafka cluster.
    + * @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
    + */
    +private[spark]
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    +  import KafkaCluster.{Err, LeaderOffset}
    +
    +  val seedBrokers: Array[(String, Int)] =
    +    kafkaParams.get("metadata.broker.list")
    +      .orElse(kafkaParams.get("bootstrap.servers"))
    +      .getOrElse(throw new Exception("Must specify metadata.broker.list or bootstrap.servers"))
    +      .split(",").map { hp =>
    +        val hpa = hp.split(":")
    +        (hpa(0), hpa(1).toInt)
    +      }
    +
    +  // ConsumerConfig isn't serializable
    +  @transient private var _config: ConsumerConfig = null
    +
    +  def config: ConsumerConfig = this.synchronized {
    +    if (_config == null) {
    +      _config = KafkaCluster.consumerConfig(kafkaParams)
    +    }
    +    _config
    +  }
    +
    +  def connect(host: String, port: Int): SimpleConsumer =
    +    new SimpleConsumer(host, port, config.socketTimeoutMs,
    +      config.socketReceiveBufferBytes, config.clientId)
    +
    +  def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] =
    +    findLeader(topic, partition).right.map(hp => connect(hp._1, hp._2))
    +
    +  // Metadata api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataAPI
    +  // scalastyle:on
    +
    +  def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, Seq(topic))
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      resp.topicsMetadata.find(_.topic == topic).flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.find(_.partitionId == partition)
    +      }.foreach { pm: PartitionMetadata =>
    +        pm.leader.foreach { leader =>
    +          return Right((leader.host, leader.port))
    +        }
    +      }
    +    }
    +    Left(errs)
    +  }
    +
    +  def findLeaders(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, (String, Int)]] = {
    +    val topics = topicAndPartitions.map(_.topic)
    +    getPartitionMetadata(topics).right.flatMap { tms: Set[TopicMetadata] =>
    +      val result = tms.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.flatMap { pm: PartitionMetadata =>
    +          val tp = TopicAndPartition(tm.topic, pm.partitionId)
    +          if (topicAndPartitions(tp)) {
    +            pm.leader.map { l =>
    +              tp -> (l.host -> l.port)
    +            }
    +          } else {
    +            None
    +          }
    +        }
    +      }.toMap
    +      if (result.keys.size == topicAndPartitions.size) {
    +        Right(result)
    +      } else {
    +        val missing = topicAndPartitions.diff(result.keySet)
    +        val err = new Err
    +        err.append(new Exception(s"Couldn't find leaders for ${missing}"))
    +        Left(err)
    +      }
    +    }
    +  }
    +
    +  def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] =
    +    getPartitionMetadata(topics).right.map { r =>
    +      r.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.map { pm: PartitionMetadata =>
    +          TopicAndPartition(tm.topic, pm.partitionId)
    +        }    
    +      }
    +    }
    +
    +  def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = {
    +    val req = TopicMetadataRequest(
    +      TopicMetadataRequest.CurrentVersion, 0, config.clientId, topics.toSeq)
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      // error codes here indicate missing / just created topic,
    +      // repeating on a different broker wont be useful
    +      return Right(resp.topicsMetadata.toSet)
    +    }
    +    Left(errs)
    +  }
    +
    +  // Leader offset api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI
    +  // scalastyle:on
    +
    +  def getLatestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime)
    +
    +  def getEarliestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime)
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long
    +  ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    +    getLeaderOffsets(topicAndPartitions, before, 1).right.map { r =>
    +      r.map { kv =>
    +        // mapValues isnt serializable, see SI-7005
    +        kv._1 -> kv._2.head
    +      }
    +    }
    +
    +  private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] =
    +    m.groupBy(_._2).map { kv =>
    +      kv._1 -> kv._2.keys.toSeq
    +    }
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long,
    +    maxNumOffsets: Int
    +  ): Either[Err, Map[TopicAndPartition, Seq[LeaderOffset]]] = {
    +    findLeaders(topicAndPartitions).right.flatMap { tpToLeader =>
    +      val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] = flip(tpToLeader)
    +      val leaders = leaderToTp.keys
    +      var result = Map[TopicAndPartition, Seq[LeaderOffset]]()
    +      val errs = new Err
    +      withBrokers(leaders, errs) { consumer =>
    +        val needed: Seq[TopicAndPartition] = leaderToTp((consumer.host, consumer.port))
    +        val req = OffsetRequest(
    --- End diff --
    
    Please break this line into 2. 
    ```
    val requestMap = needed.map { .... } 
    val req = OffsetRequest(requestMap)
    
    ```



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23822726
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +149,182 @@ 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
    +   */
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      batch: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    +    val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message)
    +    val kc = new KafkaCluster(kafkaParams)
    +    val topics = batch.map(o => TopicAndPartition(o.topic, o.partition)).toSet
    +    val leaderMap = kc.findLeaders(topics).fold(
    +      errs => throw new SparkException(errs.mkString("\n")),
    +      ok => ok
    +    )
    +    val rddParts = batch.zipWithIndex.map { case (o, i) =>
    +        val tp = TopicAndPartition(o.topic, o.partition)
    +        val (host, port) = leaderMap(tp)
    +        new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset, host, port)
    +    }.toArray
    +    new KafkaRDD[K, V, U, T, (K, V)](sc, kafkaParams, rddParts, messageHandler)
    +  }
    +
    +  /** 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 leaders Kafka leaders for each offset range in batch
    +   * @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],
    +      leaders: Array[Leader],
    +      messageHandler: MessageAndMetadata[K, V] => R
    +  ): RDD[R] with HasOffsetRanges = {
    +    val leaderMap = leaders.map(l => (l.topic, l.partition) -> (l.host, l.port)).toMap
    +    val rddParts = batch.zipWithIndex.map { case (o, i) =>
    +        val (host, port) = leaderMap((o.topic, o.partition))
    +        new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset, host, port)
    +    }.toArray
    +
    +    new KafkaRDD[K, V, U, T, R](sc, kafkaParams, rddParts, messageHandler)
    +  }
    +
    +  /**
    +   * Compared to `createStream`, the stream created by this can guarantee that each message
    --- End diff --
    
    A scala doc has to start with this method does, not with a comparison with another method. Then for understanding this method, they will have to read another method first. That's not good. In future if we deprecate the other method, then even worse. Its best to say what this method does independent of anything else, and as a second para (for those who want to understand more), compare with the other method. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by dibbhatt <gi...@git.apache.org>.
Github user dibbhatt commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23889938
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala ---
    @@ -0,0 +1,220 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.reflect.{classTag, ClassTag}
    +
    +import org.apache.spark.{Logging, Partition, SparkContext, SparkException, TaskContext}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.NextIterator
    +
    +import java.util.Properties
    +import kafka.api.{FetchRequestBuilder, FetchResponse}
    +import kafka.common.{ErrorMapping, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +import kafka.message.{MessageAndMetadata, MessageAndOffset}
    +import kafka.serializer.Decoder
    +import kafka.utils.VerifiableProperties
    +
    +/**
    + * 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 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 KafkaRDDPartition 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
    + */
    +private[spark]
    +class KafkaRDD[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag] private[spark] (
    +    sc: SparkContext,
    +    kafkaParams: Map[String, String],
    +    private[spark] val batch: Array[KafkaRDDPartition],
    +    messageHandler: MessageAndMetadata[K, V] => R
    +  ) extends RDD[R](sc, Nil) with Logging with HasOffsetRanges {
    +
    +  def offsetRanges: Array[OffsetRange] = batch.asInstanceOf[Array[OffsetRange]]
    +
    +  override def getPartitions: Array[Partition] = batch.asInstanceOf[Array[Partition]]
    +
    +  override def getPreferredLocations(thePart: Partition): Seq[String] = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    // TODO is additional hostname resolution necessary here
    +    Seq(part.host)
    +  }
    +
    +  private def errBeginAfterEnd(part: KafkaRDDPartition): String =
    +    s"Beginning offset ${part.fromOffset} is after the ending offset ${part.untilOffset} " +
    +      s"for topic ${part.topic} partition ${part.partition}. " +
    +      "You either provided an invalid fromOffset, or the Kafka topic has been damaged"
    +
    +  private def errRanOutBeforeEnd(part: KafkaRDDPartition): String =
    +    s"Ran out of messages before reaching ending offset ${part.untilOffset} " +
    +    s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." +
    +    " This should not happen, and indicates that messages may have been lost"
    +
    +  private def errOvershotEnd(itemOffset: Long, part: KafkaRDDPartition): String =
    +    s"Got ${itemOffset} > ending offset ${part.untilOffset} " +
    +    s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." +
    +    " This should not happen, and indicates a message may have been skipped"
    +
    +  override def compute(thePart: Partition, context: TaskContext): Iterator[R] = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    assert(part.fromOffset <= part.untilOffset, errBeginAfterEnd(part))
    +    if (part.fromOffset == part.untilOffset) {
    +      log.warn("Beginning offset ${part.fromOffset} is the same as ending offset " +
    +        s"skipping ${part.topic} ${part.partition}")
    +      Iterator.empty
    +    } else {
    +      new KafkaRDDIterator(part, context)
    +    }
    +  }
    +
    +  private class KafkaRDDIterator(
    +      part: KafkaRDDPartition,
    +      context: TaskContext) extends NextIterator[R] {
    +
    +    context.addTaskCompletionListener{ context => closeIfNeeded() }
    +
    +    log.info(s"Computing topic ${part.topic}, partition ${part.partition} " +
    +      s"offsets ${part.fromOffset} -> ${part.untilOffset}")
    +
    +    val kc = new KafkaCluster(kafkaParams)
    +    val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +      .newInstance(kc.config.props)
    +      .asInstanceOf[Decoder[K]]
    +    val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +      .newInstance(kc.config.props)
    +      .asInstanceOf[Decoder[V]]
    +    val consumer = connectLeader
    +    var requestOffset = part.fromOffset
    +    var iter: Iterator[MessageAndOffset] = null
    +
    +    // The idea is to use the provided preferred host, except on task retry atttempts,
    +    // to minimize number of kafka metadata requests
    +    private def connectLeader: SimpleConsumer = {
    +      if (context.attemptNumber > 0) {
    +        kc.connectLeader(part.topic, part.partition).fold(
    +          errs => throw new SparkException(
    +            s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " +
    +              errs.mkString("\n")),
    +          consumer => consumer
    +        )
    +      } else {
    +        kc.connect(part.host, part.port)
    +      }
    +    }
    +
    +    private def handleFetchErr(resp: FetchResponse) {
    +      if (resp.hasError) {
    +        val err = resp.errorCode(part.topic, part.partition)
    +        if (err == ErrorMapping.LeaderNotAvailableCode ||
    +          err == ErrorMapping.NotLeaderForPartitionCode) {
    +          log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " +
    +            s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms")
    +          Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +        }
    +        // Let normal rdd retry sort out reconnect attempts
    +        throw ErrorMapping.exceptionFor(err)
    +      }
    +    }
    +
    --- End diff --
    
    If the KafkaRdd specification is broken , then probably you need to document it how client code need to handle. In our Production Kafka cluster, this error not very rare and not very rare for others as far I know who uses Kafka in Prod.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23823493
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Leader.scala ---
    @@ -0,0 +1,68 @@
    +/*
    + * 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.streaming.kafka
    +
    +import kafka.common.TopicAndPartition
    +
    +/** Host info for the leader of a Kafka TopicAndPartition */
    +
    +trait Leader {
    +    /** kafka topic name */
    +  def topic: String
    +
    +  /** kafka partition id */
    +  def partition: Int
    +
    +  /** kafka hostname */
    +  def host: String
    +  
    +  /** kafka host's port */
    +  def port: Int
    +}
    +
    +private class LeaderImpl(
    +  override val topic: String,
    +  override val partition: Int,
    +  override val host: String,
    +  override val port: Int
    +) extends Leader
    +
    +object Leader {
    +  def create(
    +    topic: String,
    --- End diff --
    
    indent of 4 over the top line for such multiple class/method parameters. 
    See this https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide
    This is a problem with a lot of the methods, especially in KafkaCluster. I know its annoying but please fix them :)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23823428
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Leader.scala ---
    @@ -0,0 +1,68 @@
    +/*
    + * 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.streaming.kafka
    +
    +import kafka.common.TopicAndPartition
    +
    +/** Host info for the leader of a Kafka TopicAndPartition */
    +
    +trait Leader {
    +    /** kafka topic name */
    +  def topic: String
    +
    +  /** kafka partition id */
    +  def partition: Int
    +
    +  /** kafka hostname */
    +  def host: String
    +  
    +  /** kafka host's port */
    +  def port: Int
    +}
    +
    +private class LeaderImpl(
    +  override val topic: String,
    +  override val partition: Int,
    +  override val host: String,
    +  override val port: Int
    +) extends Leader
    +
    +object Leader {
    +  def create(
    +    topic: String,
    +    partition: Int,
    +    host: String,
    +    port: Int): Leader =
    +    new LeaderImpl(
    --- End diff --
    
    I think this fits in 100 chars. Should be fine to put in a line. Otherwise looks unnecessarily long. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22363576
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala ---
    @@ -0,0 +1,123 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.annotation.tailrec
    +import scala.reflect.{classTag, ClassTag}
    +
    +import kafka.common.TopicAndPartition
    +import kafka.message.MessageAndMetadata
    +import kafka.serializer.Decoder
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.rdd.kafka.{KafkaCluster, KafkaRDD}
    +import org.apache.spark.streaming.{StreamingContext, Time}
    +import org.apache.spark.streaming.dstream._
    +
    +/** A stream of {@link org.apache.spark.rdd.kafka.KafkaRDD} where
    +  * each given Kafka topic/partition corresponds to an RDD partition.
    +  * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages
    +  * per second that each '''partition''' will accept.
    +  * Starting offsets are specified in advance,
    +  * and this DStream is not responsible for committing offsets,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the stream
    +  * @param messageHandler function for translating each message into the desired type
    +  * @param maxRetries maximum number of times in a row to retry getting leaders' offsets
    +  */
    +class DeterministicKafkaInputDStream[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    @transient ssc_ : StreamingContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R,
    +    maxRetries: Int = 1
    +) extends InputDStream[R](ssc_) with Logging {
    +
    +  private val kc = new KafkaCluster(kafkaParams)
    +
    +  private val maxMessagesPerPartition: Option[Long] = {
    +    val ratePerSec = ssc.sparkContext.getConf.getInt("spark.streaming.receiver.maxRate", 0)
    --- End diff --
    
    This is a good attempt at introducing maxRate! However, what is not clear to me is how is this used. Is it this limit applied on each topic+partition? Or is it the global number of records received per second across all topics and partitions?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72790044
  
    The warning is for metadata.broker.list, since its not expected by the
    existing ConsumerConfig (its used by other config classes)
    
    Couldn't get subclassing to work, the verifiedproperties class it uses is
    very dependent on order of operations during construction.
    
    I think the simplest thing is a class that is constructed using
    kafkaparams, and uses the static defaults from the ConsumerConfig object.
    
    I'm currently waiting in an ER with my child with a 105 fever, so won't be
    getting to it for a few hours to say the least.
    On Feb 3, 2015 10:15 PM, "Tathagata Das" <no...@github.com> wrote:
    
    > I think the simplest solution is to assign zookeeper.connect. But you are
    > assigning it in KafkaCluster lines 338 - 345. So why is this warning being
    > thrown?
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798#issuecomment-72787965>.
    >



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23904486
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,249 @@ 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
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      batch: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    +    val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message)
    +    val kc = new KafkaCluster(kafkaParams)
    +    val topics = batch.map(o => TopicAndPartition(o.topic, o.partition)).toSet
    +    val leaderMap = kc.findLeaders(topics).fold(
    +      errs => throw new SparkException(errs.mkString("\n")),
    +      ok => ok
    +    )
    +    val rddParts = batch.zipWithIndex.map { case (o, i) =>
    +        val tp = TopicAndPartition(o.topic, o.partition)
    +        val (host, port) = leaderMap(tp)
    +        new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset, host, port)
    +    }.toArray
    +    new KafkaRDD[K, V, U, T, (K, V)](sc, kafkaParams, rddParts, messageHandler)
    +  }
    +
    +  /** 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 leaders Kafka leaders for each offset range in batch
    +   * @param messageHandler function for translating each message into the desired type
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      batch: Array[OffsetRange],
    +      leaders: Array[Leader],
    +      messageHandler: MessageAndMetadata[K, V] => R
    +  ): RDD[R] with HasOffsetRanges = {
    +    val leaderMap = leaders.map(l => (l.topic, l.partition) -> (l.host, l.port)).toMap
    +    val rddParts = batch.zipWithIndex.map { case (o, i) =>
    +        val (host, port) = leaderMap((o.topic, o.partition))
    +        new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset, host, port)
    +    }.toArray
    +
    +    new KafkaRDD[K, V, U, T, R](sc, kafkaParams, rddParts, messageHandler)
    +  }
    +
    +  /**
    +   * This stream can guarantee that each message from Kafka is included in transformations
    +   * (as opposed to output actions) exactly once, even in most failure situations.
    +   *
    +   * Points to note:
    +   *
    +   * Failure Recovery - You must checkpoint this stream, or save offsets yourself and provide them
    +   * as the fromOffsets parameter on restart.
    +   * Kafka must have sufficient log retention to obtain messages after failure.
    +   *
    +   * Getting offsets from the stream - see programming guide
    +   *
    +.  * Zookeeper - This does not use Zookeeper to store offsets.  For interop with Kafka monitors
    +   * that depend on Zookeeper, you must store offsets in ZK yourself.
    +   *
    +   * End-to-end semantics - This does not guarantee that any output operation will push each record
    +   * exactly once. To ensure end-to-end exactly-once semantics (that is, receiving exactly once and
    +   * outputting exactly once), you have to either ensure that the output operation is
    +   * idempotent, or transactionally store offsets with the output. See the programming guide for
    +   * more details.
    +   *
    +   * @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
    +   */
    +  @Experimental
    +  def createNewStream[
    +    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 stream can guarantee that each message from Kafka is included in transformations
    +   * (as opposed to output actions) exactly once, even in most failure situations.
    +   *
    +   * Points to note:
    +   *
    +   * Failure Recovery - You must checkpoint this stream.
    +   * Kafka must have sufficient log retention to obtain messages after failure.
    +   *
    +   * Getting offsets from the stream - see programming guide
    +   *
    +.  * Zookeeper - This does not use Zookeeper to store offsets.  For interop with Kafka monitors
    +   * that depend on Zookeeper, you must store offsets in ZK yourself.
    +   *
    +   * End-to-end semantics - This does not guarantee that any output operation will push each record
    +   * exactly once. To ensure end-to-end exactly-once semantics (that is, receiving exactly once and
    +   * outputting exactly once), you have to ensure that the output operation is idempotent.
    +   *
    +   * @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
    +   */
    +  @Experimental
    +  def createNewStream[
    +    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 {
    --- End diff --
    
    Is this "auto.offset.reset" just a hint for out-of-range re-position of offset, or just a immediate action to seek the offset to beginning or end? As I know this configuration is just a hint, will not be effective when the offset is valid, I'm not sure here your behavior, is the same as Kafka?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by dibbhatt <gi...@git.apache.org>.
Github user dibbhatt commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24010956
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,344 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.util.Random
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +import org.apache.spark.SparkException
    +
    +/**
    + * Convenience methods for interacting with a Kafka cluster.
    + * @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
    + */
    +private[spark]
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    +  import KafkaCluster.{Err, LeaderOffset}
    +
    +  val seedBrokers: Array[(String, Int)] =
    +    kafkaParams.get("metadata.broker.list")
    +      .orElse(kafkaParams.get("bootstrap.servers"))
    +      .getOrElse(throw new SparkException("Must specify metadata.broker.list or bootstrap.servers"))
    +      .split(",").map { hp =>
    +        val hpa = hp.split(":")
    +        (hpa(0), hpa(1).toInt)
    +      }
    +
    +  // ConsumerConfig isn't serializable
    +  @transient private var _config: ConsumerConfig = null
    +
    +  def config: ConsumerConfig = this.synchronized {
    +    if (_config == null) {
    +      _config = KafkaCluster.consumerConfig(kafkaParams)
    +    }
    +    _config
    +  }
    +
    +  def connect(host: String, port: Int): SimpleConsumer =
    +    new SimpleConsumer(host, port, config.socketTimeoutMs,
    +      config.socketReceiveBufferBytes, config.clientId)
    +
    +  def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] =
    +    findLeader(topic, partition).right.map(hp => connect(hp._1, hp._2))
    +
    +  // Metadata api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataAPI
    +  // scalastyle:on
    +
    +  def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, Seq(topic))
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      resp.topicsMetadata.find(_.topic == topic).flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.find(_.partitionId == partition)
    +      }.foreach { pm: PartitionMetadata =>
    +        pm.leader.foreach { leader =>
    +          return Right((leader.host, leader.port))
    +        }
    +      }
    +    }
    +    Left(errs)
    +  }
    +
    +  def findLeaders(
    +      topicAndPartitions: Set[TopicAndPartition]
    +    ): Either[Err, Map[TopicAndPartition, (String, Int)]] = {
    +    val topics = topicAndPartitions.map(_.topic)
    +    val response = getPartitionMetadata(topics).right
    +    val answer = response.flatMap { tms: Set[TopicMetadata] =>
    +      val leaderMap = tms.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.flatMap { pm: PartitionMetadata =>
    +          val tp = TopicAndPartition(tm.topic, pm.partitionId)
    +          if (topicAndPartitions(tp)) {
    +            pm.leader.map { l =>
    +              tp -> (l.host -> l.port)
    +            }
    +          } else {
    +            None
    +          }
    +        }
    +      }.toMap
    +
    +      if (leaderMap.keys.size == topicAndPartitions.size) {
    +        Right(leaderMap)
    +      } else {
    +        val missing = topicAndPartitions.diff(leaderMap.keySet)
    +        val err = new Err
    +        err.append(new SparkException(s"Couldn't find leaders for ${missing}"))
    +        Left(err)
    +      }
    +    }
    +    answer
    +  }
    +
    +  def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] = {
    +    getPartitionMetadata(topics).right.map { r =>
    +      r.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.map { pm: PartitionMetadata =>
    +          TopicAndPartition(tm.topic, pm.partitionId)
    +        }    
    +      }
    +    }
    +  }
    +
    +  def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = {
    +    val req = TopicMetadataRequest(
    +      TopicMetadataRequest.CurrentVersion, 0, config.clientId, topics.toSeq)
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      // error codes here indicate missing / just created topic,
    +      // repeating on a different broker wont be useful
    +      return Right(resp.topicsMetadata.toSet)
    +    }
    +    Left(errs)
    +  }
    +
    +  // Leader offset api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI
    +  // scalastyle:on
    +
    +  def getLatestLeaderOffsets(
    +      topicAndPartitions: Set[TopicAndPartition]
    +    ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime)
    +
    +  def getEarliestLeaderOffsets(
    +      topicAndPartitions: Set[TopicAndPartition]
    +    ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime)
    +
    +  def getLeaderOffsets(
    +      topicAndPartitions: Set[TopicAndPartition],
    +      before: Long
    +    ): Either[Err, Map[TopicAndPartition, LeaderOffset]] = {
    +    getLeaderOffsets(topicAndPartitions, before, 1).right.map { r =>
    +      r.map { kv =>
    +        // mapValues isnt serializable, see SI-7005
    +        kv._1 -> kv._2.head
    +      }
    +    }
    +  }
    +
    +  private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] =
    +    m.groupBy(_._2).map { kv =>
    +      kv._1 -> kv._2.keys.toSeq
    +    }
    +
    +  def getLeaderOffsets(
    +      topicAndPartitions: Set[TopicAndPartition],
    +      before: Long,
    +      maxNumOffsets: Int
    +    ): Either[Err, Map[TopicAndPartition, Seq[LeaderOffset]]] = {
    +    findLeaders(topicAndPartitions).right.flatMap { tpToLeader =>
    +      val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] = flip(tpToLeader)
    +      val leaders = leaderToTp.keys
    +      var result = Map[TopicAndPartition, Seq[LeaderOffset]]()
    +      val errs = new Err
    +      withBrokers(leaders, errs) { consumer =>
    +        val partitionsToGetOffsets: Seq[TopicAndPartition] =
    +          leaderToTp((consumer.host, consumer.port))
    +        val reqMap = partitionsToGetOffsets.map { tp: TopicAndPartition =>
    +          tp -> PartitionOffsetRequestInfo(before, maxNumOffsets)
    +        }.toMap
    +        val req = OffsetRequest(reqMap)
    +        val resp = consumer.getOffsetsBefore(req)
    +        val respMap = resp.partitionErrorAndOffsets
    +        partitionsToGetOffsets.foreach { tp: TopicAndPartition =>
    +          respMap.get(tp).foreach { por: PartitionOffsetsResponse =>
    +            if (por.error == ErrorMapping.NoError) {
    +              if (por.offsets.nonEmpty) {
    +                result += tp -> por.offsets.map { off =>
    +                  LeaderOffset(consumer.host, consumer.port, off)
    +                }
    +              } else {
    +                errs.append(new SparkException(
    +                  s"Empty offsets for ${tp}, is ${before} before log beginning?"))
    +              }
    +            } else {
    +              errs.append(ErrorMapping.exceptionFor(por.error))
    +            }
    +          }
    +        }
    +        if (result.keys.size == topicAndPartitions.size) {
    +          return Right(result)
    +        }
    +      }
    +      val missing = topicAndPartitions.diff(result.keySet)
    +      errs.append(new SparkException(s"Couldn't find leader offsets for ${missing}"))
    +      Left(errs)
    +    }
    +  }
    +
    +  // Consumer offset api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI
    +  // scalastyle:on
    +
    +  def getConsumerOffsets(
    +      groupId: String,
    +      topicAndPartitions: Set[TopicAndPartition]
    +    ): Either[Err, Map[TopicAndPartition, Long]] = {
    +    getConsumerOffsetMetadata(groupId, topicAndPartitions).right.map { r =>
    +      r.map { kv =>
    +        kv._1 -> kv._2.offset
    +      }
    +    }
    +  }
    +
    +  def getConsumerOffsetMetadata(
    +      groupId: String,
    +      topicAndPartitions: Set[TopicAndPartition]
    +    ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = {
    +    var result = Map[TopicAndPartition, OffsetMetadataAndError]()
    +    val req = OffsetFetchRequest(groupId, topicAndPartitions.toSeq)
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp = consumer.fetchOffsets(req)
    +      val respMap = resp.requestInfo
    +      val needed = topicAndPartitions.diff(result.keySet)
    +      needed.foreach { tp: TopicAndPartition =>
    +        respMap.get(tp).foreach { ome: OffsetMetadataAndError =>
    +          if (ome.error == ErrorMapping.NoError) {
    +            result += tp -> ome
    +          } else {
    +            errs.append(ErrorMapping.exceptionFor(ome.error))
    +          }
    +        }
    +      }
    +      if (result.keys.size == topicAndPartitions.size) {
    +        return Right(result)
    +      }
    +    }
    +    val missing = topicAndPartitions.diff(result.keySet)
    +    errs.append(new SparkException(s"Couldn't find consumer offsets for ${missing}"))
    +    Left(errs)
    +  }
    +
    +  def setConsumerOffsets(
    +      groupId: String,
    +      offsets: Map[TopicAndPartition, Long]
    +    ): Either[Err, Map[TopicAndPartition, Short]] = {
    +    setConsumerOffsetMetadata(groupId, offsets.map { kv =>
    +      kv._1 -> OffsetMetadataAndError(kv._2)
    +    })
    +  }
    +
    +  def setConsumerOffsetMetadata(
    +      groupId: String,
    +      metadata: Map[TopicAndPartition, OffsetMetadataAndError]
    +    ): Either[Err, Map[TopicAndPartition, Short]] = {
    +    var result = Map[TopicAndPartition, Short]()
    +    val req = OffsetCommitRequest(groupId, metadata)
    --- End diff --
    
    There is some incompatible wire protocol changes happened in OffsetCommitRequest from Kafka 0.8.1.1 to 0.8.2
    So I believe this API will break for Kafka 0.8.2 ?
    Some discussions you can find here : http://search-hadoop.com/m/4TaT4E02nE1&subj=Re+Consumer+and+offset+management+support+in+0+8+2+and+0+9
    
    Associated Kafka JIRA here : https://issues.apache.org/jira/browse/KAFKA-1841


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-68093197
  
    Hi @koeninger , several simple questions:
    
    1. How to map each RDD partition to Kafka partition, each Kafka partition is a RDD partition?
    2. How to do receiver injection rate control, in other words, how to decide at which offset current task should read?
    3. Do you have any consideration of fault tolerance?
    
    In general it is quite similar to what I did long ago a Kafka InputFormat (https://github.com/jerryshao/kafka-input-format) which can be loaded by HadoopRDD. I'm not sure is this  the streaming way of fixing the exact-once semantics?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71588806
  
    @tdas pinged me saying this needs more consensus on design before commenting on details so I will stop :) Anyway, I pointed out a few ways that hopefully can make this PR simpler to understand once we finalize the high level design.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24018579
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala ---
    @@ -0,0 +1,69 @@
    +/*
    + * 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.streaming.kafka
    +
    +import kafka.common.TopicAndPartition
    +
    +/** Something that has a collection of OffsetRanges */
    +trait HasOffsetRanges {
    +  def offsetRanges: Array[OffsetRange]
    +}
    +
    +/** Represents a range of offsets from a single Kafka TopicAndPartition */
    +final class OffsetRange private(
    +    /** kafka topic name */
    +    val topic: String,
    +    /** kafka partition id */
    +    val partition: Int,
    +    /** inclusive starting offset */
    +    val fromOffset: Long,
    +    /** exclusive ending offset */
    +    val untilOffset: Long) extends Serializable {
    +  import OffsetRange.OffsetRangeTuple
    +
    +  /** this is to avoid ClassNotFoundException during checkpoint restore */
    +  private[streaming]
    +  def toTuple: OffsetRangeTuple = (topic, partition, fromOffset, untilOffset)
    +}
    +
    +object OffsetRange {
    +  private[spark]
    +  type OffsetRangeTuple = (String, Int, Long, Long)
    +
    +  def create(topic: String, partition: Int, fromOffset: Long, untilOffset: Long): OffsetRange =
    --- End diff --
    
    TD thought that a static method named .create was more idiomatic for java.  It's obviously more idiomatic for scala to have a .apply method since the syntax sugar for it is baked into the language.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23823405
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala ---
    @@ -0,0 +1,71 @@
    +/*
    + * 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.streaming.kafka
    +
    +import kafka.common.TopicAndPartition
    +
    +/** Represents a range of offsets from a single Kafka TopicAndPartition */
    +trait OffsetRange {
    +  /** kafka topic name */
    +  def topic: String
    +
    +  /** kafka partition id */
    +  def partition: Int
    +
    +  /** inclusive starting offset */
    +  def fromOffset: Long
    +
    +  /** exclusive ending offset */
    +  def untilOffset: Long
    +}
    +
    +/** Something that has a collection of OffsetRanges */
    --- End diff --
    
    Same comment as Leader


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23988976
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala ---
    @@ -0,0 +1,69 @@
    +/*
    + * 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.streaming.kafka
    +
    +import kafka.common.TopicAndPartition
    +
    +/** Something that has a collection of OffsetRanges */
    +trait HasOffsetRanges {
    +  def offsetRanges: Array[OffsetRange]
    +}
    +
    +/** Represents a range of offsets from a single Kafka TopicAndPartition */
    +final class OffsetRange private(
    +    /** kafka topic name */
    +    val topic: String,
    +    /** kafka partition id */
    +    val partition: Int,
    +    /** inclusive starting offset */
    +    val fromOffset: Long,
    +    /** exclusive ending offset */
    +    val untilOffset: Long) extends Serializable {
    +  import OffsetRange.OffsetRangeTuple
    +
    +  /** this is to avoid ClassNotFoundException during checkpoint restore */
    --- End diff --
    
    This comment might be more helpful to include where `OffsetRangeTuple` is defined rather than here. I spent a long time trying to figure out why this extra class existed.
    
    Also, can you give a bit more detail. Not sure I see why you can't recover from a checkpoint safely provided that the recovering JVM has the class `OffsetRangeTuple` defined.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72159180
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/26381/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23989943
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,249 @@ 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
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    --- End diff --
    
    Isn't the returned RDD of type `RDD[R]`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-69652381
  
    Were you trying to read using the same version of Spark (your local
    development version, compiled into an assembly) as the one you were using
    to generate the checkpoint?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-69650699
  
    - Was the classloaded issue faced at the time of generating the checkpoint or reading from the checkpoint?
    - Well like RDDs, the DStream's compute method is meant for being overridden by subclasses. So subclassing is probably fine. Will have to look into the actual code to see whether it is easy to understand.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by helena <gi...@git.apache.org>.
Github user helena commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22649599
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,313 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    +  * Convenience methods for interacting with a Kafka cluster.
    +  * @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
    +  */
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    +  import KafkaCluster.Err
    +
    +  val seedBrokers: Array[(String, Int)] =
    +    kafkaParams.get("metadata.broker.list")
    +      .orElse(kafkaParams.get("bootstrap.servers"))
    +      .getOrElse(throw new Exception("Must specify metadata.broker.list or bootstrap.servers"))
    +      .split(",").map { hp =>
    +        val hpa = hp.split(":")
    +        (hpa(0), hpa(1).toInt)
    +      }
    +
    +  // ConsumerConfig isn't serializable
    +  @transient private var _config: ConsumerConfig = null
    +
    +  def config: ConsumerConfig = this.synchronized {
    +    if (_config == null) {
    +      _config = KafkaCluster.consumerConfig(kafkaParams)
    +    }
    +    _config
    +  }
    +
    +  def connect(host: String, port: Int): SimpleConsumer =
    +    new SimpleConsumer(host, port, config.socketTimeoutMs,
    +      config.socketReceiveBufferBytes, config.clientId)
    +
    +  def connect(hostAndPort: (String, Int)): SimpleConsumer =
    +    connect(hostAndPort._1, hostAndPort._2)
    +
    +  def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] =
    +    findLeader(topic, partition).right.map(connect)
    +
    +  def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, Seq(topic))
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      resp.topicsMetadata.find(_.topic == topic).flatMap { t =>
    +        t.partitionsMetadata.find(_.partitionId == partition)
    +      }.foreach { partitionMeta =>
    +        partitionMeta.leader.foreach { leader =>
    +          return Right((leader.host, leader.port))
    +        }
    +      }
    +    }
    +    Left(errs)
    +  }
    +
    +  def findLeaders(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, (String, Int)]] = {
    +    getPartitionMetadata(topicAndPartitions.map(_.topic)).right.flatMap { tms =>
    +      val result = tms.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.flatMap { pm =>
    +          val tp = TopicAndPartition(tm.topic, pm.partitionId)
    +          if (topicAndPartitions(tp)) {
    +            pm.leader.map { l =>
    +              tp -> (l.host -> l.port)
    +            }
    +          } else {
    +            None
    +          }
    +        }
    +      }.toMap
    +      if (result.keys.size == topicAndPartitions.size) {
    +        Right(result)
    +      } else {
    +        val missing = topicAndPartitions.diff(result.keys.toSet)
    +        val err = new Err
    +        err.append(new Exception(s"Couldn't find leaders for ${missing}"))
    +        Left(err)
    +      }
    +    }
    +  }
    +
    +  def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] =
    +    getPartitionMetadata(topics).right.map { r =>
    +      r.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.map { pm =>
    +          TopicAndPartition(tm.topic, pm.partitionId)
    +        }    
    +      }
    +    }
    +
    +  def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, topics.toSeq)
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      // error codes here indicate missing / just created topic,
    +      // repeating on a different broker wont be useful
    +      return Right(resp.topicsMetadata.toSet)
    +    }
    +    Left(errs)
    +  }
    +
    +  def getLatestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime)
    +
    +  def getEarliestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime)
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, before, 1).right.map { r =>
    +      r.map { kv =>
    +        // mapValues isnt serializable, see SI-7005
    +        kv._1 -> kv._2.head
    +      }
    +    }
    +
    +  private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] =
    +    m.groupBy(_._2).map { kv =>
    +      kv._1 -> kv._2.keys.toSeq
    +    }
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long,
    +    maxNumOffsets: Int
    +  ): Either[Err, Map[TopicAndPartition, Seq[Long]]] = {
    +    findLeaders(topicAndPartitions).right.flatMap { tpToLeader =>
    +      val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] = flip(tpToLeader)
    +      val leaders = leaderToTp.keys
    +      var result = Map[TopicAndPartition, Seq[Long]]()
    +      val errs = new Err
    +      withBrokers(leaders, errs) { consumer =>
    +        val needed: Seq[TopicAndPartition] = leaderToTp((consumer.host, consumer.port))
    +        val req = OffsetRequest(
    +          needed.map { tp =>
    +            tp -> PartitionOffsetRequestInfo(before, maxNumOffsets)
    +          }.toMap
    +        )
    +        val resp = consumer.getOffsetsBefore(req)
    +        val respMap = resp.partitionErrorAndOffsets
    +        needed.foreach { tp =>
    +          respMap.get(tp).foreach { errAndOffsets =>
    +            if (errAndOffsets.error == ErrorMapping.NoError) {
    +              if (errAndOffsets.offsets.nonEmpty) {
    +                result += tp -> errAndOffsets.offsets
    +              } else {
    +                errs.append(new Exception(
    +                  s"Empty offsets for ${tp}, is ${before} before log beginning?"))
    +              }
    +            } else {
    +              errs.append(ErrorMapping.exceptionFor(errAndOffsets.error))
    +            }
    +          }
    +        }
    +        if (result.keys.size == topicAndPartitions.size) {
    +          return Right(result)
    +        }
    +      }
    +      val missing = topicAndPartitions.diff(result.keys.toSet)
    +      errs.append(new Exception(s"Couldn't find leader offsets for ${missing}"))
    +      Left(errs)
    +    }
    +  }
    +
    +  def getConsumerOffsets(
    +    groupId: String,
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] = {
    +    getConsumerOffsetMetadata(groupId, topicAndPartitions).right.map { r =>
    +      r.map { kv =>
    +        kv._1 -> kv._2.offset
    +      }
    +    }
    +  }
    +
    +  def getConsumerOffsetMetadata(
    +    groupId: String,
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = {
    +    var result = Map[TopicAndPartition, OffsetMetadataAndError]()
    +    val req = OffsetFetchRequest(groupId, topicAndPartitions.toSeq)
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp = consumer.fetchOffsets(req)
    +      val respMap = resp.requestInfo
    +      val needed = topicAndPartitions.diff(result.keys.toSet)
    +      needed.foreach { tp =>
    +        respMap.get(tp).foreach { offsetMeta =>
    +          if (offsetMeta.error == ErrorMapping.NoError) {
    +            result += tp -> offsetMeta
    +          } else {
    +            errs.append(ErrorMapping.exceptionFor(offsetMeta.error))
    +          }
    +        }
    +      }
    +      if (result.keys.size == topicAndPartitions.size) {
    +        return Right(result)
    +      }
    +    }
    +    val missing = topicAndPartitions.diff(result.keys.toSet)
    +    errs.append(new Exception(s"Couldn't find consumer offsets for ${missing}"))
    +    Left(errs)
    +  }
    +
    +  def setConsumerOffsets(
    +    groupId: String,
    +    offsets: Map[TopicAndPartition, Long]
    +  ): Either[Err, Map[TopicAndPartition, Short]] = {
    +    setConsumerOffsetMetadata(groupId, offsets.map { kv =>
    +      kv._1 -> OffsetMetadataAndError(kv._2)
    +    })
    +  }
    +
    +  def setConsumerOffsetMetadata(
    +    groupId: String,
    +    metadata: Map[TopicAndPartition, OffsetMetadataAndError]
    +  ): Either[Err, Map[TopicAndPartition, Short]] = {
    +    var result = Map[TopicAndPartition, Short]()
    +    val req = OffsetCommitRequest(groupId, metadata)
    +    val errs = new Err
    +    val topicAndPartitions = metadata.keys.toSet
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp = consumer.commitOffsets(req)
    +      val respMap = resp.requestInfo
    +      val needed = topicAndPartitions.diff(result.keys.toSet)
    +      needed.foreach { tp =>
    +        respMap.get(tp).foreach { err =>
    +          if (err == ErrorMapping.NoError) {
    +            result += tp -> err
    +          } else {
    +            errs.append(ErrorMapping.exceptionFor(err))
    +          }
    +        }
    +      }
    +      if (result.keys.size == topicAndPartitions.size) {
    +        return Right(result)
    +      }
    +    }
    +    val missing = topicAndPartitions.diff(result.keys.toSet)
    +    errs.append(new Exception(s"Couldn't set offsets for ${missing}"))
    +    Left(errs)
    +  }
    +
    +  private def withBrokers(brokers: Iterable[(String, Int)], errs: Err)
    +    (fn: SimpleConsumer => Any): Unit = {
    +    brokers.foreach { hp =>
    +      var consumer: SimpleConsumer = null
    +      try {
    +        consumer = connect(hp)
    +        fn(consumer)
    +      } catch {
    +        case NonFatal(e) =>
    +          errs.append(e)
    +      } finally {
    +        if (consumer != null) consumer.close()
    --- End diff --
    
    Option(consumer) map(_.close())


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72775013
  
    Yes, Cody, I agree that this has changed a number of time. But this is not unusual for adding such a significant, publicly visible API such as this. USually this level of API arguments occurs over design docs over a period of time. However, since we were short on time in this matter and not all implementation complexities were evident from the beginning, this had to be done over the code. I can thank you enough for your cooperation!!!
    
    So the two main changes are 
    
    1. sc.createNewStream (maybe rename DeterministicKafkaStream to Direct.... )
    2. createRDD returns RDD
    
    There are smaller suggestions and issues regarding documentation, indentation, etc. However, I am willing to address them in another PR. Let's merge this after you have made these two changes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23972917
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,338 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.util.Random
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    + * Convenience methods for interacting with a Kafka cluster.
    + * @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
    + */
    +private[spark]
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    +  import KafkaCluster.{Err, LeaderOffset}
    +
    +  val seedBrokers: Array[(String, Int)] =
    +    kafkaParams.get("metadata.broker.list")
    +      .orElse(kafkaParams.get("bootstrap.servers"))
    +      .getOrElse(throw new Exception("Must specify metadata.broker.list or bootstrap.servers"))
    +      .split(",").map { hp =>
    +        val hpa = hp.split(":")
    +        (hpa(0), hpa(1).toInt)
    +      }
    +
    +  // ConsumerConfig isn't serializable
    +  @transient private var _config: ConsumerConfig = null
    +
    +  def config: ConsumerConfig = this.synchronized {
    +    if (_config == null) {
    +      _config = KafkaCluster.consumerConfig(kafkaParams)
    +    }
    +    _config
    +  }
    +
    +  def connect(host: String, port: Int): SimpleConsumer =
    +    new SimpleConsumer(host, port, config.socketTimeoutMs,
    +      config.socketReceiveBufferBytes, config.clientId)
    +
    +  def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] =
    +    findLeader(topic, partition).right.map(hp => connect(hp._1, hp._2))
    +
    +  // Metadata api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataAPI
    +  // scalastyle:on
    +
    +  def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, Seq(topic))
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      resp.topicsMetadata.find(_.topic == topic).flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.find(_.partitionId == partition)
    +      }.foreach { pm: PartitionMetadata =>
    +        pm.leader.foreach { leader =>
    +          return Right((leader.host, leader.port))
    +        }
    +      }
    +    }
    +    Left(errs)
    +  }
    +
    +  def findLeaders(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, (String, Int)]] = {
    +    val topics = topicAndPartitions.map(_.topic)
    +    getPartitionMetadata(topics).right.flatMap { tms: Set[TopicMetadata] =>
    +      val result = tms.flatMap { tm: TopicMetadata =>
    --- End diff --
    
    Can you break this up into multiple steps 
    ```
    val xxxxx = getPartitionMetadata(topcis).right
    val yyyyy = xxxxx.flatMap { tms: ...
    }
    ```
    where xxxx, yyyy are semantically meaningful names.
     



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23737948
  
    --- 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.
    --- End diff --
    
    If you're willing to add whatever is necessary, what I'm suggesting is necessary is a different method name.
    
    What's the harm in picking a more accurate method name?  
    
    Why open the possibility of someone looking at code completion in their IDE, seeing a method named "exactlyOnce", thinking it actually means exactly once, then losing money because their messages got duplicated?
    
    Many of the existing input stream classes would meet your definition of exactly once, but no where else in spark streaming is there a method named "exactlyOnce".  Why is this method special?
    
    The distinction you are drawing between receiving exactly once and outputting exactly once may be clear to you, but from talking with average users at conferences and meetups, it is not clear to them.  Judging from the comments on this thread and the design doc, it's not even perfectly clear to dedicated members of the community.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24031208
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,174 @@ 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    +    val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message)
    +    val kc = new KafkaCluster(kafkaParams)
    +    val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet
    +    val leaders = kc.findLeaders(topics).fold(
    +      errs => throw new SparkException(errs.mkString("\n")),
    +      ok => ok
    +    )
    +    new KafkaRDD[K, V, U, T, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler)
    +  }
    +
    +  /** 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   * @param leaders Kafka leaders for each offset range in batch
    +   * @param messageHandler function for translating each message into the desired type
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange],
    +      leaders: Array[Leader],
    +      messageHandler: MessageAndMetadata[K, V] => R
    +  ): RDD[R] with HasOffsetRanges = {
    +
    +    val leaderMap = leaders
    +      .map(l => TopicAndPartition(l.topic, l.partition) -> (l.host, l.port))
    +      .toMap
    +    new KafkaRDD[K, V, U, T, R](sc, kafkaParams, offsetRanges, leaderMap, messageHandler)
    +  }
    +
    +  /**
    +   * This stream can guarantee that each message from Kafka is included in transformations
    +   * (as opposed to output actions) exactly once, even in most failure situations.
    +   *
    +   * Points to note:
    +   *
    +   * Failure Recovery - You must checkpoint this stream, or save offsets yourself and provide them
    +   * as the fromOffsets parameter on restart.
    +   * Kafka must have sufficient log retention to obtain messages after failure.
    +   *
    +   * Getting offsets from the stream - see programming guide
    +   *
    +.  * Zookeeper - This does not use Zookeeper to store offsets.  For interop with Kafka monitors
    +   * that depend on Zookeeper, you must store offsets in ZK yourself.
    +   *
    +   * End-to-end semantics - This does not guarantee that any output operation will push each record
    +   * exactly once. To ensure end-to-end exactly-once semantics (that is, receiving exactly once and
    +   * outputting exactly once), you have to either ensure that the output operation is
    +   * idempotent, or transactionally store offsets with the output. See the programming guide for
    +   * more details.
    +   *
    +   * @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
    +   */
    +  @Experimental
    +  def createNewStream[
    +    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
    +  ): InputDStream[R] = {
    +    new DeterministicKafkaInputDStream[K, V, U, T, R](
    +      ssc, kafkaParams, fromOffsets, messageHandler)
    +  }
    +
    +  /**
    +   * This stream can guarantee that each message from Kafka is included in transformations
    +   * (as opposed to output actions) exactly once, even in most failure situations.
    --- End diff --
    
    I really think the best way to handle the exactly-once semantics question
    isn't in javadoc, but in separate programming guide that covers
    
    For at-most-once: do this...
    
    For at-least-once: do this...
    
    For exactly-once when you have idempotent storage of outputs: do this...
    
    For exactly-once when you have transactional storage of outputs: do this...
    
    For exactly-once when you have neither idempotent nor transactional
    storage:  Can't do it, outputs may be duplicated.
    
    
    Having the javadoc just say some variation on "this does exactly once" is
    really oversimplifying the situation IMHO.
    
    On Tue, Feb 3, 2015 at 1:03 PM, Patrick Wendell <no...@github.com>
    wrote:
    
    > In
    > external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
    > <https://github.com/apache/spark/pull/3798#discussion_r24029286>:
    >
    > > +    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
    > > +  ): InputDStream[R] = {
    > > +    new DeterministicKafkaInputDStream[K, V, U, T, R](
    > > +      ssc, kafkaParams, fromOffsets, messageHandler)
    > > +  }
    > > +
    > > +  /**
    > > +   * This stream can guarantee that each message from Kafka is included in transformations
    > > +   * (as opposed to output actions) exactly once, even in most failure situations.
    >
    > So what about saying it provides exactly once semantics subject to the
    > existing fault recovery mechanisms in streaming (i.e. checkpointing). It
    > just conveys basically no information now, it says it might work and might
    > not.
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798/files#r24029286>.
    >


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22366683
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,305 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    +  * Convenience methods for interacting with a Kafka cluster.
    +  * @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
    +  */
    +class KafkaCluster(val kafkaParams: Map[String, String]) {
    --- End diff --
    
    for example
    
    https://github.com/koeninger/kafka-exactly-once/blob/master/src/main/scala/example/IdempotentExample.scala#L60
    
    We also use it for doing things like e.g. starting a stream at the leader offsets before a given time


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24028967
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,174 @@ 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    +    val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message)
    +    val kc = new KafkaCluster(kafkaParams)
    +    val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet
    +    val leaders = kc.findLeaders(topics).fold(
    +      errs => throw new SparkException(errs.mkString("\n")),
    +      ok => ok
    +    )
    +    new KafkaRDD[K, V, U, T, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler)
    +  }
    +
    +  /** 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   * @param leaders Kafka leaders for each offset range in batch
    +   * @param messageHandler function for translating each message into the desired type
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange],
    +      leaders: Array[Leader],
    +      messageHandler: MessageAndMetadata[K, V] => R
    +  ): RDD[R] with HasOffsetRanges = {
    +
    +    val leaderMap = leaders
    +      .map(l => TopicAndPartition(l.topic, l.partition) -> (l.host, l.port))
    +      .toMap
    +    new KafkaRDD[K, V, U, T, R](sc, kafkaParams, offsetRanges, leaderMap, messageHandler)
    +  }
    +
    +  /**
    +   * This stream can guarantee that each message from Kafka is included in transformations
    --- End diff --
    
    That applies to people who already know what "createStream" does. But think about the people who is looking at KafkaUtilsl forthe first time, and happens to look upon this method first. Unless the first sentence conveys what this does, they will get extremely confused. 
    
    As a developer I think the question i need answering first is "what does this do". "what does this do different from the other one" comes second, and only after I know what this one does. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by helena <gi...@git.apache.org>.
Github user helena commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22649992
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,313 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    +  * Convenience methods for interacting with a Kafka cluster.
    +  * @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
    +  */
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    +  import KafkaCluster.Err
    +
    +  val seedBrokers: Array[(String, Int)] =
    +    kafkaParams.get("metadata.broker.list")
    +      .orElse(kafkaParams.get("bootstrap.servers"))
    +      .getOrElse(throw new Exception("Must specify metadata.broker.list or bootstrap.servers"))
    +      .split(",").map { hp =>
    +        val hpa = hp.split(":")
    +        (hpa(0), hpa(1).toInt)
    +      }
    +
    +  // ConsumerConfig isn't serializable
    +  @transient private var _config: ConsumerConfig = null
    +
    +  def config: ConsumerConfig = this.synchronized {
    +    if (_config == null) {
    +      _config = KafkaCluster.consumerConfig(kafkaParams)
    +    }
    +    _config
    +  }
    +
    +  def connect(host: String, port: Int): SimpleConsumer =
    +    new SimpleConsumer(host, port, config.socketTimeoutMs,
    +      config.socketReceiveBufferBytes, config.clientId)
    +
    +  def connect(hostAndPort: (String, Int)): SimpleConsumer =
    +    connect(hostAndPort._1, hostAndPort._2)
    +
    +  def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] =
    +    findLeader(topic, partition).right.map(connect)
    +
    +  def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, Seq(topic))
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      resp.topicsMetadata.find(_.topic == topic).flatMap { t =>
    +        t.partitionsMetadata.find(_.partitionId == partition)
    +      }.foreach { partitionMeta =>
    +        partitionMeta.leader.foreach { leader =>
    +          return Right((leader.host, leader.port))
    +        }
    +      }
    +    }
    +    Left(errs)
    +  }
    +
    +  def findLeaders(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, (String, Int)]] = {
    +    getPartitionMetadata(topicAndPartitions.map(_.topic)).right.flatMap { tms =>
    +      val result = tms.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.flatMap { pm =>
    +          val tp = TopicAndPartition(tm.topic, pm.partitionId)
    +          if (topicAndPartitions(tp)) {
    +            pm.leader.map { l =>
    +              tp -> (l.host -> l.port)
    +            }
    +          } else {
    +            None
    +          }
    +        }
    +      }.toMap
    +      if (result.keys.size == topicAndPartitions.size) {
    +        Right(result)
    +      } else {
    +        val missing = topicAndPartitions.diff(result.keys.toSet)
    +        val err = new Err
    +        err.append(new Exception(s"Couldn't find leaders for ${missing}"))
    +        Left(err)
    +      }
    +    }
    +  }
    +
    +  def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] =
    +    getPartitionMetadata(topics).right.map { r =>
    +      r.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.map { pm =>
    +          TopicAndPartition(tm.topic, pm.partitionId)
    +        }    
    +      }
    +    }
    +
    +  def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, topics.toSeq)
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      // error codes here indicate missing / just created topic,
    +      // repeating on a different broker wont be useful
    +      return Right(resp.topicsMetadata.toSet)
    +    }
    +    Left(errs)
    +  }
    +
    +  def getLatestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime)
    +
    +  def getEarliestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime)
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, before, 1).right.map { r =>
    +      r.map { kv =>
    +        // mapValues isnt serializable, see SI-7005
    +        kv._1 -> kv._2.head
    +      }
    +    }
    +
    +  private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] =
    +    m.groupBy(_._2).map { kv =>
    +      kv._1 -> kv._2.keys.toSeq
    +    }
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long,
    +    maxNumOffsets: Int
    +  ): Either[Err, Map[TopicAndPartition, Seq[Long]]] = {
    +    findLeaders(topicAndPartitions).right.flatMap { tpToLeader =>
    +      val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] = flip(tpToLeader)
    +      val leaders = leaderToTp.keys
    +      var result = Map[TopicAndPartition, Seq[Long]]()
    +      val errs = new Err
    +      withBrokers(leaders, errs) { consumer =>
    +        val needed: Seq[TopicAndPartition] = leaderToTp((consumer.host, consumer.port))
    +        val req = OffsetRequest(
    +          needed.map { tp =>
    +            tp -> PartitionOffsetRequestInfo(before, maxNumOffsets)
    +          }.toMap
    +        )
    +        val resp = consumer.getOffsetsBefore(req)
    +        val respMap = resp.partitionErrorAndOffsets
    +        needed.foreach { tp =>
    +          respMap.get(tp).foreach { errAndOffsets =>
    +            if (errAndOffsets.error == ErrorMapping.NoError) {
    +              if (errAndOffsets.offsets.nonEmpty) {
    +                result += tp -> errAndOffsets.offsets
    +              } else {
    +                errs.append(new Exception(
    +                  s"Empty offsets for ${tp}, is ${before} before log beginning?"))
    +              }
    +            } else {
    +              errs.append(ErrorMapping.exceptionFor(errAndOffsets.error))
    +            }
    +          }
    +        }
    +        if (result.keys.size == topicAndPartitions.size) {
    +          return Right(result)
    +        }
    +      }
    +      val missing = topicAndPartitions.diff(result.keys.toSet)
    +      errs.append(new Exception(s"Couldn't find leader offsets for ${missing}"))
    +      Left(errs)
    +    }
    +  }
    +
    +  def getConsumerOffsets(
    +    groupId: String,
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] = {
    +    getConsumerOffsetMetadata(groupId, topicAndPartitions).right.map { r =>
    +      r.map { kv =>
    +        kv._1 -> kv._2.offset
    +      }
    +    }
    +  }
    +
    +  def getConsumerOffsetMetadata(
    +    groupId: String,
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = {
    +    var result = Map[TopicAndPartition, OffsetMetadataAndError]()
    +    val req = OffsetFetchRequest(groupId, topicAndPartitions.toSeq)
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp = consumer.fetchOffsets(req)
    +      val respMap = resp.requestInfo
    +      val needed = topicAndPartitions.diff(result.keys.toSet)
    +      needed.foreach { tp =>
    +        respMap.get(tp).foreach { offsetMeta =>
    +          if (offsetMeta.error == ErrorMapping.NoError) {
    +            result += tp -> offsetMeta
    +          } else {
    +            errs.append(ErrorMapping.exceptionFor(offsetMeta.error))
    +          }
    +        }
    +      }
    +      if (result.keys.size == topicAndPartitions.size) {
    +        return Right(result)
    +      }
    +    }
    +    val missing = topicAndPartitions.diff(result.keys.toSet)
    +    errs.append(new Exception(s"Couldn't find consumer offsets for ${missing}"))
    +    Left(errs)
    +  }
    +
    +  def setConsumerOffsets(
    +    groupId: String,
    +    offsets: Map[TopicAndPartition, Long]
    +  ): Either[Err, Map[TopicAndPartition, Short]] = {
    +    setConsumerOffsetMetadata(groupId, offsets.map { kv =>
    +      kv._1 -> OffsetMetadataAndError(kv._2)
    +    })
    +  }
    +
    +  def setConsumerOffsetMetadata(
    +    groupId: String,
    +    metadata: Map[TopicAndPartition, OffsetMetadataAndError]
    +  ): Either[Err, Map[TopicAndPartition, Short]] = {
    +    var result = Map[TopicAndPartition, Short]()
    +    val req = OffsetCommitRequest(groupId, metadata)
    +    val errs = new Err
    +    val topicAndPartitions = metadata.keys.toSet
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp = consumer.commitOffsets(req)
    +      val respMap = resp.requestInfo
    +      val needed = topicAndPartitions.diff(result.keys.toSet)
    +      needed.foreach { tp =>
    +        respMap.get(tp).foreach { err =>
    +          if (err == ErrorMapping.NoError) {
    +            result += tp -> err
    +          } else {
    +            errs.append(ErrorMapping.exceptionFor(err))
    +          }
    +        }
    +      }
    +      if (result.keys.size == topicAndPartitions.size) {
    +        return Right(result)
    --- End diff --
    
    return? how about just 
       Right(result)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23889820
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala ---
    @@ -0,0 +1,220 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.reflect.{classTag, ClassTag}
    +
    +import org.apache.spark.{Logging, Partition, SparkContext, SparkException, TaskContext}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.NextIterator
    +
    +import java.util.Properties
    +import kafka.api.{FetchRequestBuilder, FetchResponse}
    +import kafka.common.{ErrorMapping, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +import kafka.message.{MessageAndMetadata, MessageAndOffset}
    +import kafka.serializer.Decoder
    +import kafka.utils.VerifiableProperties
    +
    +/**
    + * 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 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 KafkaRDDPartition 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
    + */
    +private[spark]
    +class KafkaRDD[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag] private[spark] (
    +    sc: SparkContext,
    +    kafkaParams: Map[String, String],
    +    private[spark] val batch: Array[KafkaRDDPartition],
    +    messageHandler: MessageAndMetadata[K, V] => R
    +  ) extends RDD[R](sc, Nil) with Logging with HasOffsetRanges {
    +
    +  def offsetRanges: Array[OffsetRange] = batch.asInstanceOf[Array[OffsetRange]]
    +
    +  override def getPartitions: Array[Partition] = batch.asInstanceOf[Array[Partition]]
    +
    +  override def getPreferredLocations(thePart: Partition): Seq[String] = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    // TODO is additional hostname resolution necessary here
    +    Seq(part.host)
    +  }
    +
    +  private def errBeginAfterEnd(part: KafkaRDDPartition): String =
    +    s"Beginning offset ${part.fromOffset} is after the ending offset ${part.untilOffset} " +
    +      s"for topic ${part.topic} partition ${part.partition}. " +
    +      "You either provided an invalid fromOffset, or the Kafka topic has been damaged"
    +
    +  private def errRanOutBeforeEnd(part: KafkaRDDPartition): String =
    +    s"Ran out of messages before reaching ending offset ${part.untilOffset} " +
    +    s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." +
    +    " This should not happen, and indicates that messages may have been lost"
    +
    +  private def errOvershotEnd(itemOffset: Long, part: KafkaRDDPartition): String =
    +    s"Got ${itemOffset} > ending offset ${part.untilOffset} " +
    +    s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." +
    +    " This should not happen, and indicates a message may have been skipped"
    +
    +  override def compute(thePart: Partition, context: TaskContext): Iterator[R] = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    assert(part.fromOffset <= part.untilOffset, errBeginAfterEnd(part))
    +    if (part.fromOffset == part.untilOffset) {
    +      log.warn("Beginning offset ${part.fromOffset} is the same as ending offset " +
    +        s"skipping ${part.topic} ${part.partition}")
    +      Iterator.empty
    +    } else {
    +      new KafkaRDDIterator(part, context)
    +    }
    +  }
    +
    +  private class KafkaRDDIterator(
    +      part: KafkaRDDPartition,
    +      context: TaskContext) extends NextIterator[R] {
    +
    +    context.addTaskCompletionListener{ context => closeIfNeeded() }
    +
    +    log.info(s"Computing topic ${part.topic}, partition ${part.partition} " +
    +      s"offsets ${part.fromOffset} -> ${part.untilOffset}")
    +
    +    val kc = new KafkaCluster(kafkaParams)
    +    val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +      .newInstance(kc.config.props)
    +      .asInstanceOf[Decoder[K]]
    +    val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +      .newInstance(kc.config.props)
    +      .asInstanceOf[Decoder[V]]
    +    val consumer = connectLeader
    +    var requestOffset = part.fromOffset
    +    var iter: Iterator[MessageAndOffset] = null
    +
    +    // The idea is to use the provided preferred host, except on task retry atttempts,
    +    // to minimize number of kafka metadata requests
    +    private def connectLeader: SimpleConsumer = {
    +      if (context.attemptNumber > 0) {
    +        kc.connectLeader(part.topic, part.partition).fold(
    +          errs => throw new SparkException(
    +            s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " +
    +              errs.mkString("\n")),
    +          consumer => consumer
    +        )
    +      } else {
    +        kc.connect(part.host, part.port)
    +      }
    +    }
    +
    +    private def handleFetchErr(resp: FetchResponse) {
    +      if (resp.hasError) {
    +        val err = resp.errorCode(part.topic, part.partition)
    +        if (err == ErrorMapping.LeaderNotAvailableCode ||
    +          err == ErrorMapping.NotLeaderForPartitionCode) {
    +          log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " +
    +            s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms")
    +          Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +        }
    +        // Let normal rdd retry sort out reconnect attempts
    +        throw ErrorMapping.exceptionFor(err)
    +      }
    +    }
    +
    --- End diff --
    
    If the offset is out of range, then there is no way to proceed without
    breaking the specification of KafkaRdd.  It should be up to the client how
    to handle that error, silently skipping to the beginning or end of the
    Kafka log isn't what we want to do.
    On Jan 31, 2015 9:42 AM, "Dibyendu Bhattacharya" <no...@github.com>
    wrote:
    
    > In
    > external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala
    > <https://github.com/apache/spark/pull/3798#discussion_r23889636>:
    >
    > > +    }
    > > +
    > > +    private def handleFetchErr(resp: FetchResponse) {
    > > +      if (resp.hasError) {
    > > +        val err = resp.errorCode(part.topic, part.partition)
    > > +        if (err == ErrorMapping.LeaderNotAvailableCode ||
    > > +          err == ErrorMapping.NotLeaderForPartitionCode) {
    > > +          log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " +
    > > +            s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms")
    > > +          Thread.sleep(kc.config.refreshLeaderBackoffMs)
    > > +        }
    > > +        // Let normal rdd retry sort out reconnect attempts
    > > +        throw ErrorMapping.exceptionFor(err)
    > > +      }
    > > +    }
    > > +
    >
    > One of the very common Kafka error may come which is OFFSET_OUT_OF_RANGE .
    > Then Normal RDD retry may not solve this problem. You may need to
    > explicitly either start from earlier offset or latest offset to solve this
    > issue.
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798/files#r23889636>.
    >


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23573027
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaRDD.scala ---
    @@ -0,0 +1,199 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.reflect.{classTag, ClassTag}
    +
    +import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.NextIterator
    +
    +import java.util.Properties
    +import kafka.api.{FetchRequestBuilder, FetchResponse}
    +import kafka.common.{ErrorMapping, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +import kafka.message.{MessageAndMetadata, MessageAndOffset}
    +import kafka.serializer.Decoder
    +import kafka.utils.VerifiableProperties
    +
    +/** A batch-oriented interface for consuming from Kafka.
    +  * Starting and ending offsets are specified in advance,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 KafkaRDDPartition 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
    +  */
    +class KafkaRDD[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    sc: SparkContext,
    +    val kafkaParams: Map[String, String],
    +    val batch: Array[KafkaRDDPartition],
    +    messageHandler: MessageAndMetadata[K, V] => R
    +  ) extends RDD[R](sc, Nil) with Logging {
    +
    +  override def getPartitions: Array[Partition] = batch.asInstanceOf[Array[Partition]]
    +
    +  override def getPreferredLocations(thePart: Partition): Seq[String] = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    // TODO is additional hostname resolution necessary here
    +    Seq(part.host)
    +  }
    +
    +  override def compute(thePart: Partition, context: TaskContext): Iterator[R] = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    if (part.fromOffset >= part.untilOffset) {
    +      log.warn("Beginning offset is same or after ending offset " +
    +        s"skipping ${part.topic} ${part.partition}")
    +      Iterator.empty
    +    } else {
    +      new NextIterator[R] {
    +        context.addTaskCompletionListener{ context => closeIfNeeded() }
    +
    +        log.info(s"Computing topic ${part.topic}, partition ${part.partition} " +
    +          s"offsets ${part.fromOffset} -> ${part.untilOffset}")
    +
    +        val kc = new KafkaCluster(kafkaParams)
    +        val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[K]]
    +        val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[V]]
    +        val consumer = connectLeader
    +        var requestOffset = part.fromOffset
    +        var iter: Iterator[MessageAndOffset] = null
    +
    +        // The idea is to use the provided preferred host, except on task retry atttempts,
    +        // to minimize number of kafka metadata requests
    +        private def connectLeader: SimpleConsumer = {
    +          if (context.attemptNumber > 0) {
    +            kc.connectLeader(part.topic, part.partition).fold(
    +              errs => throw new Exception(
    +                s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " +
    +                  errs.mkString("\n")),
    +              consumer => consumer
    +            )
    +          } else {
    +            kc.connect(part.host, part.port)
    +          }
    +        }
    +
    +        private def handleErr(resp: FetchResponse) {
    +          if (resp.hasError) {
    +            val err = resp.errorCode(part.topic, part.partition)
    +            if (err == ErrorMapping.LeaderNotAvailableCode ||
    +              err == ErrorMapping.NotLeaderForPartitionCode) {
    +              log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " +
    +                s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms")
    +              Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +            }
    +            // Let normal rdd retry sort out reconnect attempts
    +            throw ErrorMapping.exceptionFor(err)
    +          }
    +        }
    +
    +        override def close() = consumer.close()
    +
    +        override def getNext: R = {
    +          if (iter == null || !iter.hasNext) {
    +            val req = new FetchRequestBuilder().
    +              addFetch(part.topic, part.partition, requestOffset, kc.config.fetchMessageMaxBytes).
    +              build()
    +            val resp = consumer.fetch(req)
    +            handleErr(resp)
    +            // kafka may return a batch that starts before the requested offset
    +            iter = resp.messageSet(part.topic, part.partition)
    +              .iterator
    +              .dropWhile(_.offset < requestOffset)
    +          }
    +          if (!iter.hasNext) {
    +            assert(requestOffset == part.untilOffset,
    +              s"ran out of messages before reaching ending offset ${part.untilOffset} " +
    +                s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." +
    +                " This should not happen, and indicates that messages may have been lost")
    +            finished = true
    +            null.asInstanceOf[R]
    +          } else {
    +            val item = iter.next
    +            if (item.offset >= part.untilOffset) {
    +              assert(item.offset == part.untilOffset,
    +                s"got ${item.offset} > ending offset ${part.untilOffset} " +
    +                  s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." +
    +                  " This should not happen, and indicates a message may have been skipped")
    +              finished = true
    +              null.asInstanceOf[R]
    +            } else {
    +              requestOffset = item.nextOffset
    +              messageHandler(new MessageAndMetadata(
    +                part.topic, part.partition, item.message, item.offset, keyDecoder, valueDecoder))
    +            }
    --- End diff --
    
    this level of conditional / structure nesting seems scary. Any chance we can reduce this?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24020676
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,344 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.util.Random
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +import org.apache.spark.SparkException
    +
    +/**
    + * Convenience methods for interacting with a Kafka cluster.
    + * @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
    + */
    +private[spark]
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    +  import KafkaCluster.{Err, LeaderOffset}
    +
    +  val seedBrokers: Array[(String, Int)] =
    +    kafkaParams.get("metadata.broker.list")
    +      .orElse(kafkaParams.get("bootstrap.servers"))
    +      .getOrElse(throw new SparkException("Must specify metadata.broker.list or bootstrap.servers"))
    +      .split(",").map { hp =>
    +        val hpa = hp.split(":")
    +        (hpa(0), hpa(1).toInt)
    +      }
    +
    +  // ConsumerConfig isn't serializable
    +  @transient private var _config: ConsumerConfig = null
    +
    +  def config: ConsumerConfig = this.synchronized {
    +    if (_config == null) {
    +      _config = KafkaCluster.consumerConfig(kafkaParams)
    +    }
    +    _config
    +  }
    +
    +  def connect(host: String, port: Int): SimpleConsumer =
    +    new SimpleConsumer(host, port, config.socketTimeoutMs,
    +      config.socketReceiveBufferBytes, config.clientId)
    +
    +  def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] =
    +    findLeader(topic, partition).right.map(hp => connect(hp._1, hp._2))
    +
    +  // Metadata api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataAPI
    +  // scalastyle:on
    +
    +  def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, Seq(topic))
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      resp.topicsMetadata.find(_.topic == topic).flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.find(_.partitionId == partition)
    +      }.foreach { pm: PartitionMetadata =>
    +        pm.leader.foreach { leader =>
    +          return Right((leader.host, leader.port))
    +        }
    +      }
    +    }
    +    Left(errs)
    +  }
    +
    +  def findLeaders(
    +      topicAndPartitions: Set[TopicAndPartition]
    +    ): Either[Err, Map[TopicAndPartition, (String, Int)]] = {
    +    val topics = topicAndPartitions.map(_.topic)
    +    val response = getPartitionMetadata(topics).right
    +    val answer = response.flatMap { tms: Set[TopicMetadata] =>
    +      val leaderMap = tms.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.flatMap { pm: PartitionMetadata =>
    +          val tp = TopicAndPartition(tm.topic, pm.partitionId)
    +          if (topicAndPartitions(tp)) {
    +            pm.leader.map { l =>
    +              tp -> (l.host -> l.port)
    +            }
    +          } else {
    +            None
    +          }
    +        }
    +      }.toMap
    +
    +      if (leaderMap.keys.size == topicAndPartitions.size) {
    +        Right(leaderMap)
    +      } else {
    +        val missing = topicAndPartitions.diff(leaderMap.keySet)
    +        val err = new Err
    +        err.append(new SparkException(s"Couldn't find leaders for ${missing}"))
    +        Left(err)
    +      }
    +    }
    +    answer
    +  }
    +
    +  def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] = {
    +    getPartitionMetadata(topics).right.map { r =>
    +      r.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.map { pm: PartitionMetadata =>
    +          TopicAndPartition(tm.topic, pm.partitionId)
    +        }    
    +      }
    +    }
    +  }
    +
    +  def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = {
    +    val req = TopicMetadataRequest(
    +      TopicMetadataRequest.CurrentVersion, 0, config.clientId, topics.toSeq)
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      // error codes here indicate missing / just created topic,
    +      // repeating on a different broker wont be useful
    +      return Right(resp.topicsMetadata.toSet)
    +    }
    +    Left(errs)
    +  }
    +
    +  // Leader offset api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI
    +  // scalastyle:on
    +
    +  def getLatestLeaderOffsets(
    +      topicAndPartitions: Set[TopicAndPartition]
    +    ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime)
    +
    +  def getEarliestLeaderOffsets(
    +      topicAndPartitions: Set[TopicAndPartition]
    +    ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime)
    +
    +  def getLeaderOffsets(
    +      topicAndPartitions: Set[TopicAndPartition],
    +      before: Long
    +    ): Either[Err, Map[TopicAndPartition, LeaderOffset]] = {
    +    getLeaderOffsets(topicAndPartitions, before, 1).right.map { r =>
    +      r.map { kv =>
    +        // mapValues isnt serializable, see SI-7005
    +        kv._1 -> kv._2.head
    +      }
    +    }
    +  }
    +
    +  private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] =
    +    m.groupBy(_._2).map { kv =>
    +      kv._1 -> kv._2.keys.toSeq
    +    }
    +
    +  def getLeaderOffsets(
    +      topicAndPartitions: Set[TopicAndPartition],
    +      before: Long,
    +      maxNumOffsets: Int
    +    ): Either[Err, Map[TopicAndPartition, Seq[LeaderOffset]]] = {
    +    findLeaders(topicAndPartitions).right.flatMap { tpToLeader =>
    +      val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] = flip(tpToLeader)
    +      val leaders = leaderToTp.keys
    +      var result = Map[TopicAndPartition, Seq[LeaderOffset]]()
    +      val errs = new Err
    +      withBrokers(leaders, errs) { consumer =>
    +        val partitionsToGetOffsets: Seq[TopicAndPartition] =
    +          leaderToTp((consumer.host, consumer.port))
    +        val reqMap = partitionsToGetOffsets.map { tp: TopicAndPartition =>
    +          tp -> PartitionOffsetRequestInfo(before, maxNumOffsets)
    +        }.toMap
    +        val req = OffsetRequest(reqMap)
    +        val resp = consumer.getOffsetsBefore(req)
    +        val respMap = resp.partitionErrorAndOffsets
    +        partitionsToGetOffsets.foreach { tp: TopicAndPartition =>
    +          respMap.get(tp).foreach { por: PartitionOffsetsResponse =>
    +            if (por.error == ErrorMapping.NoError) {
    +              if (por.offsets.nonEmpty) {
    +                result += tp -> por.offsets.map { off =>
    +                  LeaderOffset(consumer.host, consumer.port, off)
    +                }
    +              } else {
    +                errs.append(new SparkException(
    +                  s"Empty offsets for ${tp}, is ${before} before log beginning?"))
    +              }
    +            } else {
    +              errs.append(ErrorMapping.exceptionFor(por.error))
    +            }
    +          }
    +        }
    +        if (result.keys.size == topicAndPartitions.size) {
    +          return Right(result)
    +        }
    +      }
    +      val missing = topicAndPartitions.diff(result.keySet)
    +      errs.append(new SparkException(s"Couldn't find leader offsets for ${missing}"))
    +      Left(errs)
    +    }
    +  }
    +
    +  // Consumer offset api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI
    +  // scalastyle:on
    +
    +  def getConsumerOffsets(
    +      groupId: String,
    +      topicAndPartitions: Set[TopicAndPartition]
    +    ): Either[Err, Map[TopicAndPartition, Long]] = {
    +    getConsumerOffsetMetadata(groupId, topicAndPartitions).right.map { r =>
    --- End diff --
    
    No, it just means those particular methods won't work without an 0.8.1.1 server.  Those methods aren't required for the basic solution to work, only if you're interacting with existing kafka zk backed consumer offsets.  I'll update the doc on those methods in case the ever end up getting publicly exposed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72691392
  
    Regarding naming, I agree.  The name has been a point of discussion for a
    month, how to get some consensus?
    
    Regarding Java wrappers, there have already been a number of changes
    directed at java compatibility (array of Leader instead of a map[topic,
    broker], .create in addition to .apply).  I wonder how relevant those are
    if we're doing separate java wrappers (which yes, I agree should be in a
    follow-on PR)
    
    On Tue, Feb 3, 2015 at 3:13 AM, Patrick Wendell <no...@github.com>
    wrote:
    
    > I took a pass through the public API. I'm not very familiar with Kafka so
    > it was somewhat slow going. However, some reactions:
    >
    >    1. We should try to tighten, simplify, and clarify the way we name and
    >    document everything in this public API. Most of the comments were about
    >    this. The most important IMO is coming up with a good name for the new
    >    streams returned and clearly explaining how they differ from the old Kafka
    >    stream. To me, the main differences seems to be in the way we (a) decide
    >    what goes into which batch and (b) actually ingest the data. I proposed
    >    javadoc and naming scheme that emphasizing that distinction.
    >    2. Is there plans to add a Java and Python wrappers here next? Those
    >    are straightforward and it would be good to have them. Maybe in a follow on
    >    PR?
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798#issuecomment-72617088>.
    >


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23988871
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala ---
    @@ -0,0 +1,69 @@
    +/*
    + * 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.streaming.kafka
    +
    +import kafka.common.TopicAndPartition
    +
    +/** Something that has a collection of OffsetRanges */
    +trait HasOffsetRanges {
    +  def offsetRanges: Array[OffsetRange]
    +}
    +
    +/** Represents a range of offsets from a single Kafka TopicAndPartition */
    +final class OffsetRange private(
    +    /** kafka topic name */
    +    val topic: String,
    +    /** kafka partition id */
    +    val partition: Int,
    +    /** inclusive starting offset */
    +    val fromOffset: Long,
    +    /** exclusive ending offset */
    +    val untilOffset: Long) extends Serializable {
    +  import OffsetRange.OffsetRangeTuple
    +
    +  /** this is to avoid ClassNotFoundException during checkpoint restore */
    +  private[streaming]
    +  def toTuple: OffsetRangeTuple = (topic, partition, fromOffset, untilOffset)
    +}
    +
    +object OffsetRange {
    +  private[spark]
    +  type OffsetRangeTuple = (String, Int, Long, Long)
    +
    +  def create(topic: String, partition: Int, fromOffset: Long, untilOffset: Long): OffsetRange =
    --- End diff --
    
    It's confusing to have both `create` and the apply methods here. Why not just have one way of creating these?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by dibbhatt <gi...@git.apache.org>.
Github user dibbhatt commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23999633
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,344 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.util.Random
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +import org.apache.spark.SparkException
    +
    +/**
    + * Convenience methods for interacting with a Kafka cluster.
    + * @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
    + */
    +private[spark]
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    +  import KafkaCluster.{Err, LeaderOffset}
    +
    +  val seedBrokers: Array[(String, Int)] =
    +    kafkaParams.get("metadata.broker.list")
    +      .orElse(kafkaParams.get("bootstrap.servers"))
    +      .getOrElse(throw new SparkException("Must specify metadata.broker.list or bootstrap.servers"))
    +      .split(",").map { hp =>
    +        val hpa = hp.split(":")
    +        (hpa(0), hpa(1).toInt)
    +      }
    +
    +  // ConsumerConfig isn't serializable
    +  @transient private var _config: ConsumerConfig = null
    +
    +  def config: ConsumerConfig = this.synchronized {
    +    if (_config == null) {
    +      _config = KafkaCluster.consumerConfig(kafkaParams)
    +    }
    +    _config
    +  }
    +
    +  def connect(host: String, port: Int): SimpleConsumer =
    +    new SimpleConsumer(host, port, config.socketTimeoutMs,
    +      config.socketReceiveBufferBytes, config.clientId)
    +
    +  def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] =
    +    findLeader(topic, partition).right.map(hp => connect(hp._1, hp._2))
    +
    +  // Metadata api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataAPI
    +  // scalastyle:on
    +
    +  def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, Seq(topic))
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      resp.topicsMetadata.find(_.topic == topic).flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.find(_.partitionId == partition)
    +      }.foreach { pm: PartitionMetadata =>
    +        pm.leader.foreach { leader =>
    +          return Right((leader.host, leader.port))
    +        }
    +      }
    +    }
    +    Left(errs)
    +  }
    +
    +  def findLeaders(
    +      topicAndPartitions: Set[TopicAndPartition]
    +    ): Either[Err, Map[TopicAndPartition, (String, Int)]] = {
    +    val topics = topicAndPartitions.map(_.topic)
    +    val response = getPartitionMetadata(topics).right
    +    val answer = response.flatMap { tms: Set[TopicMetadata] =>
    +      val leaderMap = tms.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.flatMap { pm: PartitionMetadata =>
    +          val tp = TopicAndPartition(tm.topic, pm.partitionId)
    +          if (topicAndPartitions(tp)) {
    +            pm.leader.map { l =>
    +              tp -> (l.host -> l.port)
    +            }
    +          } else {
    +            None
    +          }
    +        }
    +      }.toMap
    +
    +      if (leaderMap.keys.size == topicAndPartitions.size) {
    +        Right(leaderMap)
    +      } else {
    +        val missing = topicAndPartitions.diff(leaderMap.keySet)
    +        val err = new Err
    +        err.append(new SparkException(s"Couldn't find leaders for ${missing}"))
    +        Left(err)
    +      }
    +    }
    +    answer
    +  }
    +
    +  def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] = {
    +    getPartitionMetadata(topics).right.map { r =>
    +      r.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.map { pm: PartitionMetadata =>
    +          TopicAndPartition(tm.topic, pm.partitionId)
    +        }    
    +      }
    +    }
    +  }
    +
    +  def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = {
    +    val req = TopicMetadataRequest(
    +      TopicMetadataRequest.CurrentVersion, 0, config.clientId, topics.toSeq)
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      // error codes here indicate missing / just created topic,
    +      // repeating on a different broker wont be useful
    +      return Right(resp.topicsMetadata.toSet)
    +    }
    +    Left(errs)
    +  }
    +
    +  // Leader offset api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI
    +  // scalastyle:on
    +
    +  def getLatestLeaderOffsets(
    +      topicAndPartitions: Set[TopicAndPartition]
    +    ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime)
    +
    +  def getEarliestLeaderOffsets(
    +      topicAndPartitions: Set[TopicAndPartition]
    +    ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime)
    +
    +  def getLeaderOffsets(
    +      topicAndPartitions: Set[TopicAndPartition],
    +      before: Long
    +    ): Either[Err, Map[TopicAndPartition, LeaderOffset]] = {
    +    getLeaderOffsets(topicAndPartitions, before, 1).right.map { r =>
    +      r.map { kv =>
    +        // mapValues isnt serializable, see SI-7005
    +        kv._1 -> kv._2.head
    +      }
    +    }
    +  }
    +
    +  private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] =
    +    m.groupBy(_._2).map { kv =>
    +      kv._1 -> kv._2.keys.toSeq
    +    }
    +
    +  def getLeaderOffsets(
    +      topicAndPartitions: Set[TopicAndPartition],
    +      before: Long,
    +      maxNumOffsets: Int
    +    ): Either[Err, Map[TopicAndPartition, Seq[LeaderOffset]]] = {
    +    findLeaders(topicAndPartitions).right.flatMap { tpToLeader =>
    +      val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] = flip(tpToLeader)
    +      val leaders = leaderToTp.keys
    +      var result = Map[TopicAndPartition, Seq[LeaderOffset]]()
    +      val errs = new Err
    +      withBrokers(leaders, errs) { consumer =>
    +        val partitionsToGetOffsets: Seq[TopicAndPartition] =
    +          leaderToTp((consumer.host, consumer.port))
    +        val reqMap = partitionsToGetOffsets.map { tp: TopicAndPartition =>
    +          tp -> PartitionOffsetRequestInfo(before, maxNumOffsets)
    +        }.toMap
    +        val req = OffsetRequest(reqMap)
    +        val resp = consumer.getOffsetsBefore(req)
    +        val respMap = resp.partitionErrorAndOffsets
    +        partitionsToGetOffsets.foreach { tp: TopicAndPartition =>
    +          respMap.get(tp).foreach { por: PartitionOffsetsResponse =>
    +            if (por.error == ErrorMapping.NoError) {
    +              if (por.offsets.nonEmpty) {
    +                result += tp -> por.offsets.map { off =>
    +                  LeaderOffset(consumer.host, consumer.port, off)
    +                }
    +              } else {
    +                errs.append(new SparkException(
    +                  s"Empty offsets for ${tp}, is ${before} before log beginning?"))
    +              }
    +            } else {
    +              errs.append(ErrorMapping.exceptionFor(por.error))
    +            }
    +          }
    +        }
    +        if (result.keys.size == topicAndPartitions.size) {
    +          return Right(result)
    +        }
    +      }
    +      val missing = topicAndPartitions.diff(result.keySet)
    +      errs.append(new SparkException(s"Couldn't find leader offsets for ${missing}"))
    +      Left(errs)
    +    }
    +  }
    +
    +  // Consumer offset api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI
    +  // scalastyle:on
    +
    +  def getConsumerOffsets(
    +      groupId: String,
    +      topicAndPartitions: Set[TopicAndPartition]
    +    ): Either[Err, Map[TopicAndPartition, Long]] = {
    +    getConsumerOffsetMetadata(groupId, topicAndPartitions).right.map { r =>
    --- End diff --
    
    As par Kafka Documentation , these API calls are not fully functional in releases until Kafka 0.8.1.1. It will be available in the 0.8.2 release. Does that mean this solution can not work with any earlier version of Kafka ? If that is case, need to mention in documentation.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22653565
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,313 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    +  * Convenience methods for interacting with a Kafka cluster.
    +  * @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
    +  */
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    +  import KafkaCluster.Err
    +
    +  val seedBrokers: Array[(String, Int)] =
    +    kafkaParams.get("metadata.broker.list")
    +      .orElse(kafkaParams.get("bootstrap.servers"))
    +      .getOrElse(throw new Exception("Must specify metadata.broker.list or bootstrap.servers"))
    +      .split(",").map { hp =>
    +        val hpa = hp.split(":")
    +        (hpa(0), hpa(1).toInt)
    +      }
    +
    +  // ConsumerConfig isn't serializable
    +  @transient private var _config: ConsumerConfig = null
    +
    +  def config: ConsumerConfig = this.synchronized {
    +    if (_config == null) {
    +      _config = KafkaCluster.consumerConfig(kafkaParams)
    +    }
    +    _config
    +  }
    +
    +  def connect(host: String, port: Int): SimpleConsumer =
    +    new SimpleConsumer(host, port, config.socketTimeoutMs,
    +      config.socketReceiveBufferBytes, config.clientId)
    +
    +  def connect(hostAndPort: (String, Int)): SimpleConsumer =
    +    connect(hostAndPort._1, hostAndPort._2)
    +
    +  def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] =
    +    findLeader(topic, partition).right.map(connect)
    +
    +  def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, Seq(topic))
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      resp.topicsMetadata.find(_.topic == topic).flatMap { t =>
    +        t.partitionsMetadata.find(_.partitionId == partition)
    +      }.foreach { partitionMeta =>
    +        partitionMeta.leader.foreach { leader =>
    +          return Right((leader.host, leader.port))
    +        }
    +      }
    +    }
    +    Left(errs)
    +  }
    +
    +  def findLeaders(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, (String, Int)]] = {
    +    getPartitionMetadata(topicAndPartitions.map(_.topic)).right.flatMap { tms =>
    +      val result = tms.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.flatMap { pm =>
    +          val tp = TopicAndPartition(tm.topic, pm.partitionId)
    +          if (topicAndPartitions(tp)) {
    +            pm.leader.map { l =>
    +              tp -> (l.host -> l.port)
    +            }
    +          } else {
    +            None
    +          }
    +        }
    +      }.toMap
    +      if (result.keys.size == topicAndPartitions.size) {
    +        Right(result)
    +      } else {
    +        val missing = topicAndPartitions.diff(result.keys.toSet)
    +        val err = new Err
    +        err.append(new Exception(s"Couldn't find leaders for ${missing}"))
    +        Left(err)
    +      }
    +    }
    +  }
    +
    +  def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] =
    +    getPartitionMetadata(topics).right.map { r =>
    +      r.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.map { pm =>
    +          TopicAndPartition(tm.topic, pm.partitionId)
    +        }    
    +      }
    +    }
    +
    +  def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, topics.toSeq)
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      // error codes here indicate missing / just created topic,
    +      // repeating on a different broker wont be useful
    +      return Right(resp.topicsMetadata.toSet)
    +    }
    +    Left(errs)
    +  }
    +
    +  def getLatestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime)
    +
    +  def getEarliestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime)
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, before, 1).right.map { r =>
    +      r.map { kv =>
    +        // mapValues isnt serializable, see SI-7005
    +        kv._1 -> kv._2.head
    +      }
    +    }
    +
    +  private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] =
    +    m.groupBy(_._2).map { kv =>
    +      kv._1 -> kv._2.keys.toSeq
    +    }
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long,
    +    maxNumOffsets: Int
    +  ): Either[Err, Map[TopicAndPartition, Seq[Long]]] = {
    +    findLeaders(topicAndPartitions).right.flatMap { tpToLeader =>
    +      val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] = flip(tpToLeader)
    +      val leaders = leaderToTp.keys
    +      var result = Map[TopicAndPartition, Seq[Long]]()
    +      val errs = new Err
    +      withBrokers(leaders, errs) { consumer =>
    +        val needed: Seq[TopicAndPartition] = leaderToTp((consumer.host, consumer.port))
    +        val req = OffsetRequest(
    +          needed.map { tp =>
    +            tp -> PartitionOffsetRequestInfo(before, maxNumOffsets)
    +          }.toMap
    +        )
    +        val resp = consumer.getOffsetsBefore(req)
    +        val respMap = resp.partitionErrorAndOffsets
    +        needed.foreach { tp =>
    +          respMap.get(tp).foreach { errAndOffsets =>
    +            if (errAndOffsets.error == ErrorMapping.NoError) {
    +              if (errAndOffsets.offsets.nonEmpty) {
    +                result += tp -> errAndOffsets.offsets
    +              } else {
    +                errs.append(new Exception(
    +                  s"Empty offsets for ${tp}, is ${before} before log beginning?"))
    +              }
    +            } else {
    +              errs.append(ErrorMapping.exceptionFor(errAndOffsets.error))
    +            }
    +          }
    +        }
    +        if (result.keys.size == topicAndPartitions.size) {
    +          return Right(result)
    +        }
    +      }
    +      val missing = topicAndPartitions.diff(result.keys.toSet)
    +      errs.append(new Exception(s"Couldn't find leader offsets for ${missing}"))
    +      Left(errs)
    +    }
    +  }
    +
    +  def getConsumerOffsets(
    +    groupId: String,
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] = {
    +    getConsumerOffsetMetadata(groupId, topicAndPartitions).right.map { r =>
    +      r.map { kv =>
    +        kv._1 -> kv._2.offset
    +      }
    +    }
    +  }
    +
    +  def getConsumerOffsetMetadata(
    +    groupId: String,
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = {
    +    var result = Map[TopicAndPartition, OffsetMetadataAndError]()
    +    val req = OffsetFetchRequest(groupId, topicAndPartitions.toSeq)
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp = consumer.fetchOffsets(req)
    +      val respMap = resp.requestInfo
    +      val needed = topicAndPartitions.diff(result.keys.toSet)
    +      needed.foreach { tp =>
    +        respMap.get(tp).foreach { offsetMeta =>
    +          if (offsetMeta.error == ErrorMapping.NoError) {
    +            result += tp -> offsetMeta
    +          } else {
    +            errs.append(ErrorMapping.exceptionFor(offsetMeta.error))
    +          }
    +        }
    +      }
    +      if (result.keys.size == topicAndPartitions.size) {
    +        return Right(result)
    +      }
    +    }
    +    val missing = topicAndPartitions.diff(result.keys.toSet)
    +    errs.append(new Exception(s"Couldn't find consumer offsets for ${missing}"))
    +    Left(errs)
    +  }
    +
    +  def setConsumerOffsets(
    +    groupId: String,
    +    offsets: Map[TopicAndPartition, Long]
    +  ): Either[Err, Map[TopicAndPartition, Short]] = {
    +    setConsumerOffsetMetadata(groupId, offsets.map { kv =>
    +      kv._1 -> OffsetMetadataAndError(kv._2)
    +    })
    +  }
    +
    +  def setConsumerOffsetMetadata(
    +    groupId: String,
    +    metadata: Map[TopicAndPartition, OffsetMetadataAndError]
    +  ): Either[Err, Map[TopicAndPartition, Short]] = {
    +    var result = Map[TopicAndPartition, Short]()
    +    val req = OffsetCommitRequest(groupId, metadata)
    +    val errs = new Err
    +    val topicAndPartitions = metadata.keys.toSet
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp = consumer.commitOffsets(req)
    +      val respMap = resp.requestInfo
    +      val needed = topicAndPartitions.diff(result.keys.toSet)
    +      needed.foreach { tp =>
    +        respMap.get(tp).foreach { err =>
    +          if (err == ErrorMapping.NoError) {
    +            result += tp -> err
    +          } else {
    +            errs.append(ErrorMapping.exceptionFor(err))
    +          }
    +        }
    +      }
    +      if (result.keys.size == topicAndPartitions.size) {
    +        return Right(result)
    --- End diff --
    
    That's an early return from withBrokers, which would otherwise keep running the closure on each broker.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72228780
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/26401/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22446804
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala ---
    @@ -0,0 +1,123 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.annotation.tailrec
    +import scala.reflect.{classTag, ClassTag}
    +
    +import kafka.common.TopicAndPartition
    +import kafka.message.MessageAndMetadata
    +import kafka.serializer.Decoder
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.rdd.kafka.{KafkaCluster, KafkaRDD}
    +import org.apache.spark.streaming.{StreamingContext, Time}
    +import org.apache.spark.streaming.dstream._
    +
    +/** A stream of {@link org.apache.spark.rdd.kafka.KafkaRDD} where
    +  * each given Kafka topic/partition corresponds to an RDD partition.
    +  * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages
    +  * per second that each '''partition''' will accept.
    +  * Starting offsets are specified in advance,
    +  * and this DStream is not responsible for committing offsets,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the stream
    +  * @param messageHandler function for translating each message into the desired type
    +  * @param maxRetries maximum number of times in a row to retry getting leaders' offsets
    +  */
    +class DeterministicKafkaInputDStream[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    @transient ssc_ : StreamingContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R,
    +    maxRetries: Int = 1
    +) extends InputDStream[R](ssc_) with Logging {
    +
    +  private val kc = new KafkaCluster(kafkaParams)
    +
    +  private val maxMessagesPerPartition: Option[Long] = {
    +    val ratePerSec = ssc.sparkContext.getConf.getInt("spark.streaming.receiver.maxRate", 0)
    --- End diff --
    
    I'm not sure specifically what you mean by "window operations that require
    past data which needs to be pulled from Kafka every time". The current
    KafkaRDD code has a log every time compute() is called on the rdd to pull
    data from kafka, and for a job with a window operation, I only see compute
    called once for a given offset range, not repeatedly every time.
    
    Regarding the bigger question of how this approach stacks up to the two
    existing approaches... they're all different approaches with different
    tradeoffs, I don't think one has to win.  I'd still have a use for the
    original receiver based class (not the WAL one), especially if SPARK-3146
    or SPARK-4960 ever get merged.
    
    On Sat, Jan 3, 2015 at 8:57 PM, Tathagata Das <no...@github.com>
    wrote:
    
    > In
    > external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala
    > <https://github.com/apache/spark/pull/3798#discussion-diff-22436219>:
    >
    > > +  K: ClassTag,
    > > +  V: ClassTag,
    > > +  U <: Decoder[_]: ClassTag,
    > > +  T <: Decoder[_]: ClassTag,
    > > +  R: ClassTag](
    > > +    @transient ssc_ : StreamingContext,
    > > +    val kafkaParams: Map[String, String],
    > > +    val fromOffsets: Map[TopicAndPartition, Long],
    > > +    messageHandler: MessageAndMetadata[K, V] => R,
    > > +    maxRetries: Int = 1
    > > +) extends InputDStream[R](ssc_) with Logging {
    > > +
    > > +  private val kc = new KafkaCluster(kafkaParams)
    > > +
    > > +  private val maxMessagesPerPartition: Option[Long] = {
    > > +    val ratePerSec = ssc.sparkContext.getConf.getInt("spark.streaming.receiver.maxRate", 0)
    >
    > Yeah, I get it. However, this still has a different performance
    > characteristics than receiver based sources. Sometimes it is better (as you
    > are "always" pulling data in parallel across cluster, instead of the
    > default 1 receiver), sometimes it is worse (window operations that require
    > past data which needs to be pulled from Kafka every time). For this method
    > to be viable alternative to the existing, we ideally have to make sure that
    > the performance characteristics of this method is >= performance of the
    > existing method under all situations. Then using this method will be
    > justified even if the behavior is different.
    >
    > On that note, here is an idea of what we can do. We can store the data
    > pulled from the Kafka as blocks in the BlockManager, so that subsequent
    > accesses to the data (due to window or stateful ops) can be faster. One way
    > to do this is to implement a KafkaBackedBlockRDD. This is similar to
    > WriteAheadLogBackedBlockRDD which has the logic to either read from
    > BlockManager if block is present, or reload the data from the WriteAheadLog
    > based on file segment info. KafkaBackedBlockRDD can be similar, either
    > read from BlockManager, or load it from Kafka based on offsets.
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798/files#r22436219>.
    >


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22366140
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,305 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    +  * Convenience methods for interacting with a Kafka cluster.
    +  * @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
    +  */
    +class KafkaCluster(val kafkaParams: Map[String, String]) {
    --- End diff --
    
    The rdd would be really unpleasant to actually use without the convenience methods exposed by KafkaCluster, especially if you're keeping your offsets in zookeeper and doing idempotent writes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by helena <gi...@git.apache.org>.
Github user helena commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22649516
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,313 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    +  * Convenience methods for interacting with a Kafka cluster.
    +  * @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
    +  */
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    +  import KafkaCluster.Err
    +
    +  val seedBrokers: Array[(String, Int)] =
    +    kafkaParams.get("metadata.broker.list")
    +      .orElse(kafkaParams.get("bootstrap.servers"))
    +      .getOrElse(throw new Exception("Must specify metadata.broker.list or bootstrap.servers"))
    +      .split(",").map { hp =>
    +        val hpa = hp.split(":")
    +        (hpa(0), hpa(1).toInt)
    +      }
    +
    +  // ConsumerConfig isn't serializable
    +  @transient private var _config: ConsumerConfig = null
    +
    +  def config: ConsumerConfig = this.synchronized {
    +    if (_config == null) {
    +      _config = KafkaCluster.consumerConfig(kafkaParams)
    +    }
    +    _config
    +  }
    +
    +  def connect(host: String, port: Int): SimpleConsumer =
    +    new SimpleConsumer(host, port, config.socketTimeoutMs,
    +      config.socketReceiveBufferBytes, config.clientId)
    +
    +  def connect(hostAndPort: (String, Int)): SimpleConsumer =
    +    connect(hostAndPort._1, hostAndPort._2)
    +
    +  def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] =
    +    findLeader(topic, partition).right.map(connect)
    +
    +  def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, Seq(topic))
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      resp.topicsMetadata.find(_.topic == topic).flatMap { t =>
    +        t.partitionsMetadata.find(_.partitionId == partition)
    +      }.foreach { partitionMeta =>
    +        partitionMeta.leader.foreach { leader =>
    +          return Right((leader.host, leader.port))
    +        }
    +      }
    +    }
    +    Left(errs)
    +  }
    +
    +  def findLeaders(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, (String, Int)]] = {
    +    getPartitionMetadata(topicAndPartitions.map(_.topic)).right.flatMap { tms =>
    +      val result = tms.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.flatMap { pm =>
    +          val tp = TopicAndPartition(tm.topic, pm.partitionId)
    +          if (topicAndPartitions(tp)) {
    +            pm.leader.map { l =>
    +              tp -> (l.host -> l.port)
    +            }
    +          } else {
    +            None
    +          }
    +        }
    +      }.toMap
    +      if (result.keys.size == topicAndPartitions.size) {
    +        Right(result)
    +      } else {
    +        val missing = topicAndPartitions.diff(result.keys.toSet)
    +        val err = new Err
    +        err.append(new Exception(s"Couldn't find leaders for ${missing}"))
    +        Left(err)
    +      }
    +    }
    +  }
    +
    +  def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] =
    +    getPartitionMetadata(topics).right.map { r =>
    +      r.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.map { pm =>
    +          TopicAndPartition(tm.topic, pm.partitionId)
    +        }    
    +      }
    +    }
    +
    +  def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, topics.toSeq)
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      // error codes here indicate missing / just created topic,
    +      // repeating on a different broker wont be useful
    +      return Right(resp.topicsMetadata.toSet)
    +    }
    +    Left(errs)
    +  }
    +
    +  def getLatestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime)
    +
    +  def getEarliestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime)
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, before, 1).right.map { r =>
    +      r.map { kv =>
    +        // mapValues isnt serializable, see SI-7005
    +        kv._1 -> kv._2.head
    +      }
    +    }
    +
    +  private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] =
    +    m.groupBy(_._2).map { kv =>
    +      kv._1 -> kv._2.keys.toSeq
    +    }
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long,
    +    maxNumOffsets: Int
    +  ): Either[Err, Map[TopicAndPartition, Seq[Long]]] = {
    +    findLeaders(topicAndPartitions).right.flatMap { tpToLeader =>
    +      val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] = flip(tpToLeader)
    +      val leaders = leaderToTp.keys
    +      var result = Map[TopicAndPartition, Seq[Long]]()
    +      val errs = new Err
    +      withBrokers(leaders, errs) { consumer =>
    +        val needed: Seq[TopicAndPartition] = leaderToTp((consumer.host, consumer.port))
    +        val req = OffsetRequest(
    +          needed.map { tp =>
    +            tp -> PartitionOffsetRequestInfo(before, maxNumOffsets)
    +          }.toMap
    +        )
    +        val resp = consumer.getOffsetsBefore(req)
    +        val respMap = resp.partitionErrorAndOffsets
    +        needed.foreach { tp =>
    +          respMap.get(tp).foreach { errAndOffsets =>
    +            if (errAndOffsets.error == ErrorMapping.NoError) {
    +              if (errAndOffsets.offsets.nonEmpty) {
    +                result += tp -> errAndOffsets.offsets
    +              } else {
    +                errs.append(new Exception(
    +                  s"Empty offsets for ${tp}, is ${before} before log beginning?"))
    +              }
    +            } else {
    +              errs.append(ErrorMapping.exceptionFor(errAndOffsets.error))
    +            }
    +          }
    +        }
    +        if (result.keys.size == topicAndPartitions.size) {
    +          return Right(result)
    +        }
    +      }
    +      val missing = topicAndPartitions.diff(result.keys.toSet)
    +      errs.append(new Exception(s"Couldn't find leader offsets for ${missing}"))
    +      Left(errs)
    +    }
    +  }
    +
    +  def getConsumerOffsets(
    +    groupId: String,
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] = {
    +    getConsumerOffsetMetadata(groupId, topicAndPartitions).right.map { r =>
    +      r.map { kv =>
    +        kv._1 -> kv._2.offset
    +      }
    +    }
    +  }
    +
    +  def getConsumerOffsetMetadata(
    +    groupId: String,
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = {
    +    var result = Map[TopicAndPartition, OffsetMetadataAndError]()
    +    val req = OffsetFetchRequest(groupId, topicAndPartitions.toSeq)
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp = consumer.fetchOffsets(req)
    +      val respMap = resp.requestInfo
    +      val needed = topicAndPartitions.diff(result.keys.toSet)
    +      needed.foreach { tp =>
    +        respMap.get(tp).foreach { offsetMeta =>
    +          if (offsetMeta.error == ErrorMapping.NoError) {
    +            result += tp -> offsetMeta
    +          } else {
    +            errs.append(ErrorMapping.exceptionFor(offsetMeta.error))
    +          }
    +        }
    +      }
    +      if (result.keys.size == topicAndPartitions.size) {
    +        return Right(result)
    +      }
    +    }
    +    val missing = topicAndPartitions.diff(result.keys.toSet)
    +    errs.append(new Exception(s"Couldn't find consumer offsets for ${missing}"))
    +    Left(errs)
    +  }
    +
    +  def setConsumerOffsets(
    +    groupId: String,
    +    offsets: Map[TopicAndPartition, Long]
    +  ): Either[Err, Map[TopicAndPartition, Short]] = {
    +    setConsumerOffsetMetadata(groupId, offsets.map { kv =>
    +      kv._1 -> OffsetMetadataAndError(kv._2)
    +    })
    +  }
    +
    +  def setConsumerOffsetMetadata(
    +    groupId: String,
    +    metadata: Map[TopicAndPartition, OffsetMetadataAndError]
    +  ): Either[Err, Map[TopicAndPartition, Short]] = {
    +    var result = Map[TopicAndPartition, Short]()
    +    val req = OffsetCommitRequest(groupId, metadata)
    +    val errs = new Err
    +    val topicAndPartitions = metadata.keys.toSet
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp = consumer.commitOffsets(req)
    +      val respMap = resp.requestInfo
    +      val needed = topicAndPartitions.diff(result.keys.toSet)
    +      needed.foreach { tp =>
    +        respMap.get(tp).foreach { err =>
    +          if (err == ErrorMapping.NoError) {
    +            result += tp -> err
    +          } else {
    +            errs.append(ErrorMapping.exceptionFor(err))
    +          }
    +        }
    +      }
    +      if (result.keys.size == topicAndPartitions.size) {
    +        return Right(result)
    +      }
    +    }
    +    val missing = topicAndPartitions.diff(result.keys.toSet)
    +    errs.append(new Exception(s"Couldn't set offsets for ${missing}"))
    +    Left(errs)
    +  }
    +
    +  private def withBrokers(brokers: Iterable[(String, Int)], errs: Err)
    +    (fn: SimpleConsumer => Any): Unit = {
    +    brokers.foreach { hp =>
    +      var consumer: SimpleConsumer = null
    +      try {
    +        consumer = connect(hp)
    +        fn(consumer)
    +      } catch {
    +        case NonFatal(e) =>
    +          errs.append(e)
    +      } finally {
    +        if (consumer != null) consumer.close()
    +      }
    +    }
    +  }
    +}
    +
    +object KafkaCluster {
    +  type Err = ArrayBuffer[Throwable]
    +
    +  /** Make a consumer config without requiring group.id or zookeeper.connect,
    +    * since communicating with brokers also needs common settings such as timeout
    +    */
    +  def consumerConfig(kafkaParams: Map[String, String]): ConsumerConfig = {
    +    val props = new Properties()
    +    kafkaParams.foreach(param => props.put(param._1, param._2))
    +    Seq("zookeeper.connect", "group.id").foreach { s =>
    +      if (!props.contains(s)) {
    +        props.setProperty(s, "")
    +      }
    +    }
    --- End diff --
    
    or perhaps
    ```import scala.collection.JavaConversions.mapAsJavaMap 
        props.putAll(mapAsJavaMap(kafkaParams)) 
        Seq("zookeeper.connect", "group.id").collect { 
          case s if props containsKey s => props.setProperty(s, "") // containsKey is less expensive than contains
        } ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23890348
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala ---
    @@ -0,0 +1,220 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.reflect.{classTag, ClassTag}
    +
    +import org.apache.spark.{Logging, Partition, SparkContext, SparkException, TaskContext}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.NextIterator
    +
    +import java.util.Properties
    +import kafka.api.{FetchRequestBuilder, FetchResponse}
    +import kafka.common.{ErrorMapping, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +import kafka.message.{MessageAndMetadata, MessageAndOffset}
    +import kafka.serializer.Decoder
    +import kafka.utils.VerifiableProperties
    +
    +/**
    + * 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 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 KafkaRDDPartition 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
    + */
    +private[spark]
    +class KafkaRDD[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag] private[spark] (
    +    sc: SparkContext,
    +    kafkaParams: Map[String, String],
    +    private[spark] val batch: Array[KafkaRDDPartition],
    +    messageHandler: MessageAndMetadata[K, V] => R
    +  ) extends RDD[R](sc, Nil) with Logging with HasOffsetRanges {
    +
    +  def offsetRanges: Array[OffsetRange] = batch.asInstanceOf[Array[OffsetRange]]
    +
    +  override def getPartitions: Array[Partition] = batch.asInstanceOf[Array[Partition]]
    +
    +  override def getPreferredLocations(thePart: Partition): Seq[String] = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    // TODO is additional hostname resolution necessary here
    +    Seq(part.host)
    +  }
    +
    +  private def errBeginAfterEnd(part: KafkaRDDPartition): String =
    +    s"Beginning offset ${part.fromOffset} is after the ending offset ${part.untilOffset} " +
    +      s"for topic ${part.topic} partition ${part.partition}. " +
    +      "You either provided an invalid fromOffset, or the Kafka topic has been damaged"
    +
    +  private def errRanOutBeforeEnd(part: KafkaRDDPartition): String =
    +    s"Ran out of messages before reaching ending offset ${part.untilOffset} " +
    +    s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." +
    +    " This should not happen, and indicates that messages may have been lost"
    +
    +  private def errOvershotEnd(itemOffset: Long, part: KafkaRDDPartition): String =
    +    s"Got ${itemOffset} > ending offset ${part.untilOffset} " +
    +    s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." +
    +    " This should not happen, and indicates a message may have been skipped"
    +
    +  override def compute(thePart: Partition, context: TaskContext): Iterator[R] = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    assert(part.fromOffset <= part.untilOffset, errBeginAfterEnd(part))
    +    if (part.fromOffset == part.untilOffset) {
    +      log.warn("Beginning offset ${part.fromOffset} is the same as ending offset " +
    +        s"skipping ${part.topic} ${part.partition}")
    +      Iterator.empty
    +    } else {
    +      new KafkaRDDIterator(part, context)
    +    }
    +  }
    +
    +  private class KafkaRDDIterator(
    +      part: KafkaRDDPartition,
    +      context: TaskContext) extends NextIterator[R] {
    +
    +    context.addTaskCompletionListener{ context => closeIfNeeded() }
    +
    +    log.info(s"Computing topic ${part.topic}, partition ${part.partition} " +
    +      s"offsets ${part.fromOffset} -> ${part.untilOffset}")
    +
    +    val kc = new KafkaCluster(kafkaParams)
    +    val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +      .newInstance(kc.config.props)
    +      .asInstanceOf[Decoder[K]]
    +    val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +      .newInstance(kc.config.props)
    +      .asInstanceOf[Decoder[V]]
    +    val consumer = connectLeader
    +    var requestOffset = part.fromOffset
    +    var iter: Iterator[MessageAndOffset] = null
    +
    +    // The idea is to use the provided preferred host, except on task retry atttempts,
    +    // to minimize number of kafka metadata requests
    +    private def connectLeader: SimpleConsumer = {
    +      if (context.attemptNumber > 0) {
    +        kc.connectLeader(part.topic, part.partition).fold(
    +          errs => throw new SparkException(
    +            s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " +
    +              errs.mkString("\n")),
    +          consumer => consumer
    +        )
    +      } else {
    +        kc.connect(part.host, part.port)
    +      }
    +    }
    +
    +    private def handleFetchErr(resp: FetchResponse) {
    +      if (resp.hasError) {
    +        val err = resp.errorCode(part.topic, part.partition)
    +        if (err == ErrorMapping.LeaderNotAvailableCode ||
    +          err == ErrorMapping.NotLeaderForPartitionCode) {
    +          log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " +
    +            s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms")
    +          Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +        }
    +        // Let normal rdd retry sort out reconnect attempts
    +        throw ErrorMapping.exceptionFor(err)
    +      }
    +    }
    +
    --- End diff --
    
    I'm not saying the specification is inherently broken, I'm saying the
    specification is this: give a range of offsets, and they will be processed
    by spark.
    
    Of course if you give it a range of offsets that no longer exist in Kafka
    you're going to get an error.  Silently processing a different range of
    offsets than specified by the client would be horrible.  That's what would
    break the specification.
    
    I don't see why this is something that needs additional spark specific
    documentation.  Its a problem of misusing Kafka, and is documented by
    Kafka.  If you're running into that error all the time, either consume
    faster or set your log retention longer.
    On Jan 31, 2015 10:22 AM, "Dibyendu Bhattacharya" <no...@github.com>
    wrote:
    
    > In
    > external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala
    > <https://github.com/apache/spark/pull/3798#discussion_r23889938>:
    >
    > > +    }
    > > +
    > > +    private def handleFetchErr(resp: FetchResponse) {
    > > +      if (resp.hasError) {
    > > +        val err = resp.errorCode(part.topic, part.partition)
    > > +        if (err == ErrorMapping.LeaderNotAvailableCode ||
    > > +          err == ErrorMapping.NotLeaderForPartitionCode) {
    > > +          log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " +
    > > +            s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms")
    > > +          Thread.sleep(kc.config.refreshLeaderBackoffMs)
    > > +        }
    > > +        // Let normal rdd retry sort out reconnect attempts
    > > +        throw ErrorMapping.exceptionFor(err)
    > > +      }
    > > +    }
    > > +
    >
    > If the KafkaRdd specification is broken , then probably you need to
    > document it how client code need to handle. In our Production Kafka
    > cluster, this error not very rare and not very rare for others as far I
    > know who uses Kafka in Prod.
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798/files#r23889938>.
    >


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22367477
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala ---
    @@ -0,0 +1,123 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.annotation.tailrec
    +import scala.reflect.{classTag, ClassTag}
    +
    +import kafka.common.TopicAndPartition
    +import kafka.message.MessageAndMetadata
    +import kafka.serializer.Decoder
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.rdd.kafka.{KafkaCluster, KafkaRDD}
    +import org.apache.spark.streaming.{StreamingContext, Time}
    +import org.apache.spark.streaming.dstream._
    +
    +/** A stream of {@link org.apache.spark.rdd.kafka.KafkaRDD} where
    +  * each given Kafka topic/partition corresponds to an RDD partition.
    +  * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages
    +  * per second that each '''partition''' will accept.
    +  * Starting offsets are specified in advance,
    +  * and this DStream is not responsible for committing offsets,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the stream
    +  * @param messageHandler function for translating each message into the desired type
    +  * @param maxRetries maximum number of times in a row to retry getting leaders' offsets
    +  */
    +class DeterministicKafkaInputDStream[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    @transient ssc_ : StreamingContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R,
    +    maxRetries: Int = 1
    +) extends InputDStream[R](ssc_) with Logging {
    +
    +  private val kc = new KafkaCluster(kafkaParams)
    +
    +  private val maxMessagesPerPartition: Option[Long] = {
    +    val ratePerSec = ssc.sparkContext.getConf.getInt("spark.streaming.receiver.maxRate", 0)
    +    if (ratePerSec > 0) {
    +      val secsPerBatch = ssc.graph.batchDuration.milliseconds.toDouble / 1000
    +      Some((secsPerBatch * ratePerSec).toLong)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  // TODO based on the design of InputDStream's lastValidTime, it appears there isn't a
    +  // thread safety concern with private mutable state, but is this certain?
    +  private var currentOffsets = fromOffsets
    +
    +  @tailrec
    +  private def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, Long] = {
    +    val o = kc.getLatestLeaderOffsets(currentOffsets.keys.toSet)
    +    // Either.fold would confuse @tailrec, do it manually
    +    if (o.isLeft) {
    +      val err = o.left.get.toString
    +      if (retries <= 0) {
    +        throw new Exception(err)
    +      } else {
    +        log.error(err)
    +        Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +        latestLeaderOffsets(retries - 1)
    +      }
    +    } else {
    +      o.right.get
    +    }
    +  }
    +
    +  private def clamp(leaderOffsets: Map[TopicAndPartition, Long]): Map[TopicAndPartition, Long] = {
    +    maxMessagesPerPartition.map { mmp =>
    +      leaderOffsets.map { kv =>
    +        val (k, v) = kv
    +        val curr = currentOffsets(k)
    +        val diff = v - curr
    +        if (diff > mmp) (k, curr + mmp) else (k, v)
    +      }
    +    }.getOrElse(leaderOffsets)
    +  }
    +
    +  override def compute(validTime: Time): Option[KafkaRDD[K, V, U, T, R]] = {
    --- End diff --
    
    I believe your interpretation is correct. And lets focus on the logic than the names. I think this logic is good, but its not driver fault-tolerant. But I think we can extend this approach to make it so. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71786983
  
      [Test build #26206 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26206/consoleFull) for   PR 3798 at commit [`19406cc`](https://github.com/apache/spark/commit/19406cce66672d74bd0b9c1d98cd8486c186f8ee).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable `
      * `  case class LeaderOffset(host: String, port: Int, offset: Long)`
      * `class KafkaRDDPartition(`
      * `trait OffsetRange `
      * `trait HasOffsetRanges `
      * `  class DeterministicKafkaInputDStreamCheckpointData extends DStreamCheckpointData(this) `



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71564796
  
    I'm not a big fan of either either :)
    
    The issue here is that KafkaCluster is potentially dealing with multiple
    exceptions due to multiple brokers.  As a user of library code, I'd
    potentially want access to all of the exceptions, not just the first or
    last one.
    
    It sounds like TD is leaning towards not making KafkaCluster exposed, in
    which case we can do whatever makes sense for the internal usage of it.
    
    On Mon, Jan 26, 2015 at 5:03 PM, Reynold Xin <no...@github.com>
    wrote:
    
    > BTW one other thing - Either is really dangerous and very complicated to
    > use. It almost always leads to downstream code becoming a mess.
    >
    > You are mostly just using it for exception propagation. Why not just throw
    > exceptions?
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798#issuecomment-71557370>.
    >


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23586726
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaRDD.scala ---
    @@ -0,0 +1,199 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.reflect.{classTag, ClassTag}
    +
    +import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.NextIterator
    +
    +import java.util.Properties
    +import kafka.api.{FetchRequestBuilder, FetchResponse}
    +import kafka.common.{ErrorMapping, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +import kafka.message.{MessageAndMetadata, MessageAndOffset}
    +import kafka.serializer.Decoder
    +import kafka.utils.VerifiableProperties
    +
    +/** A batch-oriented interface for consuming from Kafka.
    +  * Starting and ending offsets are specified in advance,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 KafkaRDDPartition 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
    +  */
    +class KafkaRDD[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    sc: SparkContext,
    +    val kafkaParams: Map[String, String],
    +    val batch: Array[KafkaRDDPartition],
    +    messageHandler: MessageAndMetadata[K, V] => R
    +  ) extends RDD[R](sc, Nil) with Logging {
    +
    +  override def getPartitions: Array[Partition] = batch.asInstanceOf[Array[Partition]]
    +
    +  override def getPreferredLocations(thePart: Partition): Seq[String] = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    // TODO is additional hostname resolution necessary here
    +    Seq(part.host)
    +  }
    +
    +  override def compute(thePart: Partition, context: TaskContext): Iterator[R] = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    if (part.fromOffset >= part.untilOffset) {
    +      log.warn("Beginning offset is same or after ending offset " +
    +        s"skipping ${part.topic} ${part.partition}")
    +      Iterator.empty
    +    } else {
    +      new NextIterator[R] {
    +        context.addTaskCompletionListener{ context => closeIfNeeded() }
    +
    +        log.info(s"Computing topic ${part.topic}, partition ${part.partition} " +
    +          s"offsets ${part.fromOffset} -> ${part.untilOffset}")
    +
    +        val kc = new KafkaCluster(kafkaParams)
    +        val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[K]]
    +        val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[V]]
    +        val consumer = connectLeader
    +        var requestOffset = part.fromOffset
    +        var iter: Iterator[MessageAndOffset] = null
    +
    +        // The idea is to use the provided preferred host, except on task retry atttempts,
    +        // to minimize number of kafka metadata requests
    +        private def connectLeader: SimpleConsumer = {
    +          if (context.attemptNumber > 0) {
    +            kc.connectLeader(part.topic, part.partition).fold(
    +              errs => throw new Exception(
    +                s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " +
    +                  errs.mkString("\n")),
    +              consumer => consumer
    +            )
    +          } else {
    +            kc.connect(part.host, part.port)
    +          }
    +        }
    +
    +        private def handleErr(resp: FetchResponse) {
    +          if (resp.hasError) {
    +            val err = resp.errorCode(part.topic, part.partition)
    +            if (err == ErrorMapping.LeaderNotAvailableCode ||
    +              err == ErrorMapping.NotLeaderForPartitionCode) {
    +              log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " +
    +                s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms")
    +              Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +            }
    +            // Let normal rdd retry sort out reconnect attempts
    +            throw ErrorMapping.exceptionFor(err)
    +          }
    +        }
    +
    +        override def close() = consumer.close()
    +
    +        override def getNext: R = {
    +          if (iter == null || !iter.hasNext) {
    +            val req = new FetchRequestBuilder().
    +              addFetch(part.topic, part.partition, requestOffset, kc.config.fetchMessageMaxBytes).
    --- End diff --
    
    move the dot to next line for line 125 and line 126 to be consistent


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23742259
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/OffsetRange.scala ---
    @@ -0,0 +1,70 @@
    +/*
    + * 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.rdd.kafka
    +
    +/** Represents a range of offsets from a single Kafka TopicAndPartition */
    +trait OffsetRange {
    --- End diff --
    
    How about something like this. 
    ```
    class TopicPartitionOffsetRange private[kafka] (
       val topic: String, 
       val partition: .....
       val fromOffset: ...
       val untilOffset: ...
    ) {
       
    
    }
    ```
    and then we can define KafkaRDDPartition as 
    
    ```
    private[kafka] class KafkaRDDPartition(offsetRange: OffsetRange, host: String, port: Int) extends Partition
    ```
    
    - Does not expost KafkaRDDPartition
    - Does not use Scala trait or Java interface (requires separate java file, annoying)
    - People can publicly define OffsetRanges
    
    If we want to be able to define sets of OffsetRange (not sure if we need to), we can implement hashCode and equals as well. Case class would have given that for free, but the automatically defined object method that come with case classes can become annoying for Java API compatibility. Better to steer clear. 



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72758821
  
    Besides introducing 2 classes where 1 would do, it implies that there are
    (or could be) multiple implementations of the abstract class.  You're not
    using it because you're actually planning for subclassing, you're using it
    as a workaround for returning a slightly less complicated type from a
    single method, where there's an alternative... just return RDD[(K, V)] for
    that one method.
    
    This really is a situation where there's only 1 implementation for the
    foreseeable future, and a single final concrete class would be cleaner.
    
    
    On Tue, Feb 3, 2015 at 5:21 PM, Tathagata Das <no...@github.com>
    wrote:
    
    > I dont get it, what's the complication with abstract classes?
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798#issuecomment-72758029>.
    >


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24028643
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,174 @@ 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    +    val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message)
    +    val kc = new KafkaCluster(kafkaParams)
    +    val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet
    +    val leaders = kc.findLeaders(topics).fold(
    +      errs => throw new SparkException(errs.mkString("\n")),
    +      ok => ok
    +    )
    +    new KafkaRDD[K, V, U, T, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler)
    +  }
    +
    +  /** 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   * @param leaders Kafka leaders for each offset range in batch
    +   * @param messageHandler function for translating each message into the desired type
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange],
    +      leaders: Array[Leader],
    +      messageHandler: MessageAndMetadata[K, V] => R
    +  ): RDD[R] with HasOffsetRanges = {
    +
    +    val leaderMap = leaders
    +      .map(l => TopicAndPartition(l.topic, l.partition) -> (l.host, l.port))
    +      .toMap
    +    new KafkaRDD[K, V, U, T, R](sc, kafkaParams, offsetRanges, leaderMap, messageHandler)
    +  }
    +
    +  /**
    +   * This stream can guarantee that each message from Kafka is included in transformations
    +   * (as opposed to output actions) exactly once, even in most failure situations.
    +   *
    +   * Points to note:
    +   *
    +   * Failure Recovery - You must checkpoint this stream, or save offsets yourself and provide them
    +   * as the fromOffsets parameter on restart.
    +   * Kafka must have sufficient log retention to obtain messages after failure.
    +   *
    +   * Getting offsets from the stream - see programming guide
    +   *
    +.  * Zookeeper - This does not use Zookeeper to store offsets.  For interop with Kafka monitors
    +   * that depend on Zookeeper, you must store offsets in ZK yourself.
    +   *
    +   * End-to-end semantics - This does not guarantee that any output operation will push each record
    +   * exactly once. To ensure end-to-end exactly-once semantics (that is, receiving exactly once and
    +   * outputting exactly once), you have to either ensure that the output operation is
    +   * idempotent, or transactionally store offsets with the output. See the programming guide for
    +   * more details.
    +   *
    +   * @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
    +   */
    +  @Experimental
    +  def createNewStream[
    --- End diff --
    
    CreateOffsetRangeStream seems like you have give a OffsetRange to create one, whereas you can create a stream without explicitly specifying the offsetrange. 
    I have found all the names to either becoming too long, or not be conveying that it is a receiver-less stream. That includes `createDirectStream`. But `createDirectStream` does not convey much, but is definitely not overpromising.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24019815
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,174 @@ 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    +    val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message)
    +    val kc = new KafkaCluster(kafkaParams)
    +    val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet
    +    val leaders = kc.findLeaders(topics).fold(
    +      errs => throw new SparkException(errs.mkString("\n")),
    +      ok => ok
    +    )
    +    new KafkaRDD[K, V, U, T, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler)
    +  }
    +
    +  /** 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   * @param leaders Kafka leaders for each offset range in batch
    +   * @param messageHandler function for translating each message into the desired type
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange],
    +      leaders: Array[Leader],
    +      messageHandler: MessageAndMetadata[K, V] => R
    +  ): RDD[R] with HasOffsetRanges = {
    +
    +    val leaderMap = leaders
    +      .map(l => TopicAndPartition(l.topic, l.partition) -> (l.host, l.port))
    +      .toMap
    +    new KafkaRDD[K, V, U, T, R](sc, kafkaParams, offsetRanges, leaderMap, messageHandler)
    +  }
    +
    +  /**
    +   * This stream can guarantee that each message from Kafka is included in transformations
    +   * (as opposed to output actions) exactly once, even in most failure situations.
    +   *
    +   * Points to note:
    +   *
    +   * Failure Recovery - You must checkpoint this stream, or save offsets yourself and provide them
    +   * as the fromOffsets parameter on restart.
    +   * Kafka must have sufficient log retention to obtain messages after failure.
    +   *
    +   * Getting offsets from the stream - see programming guide
    +   *
    +.  * Zookeeper - This does not use Zookeeper to store offsets.  For interop with Kafka monitors
    +   * that depend on Zookeeper, you must store offsets in ZK yourself.
    +   *
    +   * End-to-end semantics - This does not guarantee that any output operation will push each record
    +   * exactly once. To ensure end-to-end exactly-once semantics (that is, receiving exactly once and
    +   * outputting exactly once), you have to either ensure that the output operation is
    +   * idempotent, or transactionally store offsets with the output. See the programming guide for
    +   * more details.
    +   *
    +   * @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
    +   */
    +  @Experimental
    +  def createNewStream[
    --- End diff --
    
    I suggested a couple of alternative names in the design doc. among them createOffsetRangeStream, which I think conveys what's going on.  I'm fine with anything that isn't over-promising ("createExactlyOnceStream")


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by helena <gi...@git.apache.org>.
Github user helena commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22647904
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala ---
    @@ -0,0 +1,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.streaming.kafka
    +
    +import scala.annotation.tailrec
    +import scala.reflect.{classTag, ClassTag}
    +
    +import kafka.common.TopicAndPartition
    +import kafka.message.MessageAndMetadata
    +import kafka.serializer.Decoder
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.rdd.kafka.{KafkaCluster, KafkaRDD}
    +import org.apache.spark.streaming.{StreamingContext, Time}
    +import org.apache.spark.streaming.dstream._
    +
    +/** A stream of {@link org.apache.spark.rdd.kafka.KafkaRDD} where
    +  * each given Kafka topic/partition corresponds to an RDD partition.
    +  * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages
    +  * per second that each '''partition''' will accept.
    +  * Starting offsets are specified in advance,
    +  * and this DStream is not responsible for committing offsets,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the stream
    +  * @param messageHandler function for translating each message into the desired type
    +  * @param maxRetries maximum number of times in a row to retry getting leaders' offsets
    +  */
    +class DeterministicKafkaInputDStream[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    @transient ssc_ : StreamingContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R,
    +    maxRetries: Int = 1
    +) extends InputDStream[R](ssc_) with Logging {
    +
    +  private val kc = new KafkaCluster(kafkaParams)
    +
    +  private val maxMessagesPerPartition: Option[Long] = {
    +    val ratePerSec = context.sparkContext.getConf.getInt("spark.streaming.receiver.maxRate", 0)
    +    if (ratePerSec > 0) {
    +      val secsPerBatch = context.graph.batchDuration.milliseconds.toDouble / 1000
    +      Some((secsPerBatch * ratePerSec).toLong)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  private var currentOffsets = fromOffsets
    +
    +  @tailrec
    +  private def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, Long] = {
    +    val o = kc.getLatestLeaderOffsets(currentOffsets.keys.toSet)
    +    // Either.fold would confuse @tailrec, do it manually
    +    if (o.isLeft) {
    +      val err = o.left.get.toString
    +      if (retries <= 0) {
    +        throw new Exception(err)
    +      } else {
    +        log.error(err)
    +        Thread.sleep(kc.config.refreshLeaderBackoffMs)
    --- End diff --
    
    another Thread.sleep - why? Is there another way to do this?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-68802901
  
    Any thoughts on my comments on driver fault-tolerance?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23967901
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala ---
    @@ -0,0 +1,150 @@
    +/*
    + * 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.streaming.kafka
    +
    +
    +import scala.annotation.tailrec
    +import scala.collection.mutable
    +import scala.reflect.{classTag, ClassTag}
    +
    +import kafka.common.TopicAndPartition
    +import kafka.message.MessageAndMetadata
    +import kafka.serializer.Decoder
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset
    +import org.apache.spark.streaming.{StreamingContext, Time}
    +import org.apache.spark.streaming.dstream._
    +
    +/** A stream of {@link org.apache.spark.rdd.kafka.KafkaRDD} where
    +  * each given Kafka topic/partition corresponds to an RDD partition.
    +  * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages
    +  * per second that each '''partition''' will accept.
    +  * Starting offsets are specified in advance,
    +  * and this DStream is not responsible for committing offsets,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the stream
    +  * @param messageHandler function for translating each message into the desired type
    +  * @param maxRetries maximum number of times in a row to retry getting leaders' offsets
    +  */
    +private[streaming]
    +class DeterministicKafkaInputDStream[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    @transient ssc_ : StreamingContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R,
    +    maxRetries: Int
    +) extends InputDStream[R](ssc_) with Logging {
    +
    +  protected[streaming] override val checkpointData =
    +    new DeterministicKafkaInputDStreamCheckpointData
    +
    +  protected val kc = new KafkaCluster(kafkaParams)
    +
    +  protected val maxMessagesPerPartition: Option[Long] = {
    +    val ratePerSec = context.sparkContext.getConf.getInt("spark.streaming.receiver.maxRate", 0)
    +    if (ratePerSec > 0) {
    +      val secsPerBatch = context.graph.batchDuration.milliseconds.toDouble / 1000
    +      Some((secsPerBatch * ratePerSec).toLong)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  protected var currentOffsets = fromOffsets
    +
    +  @tailrec
    +  protected final def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, LeaderOffset] = {
    +    val o = kc.getLatestLeaderOffsets(currentOffsets.keySet)
    +    // Either.fold would confuse @tailrec, do it manually
    +    if (o.isLeft) {
    +      val err = o.left.get.toString
    +      if (retries <= 0) {
    +        throw new Exception(err)
    +      } else {
    +        log.error(err)
    +        Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +        latestLeaderOffsets(retries - 1)
    +      }
    +    } else {
    +      o.right.get
    +    }
    +  }
    +
    +  protected def clamp(
    +    leaderOffsets: Map[TopicAndPartition, LeaderOffset]): Map[TopicAndPartition, LeaderOffset] = {
    +    maxMessagesPerPartition.map { mmp =>
    +      leaderOffsets.map { case (tp, lo) =>
    +        tp -> lo.copy(offset = Math.min(currentOffsets(tp) + mmp, lo.offset))
    +      }
    +    }.getOrElse(leaderOffsets)
    +  }
    +
    +  override def compute(validTime: Time): Option[KafkaRDD[K, V, U, T, R]] = {
    +    val untilOffsets = clamp(latestLeaderOffsets(maxRetries))
    +    val rdd = KafkaRDD[K, V, U, T, R](
    +      context.sparkContext, kafkaParams, currentOffsets, untilOffsets, messageHandler)
    +
    +    currentOffsets = untilOffsets.map(kv => kv._1 -> kv._2.offset)
    +    Some(rdd)
    +  }
    +
    +  override def start(): Unit = {
    +  }
    +
    +  def stop(): Unit = {
    +  }
    +
    +  private[streaming]
    +  class DeterministicKafkaInputDStreamCheckpointData extends DStreamCheckpointData(this) {
    +    def batchForTime = data.asInstanceOf[mutable.HashMap[
    +      Time, Array[(Int, String, Int, Long, Long, String, Int)]]]
    --- End diff --
    
    @koeninger Yes, please making this a specific type. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71667244
  
    Most of Either's problems can be fixed with a one-line implicit conversion
    to RightProjection.  I've seen scalactic before, seems like overkill by
    comparison.
    
    On Mon, Jan 26, 2015 at 8:08 PM, Imran Rashid <no...@github.com>
    wrote:
    
    > @koeninger <https://github.com/koeninger>
    > I doubt that we want to go this route in this case, but just in case
    > you're interested, I think a much better way to handle multiple errors
    > gracefully is with scalactic's Or
    > <http://www.scalactic.org/user_guide/OrAndEvery>. Its much better than
    > Either for this case of building up a set of errors to report back to the
    > user. And scalactic is a nicely designed, small library (eg. you're not
    > pulling scalaz). Probably not worth it for this one case, but thought you
    > might find it interesting :)
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798#issuecomment-71577982>.
    >


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23822761
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +149,182 @@ 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
    +   */
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      batch: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    +    val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message)
    +    val kc = new KafkaCluster(kafkaParams)
    +    val topics = batch.map(o => TopicAndPartition(o.topic, o.partition)).toSet
    +    val leaderMap = kc.findLeaders(topics).fold(
    +      errs => throw new SparkException(errs.mkString("\n")),
    +      ok => ok
    +    )
    +    val rddParts = batch.zipWithIndex.map { case (o, i) =>
    +        val tp = TopicAndPartition(o.topic, o.partition)
    +        val (host, port) = leaderMap(tp)
    +        new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset, host, port)
    +    }.toArray
    +    new KafkaRDD[K, V, U, T, (K, V)](sc, kafkaParams, rddParts, messageHandler)
    +  }
    +
    +  /** 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 leaders Kafka leaders for each offset range in batch
    +   * @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],
    +      leaders: Array[Leader],
    +      messageHandler: MessageAndMetadata[K, V] => R
    +  ): RDD[R] with HasOffsetRanges = {
    +    val leaderMap = leaders.map(l => (l.topic, l.partition) -> (l.host, l.port)).toMap
    +    val rddParts = batch.zipWithIndex.map { case (o, i) =>
    +        val (host, port) = leaderMap((o.topic, o.partition))
    +        new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset, host, port)
    +    }.toArray
    +
    +    new KafkaRDD[K, V, U, T, R](sc, kafkaParams, rddParts, messageHandler)
    +  }
    +
    +  /**
    +   * Compared to `createStream`, the stream created by this can guarantee that each message
    +   * from Kafka is included in transformations (as opposed to output actions) exactly once,
    +   * even in most failure situations.
    +   *
    +   * Points to note:
    +   *
    +   * Failure Recovery - You must checkpoint this stream, or save offsets yourself and provide them
    +   * as the fromOffsets parameter on restart.
    +   * Kafka must have sufficient log retention to obtain messages after failure.
    +   *
    +   * Getting offsets from the stream - see programming guide
    +   *
    +.  * Zookeeper - This does not use Zookeeper to store offsets.  For interop with Kafka monitors
    +   * that depend on Zookeeper, you must store offsets in ZK yourself.
    +   *
    +   * End-to-end semantics - This does not guarantee that any output operation will push each record
    +   * exactly once. To ensure end-to-end exactly-once semantics (that is, receiving exactly once and
    +   * outputting exactly once), you have to either ensure that the output operation is
    +   * idempotent, or transactionally store offsets with the output. See the programming guide for
    +   * more details.
    +   *
    +   * @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 createNewStream[
    --- End diff --
    
    BTW, all these public methods needs to annotated with "@Experimental" (see org.apache.spark.annotation.Experimental and its uses).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72773257
  
    To be clear, I'm ok with any solution that gives me access to what I need,
    which in this case are offsets.
    
    What's coming across as me feeling strongly about it is, I think, really
    because I'm not clear on what your principles with regard to the interface
    are ... well, that and frustration because it's been changed 3 times
    already :)
    
    For instance, why would you be willing to take on the fragile base class
    maintenance problem in exposing KafkaRDD as something that could be
    subclassed... but not ok with exposing the DStream (so that people could
    override the batch generation policy)?
    
    In the interests of moving this forward, if we're really just talking about
    changing KafkaUtil's use of
    
    RDD[..] with HasOffsetRanges
    to
    RDD[..]
    
    I can make that that change
    
    
    On Tue, Feb 3, 2015 at 7:02 PM, Tathagata Das <no...@github.com>
    wrote:
    
    > Okay here are the two options.
    >
    >    1. createRDD returns RDD[(K,V)] or RDD[R], and DStream.foreachRDD uses
    >    rdd.asInstanceOf[HasOffsetRanges]
    >    2. createRDD returns KafkaRDD[(K,V)] or KafkaRDD[R] and
    >    DStream.foreachRDD uses rdd.asInstanceOf[KafkaRDD[_]]
    >
    > I think I am okay with either one. Stepping back, my original concern was
    > returning something that had no binary compatibility issues. Both solution
    > suffices. Between these two, since you feel so strongly against (2), lets
    > go with (1).
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798#issuecomment-72770451>.
    >



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72780349
  
    High level consumers connect to ZK.
    
    Simple consumers (which is what this is using) connect to brokers directly
    instead.  See
    https://cwiki.apache.org/confluence/display/KAFKA/0.8.0+SimpleConsumer+Example
    
    I chose to accept either of the two existing means in Kafka of specifying a
    list of seed brokers, rather than making up yet a third way
    
    
    
    On Tue, Feb 3, 2015 at 8:36 PM, Tathagata Das <no...@github.com>
    wrote:
    
    > Why did you choose the parameters "metadata.broker.list" and the
    > "bootstrap.servers" as the required kafka params? I looked at the Kafka
    > docs, and it says that for consumers, the necessary properties are
    > "zookeeper.connect" and "group.id". And intuitively the application is
    > consuming, so the consumer configs should apply (not "group.id", but
    > "zookeeper.connect"). So our interface should also require
    > "zookeeper.connect" and not other two. Isnt it?
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798#issuecomment-72779779>.
    >



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72779615
  
    Yeah, there's a weird distinction in Kafka between simple consumers and
    high level consumers in that they have a lot of common configuration
    parameters, but one of them talks directly to brokers and the other goes
    through zk.
    
    I'll see if I can make a private subclass of ConsumerConfig to shut that
    warning up.
    
    On Tue, Feb 3, 2015 at 8:28 PM, Tathagata Das <no...@github.com>
    wrote:
    
    > Hey Cody, I was trying it and I found a odd behavior. It was printing this
    > repeatedly.
    >
    > 15/02/03 18:22:08 WARN VerifiableProperties: Property metadata.broker.list is not valid
    >
    > I was using this code.
    >
    > val kafkaParams = Map[String, String]("metadata.broker.list" -> brokerList)
    >     val lines = KafkaUtils.createNewStream[String, String, StringDecoder, StringDecoder](
    >       ssc, kafkaParams, topicsSet)
    >
    > I chose "metadata.broker.list" from the code in KafkaCluster, because
    > without that I was getting exception from the KafkaCluster.
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798#issuecomment-72779120>.
    >



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24021621
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,344 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.util.Random
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +import org.apache.spark.SparkException
    +
    +/**
    + * Convenience methods for interacting with a Kafka cluster.
    + * @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
    + */
    +private[spark]
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    +  import KafkaCluster.{Err, LeaderOffset}
    +
    +  val seedBrokers: Array[(String, Int)] =
    +    kafkaParams.get("metadata.broker.list")
    +      .orElse(kafkaParams.get("bootstrap.servers"))
    +      .getOrElse(throw new SparkException("Must specify metadata.broker.list or bootstrap.servers"))
    +      .split(",").map { hp =>
    +        val hpa = hp.split(":")
    +        (hpa(0), hpa(1).toInt)
    +      }
    +
    +  // ConsumerConfig isn't serializable
    +  @transient private var _config: ConsumerConfig = null
    +
    +  def config: ConsumerConfig = this.synchronized {
    +    if (_config == null) {
    +      _config = KafkaCluster.consumerConfig(kafkaParams)
    +    }
    +    _config
    +  }
    +
    +  def connect(host: String, port: Int): SimpleConsumer =
    +    new SimpleConsumer(host, port, config.socketTimeoutMs,
    +      config.socketReceiveBufferBytes, config.clientId)
    +
    +  def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] =
    +    findLeader(topic, partition).right.map(hp => connect(hp._1, hp._2))
    +
    +  // Metadata api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataAPI
    +  // scalastyle:on
    +
    +  def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, Seq(topic))
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      resp.topicsMetadata.find(_.topic == topic).flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.find(_.partitionId == partition)
    +      }.foreach { pm: PartitionMetadata =>
    +        pm.leader.foreach { leader =>
    +          return Right((leader.host, leader.port))
    +        }
    +      }
    +    }
    +    Left(errs)
    +  }
    +
    +  def findLeaders(
    +      topicAndPartitions: Set[TopicAndPartition]
    +    ): Either[Err, Map[TopicAndPartition, (String, Int)]] = {
    +    val topics = topicAndPartitions.map(_.topic)
    +    val response = getPartitionMetadata(topics).right
    +    val answer = response.flatMap { tms: Set[TopicMetadata] =>
    +      val leaderMap = tms.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.flatMap { pm: PartitionMetadata =>
    +          val tp = TopicAndPartition(tm.topic, pm.partitionId)
    +          if (topicAndPartitions(tp)) {
    +            pm.leader.map { l =>
    +              tp -> (l.host -> l.port)
    +            }
    +          } else {
    +            None
    +          }
    +        }
    +      }.toMap
    +
    +      if (leaderMap.keys.size == topicAndPartitions.size) {
    +        Right(leaderMap)
    +      } else {
    +        val missing = topicAndPartitions.diff(leaderMap.keySet)
    +        val err = new Err
    +        err.append(new SparkException(s"Couldn't find leaders for ${missing}"))
    +        Left(err)
    +      }
    +    }
    +    answer
    +  }
    +
    +  def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] = {
    +    getPartitionMetadata(topics).right.map { r =>
    +      r.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.map { pm: PartitionMetadata =>
    +          TopicAndPartition(tm.topic, pm.partitionId)
    +        }    
    +      }
    +    }
    +  }
    +
    +  def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = {
    +    val req = TopicMetadataRequest(
    +      TopicMetadataRequest.CurrentVersion, 0, config.clientId, topics.toSeq)
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      // error codes here indicate missing / just created topic,
    +      // repeating on a different broker wont be useful
    +      return Right(resp.topicsMetadata.toSet)
    +    }
    +    Left(errs)
    +  }
    +
    +  // Leader offset api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI
    +  // scalastyle:on
    +
    +  def getLatestLeaderOffsets(
    +      topicAndPartitions: Set[TopicAndPartition]
    +    ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime)
    +
    +  def getEarliestLeaderOffsets(
    +      topicAndPartitions: Set[TopicAndPartition]
    +    ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime)
    +
    +  def getLeaderOffsets(
    +      topicAndPartitions: Set[TopicAndPartition],
    +      before: Long
    +    ): Either[Err, Map[TopicAndPartition, LeaderOffset]] = {
    +    getLeaderOffsets(topicAndPartitions, before, 1).right.map { r =>
    +      r.map { kv =>
    +        // mapValues isnt serializable, see SI-7005
    +        kv._1 -> kv._2.head
    +      }
    +    }
    +  }
    +
    +  private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] =
    +    m.groupBy(_._2).map { kv =>
    +      kv._1 -> kv._2.keys.toSeq
    +    }
    +
    +  def getLeaderOffsets(
    +      topicAndPartitions: Set[TopicAndPartition],
    +      before: Long,
    +      maxNumOffsets: Int
    +    ): Either[Err, Map[TopicAndPartition, Seq[LeaderOffset]]] = {
    +    findLeaders(topicAndPartitions).right.flatMap { tpToLeader =>
    +      val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] = flip(tpToLeader)
    +      val leaders = leaderToTp.keys
    +      var result = Map[TopicAndPartition, Seq[LeaderOffset]]()
    +      val errs = new Err
    +      withBrokers(leaders, errs) { consumer =>
    +        val partitionsToGetOffsets: Seq[TopicAndPartition] =
    +          leaderToTp((consumer.host, consumer.port))
    +        val reqMap = partitionsToGetOffsets.map { tp: TopicAndPartition =>
    +          tp -> PartitionOffsetRequestInfo(before, maxNumOffsets)
    +        }.toMap
    +        val req = OffsetRequest(reqMap)
    +        val resp = consumer.getOffsetsBefore(req)
    +        val respMap = resp.partitionErrorAndOffsets
    +        partitionsToGetOffsets.foreach { tp: TopicAndPartition =>
    +          respMap.get(tp).foreach { por: PartitionOffsetsResponse =>
    +            if (por.error == ErrorMapping.NoError) {
    +              if (por.offsets.nonEmpty) {
    +                result += tp -> por.offsets.map { off =>
    +                  LeaderOffset(consumer.host, consumer.port, off)
    +                }
    +              } else {
    +                errs.append(new SparkException(
    +                  s"Empty offsets for ${tp}, is ${before} before log beginning?"))
    +              }
    +            } else {
    +              errs.append(ErrorMapping.exceptionFor(por.error))
    +            }
    +          }
    +        }
    +        if (result.keys.size == topicAndPartitions.size) {
    +          return Right(result)
    +        }
    +      }
    +      val missing = topicAndPartitions.diff(result.keySet)
    +      errs.append(new SparkException(s"Couldn't find leader offsets for ${missing}"))
    +      Left(errs)
    +    }
    +  }
    +
    +  // Consumer offset api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI
    +  // scalastyle:on
    +
    +  def getConsumerOffsets(
    +      groupId: String,
    +      topicAndPartitions: Set[TopicAndPartition]
    +    ): Either[Err, Map[TopicAndPartition, Long]] = {
    +    getConsumerOffsetMetadata(groupId, topicAndPartitions).right.map { r =>
    +      r.map { kv =>
    +        kv._1 -> kv._2.offset
    +      }
    +    }
    +  }
    +
    +  def getConsumerOffsetMetadata(
    +      groupId: String,
    +      topicAndPartitions: Set[TopicAndPartition]
    +    ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = {
    +    var result = Map[TopicAndPartition, OffsetMetadataAndError]()
    +    val req = OffsetFetchRequest(groupId, topicAndPartitions.toSeq)
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp = consumer.fetchOffsets(req)
    +      val respMap = resp.requestInfo
    +      val needed = topicAndPartitions.diff(result.keySet)
    +      needed.foreach { tp: TopicAndPartition =>
    +        respMap.get(tp).foreach { ome: OffsetMetadataAndError =>
    +          if (ome.error == ErrorMapping.NoError) {
    +            result += tp -> ome
    +          } else {
    +            errs.append(ErrorMapping.exceptionFor(ome.error))
    +          }
    +        }
    +      }
    +      if (result.keys.size == topicAndPartitions.size) {
    +        return Right(result)
    +      }
    +    }
    +    val missing = topicAndPartitions.diff(result.keySet)
    +    errs.append(new SparkException(s"Couldn't find consumer offsets for ${missing}"))
    +    Left(errs)
    +  }
    +
    +  def setConsumerOffsets(
    +      groupId: String,
    +      offsets: Map[TopicAndPartition, Long]
    +    ): Either[Err, Map[TopicAndPartition, Short]] = {
    +    setConsumerOffsetMetadata(groupId, offsets.map { kv =>
    +      kv._1 -> OffsetMetadataAndError(kv._2)
    +    })
    +  }
    +
    +  def setConsumerOffsetMetadata(
    +      groupId: String,
    +      metadata: Map[TopicAndPartition, OffsetMetadataAndError]
    +    ): Either[Err, Map[TopicAndPartition, Short]] = {
    +    var result = Map[TopicAndPartition, Short]()
    +    val req = OffsetCommitRequest(groupId, metadata)
    --- End diff --
    
    Well, the change to kafka-topic-backed rather than ZK-backed offsets is bigger than any of those concerns, and no, using ZK client directly doesn't solve that problem.
    
    Bottom line is this:  Kafka consumer offset management api is a moving target.  This PR doesn't expose public methods for dealing with kafka consumer offsets.  Nothing outside of the testing code uses the private methods it does provide.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23571892
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,318 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.util.Random
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    --- End diff --
    
    can you turn all scaladoc style into javadoc for this pr? 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23586464
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,318 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.util.Random
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    +  * Convenience methods for interacting with a Kafka cluster.
    +  * @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
    +  */
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    +  import KafkaCluster.{Err, LeaderOffset}
    +
    +  val seedBrokers: Array[(String, Int)] =
    +    kafkaParams.get("metadata.broker.list")
    +      .orElse(kafkaParams.get("bootstrap.servers"))
    +      .getOrElse(throw new Exception("Must specify metadata.broker.list or bootstrap.servers"))
    +      .split(",").map { hp =>
    +        val hpa = hp.split(":")
    +        (hpa(0), hpa(1).toInt)
    +      }
    +
    +  // ConsumerConfig isn't serializable
    +  @transient private var _config: ConsumerConfig = null
    +
    +  def config: ConsumerConfig = this.synchronized {
    +    if (_config == null) {
    +      _config = KafkaCluster.consumerConfig(kafkaParams)
    +    }
    +    _config
    +  }
    +
    +  def connect(host: String, port: Int): SimpleConsumer =
    +    new SimpleConsumer(host, port, config.socketTimeoutMs,
    +      config.socketReceiveBufferBytes, config.clientId)
    +
    +  def connect(hostAndPort: (String, Int)): SimpleConsumer =
    +    connect(hostAndPort._1, hostAndPort._2)
    +
    +  def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] =
    +    findLeader(topic, partition).right.map(connect)
    +
    +  def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, Seq(topic))
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      resp.topicsMetadata.find(_.topic == topic).flatMap { t =>
    --- End diff --
    
    this block of code is pretty hard to understand with so many level of nesting. can you rewrite it? maybe by introducing variables and adding comments to explain what is going on. overall I feel this PR went slightly overboard with Scala. With no explicit type, intermediate variable, and comment, it is pretty hard to understand a lot of blocks


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72902696
  
    That's a nifty solution :) I like it.
    Lets merge this as soon as the tests pass. Smaller changes like docs, etc. , we can do it in the next PR. 
    @jerryshao I will add the example in a different PR. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23729871
  
    --- 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 --
    
    There are multiple possible underlying exceptions, because it may have tried multiple brokers before failing or succeeeding


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23973371
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,338 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.util.Random
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    + * Convenience methods for interacting with a Kafka cluster.
    + * @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
    + */
    +private[spark]
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    +  import KafkaCluster.{Err, LeaderOffset}
    +
    +  val seedBrokers: Array[(String, Int)] =
    +    kafkaParams.get("metadata.broker.list")
    +      .orElse(kafkaParams.get("bootstrap.servers"))
    +      .getOrElse(throw new Exception("Must specify metadata.broker.list or bootstrap.servers"))
    +      .split(",").map { hp =>
    +        val hpa = hp.split(":")
    +        (hpa(0), hpa(1).toInt)
    +      }
    +
    +  // ConsumerConfig isn't serializable
    +  @transient private var _config: ConsumerConfig = null
    +
    +  def config: ConsumerConfig = this.synchronized {
    +    if (_config == null) {
    +      _config = KafkaCluster.consumerConfig(kafkaParams)
    +    }
    +    _config
    +  }
    +
    +  def connect(host: String, port: Int): SimpleConsumer =
    +    new SimpleConsumer(host, port, config.socketTimeoutMs,
    +      config.socketReceiveBufferBytes, config.clientId)
    +
    +  def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] =
    +    findLeader(topic, partition).right.map(hp => connect(hp._1, hp._2))
    +
    +  // Metadata api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataAPI
    +  // scalastyle:on
    +
    +  def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, Seq(topic))
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      resp.topicsMetadata.find(_.topic == topic).flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.find(_.partitionId == partition)
    +      }.foreach { pm: PartitionMetadata =>
    +        pm.leader.foreach { leader =>
    +          return Right((leader.host, leader.port))
    +        }
    +      }
    +    }
    +    Left(errs)
    +  }
    +
    +  def findLeaders(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, (String, Int)]] = {
    +    val topics = topicAndPartitions.map(_.topic)
    +    getPartitionMetadata(topics).right.flatMap { tms: Set[TopicMetadata] =>
    +      val result = tms.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.flatMap { pm: PartitionMetadata =>
    +          val tp = TopicAndPartition(tm.topic, pm.partitionId)
    +          if (topicAndPartitions(tp)) {
    +            pm.leader.map { l =>
    +              tp -> (l.host -> l.port)
    +            }
    +          } else {
    +            None
    +          }
    +        }
    +      }.toMap
    +      if (result.keys.size == topicAndPartitions.size) {
    +        Right(result)
    +      } else {
    +        val missing = topicAndPartitions.diff(result.keySet)
    +        val err = new Err
    +        err.append(new Exception(s"Couldn't find leaders for ${missing}"))
    +        Left(err)
    +      }
    +    }
    +  }
    +
    +  def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] =
    +    getPartitionMetadata(topics).right.map { r =>
    +      r.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.map { pm: PartitionMetadata =>
    +          TopicAndPartition(tm.topic, pm.partitionId)
    +        }    
    +      }
    +    }
    +
    +  def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = {
    +    val req = TopicMetadataRequest(
    +      TopicMetadataRequest.CurrentVersion, 0, config.clientId, topics.toSeq)
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      // error codes here indicate missing / just created topic,
    +      // repeating on a different broker wont be useful
    +      return Right(resp.topicsMetadata.toSet)
    +    }
    +    Left(errs)
    +  }
    +
    +  // Leader offset api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI
    +  // scalastyle:on
    +
    +  def getLatestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime)
    +
    +  def getEarliestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime)
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long
    +  ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    +    getLeaderOffsets(topicAndPartitions, before, 1).right.map { r =>
    +      r.map { kv =>
    +        // mapValues isnt serializable, see SI-7005
    +        kv._1 -> kv._2.head
    +      }
    +    }
    +
    +  private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] =
    +    m.groupBy(_._2).map { kv =>
    +      kv._1 -> kv._2.keys.toSeq
    +    }
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long,
    +    maxNumOffsets: Int
    +  ): Either[Err, Map[TopicAndPartition, Seq[LeaderOffset]]] = {
    +    findLeaders(topicAndPartitions).right.flatMap { tpToLeader =>
    +      val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] = flip(tpToLeader)
    +      val leaders = leaderToTp.keys
    +      var result = Map[TopicAndPartition, Seq[LeaderOffset]]()
    +      val errs = new Err
    +      withBrokers(leaders, errs) { consumer =>
    +        val needed: Seq[TopicAndPartition] = leaderToTp((consumer.host, consumer.port))
    +        val req = OffsetRequest(
    +          needed.map { tp: TopicAndPartition =>
    +            tp -> PartitionOffsetRequestInfo(before, maxNumOffsets)
    +          }.toMap
    +        )
    +        val resp = consumer.getOffsetsBefore(req)
    +        val respMap = resp.partitionErrorAndOffsets
    +        needed.foreach { tp: TopicAndPartition =>
    +          respMap.get(tp).foreach { por: PartitionOffsetsResponse =>
    +            if (por.error == ErrorMapping.NoError) {
    +              if (por.offsets.nonEmpty) {
    +                result += tp -> por.offsets.map { off =>
    +                  LeaderOffset(consumer.host, consumer.port, off)
    +                }
    +              } else {
    +                errs.append(new Exception(
    +                  s"Empty offsets for ${tp}, is ${before} before log beginning?"))
    +              }
    +            } else {
    +              errs.append(ErrorMapping.exceptionFor(por.error))
    +            }
    +          }
    +        }
    +        if (result.keys.size == topicAndPartitions.size) {
    +          return Right(result)
    +        }
    +      }
    +      val missing = topicAndPartitions.diff(result.keySet)
    +      errs.append(new Exception(s"Couldn't find leader offsets for ${missing}"))
    +      Left(errs)
    +    }
    +  }
    +
    +  // Consumer offset api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI
    +  // scalastyle:on
    +
    +  def getConsumerOffsets(
    +      groupId: String,
    +      topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] = {
    +    getConsumerOffsetMetadata(groupId, topicAndPartitions).right.map { r =>
    +      r.map { kv =>
    +        kv._1 -> kv._2.offset
    +      }
    +    }
    +  }
    +
    +  def getConsumerOffsetMetadata(
    +      groupId: String,
    +      topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = {
    +    var result = Map[TopicAndPartition, OffsetMetadataAndError]()
    +    val req = OffsetFetchRequest(groupId, topicAndPartitions.toSeq)
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp = consumer.fetchOffsets(req)
    +      val respMap = resp.requestInfo
    +      val needed = topicAndPartitions.diff(result.keySet)
    +      needed.foreach { tp: TopicAndPartition =>
    +        respMap.get(tp).foreach { ome: OffsetMetadataAndError =>
    +          if (ome.error == ErrorMapping.NoError) {
    +            result += tp -> ome
    +          } else {
    +            errs.append(ErrorMapping.exceptionFor(ome.error))
    +          }
    +        }
    +      }
    +      if (result.keys.size == topicAndPartitions.size) {
    +        return Right(result)
    +      }
    +    }
    +    val missing = topicAndPartitions.diff(result.keySet)
    +    errs.append(new Exception(s"Couldn't find consumer offsets for ${missing}"))
    +    Left(errs)
    +  }
    +
    +  def setConsumerOffsets(
    +      groupId: String,
    +      offsets: Map[TopicAndPartition, Long]
    +  ): Either[Err, Map[TopicAndPartition, Short]] = {
    --- End diff --
    
    indentation issue.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by dibbhatt <gi...@git.apache.org>.
Github user dibbhatt commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23890423
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala ---
    @@ -0,0 +1,220 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.reflect.{classTag, ClassTag}
    +
    +import org.apache.spark.{Logging, Partition, SparkContext, SparkException, TaskContext}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.NextIterator
    +
    +import java.util.Properties
    +import kafka.api.{FetchRequestBuilder, FetchResponse}
    +import kafka.common.{ErrorMapping, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +import kafka.message.{MessageAndMetadata, MessageAndOffset}
    +import kafka.serializer.Decoder
    +import kafka.utils.VerifiableProperties
    +
    +/**
    + * 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 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 KafkaRDDPartition 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
    + */
    +private[spark]
    +class KafkaRDD[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag] private[spark] (
    +    sc: SparkContext,
    +    kafkaParams: Map[String, String],
    +    private[spark] val batch: Array[KafkaRDDPartition],
    +    messageHandler: MessageAndMetadata[K, V] => R
    +  ) extends RDD[R](sc, Nil) with Logging with HasOffsetRanges {
    +
    +  def offsetRanges: Array[OffsetRange] = batch.asInstanceOf[Array[OffsetRange]]
    +
    +  override def getPartitions: Array[Partition] = batch.asInstanceOf[Array[Partition]]
    +
    +  override def getPreferredLocations(thePart: Partition): Seq[String] = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    // TODO is additional hostname resolution necessary here
    +    Seq(part.host)
    +  }
    +
    +  private def errBeginAfterEnd(part: KafkaRDDPartition): String =
    +    s"Beginning offset ${part.fromOffset} is after the ending offset ${part.untilOffset} " +
    +      s"for topic ${part.topic} partition ${part.partition}. " +
    +      "You either provided an invalid fromOffset, or the Kafka topic has been damaged"
    +
    +  private def errRanOutBeforeEnd(part: KafkaRDDPartition): String =
    +    s"Ran out of messages before reaching ending offset ${part.untilOffset} " +
    +    s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." +
    +    " This should not happen, and indicates that messages may have been lost"
    +
    +  private def errOvershotEnd(itemOffset: Long, part: KafkaRDDPartition): String =
    +    s"Got ${itemOffset} > ending offset ${part.untilOffset} " +
    +    s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." +
    +    " This should not happen, and indicates a message may have been skipped"
    +
    +  override def compute(thePart: Partition, context: TaskContext): Iterator[R] = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    assert(part.fromOffset <= part.untilOffset, errBeginAfterEnd(part))
    +    if (part.fromOffset == part.untilOffset) {
    +      log.warn("Beginning offset ${part.fromOffset} is the same as ending offset " +
    +        s"skipping ${part.topic} ${part.partition}")
    +      Iterator.empty
    +    } else {
    +      new KafkaRDDIterator(part, context)
    +    }
    +  }
    +
    +  private class KafkaRDDIterator(
    +      part: KafkaRDDPartition,
    +      context: TaskContext) extends NextIterator[R] {
    +
    +    context.addTaskCompletionListener{ context => closeIfNeeded() }
    +
    +    log.info(s"Computing topic ${part.topic}, partition ${part.partition} " +
    +      s"offsets ${part.fromOffset} -> ${part.untilOffset}")
    +
    +    val kc = new KafkaCluster(kafkaParams)
    +    val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +      .newInstance(kc.config.props)
    +      .asInstanceOf[Decoder[K]]
    +    val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +      .newInstance(kc.config.props)
    +      .asInstanceOf[Decoder[V]]
    +    val consumer = connectLeader
    +    var requestOffset = part.fromOffset
    +    var iter: Iterator[MessageAndOffset] = null
    +
    +    // The idea is to use the provided preferred host, except on task retry atttempts,
    +    // to minimize number of kafka metadata requests
    +    private def connectLeader: SimpleConsumer = {
    +      if (context.attemptNumber > 0) {
    +        kc.connectLeader(part.topic, part.partition).fold(
    +          errs => throw new SparkException(
    +            s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " +
    +              errs.mkString("\n")),
    +          consumer => consumer
    +        )
    +      } else {
    +        kc.connect(part.host, part.port)
    +      }
    +    }
    +
    +    private def handleFetchErr(resp: FetchResponse) {
    +      if (resp.hasError) {
    +        val err = resp.errorCode(part.topic, part.partition)
    +        if (err == ErrorMapping.LeaderNotAvailableCode ||
    +          err == ErrorMapping.NotLeaderForPartitionCode) {
    +          log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " +
    +            s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms")
    +          Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +        }
    +        // Let normal rdd retry sort out reconnect attempts
    +        throw ErrorMapping.exceptionFor(err)
    +      }
    +    }
    +
    --- End diff --
    
    Hi Cody, I believe in this implementation the range of offset of given only at the start, for rest of the flow the offset range is calculated automatically. What I say, if Offset_Out_Of_Range comes in those RDDs where you calculate the offset, this implementation can not recover from this failure. I did not say that we always get this error all time , but this error is not very rare cases also. and if the Receiver stops because of this, that's an issue. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23576495
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaRDD.scala ---
    @@ -0,0 +1,199 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.reflect.{classTag, ClassTag}
    +
    +import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.NextIterator
    +
    +import java.util.Properties
    +import kafka.api.{FetchRequestBuilder, FetchResponse}
    +import kafka.common.{ErrorMapping, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +import kafka.message.{MessageAndMetadata, MessageAndOffset}
    +import kafka.serializer.Decoder
    +import kafka.utils.VerifiableProperties
    +
    +/** A batch-oriented interface for consuming from Kafka.
    +  * Starting and ending offsets are specified in advance,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 KafkaRDDPartition 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
    +  */
    +class KafkaRDD[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    sc: SparkContext,
    +    val kafkaParams: Map[String, String],
    +    val batch: Array[KafkaRDDPartition],
    +    messageHandler: MessageAndMetadata[K, V] => R
    +  ) extends RDD[R](sc, Nil) with Logging {
    +
    +  override def getPartitions: Array[Partition] = batch.asInstanceOf[Array[Partition]]
    +
    +  override def getPreferredLocations(thePart: Partition): Seq[String] = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    // TODO is additional hostname resolution necessary here
    +    Seq(part.host)
    +  }
    +
    +  override def compute(thePart: Partition, context: TaskContext): Iterator[R] = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    if (part.fromOffset >= part.untilOffset) {
    +      log.warn("Beginning offset is same or after ending offset " +
    +        s"skipping ${part.topic} ${part.partition}")
    +      Iterator.empty
    +    } else {
    +      new NextIterator[R] {
    +        context.addTaskCompletionListener{ context => closeIfNeeded() }
    +
    +        log.info(s"Computing topic ${part.topic}, partition ${part.partition} " +
    +          s"offsets ${part.fromOffset} -> ${part.untilOffset}")
    +
    +        val kc = new KafkaCluster(kafkaParams)
    +        val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[K]]
    +        val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[V]]
    +        val consumer = connectLeader
    +        var requestOffset = part.fromOffset
    +        var iter: Iterator[MessageAndOffset] = null
    +
    +        // The idea is to use the provided preferred host, except on task retry atttempts,
    +        // to minimize number of kafka metadata requests
    +        private def connectLeader: SimpleConsumer = {
    +          if (context.attemptNumber > 0) {
    +            kc.connectLeader(part.topic, part.partition).fold(
    +              errs => throw new Exception(
    +                s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " +
    +                  errs.mkString("\n")),
    +              consumer => consumer
    +            )
    +          } else {
    +            kc.connect(part.host, part.port)
    +          }
    +        }
    +
    +        private def handleErr(resp: FetchResponse) {
    +          if (resp.hasError) {
    +            val err = resp.errorCode(part.topic, part.partition)
    +            if (err == ErrorMapping.LeaderNotAvailableCode ||
    +              err == ErrorMapping.NotLeaderForPartitionCode) {
    +              log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " +
    +                s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms")
    +              Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +            }
    +            // Let normal rdd retry sort out reconnect attempts
    +            throw ErrorMapping.exceptionFor(err)
    +          }
    +        }
    +
    +        override def close() = consumer.close()
    +
    +        override def getNext: R = {
    +          if (iter == null || !iter.hasNext) {
    +            val req = new FetchRequestBuilder().
    +              addFetch(part.topic, part.partition, requestOffset, kc.config.fetchMessageMaxBytes).
    +              build()
    +            val resp = consumer.fetch(req)
    +            handleErr(resp)
    +            // kafka may return a batch that starts before the requested offset
    +            iter = resp.messageSet(part.topic, part.partition)
    +              .iterator
    +              .dropWhile(_.offset < requestOffset)
    +          }
    +          if (!iter.hasNext) {
    +            assert(requestOffset == part.untilOffset,
    +              s"ran out of messages before reaching ending offset ${part.untilOffset} " +
    +                s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." +
    +                " This should not happen, and indicates that messages may have been lost")
    +            finished = true
    +            null.asInstanceOf[R]
    +          } else {
    +            val item = iter.next
    +            if (item.offset >= part.untilOffset) {
    +              assert(item.offset == part.untilOffset,
    +                s"got ${item.offset} > ending offset ${part.untilOffset} " +
    +                  s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." +
    +                  " This should not happen, and indicates a message may have been skipped")
    +              finished = true
    +              null.asInstanceOf[R]
    +            } else {
    +              requestOffset = item.nextOffset
    +              messageHandler(new MessageAndMetadata(
    +                part.topic, part.partition, item.message, item.offset, keyDecoder, valueDecoder))
    +            }
    --- End diff --
    
    I agree that's one of the ugliest parts of the code, but unfortunately I
    think those are the cases that need to be handled.  I'll see if there's a
    way to flatten it.
    
    On Mon, Jan 26, 2015 at 5:06 PM, Reynold Xin <no...@github.com>
    wrote:
    
    > In external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaRDD.scala
    > <https://github.com/apache/spark/pull/3798#discussion_r23573027>:
    >
    > > +            finished = true
    > > +            null.asInstanceOf[R]
    > > +          } else {
    > > +            val item = iter.next
    > > +            if (item.offset >= part.untilOffset) {
    > > +              assert(item.offset == part.untilOffset,
    > > +                s"got ${item.offset} > ending offset ${part.untilOffset} " +
    > > +                  s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." +
    > > +                  " This should not happen, and indicates a message may have been skipped")
    > > +              finished = true
    > > +              null.asInstanceOf[R]
    > > +            } else {
    > > +              requestOffset = item.nextOffset
    > > +              messageHandler(new MessageAndMetadata(
    > > +                part.topic, part.partition, item.message, item.offset, keyDecoder, valueDecoder))
    > > +            }
    >
    > this level of conditional / structure nesting seems scary. Any chance we
    > can reduce this?
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798/files#r23573027>.
    >


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71577982
  
    @koeninger
    I doubt that we want to go this route in this case, but just in case you're interested, I think a much better way to handle multiple errors gracefully is with [scalactic's `Or`](http://www.scalactic.org/user_guide/OrAndEvery).  Its much better than `Either` for this case of building up a set of errors to report back to the user.  And scalactic is a nicely designed, small library (eg. you're not pulling scalaz).  Probably not worth it for this one case, but thought you might find it interesting :)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23988994
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Leader.scala ---
    @@ -0,0 +1,46 @@
    +/*
    + * 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.streaming.kafka
    +
    +import kafka.common.TopicAndPartition
    +
    +/** Host info for the leader of a Kafka TopicAndPartition */
    +final class Leader private(
    +    /** kafka topic name */
    +    val topic: String,
    +    /** kafka partition id */
    +    val partition: Int,
    +    /** kafka hostname */
    +    val host: String,
    +    /** kafka host's port */
    +    val port: Int) extends Serializable
    +
    +object Leader {
    +  def create(topic: String, partition: Int, host: String, port: Int): Leader =
    --- End diff --
    
    Similar with offset ranges, can't we just have a single way to construct these?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71131618
  
    Makes sense. I guess this is good to go then!


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by dibbhatt <gi...@git.apache.org>.
Github user dibbhatt commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24001371
  
    --- Diff: external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala ---
    @@ -0,0 +1,99 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.util.Random
    +
    +import kafka.serializer.StringDecoder
    +import kafka.common.TopicAndPartition
    +import org.scalatest.BeforeAndAfter
    +
    +import org.apache.spark._
    +import org.apache.spark.SparkContext._
    +
    +class KafkaRDDSuite extends KafkaStreamSuiteBase with BeforeAndAfter {
    +  var sc: SparkContext = _
    +  before {
    +    setupKafka()
    +  }
    +
    +  after {
    +    if (sc != null) {
    +      sc.stop
    +      sc = null
    +    }
    +    tearDownKafka()
    +  }
    +
    +  test("Kafka RDD") {
    +    val sparkConf = new SparkConf().setMaster("local[4]").setAppName(this.getClass.getSimpleName)
    +    sc = new SparkContext(sparkConf)
    +    val topic = "topic1"
    +    val sent = Map("a" -> 5, "b" -> 3, "c" -> 10)
    +    createTopic(topic)
    +    produceAndSendMessage(topic, sent)
    +
    +    val kafkaParams = Map("metadata.broker.list" -> s"localhost:$brokerPort",
    +      "group.id" -> s"test-consumer-${Random.nextInt(10000)}")
    +
    +    val kc = new KafkaCluster(kafkaParams)
    +
    +    val rdd = getRdd(kc, Set(topic))
    +    // this is the "lots of messages" case
    +    // make sure we get all of them
    +    assert(rdd.isDefined)
    +    assert(rdd.get.count === sent.values.sum)
    +
    +    kc.setConsumerOffsets(
    --- End diff --
    
    Here you are committing the RDD offset to ZK via SimpleConsumer commitOffset call for next getRdd call . If committOffset call fails , what to do ? Isn't it breaking the logic of keeping the offset along with RDD concept mentioned in the design document ?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71548572
  
    @koeninger Can you run `sbt scalastyle` and fix the style violations? There are like 2 instances where there are lines > 100 chars.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72759859
  
    But of course there can be multiple implementations! For example, there is both KafkaReceiver and ReliableKafkaReceiver. The second was introduced so that the code path for existing uses is not disturbed when we are introducing experimental code paths that are optionally enabled with flags. We never envisioned that happening, but when it occur, we could do this because the KafkaReceiver was not exposed, only the Receiver interface was exposed. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72586127
  
      [Test build #26590 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26590/consoleFull) for   PR 3798 at commit [`825110f`](https://github.com/apache/spark/commit/825110fc22abc8ba3bb8c0e685a35ed5e939ffb9).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  class DeterministicKafkaInputDStreamCheckpointData extends DStreamCheckpointData(this) `
      * `class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable `
      * `  case class LeaderOffset(host: String, port: Int, offset: Long)`
      * `class KafkaRDDPartition(`
      * `trait HasOffsetRanges `



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71571551
  
    Great! Perhaps we can add some documentation to the Programming Guide as well, explaining this.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71552428
  
    Just a general comment for other reviewers: 
    
    This PR actually gives us a way of receiving data from Kafka without having to write data to the HDFS-based WAL. This is a really great optimization and would gives us better performance when we use it with Kafka.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by helena <gi...@git.apache.org>.
Github user helena commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22648458
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala ---
    @@ -0,0 +1,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.streaming.kafka
    +
    +import scala.annotation.tailrec
    +import scala.reflect.{classTag, ClassTag}
    +
    +import kafka.common.TopicAndPartition
    +import kafka.message.MessageAndMetadata
    +import kafka.serializer.Decoder
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.rdd.kafka.{KafkaCluster, KafkaRDD}
    +import org.apache.spark.streaming.{StreamingContext, Time}
    +import org.apache.spark.streaming.dstream._
    +
    +/** A stream of {@link org.apache.spark.rdd.kafka.KafkaRDD} where
    +  * each given Kafka topic/partition corresponds to an RDD partition.
    +  * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages
    +  * per second that each '''partition''' will accept.
    +  * Starting offsets are specified in advance,
    +  * and this DStream is not responsible for committing offsets,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the stream
    +  * @param messageHandler function for translating each message into the desired type
    +  * @param maxRetries maximum number of times in a row to retry getting leaders' offsets
    +  */
    +class DeterministicKafkaInputDStream[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    @transient ssc_ : StreamingContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R,
    +    maxRetries: Int = 1
    +) extends InputDStream[R](ssc_) with Logging {
    +
    +  private val kc = new KafkaCluster(kafkaParams)
    +
    +  private val maxMessagesPerPartition: Option[Long] = {
    +    val ratePerSec = context.sparkContext.getConf.getInt("spark.streaming.receiver.maxRate", 0)
    +    if (ratePerSec > 0) {
    +      val secsPerBatch = context.graph.batchDuration.milliseconds.toDouble / 1000
    +      Some((secsPerBatch * ratePerSec).toLong)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  private var currentOffsets = fromOffsets
    +
    +  @tailrec
    +  private def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, Long] = {
    +    val o = kc.getLatestLeaderOffsets(currentOffsets.keys.toSet)
    +    // Either.fold would confuse @tailrec, do it manually
    +    if (o.isLeft) {
    +      val err = o.left.get.toString
    +      if (retries <= 0) {
    +        throw new Exception(err)
    +      } else {
    +        log.error(err)
    +        Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +        latestLeaderOffsets(retries - 1)
    +      }
    +    } else {
    +      o.right.get
    +    }
    +  }
    +
    +  private def clamp(leaderOffsets: Map[TopicAndPartition, Long]): Map[TopicAndPartition, Long] = {
    +    maxMessagesPerPartition.map { mmp =>
    +      leaderOffsets.map { kv =>
    --- End diff --
    
    Perhaps use scala math vs Java, and case (k,v) in one line vs creating a val
    ```maxMessagesPerPartition.map { mmp =>
          leaderOffsets.map { case (k, v) => k -> math.min(currentOffsets(k) + mmp, v) }
        }.getOrElse(leaderOffsets)```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72758029
  
    I dont get it, what's the complication with abstract classes?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23971976
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala ---
    @@ -0,0 +1,150 @@
    +/*
    + * 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.streaming.kafka
    +
    +
    +import scala.annotation.tailrec
    +import scala.collection.mutable
    +import scala.reflect.{classTag, ClassTag}
    +
    +import kafka.common.TopicAndPartition
    +import kafka.message.MessageAndMetadata
    +import kafka.serializer.Decoder
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset
    +import org.apache.spark.streaming.{StreamingContext, Time}
    +import org.apache.spark.streaming.dstream._
    +
    +/** A stream of {@link org.apache.spark.rdd.kafka.KafkaRDD} where
    +  * each given Kafka topic/partition corresponds to an RDD partition.
    +  * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages
    +  * per second that each '''partition''' will accept.
    +  * Starting offsets are specified in advance,
    +  * and this DStream is not responsible for committing offsets,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the stream
    +  * @param messageHandler function for translating each message into the desired type
    +  * @param maxRetries maximum number of times in a row to retry getting leaders' offsets
    +  */
    +private[streaming]
    +class DeterministicKafkaInputDStream[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    @transient ssc_ : StreamingContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R,
    +    maxRetries: Int
    +) extends InputDStream[R](ssc_) with Logging {
    +
    +  protected[streaming] override val checkpointData =
    +    new DeterministicKafkaInputDStreamCheckpointData
    +
    +  protected val kc = new KafkaCluster(kafkaParams)
    +
    +  protected val maxMessagesPerPartition: Option[Long] = {
    +    val ratePerSec = context.sparkContext.getConf.getInt("spark.streaming.receiver.maxRate", 0)
    +    if (ratePerSec > 0) {
    +      val secsPerBatch = context.graph.batchDuration.milliseconds.toDouble / 1000
    +      Some((secsPerBatch * ratePerSec).toLong)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  protected var currentOffsets = fromOffsets
    +
    +  @tailrec
    +  protected final def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, LeaderOffset] = {
    --- End diff --
    
    This function should be part of KafkaCluster. The getLatestLeaderOffset could take an optional parameter of retries. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by shaneknapp <gi...@git.apache.org>.
Github user shaneknapp commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71975308
  
    jenkins, test this please.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22362375
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaRDD.scala ---
    @@ -0,0 +1,157 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.reflect.{classTag, ClassTag}
    +
    +import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.NextIterator
    +
    +import java.util.Properties
    +import kafka.api.FetchRequestBuilder
    +import kafka.common.{ErrorMapping, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +import kafka.message.{MessageAndMetadata, MessageAndOffset}
    +import kafka.serializer.Decoder
    +import kafka.utils.VerifiableProperties
    +
    +case class KafkaRDDPartition(
    +  override val index: Int,
    +  topic: String,
    +  partition: Int,
    +  fromOffset: Long,
    +  untilOffset: Long
    +) extends Partition
    +
    +/** A batch-oriented interface for consuming from Kafka.
    +  * Each given Kafka topic/partition corresponds to an RDD partition.
    +  * Starting and ending offsets are specified in advance,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the batch
    +  * @param untilOffsets per-topic/partition Kafka offsets defining the (exclusive)
    +  *  ending point of the batch
    +  * @param messageHandler function for translating each message into the desired type
    +  */
    +class KafkaRDD[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    sc: SparkContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    val untilOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R
    +  ) extends RDD[R](sc, Nil) with Logging {
    +
    +  assert(fromOffsets.keys == untilOffsets.keys,
    +    "Must provide both from and until offsets for each topic/partition")
    +
    +  override def getPartitions: Array[Partition] = fromOffsets.zipWithIndex.map { kvi =>
    +    val ((tp, from), index) = kvi
    +    new KafkaRDDPartition(index, tp.topic, tp.partition, from, untilOffsets(tp))
    +  }.toArray
    +
    +  override def compute(thePart: Partition, context: TaskContext) = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    if (part.fromOffset >= part.untilOffset) {
    +      log.warn("Beginning offset is same or after ending offset " +
    +        s"skipping ${part.topic} ${part.partition}")
    +      Iterator.empty
    +    } else {
    +      new NextIterator[R] {
    +        context.addTaskCompletionListener{ context => closeIfNeeded() }
    +
    +        val kc = new KafkaCluster(kafkaParams)
    +        log.info(s"Computing topic ${part.topic}, partition ${part.partition} " +
    +          s"offsets ${part.fromOffset} -> ${part.untilOffset}")
    +        val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[K]]
    +        val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[V]]
    +        val consumer: SimpleConsumer = kc.connectLeader(part.topic, part.partition).fold(
    +          errs => throw new Exception(
    +            s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " +
    +              errs.mkString("\n")),
    +          consumer => consumer
    +        )
    +        var requestOffset = part.fromOffset
    +        var iter: Iterator[MessageAndOffset] = null
    +
    +        override def close() = consumer.close()
    +
    +        override def getNext: R = {
    +          if (iter == null || !iter.hasNext) {
    +            val req = new FetchRequestBuilder().
    +              addFetch(part.topic, part.partition, requestOffset, kc.config.fetchMessageMaxBytes).
    +              build()
    +            val resp = consumer.fetch(req)
    +            if (resp.hasError) {
    +              val err = resp.errorCode(part.topic, part.partition)
    +              if (err == ErrorMapping.LeaderNotAvailableCode ||
    +                err == ErrorMapping.NotLeaderForPartitionCode) {
    +                log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " +
    +                  s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms")
    +                Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +              }
    +              // Let normal rdd retry sort out reconnect attempts
    +              throw ErrorMapping.exceptionFor(err)
    +            }
    +            iter = resp.messageSet(part.topic, part.partition)
    +              .iterator
    +              .dropWhile(_.offset < requestOffset)
    --- End diff --
    
    https://cwiki.apache.org/confluence/display/KAFKA/0.8.0+SimpleConsumer+Example
    
    "Also note that we are explicitly checking that the offset being read is
    not less than the offset that we requested. This is needed since if Kafka
    is compressing the messages, the fetch request will return an entire
    compressed block even if the requested offset isn't the beginning of the
    compressed block. Thus a message we saw previously may be returned again."
    
    On Tue, Dec 30, 2014 at 2:00 PM, Tathagata Das <no...@github.com>
    wrote:
    
    > In external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaRDD.scala
    > <https://github.com/apache/spark/pull/3798#discussion-diff-22362167>:
    >
    > > +              build()
    > > +            val resp = consumer.fetch(req)
    > > +            if (resp.hasError) {
    > > +              val err = resp.errorCode(part.topic, part.partition)
    > > +              if (err == ErrorMapping.LeaderNotAvailableCode ||
    > > +                err == ErrorMapping.NotLeaderForPartitionCode) {
    > > +                log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " +
    > > +                  s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms")
    > > +                Thread.sleep(kc.config.refreshLeaderBackoffMs)
    > > +              }
    > > +              // Let normal rdd retry sort out reconnect attempts
    > > +              throw ErrorMapping.exceptionFor(err)
    > > +            }
    > > +            iter = resp.messageSet(part.topic, part.partition)
    > > +              .iterator
    > > +              .dropWhile(_.offset < requestOffset)
    >
    > Why is there a drop here? Doesnt the response return messages for the
    > requested offset?
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798/files#r22362167>.
    >


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-69695353
  
    Can you confirm the following.
    1. In your SBT/maven app used for testing, you are using your development
    Spark version to compile? That is, the dev version is locally publish and
    you are compiling your app against spark version 1.3.0-SNAPSHOT?
    2. Do you have spark-streaming dependency as "provided" scope or the
    default "compile" scope? And then are you creating uber jar of the app?
    3. Are you submitting the app through spark-submit to the same development
    Spark version to compile?
    
    On Mon, Jan 12, 2015 at 2:13 PM, Cody Koeninger <no...@github.com>
    wrote:
    
    > Yeah, this is on a local development version, after assembly / publish
    > local.
    >
    > Here's a gist of the exception and the diff that causes it (using
    > KafkaRDDPartition instead of a tuple)
    >
    > https://gist.github.com/koeninger/561a61482cd1b5b3600c
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798#issuecomment-69656800>.
    >


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72775631
  
    Hey man, I'd rather talk about the code anyway.  I think there's just something I'm missing as far as your underlying assumptions about interfaces go :)  Thanks for your help on this.
    
    Just made the createRDD change.  Not clear on what createNewStream change you mean. Rename it to createStream, or something else?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72756440
  
    I spent some time talking to Patrick offline about this. If we expose the KafkaRDD as is (while keeping its constructor private), then the simplified createRDD would be 
    ```
    KafkaUtils.createRDD[K, V, KD, VD](....): KafkaRDD[K, V, KD, VD, (K, V)]
    ```
    Imagine how one would use it in Java. 
    ```
    KafkaRDD<String, String, StringDecoder, StringDecode, Product2<String, String>> rdd = KafkaUtils.createRDD(....)
    ```
    That's not very Java friendly if you ask a Java developer. And we a huge fraction of the community as Java developers. Furthemore, we want to add Python API as well, and that also requires the interfaces to be Java-friendly. Here is the alternative (I think) with what I proposed.
    ```
    KafkaRDD<String, String> rdd = KafkaUtils.createRDD(....)
    ```
    Much simpler. 
    
    
    Regarding casting, there are two cases, 
    1. casting RDD generated from createRDD - If we take off `HasOffsetRanges` (`KafkaUtils.createRDD` returns only RDD), then user have to cast. But if we return abstract class KafkaRDD, then no casting necessary.
    2. casting RDD in `DStream.foreachRDD` - Casting is necessary either ways. But isnt it more intuitive to write `rdd.asInstanceOf[KafkaRDD]` than `rdd.asInstanceOf[HasOffsetRanges]`?
    
    
    
    



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23973357
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,338 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.util.Random
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    + * Convenience methods for interacting with a Kafka cluster.
    + * @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
    + */
    +private[spark]
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    +  import KafkaCluster.{Err, LeaderOffset}
    +
    +  val seedBrokers: Array[(String, Int)] =
    +    kafkaParams.get("metadata.broker.list")
    +      .orElse(kafkaParams.get("bootstrap.servers"))
    +      .getOrElse(throw new Exception("Must specify metadata.broker.list or bootstrap.servers"))
    +      .split(",").map { hp =>
    +        val hpa = hp.split(":")
    +        (hpa(0), hpa(1).toInt)
    +      }
    +
    +  // ConsumerConfig isn't serializable
    +  @transient private var _config: ConsumerConfig = null
    +
    +  def config: ConsumerConfig = this.synchronized {
    +    if (_config == null) {
    +      _config = KafkaCluster.consumerConfig(kafkaParams)
    +    }
    +    _config
    +  }
    +
    +  def connect(host: String, port: Int): SimpleConsumer =
    +    new SimpleConsumer(host, port, config.socketTimeoutMs,
    +      config.socketReceiveBufferBytes, config.clientId)
    +
    +  def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] =
    +    findLeader(topic, partition).right.map(hp => connect(hp._1, hp._2))
    +
    +  // Metadata api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataAPI
    +  // scalastyle:on
    +
    +  def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, Seq(topic))
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      resp.topicsMetadata.find(_.topic == topic).flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.find(_.partitionId == partition)
    +      }.foreach { pm: PartitionMetadata =>
    +        pm.leader.foreach { leader =>
    +          return Right((leader.host, leader.port))
    +        }
    +      }
    +    }
    +    Left(errs)
    +  }
    +
    +  def findLeaders(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, (String, Int)]] = {
    +    val topics = topicAndPartitions.map(_.topic)
    +    getPartitionMetadata(topics).right.flatMap { tms: Set[TopicMetadata] =>
    +      val result = tms.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.flatMap { pm: PartitionMetadata =>
    +          val tp = TopicAndPartition(tm.topic, pm.partitionId)
    +          if (topicAndPartitions(tp)) {
    +            pm.leader.map { l =>
    +              tp -> (l.host -> l.port)
    +            }
    +          } else {
    +            None
    +          }
    +        }
    +      }.toMap
    +      if (result.keys.size == topicAndPartitions.size) {
    +        Right(result)
    +      } else {
    +        val missing = topicAndPartitions.diff(result.keySet)
    +        val err = new Err
    +        err.append(new Exception(s"Couldn't find leaders for ${missing}"))
    +        Left(err)
    +      }
    +    }
    +  }
    +
    +  def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] =
    +    getPartitionMetadata(topics).right.map { r =>
    +      r.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.map { pm: PartitionMetadata =>
    +          TopicAndPartition(tm.topic, pm.partitionId)
    +        }    
    +      }
    +    }
    +
    +  def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = {
    +    val req = TopicMetadataRequest(
    +      TopicMetadataRequest.CurrentVersion, 0, config.clientId, topics.toSeq)
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      // error codes here indicate missing / just created topic,
    +      // repeating on a different broker wont be useful
    +      return Right(resp.topicsMetadata.toSet)
    +    }
    +    Left(errs)
    +  }
    +
    +  // Leader offset api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI
    +  // scalastyle:on
    +
    +  def getLatestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime)
    +
    +  def getEarliestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime)
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long
    +  ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    +    getLeaderOffsets(topicAndPartitions, before, 1).right.map { r =>
    +      r.map { kv =>
    +        // mapValues isnt serializable, see SI-7005
    +        kv._1 -> kv._2.head
    +      }
    +    }
    +
    +  private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] =
    +    m.groupBy(_._2).map { kv =>
    +      kv._1 -> kv._2.keys.toSeq
    +    }
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long,
    +    maxNumOffsets: Int
    +  ): Either[Err, Map[TopicAndPartition, Seq[LeaderOffset]]] = {
    +    findLeaders(topicAndPartitions).right.flatMap { tpToLeader =>
    +      val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] = flip(tpToLeader)
    +      val leaders = leaderToTp.keys
    +      var result = Map[TopicAndPartition, Seq[LeaderOffset]]()
    +      val errs = new Err
    +      withBrokers(leaders, errs) { consumer =>
    +        val needed: Seq[TopicAndPartition] = leaderToTp((consumer.host, consumer.port))
    +        val req = OffsetRequest(
    +          needed.map { tp: TopicAndPartition =>
    +            tp -> PartitionOffsetRequestInfo(before, maxNumOffsets)
    +          }.toMap
    +        )
    +        val resp = consumer.getOffsetsBefore(req)
    +        val respMap = resp.partitionErrorAndOffsets
    +        needed.foreach { tp: TopicAndPartition =>
    +          respMap.get(tp).foreach { por: PartitionOffsetsResponse =>
    +            if (por.error == ErrorMapping.NoError) {
    +              if (por.offsets.nonEmpty) {
    +                result += tp -> por.offsets.map { off =>
    +                  LeaderOffset(consumer.host, consumer.port, off)
    +                }
    +              } else {
    +                errs.append(new Exception(
    +                  s"Empty offsets for ${tp}, is ${before} before log beginning?"))
    +              }
    +            } else {
    +              errs.append(ErrorMapping.exceptionFor(por.error))
    +            }
    +          }
    +        }
    +        if (result.keys.size == topicAndPartitions.size) {
    +          return Right(result)
    +        }
    +      }
    +      val missing = topicAndPartitions.diff(result.keySet)
    +      errs.append(new Exception(s"Couldn't find leader offsets for ${missing}"))
    +      Left(errs)
    +    }
    +  }
    +
    +  // Consumer offset api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI
    +  // scalastyle:on
    +
    +  def getConsumerOffsets(
    +      groupId: String,
    +      topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] = {
    --- End diff --
    
    Indentation issue.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22364358
  
    --- Diff: core/src/main/scala/org/apache/spark/rdd/RDD.scala ---
    @@ -788,6 +788,20 @@ abstract class RDD[T: ClassTag](
       }
     
       /**
    +   * Applies a function to each partition of this RDD, while tracking the index
    +   * of the original partition.
    +   */
    +  def foreachPartitionWithIndex(
    --- End diff --
    
    The code for actually using the rdd and committing offsets transactionally is quite awkward without that method, see
    
    https://github.com/koeninger/kafka-exactly-once/commit/cb812c918f724b3ae5e57c66618276b6947ed30d
    



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24030535
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala ---
    @@ -0,0 +1,69 @@
    +/*
    + * 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.streaming.kafka
    +
    +import kafka.common.TopicAndPartition
    +
    +/** Something that has a collection of OffsetRanges */
    +trait HasOffsetRanges {
    +  def offsetRanges: Array[OffsetRange]
    +}
    +
    +/** Represents a range of offsets from a single Kafka TopicAndPartition */
    +final class OffsetRange private(
    +    /** kafka topic name */
    +    val topic: String,
    +    /** kafka partition id */
    +    val partition: Int,
    +    /** inclusive starting offset */
    +    val fromOffset: Long,
    +    /** exclusive ending offset */
    +    val untilOffset: Long) extends Serializable {
    +  import OffsetRange.OffsetRangeTuple
    +
    +  /** this is to avoid ClassNotFoundException during checkpoint restore */
    --- End diff --
    
    Created this so we don't lose track: https://issues.apache.org/jira/browse/SPARK-5569


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72228197
  
      [Test build #26404 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26404/consoleFull) for   PR 3798 at commit [`9adaa0a`](https://github.com/apache/spark/commit/9adaa0a0e13a1c7eef80569b90ff7eaa2b6db190).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  class DeterministicKafkaInputDStreamCheckpointData extends DStreamCheckpointData(this) `
      * `class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable `
      * `  case class LeaderOffset(host: String, port: Int, offset: Long)`
      * `class KafkaRDDPartition(`



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23890594
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala ---
    @@ -0,0 +1,220 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.reflect.{classTag, ClassTag}
    +
    +import org.apache.spark.{Logging, Partition, SparkContext, SparkException, TaskContext}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.NextIterator
    +
    +import java.util.Properties
    +import kafka.api.{FetchRequestBuilder, FetchResponse}
    +import kafka.common.{ErrorMapping, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +import kafka.message.{MessageAndMetadata, MessageAndOffset}
    +import kafka.serializer.Decoder
    +import kafka.utils.VerifiableProperties
    +
    +/**
    + * 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 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 KafkaRDDPartition 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
    + */
    +private[spark]
    +class KafkaRDD[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag] private[spark] (
    +    sc: SparkContext,
    +    kafkaParams: Map[String, String],
    +    private[spark] val batch: Array[KafkaRDDPartition],
    +    messageHandler: MessageAndMetadata[K, V] => R
    +  ) extends RDD[R](sc, Nil) with Logging with HasOffsetRanges {
    +
    +  def offsetRanges: Array[OffsetRange] = batch.asInstanceOf[Array[OffsetRange]]
    +
    +  override def getPartitions: Array[Partition] = batch.asInstanceOf[Array[Partition]]
    +
    +  override def getPreferredLocations(thePart: Partition): Seq[String] = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    // TODO is additional hostname resolution necessary here
    +    Seq(part.host)
    +  }
    +
    +  private def errBeginAfterEnd(part: KafkaRDDPartition): String =
    +    s"Beginning offset ${part.fromOffset} is after the ending offset ${part.untilOffset} " +
    +      s"for topic ${part.topic} partition ${part.partition}. " +
    +      "You either provided an invalid fromOffset, or the Kafka topic has been damaged"
    +
    +  private def errRanOutBeforeEnd(part: KafkaRDDPartition): String =
    +    s"Ran out of messages before reaching ending offset ${part.untilOffset} " +
    +    s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." +
    +    " This should not happen, and indicates that messages may have been lost"
    +
    +  private def errOvershotEnd(itemOffset: Long, part: KafkaRDDPartition): String =
    +    s"Got ${itemOffset} > ending offset ${part.untilOffset} " +
    +    s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." +
    +    " This should not happen, and indicates a message may have been skipped"
    +
    +  override def compute(thePart: Partition, context: TaskContext): Iterator[R] = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    assert(part.fromOffset <= part.untilOffset, errBeginAfterEnd(part))
    +    if (part.fromOffset == part.untilOffset) {
    +      log.warn("Beginning offset ${part.fromOffset} is the same as ending offset " +
    +        s"skipping ${part.topic} ${part.partition}")
    +      Iterator.empty
    +    } else {
    +      new KafkaRDDIterator(part, context)
    +    }
    +  }
    +
    +  private class KafkaRDDIterator(
    +      part: KafkaRDDPartition,
    +      context: TaskContext) extends NextIterator[R] {
    +
    +    context.addTaskCompletionListener{ context => closeIfNeeded() }
    +
    +    log.info(s"Computing topic ${part.topic}, partition ${part.partition} " +
    +      s"offsets ${part.fromOffset} -> ${part.untilOffset}")
    +
    +    val kc = new KafkaCluster(kafkaParams)
    +    val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +      .newInstance(kc.config.props)
    +      .asInstanceOf[Decoder[K]]
    +    val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +      .newInstance(kc.config.props)
    +      .asInstanceOf[Decoder[V]]
    +    val consumer = connectLeader
    +    var requestOffset = part.fromOffset
    +    var iter: Iterator[MessageAndOffset] = null
    +
    +    // The idea is to use the provided preferred host, except on task retry atttempts,
    +    // to minimize number of kafka metadata requests
    +    private def connectLeader: SimpleConsumer = {
    +      if (context.attemptNumber > 0) {
    +        kc.connectLeader(part.topic, part.partition).fold(
    +          errs => throw new SparkException(
    +            s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " +
    +              errs.mkString("\n")),
    +          consumer => consumer
    +        )
    +      } else {
    +        kc.connect(part.host, part.port)
    +      }
    +    }
    +
    +    private def handleFetchErr(resp: FetchResponse) {
    +      if (resp.hasError) {
    +        val err = resp.errorCode(part.topic, part.partition)
    +        if (err == ErrorMapping.LeaderNotAvailableCode ||
    +          err == ErrorMapping.NotLeaderForPartitionCode) {
    +          log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " +
    +            s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms")
    +          Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +        }
    +        // Let normal rdd retry sort out reconnect attempts
    +        throw ErrorMapping.exceptionFor(err)
    +      }
    +    }
    +
    --- End diff --
    
    My documentation for the stream already says you need to have sufficient
    Kafka log retention.  The Kafka documentation for OffsetOutOfRangeException
    already says it Indicates the client has requested a range no longer
    available on the server.
    
    The stream stopping is exactly what should happen in that case.  Then its
    the clients choice as to how to handle the situation.  If someone really
    wants to handle the fact that they underprovisioned Kafka by just
    restarting the stream at the head of the log and losing data, they can do
    so by starting up the stream again with auto.offset.reset set to largest
    (which is also already documented). But they should have to do that
    themselves, we shouldn't be building a library that silently throws away
    data in failure situations.
    
    I think that's fairly clear, but if you want to write up something for
    including in the streaming programming guide I'll take a look.
    On Jan 31, 2015 11:27 AM, "Dibyendu Bhattacharya" <no...@github.com>
    wrote:
    
    > In
    > external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala
    > <https://github.com/apache/spark/pull/3798#discussion_r23890423>:
    >
    > > +    }
    > > +
    > > +    private def handleFetchErr(resp: FetchResponse) {
    > > +      if (resp.hasError) {
    > > +        val err = resp.errorCode(part.topic, part.partition)
    > > +        if (err == ErrorMapping.LeaderNotAvailableCode ||
    > > +          err == ErrorMapping.NotLeaderForPartitionCode) {
    > > +          log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " +
    > > +            s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms")
    > > +          Thread.sleep(kc.config.refreshLeaderBackoffMs)
    > > +        }
    > > +        // Let normal rdd retry sort out reconnect attempts
    > > +        throw ErrorMapping.exceptionFor(err)
    > > +      }
    > > +    }
    > > +
    >
    > Hi Cody, I believe in this implementation the range of offset of given
    > only at the start, for rest of the flow the offset range is calculated
    > automatically. What I say, if Offset_Out_Of_Range comes in those RDDs where
    > you calculate the offset, this implementation can not recover from this
    > failure. I did not say that we always get this error all time , but this
    > error is not very rare cases also. and if the Receiver stops because of
    > this, that's an issue.
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798/files#r23890423>.
    >


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23418917
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala ---
    @@ -0,0 +1,123 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.annotation.tailrec
    +import scala.reflect.{classTag, ClassTag}
    +
    +import kafka.common.TopicAndPartition
    +import kafka.message.MessageAndMetadata
    +import kafka.serializer.Decoder
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.rdd.kafka.{KafkaCluster, KafkaRDD}
    +import org.apache.spark.streaming.{StreamingContext, Time}
    +import org.apache.spark.streaming.dstream._
    +
    +/** A stream of {@link org.apache.spark.rdd.kafka.KafkaRDD} where
    +  * each given Kafka topic/partition corresponds to an RDD partition.
    +  * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages
    +  * per second that each '''partition''' will accept.
    +  * Starting offsets are specified in advance,
    +  * and this DStream is not responsible for committing offsets,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the stream
    +  * @param messageHandler function for translating each message into the desired type
    +  * @param maxRetries maximum number of times in a row to retry getting leaders' offsets
    +  */
    +class DeterministicKafkaInputDStream[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    @transient ssc_ : StreamingContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R,
    +    maxRetries: Int = 1
    +) extends InputDStream[R](ssc_) with Logging {
    +
    +  private val kc = new KafkaCluster(kafkaParams)
    +
    +  private val maxMessagesPerPartition: Option[Long] = {
    +    val ratePerSec = ssc.sparkContext.getConf.getInt("spark.streaming.receiver.maxRate", 0)
    +    if (ratePerSec > 0) {
    +      val secsPerBatch = ssc.graph.batchDuration.milliseconds.toDouble / 1000
    +      Some((secsPerBatch * ratePerSec).toLong)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  // TODO based on the design of InputDStream's lastValidTime, it appears there isn't a
    +  // thread safety concern with private mutable state, but is this certain?
    +  private var currentOffsets = fromOffsets
    +
    +  @tailrec
    +  private def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, Long] = {
    +    val o = kc.getLatestLeaderOffsets(currentOffsets.keys.toSet)
    +    // Either.fold would confuse @tailrec, do it manually
    +    if (o.isLeft) {
    +      val err = o.left.get.toString
    +      if (retries <= 0) {
    +        throw new Exception(err)
    +      } else {
    +        log.error(err)
    +        Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +        latestLeaderOffsets(retries - 1)
    +      }
    +    } else {
    +      o.right.get
    +    }
    +  }
    +
    +  private def clamp(leaderOffsets: Map[TopicAndPartition, Long]): Map[TopicAndPartition, Long] = {
    +    maxMessagesPerPartition.map { mmp =>
    +      leaderOffsets.map { kv =>
    +        val (k, v) = kv
    +        val curr = currentOffsets(k)
    +        val diff = v - curr
    +        if (diff > mmp) (k, curr + mmp) else (k, v)
    +      }
    +    }.getOrElse(leaderOffsets)
    +  }
    +
    +  override def compute(validTime: Time): Option[KafkaRDD[K, V, U, T, R]] = {
    --- End diff --
    
    Sorry, posted the comment before I reached that part of the code.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72709829
  
      [Test build #26658 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26658/consoleFull) for   PR 3798 at commit [`0df3ebe`](https://github.com/apache/spark/commit/0df3ebe1eed5b149c03a828db621fbc60e5555df).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  class DeterministicKafkaInputDStreamCheckpointData extends DStreamCheckpointData(this) `
      * `class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable `
      * `  case class LeaderOffset(host: String, port: Int, offset: Long)`
      * `class KafkaRDDPartition(`
      * `trait HasOffsetRanges `



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23569836
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala ---
    @@ -0,0 +1,149 @@
    +/*
    + * 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.streaming.kafka
    +
    +
    +import scala.annotation.tailrec
    +import scala.collection.mutable
    +import scala.reflect.{classTag, ClassTag}
    +
    +import kafka.common.TopicAndPartition
    +import kafka.message.MessageAndMetadata
    +import kafka.serializer.Decoder
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.rdd.kafka.{KafkaCluster, KafkaRDD, KafkaRDDPartition}
    +import org.apache.spark.rdd.kafka.KafkaCluster.LeaderOffset
    +import org.apache.spark.streaming.{StreamingContext, Time}
    +import org.apache.spark.streaming.dstream._
    +
    +/** A stream of {@link org.apache.spark.rdd.kafka.KafkaRDD} where
    +  * each given Kafka topic/partition corresponds to an RDD partition.
    +  * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages
    +  * per second that each '''partition''' will accept.
    +  * Starting offsets are specified in advance,
    +  * and this DStream is not responsible for committing offsets,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the stream
    +  * @param messageHandler function for translating each message into the desired type
    +  * @param maxRetries maximum number of times in a row to retry getting leaders' offsets
    +  */
    +class DeterministicKafkaInputDStream[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    @transient ssc_ : StreamingContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R,
    +    maxRetries: Int = 1
    +) extends InputDStream[R](ssc_) with Logging {
    +
    +  protected[streaming] override val checkpointData = new DeterministicKafkaInputDStreamCheckpointData
    --- End diff --
    
    looks like this line is too long ...


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23971703
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala ---
    @@ -0,0 +1,150 @@
    +/*
    + * 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.streaming.kafka
    +
    +
    +import scala.annotation.tailrec
    +import scala.collection.mutable
    +import scala.reflect.{classTag, ClassTag}
    +
    +import kafka.common.TopicAndPartition
    +import kafka.message.MessageAndMetadata
    +import kafka.serializer.Decoder
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset
    +import org.apache.spark.streaming.{StreamingContext, Time}
    +import org.apache.spark.streaming.dstream._
    +
    +/** A stream of {@link org.apache.spark.rdd.kafka.KafkaRDD} where
    +  * each given Kafka topic/partition corresponds to an RDD partition.
    +  * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages
    +  * per second that each '''partition''' will accept.
    +  * Starting offsets are specified in advance,
    +  * and this DStream is not responsible for committing offsets,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the stream
    +  * @param messageHandler function for translating each message into the desired type
    +  * @param maxRetries maximum number of times in a row to retry getting leaders' offsets
    +  */
    +private[streaming]
    +class DeterministicKafkaInputDStream[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    @transient ssc_ : StreamingContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R,
    +    maxRetries: Int
    +) extends InputDStream[R](ssc_) with Logging {
    +
    +  protected[streaming] override val checkpointData =
    +    new DeterministicKafkaInputDStreamCheckpointData
    +
    +  protected val kc = new KafkaCluster(kafkaParams)
    +
    +  protected val maxMessagesPerPartition: Option[Long] = {
    +    val ratePerSec = context.sparkContext.getConf.getInt("spark.streaming.receiver.maxRate", 0)
    --- End diff --
    
    Lets rename this configuration. Its very confusing to overload this configuration because the system does not behave in the same way. `receiver.maxRate` applies to receivers which is not used by this stream. In fact the mechanism used here is very specific to this input stream and applies to not other input stream. So lets rename it to something like 
    `spark.streaming.kafka.maxRatePerPartition`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24017652
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,174 @@ 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    +    val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message)
    +    val kc = new KafkaCluster(kafkaParams)
    +    val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet
    +    val leaders = kc.findLeaders(topics).fold(
    +      errs => throw new SparkException(errs.mkString("\n")),
    +      ok => ok
    +    )
    +    new KafkaRDD[K, V, U, T, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler)
    +  }
    +
    +  /** 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   * @param leaders Kafka leaders for each offset range in batch
    +   * @param messageHandler function for translating each message into the desired type
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange],
    +      leaders: Array[Leader],
    +      messageHandler: MessageAndMetadata[K, V] => R
    +  ): RDD[R] with HasOffsetRanges = {
    +
    +    val leaderMap = leaders
    +      .map(l => TopicAndPartition(l.topic, l.partition) -> (l.host, l.port))
    +      .toMap
    +    new KafkaRDD[K, V, U, T, R](sc, kafkaParams, offsetRanges, leaderMap, messageHandler)
    +  }
    +
    +  /**
    +   * This stream can guarantee that each message from Kafka is included in transformations
    +   * (as opposed to output actions) exactly once, even in most failure situations.
    +   *
    +   * Points to note:
    +   *
    +   * Failure Recovery - You must checkpoint this stream, or save offsets yourself and provide them
    +   * as the fromOffsets parameter on restart.
    +   * Kafka must have sufficient log retention to obtain messages after failure.
    +   *
    +   * Getting offsets from the stream - see programming guide
    +   *
    +.  * Zookeeper - This does not use Zookeeper to store offsets.  For interop with Kafka monitors
    +   * that depend on Zookeeper, you must store offsets in ZK yourself.
    +   *
    +   * End-to-end semantics - This does not guarantee that any output operation will push each record
    +   * exactly once. To ensure end-to-end exactly-once semantics (that is, receiving exactly once and
    +   * outputting exactly once), you have to either ensure that the output operation is
    +   * idempotent, or transactionally store offsets with the output. See the programming guide for
    +   * more details.
    +   *
    +   * @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
    +   */
    +  @Experimental
    +  def createNewStream[
    +    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
    +  ): InputDStream[R] = {
    +    new DeterministicKafkaInputDStream[K, V, U, T, R](
    +      ssc, kafkaParams, fromOffsets, messageHandler)
    +  }
    +
    +  /**
    +   * This stream can guarantee that each message from Kafka is included in transformations
    +   * (as opposed to output actions) exactly once, even in most failure situations.
    --- End diff --
    
    I'd rather make a weak true statement than a strong false statement.
    
    The most obvious known failure case here is if someone's relying on checkpointing, and the checkpoint can't be recovered (because code needs to change, etc).  TD says he's working on this for 1.4, but as of now it's still an issue.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71120030
  
    Yeah, it's pulled down every batch interval.  That way you know exactly
    what the upper and lower bounds of the offsets are.
    
    On Thu, Jan 22, 2015 at 5:15 PM, Hari Shreedharan <no...@github.com>
    wrote:
    
    > I like this! I didn't try building it, but the logic is great!
    >
    > So, to sum up the idea - the key detail here is that the checkpoint
    > contains the metadata to regenerate the RDDs, thus original order and
    > batches are recovered. That looks good - it was the same thing I was trying
    > to see if we could do in the Kafka receiver, but it would be difficult
    > without some API changes.
    >
    > That brings me to a question - so in this PR, is the data pulled down from
    > Kafka only once every batch interval - say every 2 seconds, or is there a
    > way to generate it continuously rather than have spikes?
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798#issuecomment-71119114>.
    >


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23746302
  
    --- Diff: external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala ---
    @@ -130,7 +130,7 @@ abstract class KafkaStreamSuiteBase extends FunSuite with Eventually with Loggin
       }
     
       def createTopic(topic: String) {
    -    CreateTopicCommand.createTopic(zkClient, topic, 1, 1, "0")
    --- End diff --
    
    Yeah, they just shuffled some of their classes around.  You'll notice the PR floating around for writing to kafka does the same thing, because it also upgrades kafka


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23726705
  
    --- 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.
    --- End diff --
    
    Same as above, please update the docs.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71560068
  
      [Test build #26118 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26118/consoleFull) for   PR 3798 at commit [`bb80bbe`](https://github.com/apache/spark/commit/bb80bbea59d11af73430961c925c80dbb942d056).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable `
      * `  case class LeaderOffset(host: String, port: Int, offset: Long)`
      * `class KafkaRDDPartition(`
      * `  class DeterministicKafkaInputDStreamCheckpointData extends DStreamCheckpointData(this) `



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23570025
  
    --- Diff: core/src/main/scala/org/apache/spark/rdd/RDD.scala ---
    @@ -788,6 +788,20 @@ abstract class RDD[T: ClassTag](
       }
     
       /**
    +   * Applies a function to each partition of this RDD, while tracking the index
    +   * of the original partition.
    +   */
    +  def foreachPartitionWithIndex(
    --- End diff --
    
    I disagree -- I think it should stay here.  It seems like a pretty obvious omission from the RDD api.  Probably its never been added b/c there wasn't a good use case.  Well, now we've got a good use case.  IMO no sense in creating another PR that needs review just for that.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72783141
  
    Yeah, more importantly it's so defaults for things like connection timeouts
    match what kafka provides.
    
    It's possible to assign fake zookeeper.connect and have it pass
    verification, that's what existing code does.
    
    Unfortunately ConsumerConfig has a private constructor so subclassing it in
    order for the broker list to pass verification without that warning may
    prove to be tricky.  Worst case scenario I'll re-implement a config that
    uses the kafka defaults.
    
    On Tue, Feb 3, 2015 at 9:05 PM, Tathagata Das <no...@github.com>
    wrote:
    
    > I see. ConsumerConfig is really necessary only for high-level consumer,
    > but you are using it configure stuff in the low level consumer as well.
    > That is so that you dont have to introduce parameter strings to configure
    > them yourselves.
    >
    > Is it possible to assign fake but verified zookeeper.connect ?
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798#issuecomment-72782434>.
    >



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71122072
  
    I need to know, perhaps even at the driver, what the ending offset is in
    order to be able to commit it.
    
    I also have several use cases where I want to end a batch at a specific
    point which may or may not be "now".
    On Jan 22, 2015 5:33 PM, "Hari Shreedharan" <no...@github.com>
    wrote:
    
    > OK.
    >
    > Just a thought: Do you think there might be a way to avoid the spikes?
    > Once the current RDD is checkpointed, create a "new" pending RDD, which
    > continuously receives data, until the compute method is called. When
    > compute gets called, the last offset we received can be considered to be
    > the upper bound, and the data is now available for transformations. That
    > way, we could spread out network transfers from Kafka over a larger period.
    >
    > Not sure if there are holes in that algorithm, but it looks almost
    > equivalent to the current model, no?
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798#issuecomment-71121466>.
    >


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72893647
  
    Here's a solution for subclassing ConsumerConfig while still silencing the warning.
    My son is doing ok(ish) now, thanks for the concern.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23990592
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,174 @@ 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    +    val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message)
    +    val kc = new KafkaCluster(kafkaParams)
    +    val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet
    +    val leaders = kc.findLeaders(topics).fold(
    +      errs => throw new SparkException(errs.mkString("\n")),
    +      ok => ok
    +    )
    +    new KafkaRDD[K, V, U, T, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler)
    +  }
    +
    +  /** 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   * @param leaders Kafka leaders for each offset range in batch
    +   * @param messageHandler function for translating each message into the desired type
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange],
    +      leaders: Array[Leader],
    +      messageHandler: MessageAndMetadata[K, V] => R
    +  ): RDD[R] with HasOffsetRanges = {
    +
    +    val leaderMap = leaders
    +      .map(l => TopicAndPartition(l.topic, l.partition) -> (l.host, l.port))
    +      .toMap
    +    new KafkaRDD[K, V, U, T, R](sc, kafkaParams, offsetRanges, leaderMap, messageHandler)
    +  }
    +
    +  /**
    +   * This stream can guarantee that each message from Kafka is included in transformations
    --- End diff --
    
    For the top level doc here isn't it something like:
    
    ```
    Returns a Kafka stream that computes a specific offset range for each
    partition, then reads those offsets directly from Kafka without the use of receivers.
    
    Because this stream deterministically assigns offset ranges to specific batches, it can
    support "exactly once" semantics (as defined in the programming guide). Specifically,
    a streaming program experiences task failures or restarts from a job failure, output
    batches appear as if each record was ingested and processed exactly once.
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22329463
  
    --- Diff: external/kafka/pom.xml ---
    @@ -44,7 +44,7 @@
         <dependency>
           <groupId>org.apache.kafka</groupId>
           <artifactId>kafka_${scala.binary.version}</artifactId>
    -      <version>0.8.0</version>
    --- End diff --
    
    Why was this necessary? What aspect of this PR depends on this updated version?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23586712
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaRDD.scala ---
    @@ -0,0 +1,199 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.reflect.{classTag, ClassTag}
    +
    +import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.NextIterator
    +
    +import java.util.Properties
    +import kafka.api.{FetchRequestBuilder, FetchResponse}
    +import kafka.common.{ErrorMapping, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +import kafka.message.{MessageAndMetadata, MessageAndOffset}
    +import kafka.serializer.Decoder
    +import kafka.utils.VerifiableProperties
    +
    +/** A batch-oriented interface for consuming from Kafka.
    +  * Starting and ending offsets are specified in advance,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 KafkaRDDPartition 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
    +  */
    +class KafkaRDD[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    sc: SparkContext,
    +    val kafkaParams: Map[String, String],
    +    val batch: Array[KafkaRDDPartition],
    +    messageHandler: MessageAndMetadata[K, V] => R
    +  ) extends RDD[R](sc, Nil) with Logging {
    +
    +  override def getPartitions: Array[Partition] = batch.asInstanceOf[Array[Partition]]
    +
    +  override def getPreferredLocations(thePart: Partition): Seq[String] = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    // TODO is additional hostname resolution necessary here
    +    Seq(part.host)
    +  }
    +
    +  override def compute(thePart: Partition, context: TaskContext): Iterator[R] = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    if (part.fromOffset >= part.untilOffset) {
    +      log.warn("Beginning offset is same or after ending offset " +
    +        s"skipping ${part.topic} ${part.partition}")
    +      Iterator.empty
    +    } else {
    +      new NextIterator[R] {
    +        context.addTaskCompletionListener{ context => closeIfNeeded() }
    +
    +        log.info(s"Computing topic ${part.topic}, partition ${part.partition} " +
    +          s"offsets ${part.fromOffset} -> ${part.untilOffset}")
    +
    +        val kc = new KafkaCluster(kafkaParams)
    +        val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[K]]
    +        val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[V]]
    +        val consumer = connectLeader
    +        var requestOffset = part.fromOffset
    +        var iter: Iterator[MessageAndOffset] = null
    +
    +        // The idea is to use the provided preferred host, except on task retry atttempts,
    +        // to minimize number of kafka metadata requests
    +        private def connectLeader: SimpleConsumer = {
    +          if (context.attemptNumber > 0) {
    +            kc.connectLeader(part.topic, part.partition).fold(
    +              errs => throw new Exception(
    +                s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " +
    +                  errs.mkString("\n")),
    +              consumer => consumer
    +            )
    +          } else {
    +            kc.connect(part.host, part.port)
    +          }
    +        }
    +
    +        private def handleErr(resp: FetchResponse) {
    +          if (resp.hasError) {
    +            val err = resp.errorCode(part.topic, part.partition)
    +            if (err == ErrorMapping.LeaderNotAvailableCode ||
    +              err == ErrorMapping.NotLeaderForPartitionCode) {
    +              log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " +
    +                s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms")
    +              Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +            }
    +            // Let normal rdd retry sort out reconnect attempts
    +            throw ErrorMapping.exceptionFor(err)
    +          }
    +        }
    +
    +        override def close() = consumer.close()
    +
    +        override def getNext: R = {
    --- End diff --
    
    getNext should have parentheses (since it has side effect and it was defined with parentheses in NextIterator)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23983277
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,174 @@ 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    +    val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message)
    +    val kc = new KafkaCluster(kafkaParams)
    +    val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet
    +    val leaders = kc.findLeaders(topics).fold(
    +      errs => throw new SparkException(errs.mkString("\n")),
    +      ok => ok
    +    )
    +    new KafkaRDD[K, V, U, T, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler)
    +  }
    +
    +  /** 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   * @param leaders Kafka leaders for each offset range in batch
    +   * @param messageHandler function for translating each message into the desired type
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange],
    +      leaders: Array[Leader],
    +      messageHandler: MessageAndMetadata[K, V] => R
    +  ): RDD[R] with HasOffsetRanges = {
    +
    +    val leaderMap = leaders
    +      .map(l => TopicAndPartition(l.topic, l.partition) -> (l.host, l.port))
    +      .toMap
    +    new KafkaRDD[K, V, U, T, R](sc, kafkaParams, offsetRanges, leaderMap, messageHandler)
    +  }
    +
    +  /**
    +   * This stream can guarantee that each message from Kafka is included in transformations
    +   * (as opposed to output actions) exactly once, even in most failure situations.
    +   *
    +   * Points to note:
    +   *
    +   * Failure Recovery - You must checkpoint this stream, or save offsets yourself and provide them
    +   * as the fromOffsets parameter on restart.
    +   * Kafka must have sufficient log retention to obtain messages after failure.
    +   *
    +   * Getting offsets from the stream - see programming guide
    +   *
    +.  * Zookeeper - This does not use Zookeeper to store offsets.  For interop with Kafka monitors
    +   * that depend on Zookeeper, you must store offsets in ZK yourself.
    +   *
    +   * End-to-end semantics - This does not guarantee that any output operation will push each record
    +   * exactly once. To ensure end-to-end exactly-once semantics (that is, receiving exactly once and
    +   * outputting exactly once), you have to either ensure that the output operation is
    +   * idempotent, or transactionally store offsets with the output. See the programming guide for
    +   * more details.
    +   *
    +   * @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
    +   */
    +  @Experimental
    +  def createNewStream[
    +    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
    +  ): InputDStream[R] = {
    +    new DeterministicKafkaInputDStream[K, V, U, T, R](
    +      ssc, kafkaParams, fromOffsets, messageHandler)
    +  }
    +
    +  /**
    +   * This stream can guarantee that each message from Kafka is included in transformations
    +   * (as opposed to output actions) exactly once, even in most failure situations.
    --- End diff --
    
    This is a weak statement: "in most failure situations". Shouldn't it say "in all known failure situations"?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24019256
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala ---
    @@ -0,0 +1,69 @@
    +/*
    + * 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.streaming.kafka
    +
    +import kafka.common.TopicAndPartition
    +
    +/** Something that has a collection of OffsetRanges */
    +trait HasOffsetRanges {
    +  def offsetRanges: Array[OffsetRange]
    +}
    +
    +/** Represents a range of offsets from a single Kafka TopicAndPartition */
    +final class OffsetRange private(
    +    /** kafka topic name */
    +    val topic: String,
    +    /** kafka partition id */
    +    val partition: Int,
    +    /** inclusive starting offset */
    +    val fromOffset: Long,
    +    /** exclusive ending offset */
    +    val untilOffset: Long) extends Serializable {
    +  import OffsetRange.OffsetRangeTuple
    +
    +  /** this is to avoid ClassNotFoundException during checkpoint restore */
    --- End diff --
    
    There was discussion of this earlier that has since gotten buried.  Here's the gist:
    
    https://gist.github.com/koeninger/561a61482cd1b5b3600c
    
    The classloader being used for restoring the checkpoint doesn't have that class, probably because it's in external (and thus included in the user assembly), rather than one of the spark jars thats on the default classpath.
    
    I went ahead and duplicated that comment.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72782343
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/26701/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by helena <gi...@git.apache.org>.
Github user helena commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22649891
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,313 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    +  * Convenience methods for interacting with a Kafka cluster.
    +  * @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
    +  */
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    +  import KafkaCluster.Err
    +
    +  val seedBrokers: Array[(String, Int)] =
    +    kafkaParams.get("metadata.broker.list")
    +      .orElse(kafkaParams.get("bootstrap.servers"))
    +      .getOrElse(throw new Exception("Must specify metadata.broker.list or bootstrap.servers"))
    +      .split(",").map { hp =>
    +        val hpa = hp.split(":")
    +        (hpa(0), hpa(1).toInt)
    +      }
    +
    +  // ConsumerConfig isn't serializable
    +  @transient private var _config: ConsumerConfig = null
    +
    +  def config: ConsumerConfig = this.synchronized {
    +    if (_config == null) {
    +      _config = KafkaCluster.consumerConfig(kafkaParams)
    +    }
    +    _config
    +  }
    +
    +  def connect(host: String, port: Int): SimpleConsumer =
    +    new SimpleConsumer(host, port, config.socketTimeoutMs,
    +      config.socketReceiveBufferBytes, config.clientId)
    +
    +  def connect(hostAndPort: (String, Int)): SimpleConsumer =
    +    connect(hostAndPort._1, hostAndPort._2)
    +
    +  def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] =
    +    findLeader(topic, partition).right.map(connect)
    +
    +  def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, Seq(topic))
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      resp.topicsMetadata.find(_.topic == topic).flatMap { t =>
    +        t.partitionsMetadata.find(_.partitionId == partition)
    +      }.foreach { partitionMeta =>
    +        partitionMeta.leader.foreach { leader =>
    +          return Right((leader.host, leader.port))
    +        }
    +      }
    +    }
    +    Left(errs)
    +  }
    +
    +  def findLeaders(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, (String, Int)]] = {
    +    getPartitionMetadata(topicAndPartitions.map(_.topic)).right.flatMap { tms =>
    +      val result = tms.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.flatMap { pm =>
    +          val tp = TopicAndPartition(tm.topic, pm.partitionId)
    +          if (topicAndPartitions(tp)) {
    +            pm.leader.map { l =>
    +              tp -> (l.host -> l.port)
    +            }
    +          } else {
    +            None
    +          }
    +        }
    +      }.toMap
    +      if (result.keys.size == topicAndPartitions.size) {
    +        Right(result)
    +      } else {
    +        val missing = topicAndPartitions.diff(result.keys.toSet)
    +        val err = new Err
    +        err.append(new Exception(s"Couldn't find leaders for ${missing}"))
    +        Left(err)
    +      }
    +    }
    +  }
    +
    +  def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] =
    +    getPartitionMetadata(topics).right.map { r =>
    +      r.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.map { pm =>
    +          TopicAndPartition(tm.topic, pm.partitionId)
    +        }    
    +      }
    +    }
    +
    +  def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, topics.toSeq)
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      // error codes here indicate missing / just created topic,
    +      // repeating on a different broker wont be useful
    +      return Right(resp.topicsMetadata.toSet)
    +    }
    +    Left(errs)
    +  }
    +
    +  def getLatestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime)
    +
    +  def getEarliestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime)
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, before, 1).right.map { r =>
    +      r.map { kv =>
    +        // mapValues isnt serializable, see SI-7005
    +        kv._1 -> kv._2.head
    +      }
    +    }
    +
    +  private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] =
    +    m.groupBy(_._2).map { kv =>
    +      kv._1 -> kv._2.keys.toSeq
    +    }
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long,
    +    maxNumOffsets: Int
    +  ): Either[Err, Map[TopicAndPartition, Seq[Long]]] = {
    +    findLeaders(topicAndPartitions).right.flatMap { tpToLeader =>
    +      val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] = flip(tpToLeader)
    +      val leaders = leaderToTp.keys
    +      var result = Map[TopicAndPartition, Seq[Long]]()
    +      val errs = new Err
    +      withBrokers(leaders, errs) { consumer =>
    +        val needed: Seq[TopicAndPartition] = leaderToTp((consumer.host, consumer.port))
    +        val req = OffsetRequest(
    +          needed.map { tp =>
    +            tp -> PartitionOffsetRequestInfo(before, maxNumOffsets)
    +          }.toMap
    +        )
    +        val resp = consumer.getOffsetsBefore(req)
    +        val respMap = resp.partitionErrorAndOffsets
    +        needed.foreach { tp =>
    +          respMap.get(tp).foreach { errAndOffsets =>
    +            if (errAndOffsets.error == ErrorMapping.NoError) {
    +              if (errAndOffsets.offsets.nonEmpty) {
    +                result += tp -> errAndOffsets.offsets
    +              } else {
    +                errs.append(new Exception(
    +                  s"Empty offsets for ${tp}, is ${before} before log beginning?"))
    +              }
    +            } else {
    +              errs.append(ErrorMapping.exceptionFor(errAndOffsets.error))
    +            }
    +          }
    +        }
    +        if (result.keys.size == topicAndPartitions.size) {
    +          return Right(result)
    +        }
    +      }
    +      val missing = topicAndPartitions.diff(result.keys.toSet)
    +      errs.append(new Exception(s"Couldn't find leader offsets for ${missing}"))
    +      Left(errs)
    +    }
    +  }
    +
    +  def getConsumerOffsets(
    +    groupId: String,
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] = {
    +    getConsumerOffsetMetadata(groupId, topicAndPartitions).right.map { r =>
    +      r.map { kv =>
    +        kv._1 -> kv._2.offset
    +      }
    +    }
    +  }
    +
    +  def getConsumerOffsetMetadata(
    +    groupId: String,
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = {
    +    var result = Map[TopicAndPartition, OffsetMetadataAndError]()
    +    val req = OffsetFetchRequest(groupId, topicAndPartitions.toSeq)
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp = consumer.fetchOffsets(req)
    +      val respMap = resp.requestInfo
    +      val needed = topicAndPartitions.diff(result.keys.toSet)
    +      needed.foreach { tp =>
    +        respMap.get(tp).foreach { offsetMeta =>
    +          if (offsetMeta.error == ErrorMapping.NoError) {
    +            result += tp -> offsetMeta
    +          } else {
    +            errs.append(ErrorMapping.exceptionFor(offsetMeta.error))
    +          }
    +        }
    +      }
    +      if (result.keys.size == topicAndPartitions.size) {
    +        return Right(result)
    +      }
    +    }
    +    val missing = topicAndPartitions.diff(result.keys.toSet)
    +    errs.append(new Exception(s"Couldn't find consumer offsets for ${missing}"))
    +    Left(errs)
    +  }
    +
    +  def setConsumerOffsets(
    +    groupId: String,
    +    offsets: Map[TopicAndPartition, Long]
    +  ): Either[Err, Map[TopicAndPartition, Short]] = {
    +    setConsumerOffsetMetadata(groupId, offsets.map { kv =>
    +      kv._1 -> OffsetMetadataAndError(kv._2)
    +    })
    +  }
    +
    +  def setConsumerOffsetMetadata(
    +    groupId: String,
    +    metadata: Map[TopicAndPartition, OffsetMetadataAndError]
    +  ): Either[Err, Map[TopicAndPartition, Short]] = {
    +    var result = Map[TopicAndPartition, Short]()
    +    val req = OffsetCommitRequest(groupId, metadata)
    +    val errs = new Err
    +    val topicAndPartitions = metadata.keys.toSet
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp = consumer.commitOffsets(req)
    +      val respMap = resp.requestInfo
    +      val needed = topicAndPartitions.diff(result.keys.toSet)
    +      needed.foreach { tp =>
    +        respMap.get(tp).foreach { err =>
    +          if (err == ErrorMapping.NoError) {
    +            result += tp -> err
    +          } else {
    +            errs.append(ErrorMapping.exceptionFor(err))
    +          }
    +        }
    +      }
    +      if (result.keys.size == topicAndPartitions.size) {
    +        return Right(result)
    +      }
    +    }
    +    val missing = topicAndPartitions.diff(result.keys.toSet)
    +    errs.append(new Exception(s"Couldn't set offsets for ${missing}"))
    +    Left(errs)
    +  }
    +
    +  private def withBrokers(brokers: Iterable[(String, Int)], errs: Err)
    +    (fn: SimpleConsumer => Any): Unit = {
    +    brokers.foreach { hp =>
    +      var consumer: SimpleConsumer = null
    +      try {
    +        consumer = connect(hp)
    +        fn(consumer)
    +      } catch {
    +        case NonFatal(e) =>
    +          errs.append(e)
    +      } finally {
    +        if (consumer != null) consumer.close()
    +      }
    --- End diff --
    
    Or you can stay away from java nulls     
    
        var consumer: Option[SimpleConsumer] = None
        try {
          consumer = Some(connect(hp))
          consumer map (c => fn(c)) 
        } catch {
          case NonFatal(e) => errs.append(e)
        } finally consumer map (_.close())4


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by helena <gi...@git.apache.org>.
Github user helena commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22647698
  
    --- Diff: external/kafka/pom.xml ---
    @@ -44,7 +44,7 @@
         <dependency>
           <groupId>org.apache.kafka</groupId>
           <artifactId>kafka_${scala.binary.version}</artifactId>
    -      <version>0.8.0</version>
    --- End diff --
    
    @huitseeker @koeninger @tdas I do have the initial Kafka 0.8.2 PR in, just waiting to update the version to GA vs beta and re-test and check for any changes/regression.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23729727
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/OffsetRange.scala ---
    @@ -0,0 +1,70 @@
    +/*
    + * 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.rdd.kafka
    +
    +/** Represents a range of offsets from a single Kafka TopicAndPartition */
    +trait OffsetRange {
    --- End diff --
    
    It's a trait with no implemented methods, so it will get compiled to a single class file with the same bytecode as a java interface (plus a scala signature annotation).  It won't make a separate OffsetRange$class.class file as you may have seen for scala traits with default method implementations.
    
    The point of the trait/interface is that, as far as I understood, you were concerned about publicly exposing KafkaRDDPartition (which already is just a simple class, not a case class).  If you want one common supertype for both KafkaRDDPartition and whatever people pass into public methods to construct a KafkaRDD, your choices are an interface or a (possibly abstract) class.  I think an interface is cleaner.
    
    TLDR 
    - if you're fine with exposing KafkaRDDPartition, let's just do that.
    - if you're super concerned that a trait with no implementation can't be used from java, i'll move it to java code and change "trait" for "interface"



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23727132
  
    --- 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")) {
    --- End diff --
    
    This code is quite convoluted thought. It can definitely be simplified.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72749763
  
    @koeninger Thank you very much for all the changes. They are looking good. Unfortunately the issue with `createRDD` returning `RDD[] with OffsetRange` (i.e., the issue that @pwendell raised) could be a problem in the future in terms of binary compatibility. Basically, we have not used such things in the rest of Spark to keep things as Java-friendly and binary compatible as possible. Also in the generated Java doc this looks scary. So here is an alternate suggestion that I am trying to implement on top of your PR branch. How about this. We effectively combine KafkaRDD and HasOffsetRanges into a abstract class. 
    
    
    ```
    abstract class KafkaRDD[T] (val offsetRanges: Array[OffsetRanges], sc: SparkContext) 
       extends RDD[T](sc, Nil)
    
    private[kafka]
    class KafkaRDDImpl[K, V, KD, VD, R] extends KafkaRDD[R] {
      ...
    }
    
    KafkaUtils.createRDD(...simple one without messageHandler...): KafkaRDD[(K, V)] = {
       // return KafkaRDDImpl[K, V, KD, VD, (K, V)]
    }
    
    KafkaUtils.createRDD(...simple one WITH messageHandler...): KafkaRDD[R] = {
       // return KafkaRDDImpl[K, V, KD, VD, R]
    }
    
    ```
    
    Advantages
    - No binary compatibility issues
    - Easy to read from Java
    - KafkaRDD implementation and constructor all hidden as before
    - Can still extend KafkaRDD to expose more methods in future.  
    
    What do you think?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72228772
  
      [Test build #26401 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26401/consoleFull) for   PR 3798 at commit [`0090553`](https://github.com/apache/spark/commit/0090553eba09240b6ad4cf508ea33503705b12d9).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  class DeterministicKafkaInputDStreamCheckpointData extends DStreamCheckpointData(this) `
      * `class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable `
      * `  case class LeaderOffset(host: String, port: Int, offset: Long)`
      * `class KafkaRDDPartition(`



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24029555
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,174 @@ 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    +    val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message)
    +    val kc = new KafkaCluster(kafkaParams)
    +    val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet
    +    val leaders = kc.findLeaders(topics).fold(
    +      errs => throw new SparkException(errs.mkString("\n")),
    +      ok => ok
    +    )
    +    new KafkaRDD[K, V, U, T, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler)
    +  }
    +
    +  /** 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   * @param leaders Kafka leaders for each offset range in batch
    +   * @param messageHandler function for translating each message into the desired type
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange],
    +      leaders: Array[Leader],
    +      messageHandler: MessageAndMetadata[K, V] => R
    +  ): RDD[R] with HasOffsetRanges = {
    +
    +    val leaderMap = leaders
    +      .map(l => TopicAndPartition(l.topic, l.partition) -> (l.host, l.port))
    +      .toMap
    +    new KafkaRDD[K, V, U, T, R](sc, kafkaParams, offsetRanges, leaderMap, messageHandler)
    +  }
    +
    +  /**
    +   * This stream can guarantee that each message from Kafka is included in transformations
    +   * (as opposed to output actions) exactly once, even in most failure situations.
    +   *
    +   * Points to note:
    +   *
    +   * Failure Recovery - You must checkpoint this stream, or save offsets yourself and provide them
    +   * as the fromOffsets parameter on restart.
    +   * Kafka must have sufficient log retention to obtain messages after failure.
    +   *
    +   * Getting offsets from the stream - see programming guide
    +   *
    +.  * Zookeeper - This does not use Zookeeper to store offsets.  For interop with Kafka monitors
    +   * that depend on Zookeeper, you must store offsets in ZK yourself.
    +   *
    +   * End-to-end semantics - This does not guarantee that any output operation will push each record
    +   * exactly once. To ensure end-to-end exactly-once semantics (that is, receiving exactly once and
    +   * outputting exactly once), you have to either ensure that the output operation is
    +   * idempotent, or transactionally store offsets with the output. See the programming guide for
    +   * more details.
    +   *
    +   * @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
    +   */
    +  @Experimental
    +  def createNewStream[
    --- End diff --
    
    Okay - how about `createDirectStream` then?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72907304
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/26763/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24031550
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,249 @@ 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
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    --- End diff --
    
    The comment about unused R was referring to a prior version of the PR that
    had a copy-pasted type-level R parameter even for the version that returned
    RDD[(K, V)].  Github probably just got confused because the comment wasn't
    attached to the particular line in question.
    
    Pretty sure things are correct at this point.
    
    On Tue, Feb 3, 2015 at 1:05 PM, Patrick Wendell <no...@github.com>
    wrote:
    
    > In
    > external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
    > <https://github.com/apache/spark/pull/3798#discussion_r24029463>:
    >
    > > +   * 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
    > > +   */
    > > +  @Experimental
    > > +  def createRDD[
    > > +    K: ClassTag,
    > > +    V: ClassTag,
    > > +    U <: Decoder[_]: ClassTag,
    > > +    T <: Decoder[_]: ClassTag,
    > > +    R: ClassTag] (
    >
    > Yeah, makes sense. But the comment here suggests R is not used, however,
    > I see R being used in the return type. So that was my point.
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798/files#r24029463>.
    >


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71557370
  
    BTW one other thing - Either is really dangerous and very complicated to use. It almost always leads to downstream code becoming a mess. 
    
    You are mostly just using it for exception propagation. Why not just throw exceptions?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23747450
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,320 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.util.Random
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    +  * Convenience methods for interacting with a Kafka cluster.
    +  * @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
    +  */
    +private[spark]
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    --- End diff --
    
    Hmmm, that increases the checkpoint size. Though not by much. A utility class like this should not be serialized with the DAG of DStreams, but that's okay for now. We can deal with later. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23988445
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,174 @@ 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    --- End diff --
    
    I noticed that KafkaRDD isn't exposed, so maybe this is why. Not sure I see a big issue with exposing KafkaRDD and its constructor given that it's basically the same level of visibility as this static factory function.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by helena <gi...@git.apache.org>.
Github user helena commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22662976
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,313 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    +  * Convenience methods for interacting with a Kafka cluster.
    +  * @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
    +  */
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    +  import KafkaCluster.Err
    +
    +  val seedBrokers: Array[(String, Int)] =
    +    kafkaParams.get("metadata.broker.list")
    +      .orElse(kafkaParams.get("bootstrap.servers"))
    +      .getOrElse(throw new Exception("Must specify metadata.broker.list or bootstrap.servers"))
    +      .split(",").map { hp =>
    +        val hpa = hp.split(":")
    +        (hpa(0), hpa(1).toInt)
    +      }
    +
    +  // ConsumerConfig isn't serializable
    +  @transient private var _config: ConsumerConfig = null
    +
    +  def config: ConsumerConfig = this.synchronized {
    +    if (_config == null) {
    +      _config = KafkaCluster.consumerConfig(kafkaParams)
    +    }
    +    _config
    +  }
    +
    +  def connect(host: String, port: Int): SimpleConsumer =
    +    new SimpleConsumer(host, port, config.socketTimeoutMs,
    +      config.socketReceiveBufferBytes, config.clientId)
    +
    +  def connect(hostAndPort: (String, Int)): SimpleConsumer =
    +    connect(hostAndPort._1, hostAndPort._2)
    +
    +  def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] =
    +    findLeader(topic, partition).right.map(connect)
    +
    +  def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, Seq(topic))
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      resp.topicsMetadata.find(_.topic == topic).flatMap { t =>
    +        t.partitionsMetadata.find(_.partitionId == partition)
    +      }.foreach { partitionMeta =>
    +        partitionMeta.leader.foreach { leader =>
    +          return Right((leader.host, leader.port))
    +        }
    +      }
    +    }
    +    Left(errs)
    +  }
    +
    +  def findLeaders(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, (String, Int)]] = {
    +    getPartitionMetadata(topicAndPartitions.map(_.topic)).right.flatMap { tms =>
    +      val result = tms.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.flatMap { pm =>
    +          val tp = TopicAndPartition(tm.topic, pm.partitionId)
    +          if (topicAndPartitions(tp)) {
    +            pm.leader.map { l =>
    +              tp -> (l.host -> l.port)
    +            }
    +          } else {
    +            None
    +          }
    +        }
    +      }.toMap
    +      if (result.keys.size == topicAndPartitions.size) {
    +        Right(result)
    +      } else {
    +        val missing = topicAndPartitions.diff(result.keys.toSet)
    +        val err = new Err
    +        err.append(new Exception(s"Couldn't find leaders for ${missing}"))
    +        Left(err)
    +      }
    +    }
    +  }
    +
    +  def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] =
    +    getPartitionMetadata(topics).right.map { r =>
    +      r.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.map { pm =>
    +          TopicAndPartition(tm.topic, pm.partitionId)
    +        }    
    +      }
    +    }
    +
    +  def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, topics.toSeq)
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      // error codes here indicate missing / just created topic,
    +      // repeating on a different broker wont be useful
    +      return Right(resp.topicsMetadata.toSet)
    +    }
    +    Left(errs)
    +  }
    +
    +  def getLatestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime)
    +
    +  def getEarliestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime)
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, before, 1).right.map { r =>
    +      r.map { kv =>
    +        // mapValues isnt serializable, see SI-7005
    +        kv._1 -> kv._2.head
    +      }
    +    }
    +
    +  private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] =
    +    m.groupBy(_._2).map { kv =>
    +      kv._1 -> kv._2.keys.toSeq
    +    }
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long,
    +    maxNumOffsets: Int
    +  ): Either[Err, Map[TopicAndPartition, Seq[Long]]] = {
    +    findLeaders(topicAndPartitions).right.flatMap { tpToLeader =>
    +      val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] = flip(tpToLeader)
    +      val leaders = leaderToTp.keys
    +      var result = Map[TopicAndPartition, Seq[Long]]()
    +      val errs = new Err
    +      withBrokers(leaders, errs) { consumer =>
    +        val needed: Seq[TopicAndPartition] = leaderToTp((consumer.host, consumer.port))
    +        val req = OffsetRequest(
    +          needed.map { tp =>
    +            tp -> PartitionOffsetRequestInfo(before, maxNumOffsets)
    +          }.toMap
    +        )
    +        val resp = consumer.getOffsetsBefore(req)
    +        val respMap = resp.partitionErrorAndOffsets
    +        needed.foreach { tp =>
    +          respMap.get(tp).foreach { errAndOffsets =>
    +            if (errAndOffsets.error == ErrorMapping.NoError) {
    +              if (errAndOffsets.offsets.nonEmpty) {
    +                result += tp -> errAndOffsets.offsets
    +              } else {
    +                errs.append(new Exception(
    +                  s"Empty offsets for ${tp}, is ${before} before log beginning?"))
    +              }
    +            } else {
    +              errs.append(ErrorMapping.exceptionFor(errAndOffsets.error))
    +            }
    +          }
    +        }
    +        if (result.keys.size == topicAndPartitions.size) {
    +          return Right(result)
    +        }
    +      }
    +      val missing = topicAndPartitions.diff(result.keys.toSet)
    +      errs.append(new Exception(s"Couldn't find leader offsets for ${missing}"))
    +      Left(errs)
    +    }
    +  }
    +
    +  def getConsumerOffsets(
    +    groupId: String,
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] = {
    +    getConsumerOffsetMetadata(groupId, topicAndPartitions).right.map { r =>
    +      r.map { kv =>
    +        kv._1 -> kv._2.offset
    +      }
    +    }
    +  }
    +
    +  def getConsumerOffsetMetadata(
    +    groupId: String,
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = {
    +    var result = Map[TopicAndPartition, OffsetMetadataAndError]()
    +    val req = OffsetFetchRequest(groupId, topicAndPartitions.toSeq)
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp = consumer.fetchOffsets(req)
    +      val respMap = resp.requestInfo
    +      val needed = topicAndPartitions.diff(result.keys.toSet)
    +      needed.foreach { tp =>
    +        respMap.get(tp).foreach { offsetMeta =>
    +          if (offsetMeta.error == ErrorMapping.NoError) {
    +            result += tp -> offsetMeta
    +          } else {
    +            errs.append(ErrorMapping.exceptionFor(offsetMeta.error))
    +          }
    +        }
    +      }
    +      if (result.keys.size == topicAndPartitions.size) {
    +        return Right(result)
    +      }
    +    }
    +    val missing = topicAndPartitions.diff(result.keys.toSet)
    +    errs.append(new Exception(s"Couldn't find consumer offsets for ${missing}"))
    +    Left(errs)
    +  }
    +
    +  def setConsumerOffsets(
    +    groupId: String,
    +    offsets: Map[TopicAndPartition, Long]
    +  ): Either[Err, Map[TopicAndPartition, Short]] = {
    +    setConsumerOffsetMetadata(groupId, offsets.map { kv =>
    +      kv._1 -> OffsetMetadataAndError(kv._2)
    +    })
    +  }
    +
    +  def setConsumerOffsetMetadata(
    +    groupId: String,
    +    metadata: Map[TopicAndPartition, OffsetMetadataAndError]
    +  ): Either[Err, Map[TopicAndPartition, Short]] = {
    +    var result = Map[TopicAndPartition, Short]()
    +    val req = OffsetCommitRequest(groupId, metadata)
    +    val errs = new Err
    +    val topicAndPartitions = metadata.keys.toSet
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp = consumer.commitOffsets(req)
    +      val respMap = resp.requestInfo
    +      val needed = topicAndPartitions.diff(result.keys.toSet)
    +      needed.foreach { tp =>
    +        respMap.get(tp).foreach { err =>
    +          if (err == ErrorMapping.NoError) {
    +            result += tp -> err
    +          } else {
    +            errs.append(ErrorMapping.exceptionFor(err))
    +          }
    +        }
    +      }
    +      if (result.keys.size == topicAndPartitions.size) {
    +        return Right(result)
    +      }
    +    }
    +    val missing = topicAndPartitions.diff(result.keys.toSet)
    +    errs.append(new Exception(s"Couldn't set offsets for ${missing}"))
    +    Left(errs)
    +  }
    +
    +  private def withBrokers(brokers: Iterable[(String, Int)], errs: Err)
    +    (fn: SimpleConsumer => Any): Unit = {
    +    brokers.foreach { hp =>
    +      var consumer: SimpleConsumer = null
    +      try {
    +        consumer = connect(hp)
    +        fn(consumer)
    +      } catch {
    +        case NonFatal(e) =>
    +          errs.append(e)
    +      } finally {
    +        if (consumer != null) consumer.close()
    +      }
    --- End diff --
    
    True, it could because I was not conclusive on the suggestion adding None where applicable. catch => None


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23973281
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,338 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.util.Random
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    + * Convenience methods for interacting with a Kafka cluster.
    + * @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
    + */
    +private[spark]
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    +  import KafkaCluster.{Err, LeaderOffset}
    +
    +  val seedBrokers: Array[(String, Int)] =
    +    kafkaParams.get("metadata.broker.list")
    +      .orElse(kafkaParams.get("bootstrap.servers"))
    +      .getOrElse(throw new Exception("Must specify metadata.broker.list or bootstrap.servers"))
    +      .split(",").map { hp =>
    +        val hpa = hp.split(":")
    +        (hpa(0), hpa(1).toInt)
    +      }
    +
    +  // ConsumerConfig isn't serializable
    +  @transient private var _config: ConsumerConfig = null
    +
    +  def config: ConsumerConfig = this.synchronized {
    +    if (_config == null) {
    +      _config = KafkaCluster.consumerConfig(kafkaParams)
    +    }
    +    _config
    +  }
    +
    +  def connect(host: String, port: Int): SimpleConsumer =
    +    new SimpleConsumer(host, port, config.socketTimeoutMs,
    +      config.socketReceiveBufferBytes, config.clientId)
    +
    +  def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] =
    +    findLeader(topic, partition).right.map(hp => connect(hp._1, hp._2))
    +
    +  // Metadata api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataAPI
    +  // scalastyle:on
    +
    +  def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, Seq(topic))
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      resp.topicsMetadata.find(_.topic == topic).flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.find(_.partitionId == partition)
    +      }.foreach { pm: PartitionMetadata =>
    +        pm.leader.foreach { leader =>
    +          return Right((leader.host, leader.port))
    +        }
    +      }
    +    }
    +    Left(errs)
    +  }
    +
    +  def findLeaders(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, (String, Int)]] = {
    +    val topics = topicAndPartitions.map(_.topic)
    +    getPartitionMetadata(topics).right.flatMap { tms: Set[TopicMetadata] =>
    +      val result = tms.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.flatMap { pm: PartitionMetadata =>
    +          val tp = TopicAndPartition(tm.topic, pm.partitionId)
    +          if (topicAndPartitions(tp)) {
    +            pm.leader.map { l =>
    +              tp -> (l.host -> l.port)
    +            }
    +          } else {
    +            None
    +          }
    +        }
    +      }.toMap
    +      if (result.keys.size == topicAndPartitions.size) {
    +        Right(result)
    +      } else {
    +        val missing = topicAndPartitions.diff(result.keySet)
    +        val err = new Err
    +        err.append(new Exception(s"Couldn't find leaders for ${missing}"))
    +        Left(err)
    +      }
    +    }
    +  }
    +
    +  def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] =
    +    getPartitionMetadata(topics).right.map { r =>
    +      r.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.map { pm: PartitionMetadata =>
    +          TopicAndPartition(tm.topic, pm.partitionId)
    +        }    
    +      }
    +    }
    +
    +  def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = {
    +    val req = TopicMetadataRequest(
    +      TopicMetadataRequest.CurrentVersion, 0, config.clientId, topics.toSeq)
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      // error codes here indicate missing / just created topic,
    +      // repeating on a different broker wont be useful
    +      return Right(resp.topicsMetadata.toSet)
    +    }
    +    Left(errs)
    +  }
    +
    +  // Leader offset api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI
    +  // scalastyle:on
    +
    +  def getLatestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime)
    +
    +  def getEarliestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime)
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long
    +  ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    +    getLeaderOffsets(topicAndPartitions, before, 1).right.map { r =>
    +      r.map { kv =>
    +        // mapValues isnt serializable, see SI-7005
    +        kv._1 -> kv._2.head
    +      }
    +    }
    +
    +  private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] =
    +    m.groupBy(_._2).map { kv =>
    +      kv._1 -> kv._2.keys.toSeq
    +    }
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long,
    +    maxNumOffsets: Int
    +  ): Either[Err, Map[TopicAndPartition, Seq[LeaderOffset]]] = {
    +    findLeaders(topicAndPartitions).right.flatMap { tpToLeader =>
    +      val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] = flip(tpToLeader)
    +      val leaders = leaderToTp.keys
    +      var result = Map[TopicAndPartition, Seq[LeaderOffset]]()
    +      val errs = new Err
    +      withBrokers(leaders, errs) { consumer =>
    +        val needed: Seq[TopicAndPartition] = leaderToTp((consumer.host, consumer.port))
    --- End diff --
    
    It will be easier to understand if `needed` is renamed to `partitionsToGetOffsets` (at least that is my interpretation of this code).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23904854
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,249 @@ 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
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      batch: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    +    val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message)
    +    val kc = new KafkaCluster(kafkaParams)
    +    val topics = batch.map(o => TopicAndPartition(o.topic, o.partition)).toSet
    +    val leaderMap = kc.findLeaders(topics).fold(
    +      errs => throw new SparkException(errs.mkString("\n")),
    +      ok => ok
    +    )
    +    val rddParts = batch.zipWithIndex.map { case (o, i) =>
    +        val tp = TopicAndPartition(o.topic, o.partition)
    +        val (host, port) = leaderMap(tp)
    +        new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset, host, port)
    +    }.toArray
    +    new KafkaRDD[K, V, U, T, (K, V)](sc, kafkaParams, rddParts, messageHandler)
    +  }
    +
    +  /** 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 leaders Kafka leaders for each offset range in batch
    +   * @param messageHandler function for translating each message into the desired type
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      batch: Array[OffsetRange],
    +      leaders: Array[Leader],
    +      messageHandler: MessageAndMetadata[K, V] => R
    +  ): RDD[R] with HasOffsetRanges = {
    +    val leaderMap = leaders.map(l => (l.topic, l.partition) -> (l.host, l.port)).toMap
    +    val rddParts = batch.zipWithIndex.map { case (o, i) =>
    +        val (host, port) = leaderMap((o.topic, o.partition))
    +        new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset, host, port)
    +    }.toArray
    +
    +    new KafkaRDD[K, V, U, T, R](sc, kafkaParams, rddParts, messageHandler)
    +  }
    +
    +  /**
    +   * This stream can guarantee that each message from Kafka is included in transformations
    +   * (as opposed to output actions) exactly once, even in most failure situations.
    +   *
    +   * Points to note:
    +   *
    +   * Failure Recovery - You must checkpoint this stream, or save offsets yourself and provide them
    +   * as the fromOffsets parameter on restart.
    +   * Kafka must have sufficient log retention to obtain messages after failure.
    +   *
    +   * Getting offsets from the stream - see programming guide
    +   *
    +.  * Zookeeper - This does not use Zookeeper to store offsets.  For interop with Kafka monitors
    +   * that depend on Zookeeper, you must store offsets in ZK yourself.
    +   *
    +   * End-to-end semantics - This does not guarantee that any output operation will push each record
    +   * exactly once. To ensure end-to-end exactly-once semantics (that is, receiving exactly once and
    +   * outputting exactly once), you have to either ensure that the output operation is
    +   * idempotent, or transactionally store offsets with the output. See the programming guide for
    +   * more details.
    +   *
    +   * @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
    +   */
    +  @Experimental
    +  def createNewStream[
    +    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 stream can guarantee that each message from Kafka is included in transformations
    +   * (as opposed to output actions) exactly once, even in most failure situations.
    +   *
    +   * Points to note:
    +   *
    +   * Failure Recovery - You must checkpoint this stream.
    +   * Kafka must have sufficient log retention to obtain messages after failure.
    +   *
    +   * Getting offsets from the stream - see programming guide
    +   *
    +.  * Zookeeper - This does not use Zookeeper to store offsets.  For interop with Kafka monitors
    +   * that depend on Zookeeper, you must store offsets in ZK yourself.
    +   *
    +   * End-to-end semantics - This does not guarantee that any output operation will push each record
    +   * exactly once. To ensure end-to-end exactly-once semantics (that is, receiving exactly once and
    +   * outputting exactly once), you have to ensure that the output operation is idempotent.
    +   *
    +   * @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
    +   */
    +  @Experimental
    +  def createNewStream[
    +    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 {
    --- End diff --
    
    OK, I got it. So the meaning of "auto.offset.reset" may be different from what Kafka originally means, I think maybe we could give this configuration another name, otherwise user may be confused with it.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71549132
  
    Just updated it
    
    On Mon, Jan 26, 2015 at 4:06 PM, Hari Shreedharan <no...@github.com>
    wrote:
    
    > @koeninger <https://github.com/koeninger> Can you run sbt scalastyle and
    > fix the style violations? There are like 2 instances where there are lines
    > > 100 chars.
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798#issuecomment-71548572>.
    >


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72893619
  
      [Test build #26763 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26763/consoleFull) for   PR 3798 at commit [`1dc2941`](https://github.com/apache/spark/commit/1dc29415e3c0ac23a4207513686dfe5ee5ab2725).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72902389
  
    Glad to hear that your son is doing ok, hope he gets better (okish--> great) real real soon. :)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23972610
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala ---
    @@ -0,0 +1,150 @@
    +/*
    + * 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.streaming.kafka
    +
    +
    +import scala.annotation.tailrec
    +import scala.collection.mutable
    +import scala.reflect.{classTag, ClassTag}
    +
    +import kafka.common.TopicAndPartition
    +import kafka.message.MessageAndMetadata
    +import kafka.serializer.Decoder
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset
    +import org.apache.spark.streaming.{StreamingContext, Time}
    +import org.apache.spark.streaming.dstream._
    +
    +/** A stream of {@link org.apache.spark.rdd.kafka.KafkaRDD} where
    +  * each given Kafka topic/partition corresponds to an RDD partition.
    +  * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages
    +  * per second that each '''partition''' will accept.
    +  * Starting offsets are specified in advance,
    +  * and this DStream is not responsible for committing offsets,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the stream
    +  * @param messageHandler function for translating each message into the desired type
    +  * @param maxRetries maximum number of times in a row to retry getting leaders' offsets
    +  */
    +private[streaming]
    +class DeterministicKafkaInputDStream[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    @transient ssc_ : StreamingContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R,
    +    maxRetries: Int
    +) extends InputDStream[R](ssc_) with Logging {
    +
    +  protected[streaming] override val checkpointData =
    +    new DeterministicKafkaInputDStreamCheckpointData
    +
    +  protected val kc = new KafkaCluster(kafkaParams)
    +
    +  protected val maxMessagesPerPartition: Option[Long] = {
    +    val ratePerSec = context.sparkContext.getConf.getInt("spark.streaming.receiver.maxRate", 0)
    +    if (ratePerSec > 0) {
    +      val secsPerBatch = context.graph.batchDuration.milliseconds.toDouble / 1000
    +      Some((secsPerBatch * ratePerSec).toLong)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  protected var currentOffsets = fromOffsets
    +
    +  @tailrec
    +  protected final def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, LeaderOffset] = {
    --- End diff --
    
    I don't want KafkaCluster throwing exceptions though
    
    On Mon, Feb 2, 2015 at 6:12 PM, Tathagata Das <no...@github.com>
    wrote:
    
    > In
    > external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala
    > <https://github.com/apache/spark/pull/3798#discussion_r23971976>:
    >
    > > +  protected val kc = new KafkaCluster(kafkaParams)
    > > +
    > > +  protected val maxMessagesPerPartition: Option[Long] = {
    > > +    val ratePerSec = context.sparkContext.getConf.getInt("spark.streaming.receiver.maxRate", 0)
    > > +    if (ratePerSec > 0) {
    > > +      val secsPerBatch = context.graph.batchDuration.milliseconds.toDouble / 1000
    > > +      Some((secsPerBatch * ratePerSec).toLong)
    > > +    } else {
    > > +      None
    > > +    }
    > > +  }
    > > +
    > > +  protected var currentOffsets = fromOffsets
    > > +
    > > +  @tailrec
    > > +  protected final def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, LeaderOffset] = {
    >
    > This function should be part of KafkaCluster. The getLatestLeaderOffset
    > could take an optional parameter of retries.
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798/files#r23971976>.
    >


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72592274
  
      [Test build #26593 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26593/consoleFull) for   PR 3798 at commit [`8991017`](https://github.com/apache/spark/commit/8991017cdf6c33527cbabab5dcda29519579692f).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  class DeterministicKafkaInputDStreamCheckpointData extends DStreamCheckpointData(this) `
      * `class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable `
      * `  case class LeaderOffset(host: String, port: Int, offset: Long)`
      * `class KafkaRDDPartition(`
      * `trait HasOffsetRanges `



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23973072
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,338 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.util.Random
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    + * Convenience methods for interacting with a Kafka cluster.
    + * @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
    + */
    +private[spark]
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    +  import KafkaCluster.{Err, LeaderOffset}
    +
    +  val seedBrokers: Array[(String, Int)] =
    +    kafkaParams.get("metadata.broker.list")
    +      .orElse(kafkaParams.get("bootstrap.servers"))
    +      .getOrElse(throw new Exception("Must specify metadata.broker.list or bootstrap.servers"))
    +      .split(",").map { hp =>
    +        val hpa = hp.split(":")
    +        (hpa(0), hpa(1).toInt)
    +      }
    +
    +  // ConsumerConfig isn't serializable
    +  @transient private var _config: ConsumerConfig = null
    +
    +  def config: ConsumerConfig = this.synchronized {
    +    if (_config == null) {
    +      _config = KafkaCluster.consumerConfig(kafkaParams)
    +    }
    +    _config
    +  }
    +
    +  def connect(host: String, port: Int): SimpleConsumer =
    +    new SimpleConsumer(host, port, config.socketTimeoutMs,
    +      config.socketReceiveBufferBytes, config.clientId)
    +
    +  def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] =
    +    findLeader(topic, partition).right.map(hp => connect(hp._1, hp._2))
    +
    +  // Metadata api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataAPI
    +  // scalastyle:on
    +
    +  def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, Seq(topic))
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      resp.topicsMetadata.find(_.topic == topic).flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.find(_.partitionId == partition)
    +      }.foreach { pm: PartitionMetadata =>
    +        pm.leader.foreach { leader =>
    +          return Right((leader.host, leader.port))
    +        }
    +      }
    +    }
    +    Left(errs)
    +  }
    +
    +  def findLeaders(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, (String, Int)]] = {
    +    val topics = topicAndPartitions.map(_.topic)
    +    getPartitionMetadata(topics).right.flatMap { tms: Set[TopicMetadata] =>
    +      val result = tms.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.flatMap { pm: PartitionMetadata =>
    +          val tp = TopicAndPartition(tm.topic, pm.partitionId)
    +          if (topicAndPartitions(tp)) {
    +            pm.leader.map { l =>
    +              tp -> (l.host -> l.port)
    +            }
    +          } else {
    +            None
    +          }
    +        }
    +      }.toMap
    +      if (result.keys.size == topicAndPartitions.size) {
    +        Right(result)
    +      } else {
    +        val missing = topicAndPartitions.diff(result.keySet)
    +        val err = new Err
    +        err.append(new Exception(s"Couldn't find leaders for ${missing}"))
    +        Left(err)
    +      }
    +    }
    +  }
    +
    +  def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] =
    --- End diff --
    
    Multi-line methods must use `{ ... }`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23823395
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala ---
    @@ -0,0 +1,71 @@
    +/*
    + * 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.streaming.kafka
    +
    +import kafka.common.TopicAndPartition
    +
    +/** Represents a range of offsets from a single Kafka TopicAndPartition */
    +trait OffsetRange {
    --- End diff --
    
    Same comment as Leader.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23572086
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,318 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.util.Random
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    +  * Convenience methods for interacting with a Kafka cluster.
    +  * @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
    +  */
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    +  import KafkaCluster.{Err, LeaderOffset}
    +
    +  val seedBrokers: Array[(String, Int)] =
    +    kafkaParams.get("metadata.broker.list")
    +      .orElse(kafkaParams.get("bootstrap.servers"))
    +      .getOrElse(throw new Exception("Must specify metadata.broker.list or bootstrap.servers"))
    +      .split(",").map { hp =>
    +        val hpa = hp.split(":")
    +        (hpa(0), hpa(1).toInt)
    +      }
    +
    +  // ConsumerConfig isn't serializable
    +  @transient private var _config: ConsumerConfig = null
    +
    +  def config: ConsumerConfig = this.synchronized {
    +    if (_config == null) {
    +      _config = KafkaCluster.consumerConfig(kafkaParams)
    +    }
    +    _config
    +  }
    +
    +  def connect(host: String, port: Int): SimpleConsumer =
    +    new SimpleConsumer(host, port, config.socketTimeoutMs,
    +      config.socketReceiveBufferBytes, config.clientId)
    +
    +  def connect(hostAndPort: (String, Int)): SimpleConsumer =
    --- End diff --
    
    better remove this method since it doesn't do much ...


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22366478
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,305 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    +  * Convenience methods for interacting with a Kafka cluster.
    +  * @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
    +  */
    +class KafkaCluster(val kafkaParams: Map[String, String]) {
    --- End diff --
    
    I see - sorry let me look more, I didn't realize this is necessary for users.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23748806
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/OffsetRange.scala ---
    @@ -0,0 +1,70 @@
    +/*
    + * 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.rdd.kafka
    +
    +/** Represents a range of offsets from a single Kafka TopicAndPartition */
    +trait OffsetRange {
    --- End diff --
    
    No, it's not attaching the offset to every record, that's what i'm trying
    to avoid.  It's dealing with the offsets either once per rdd, or once per
    partition, depending on what is necessary from a client semantics point of
    view.
    
    Hopefully accessing the RDD from inside foreachRDD isn't contentious?
    
    As for accessing the partition, yeah its pretty weird to have to go index
    the rdd to get the partition... but it's also pretty weird that the
    existing apis named 'mapPartition*' don't actually give you access to the
    partition... after all, the partition is serializable.
    
    
    
    On Wed, Jan 28, 2015 at 11:47 PM, Tathagata Das <no...@github.com>
    wrote:
    
    > In
    > external/kafka/src/main/scala/org/apache/spark/rdd/kafka/OffsetRange.scala
    > <https://github.com/apache/spark/pull/3798#discussion_r23747942>:
    >
    > > + * (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.rdd.kafka
    > > +
    > > +/** Represents a range of offsets from a single Kafka TopicAndPartition */
    > > +trait OffsetRange {
    >
    > Right, I get your point. Though I thought about the usage based on the
    > example, and I think we need to think this a little bit more. From what I
    > understood, you are attaching the offset in every records, and shuffling
    > everything with that offset attached. That is quite a loss of efficiency.
    > Also, accessing the RDD and its partition object from within the
    > mapPartition function is very confusing, and ... does it actually work????
    > If at all this works, thats not even the recommended RDD operation!
    >
    > We really need to come up with a better way to expose offsets.
    > Brainstorming a little more on this.
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798/files#r23747942>.
    >


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23743523
  
    --- Diff: external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala ---
    @@ -130,7 +130,7 @@ abstract class KafkaStreamSuiteBase extends FunSuite with Eventually with Loggin
       }
     
       def createTopic(topic: String) {
    -    CreateTopicCommand.createTopic(zkClient, topic, 1, 1, "0")
    --- End diff --
    
    Was this necessary because of Kafka 0.8.0 to 0.8.1.1 change?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23904616
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,249 @@ 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
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      batch: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    +    val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message)
    +    val kc = new KafkaCluster(kafkaParams)
    +    val topics = batch.map(o => TopicAndPartition(o.topic, o.partition)).toSet
    +    val leaderMap = kc.findLeaders(topics).fold(
    +      errs => throw new SparkException(errs.mkString("\n")),
    +      ok => ok
    +    )
    +    val rddParts = batch.zipWithIndex.map { case (o, i) =>
    +        val tp = TopicAndPartition(o.topic, o.partition)
    +        val (host, port) = leaderMap(tp)
    +        new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset, host, port)
    +    }.toArray
    +    new KafkaRDD[K, V, U, T, (K, V)](sc, kafkaParams, rddParts, messageHandler)
    +  }
    +
    +  /** 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 leaders Kafka leaders for each offset range in batch
    +   * @param messageHandler function for translating each message into the desired type
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      batch: Array[OffsetRange],
    +      leaders: Array[Leader],
    +      messageHandler: MessageAndMetadata[K, V] => R
    +  ): RDD[R] with HasOffsetRanges = {
    +    val leaderMap = leaders.map(l => (l.topic, l.partition) -> (l.host, l.port)).toMap
    +    val rddParts = batch.zipWithIndex.map { case (o, i) =>
    +        val (host, port) = leaderMap((o.topic, o.partition))
    +        new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset, host, port)
    +    }.toArray
    +
    +    new KafkaRDD[K, V, U, T, R](sc, kafkaParams, rddParts, messageHandler)
    +  }
    +
    +  /**
    +   * This stream can guarantee that each message from Kafka is included in transformations
    +   * (as opposed to output actions) exactly once, even in most failure situations.
    +   *
    +   * Points to note:
    +   *
    +   * Failure Recovery - You must checkpoint this stream, or save offsets yourself and provide them
    +   * as the fromOffsets parameter on restart.
    +   * Kafka must have sufficient log retention to obtain messages after failure.
    +   *
    +   * Getting offsets from the stream - see programming guide
    +   *
    +.  * Zookeeper - This does not use Zookeeper to store offsets.  For interop with Kafka monitors
    +   * that depend on Zookeeper, you must store offsets in ZK yourself.
    +   *
    +   * End-to-end semantics - This does not guarantee that any output operation will push each record
    +   * exactly once. To ensure end-to-end exactly-once semantics (that is, receiving exactly once and
    +   * outputting exactly once), you have to either ensure that the output operation is
    +   * idempotent, or transactionally store offsets with the output. See the programming guide for
    +   * more details.
    +   *
    +   * @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
    +   */
    +  @Experimental
    +  def createNewStream[
    +    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 stream can guarantee that each message from Kafka is included in transformations
    +   * (as opposed to output actions) exactly once, even in most failure situations.
    +   *
    +   * Points to note:
    +   *
    +   * Failure Recovery - You must checkpoint this stream.
    +   * Kafka must have sufficient log retention to obtain messages after failure.
    +   *
    +   * Getting offsets from the stream - see programming guide
    +   *
    +.  * Zookeeper - This does not use Zookeeper to store offsets.  For interop with Kafka monitors
    +   * that depend on Zookeeper, you must store offsets in ZK yourself.
    +   *
    +   * End-to-end semantics - This does not guarantee that any output operation will push each record
    +   * exactly once. To ensure end-to-end exactly-once semantics (that is, receiving exactly once and
    +   * outputting exactly once), you have to ensure that the output operation is idempotent.
    +   *
    +   * @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
    +   */
    +  @Experimental
    +  def createNewStream[
    +    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 {
    --- End diff --
    
    The method you're commenting on doesn't take an offset, and the doc already
    says it doesn't store offsets in zookeeper, so there is no other valid
    offset to be used.  It will just use whatever is specified by
    auto.offset.reset
    On Feb 1, 2015 8:09 PM, "Saisai Shao" <no...@github.com> wrote:
    
    > In
    > external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
    > <https://github.com/apache/spark/pull/3798#discussion_r23904486>:
    >
    > > +      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 {
    >
    > Is this "auto.offset.reset" just a hint for out-of-range re-position of
    > offset, or just a immediate action to seek the offset to beginning or end?
    > As I know this configuration is just a hint, will not be effective when the
    > offset is valid, I'm not sure here your behavior, is the same as Kafka?
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798/files#r23904486>.
    >


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71616661
  
    For everyone's reference, I have updated the JIRA with this design doc. 
    https://docs.google.com/document/d/1IuvZhg9cOueTf1mq4qwc1fhPb5FVcaRLcyjrtG4XU1k/edit?usp=sharing
    
    Please take a look at the design. Once we have converged on the design, this PR can be updated accordingly. 



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22330325
  
    --- Diff: external/kafka/pom.xml ---
    @@ -44,7 +44,7 @@
         <dependency>
           <groupId>org.apache.kafka</groupId>
           <artifactId>kafka_${scala.binary.version}</artifactId>
    -      <version>0.8.0</version>
    --- End diff --
    
    It's due to the consumer offset management api only being available in 0.8.1
    
    https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI
    



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23823438
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Leader.scala ---
    @@ -0,0 +1,68 @@
    +/*
    + * 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.streaming.kafka
    +
    +import kafka.common.TopicAndPartition
    +
    +/** Host info for the leader of a Kafka TopicAndPartition */
    +
    +trait Leader {
    +    /** kafka topic name */
    +  def topic: String
    +
    +  /** kafka partition id */
    +  def partition: Int
    +
    +  /** kafka hostname */
    +  def host: String
    +  
    +  /** kafka host's port */
    +  def port: Int
    +}
    +
    +private class LeaderImpl(
    +  override val topic: String,
    +  override val partition: Int,
    +  override val host: String,
    +  override val port: Int
    +) extends Leader
    +
    +object Leader {
    +  def create(
    +    topic: String,
    +    partition: Int,
    +    host: String,
    +    port: Int): Leader =
    +    new LeaderImpl(
    +      topic,
    +      partition,
    +      host,
    +      port)
    +
    +    def create(
    --- End diff --
    
    incorrect indentation.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-69446001
  
    I went ahead and implemented locality and checkpointing of generated rdds.
    Couple of points
    
    - still depends on SPARK-4014 eventually being merged, for efficiency's
    sake.
    
    - I ran into classloader / class not found issues trying to checkpoint
    KafkaRDDPartition directly.  Current solution is to transform them to/from
    tuples, ugly but it works.  If you know what the issue is there, let me
    know.
    
    - I've got a use case that requires overriding the compute method on the
    DStream (basically, modifying offsets to a fixed delay rather than "now").
    I'm assuming you'd prefer a user supplied function to do the transformation
    rather than subclassing, but let me know.
    
    On Mon, Jan 5, 2015 at 7:59 PM, Tathagata Das <no...@github.com>
    wrote:
    
    > Great! Keep me posted.
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798#issuecomment-68815205>.
    >


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72779779
  
    Why did you choose the parameters "metadata.broker.list" and the "bootstrap.servers" as the required kafka params? I looked at the Kafka docs, and it says that for consumers, the necessary properties are 
    "zookeeper.connect"  and "group.id". And intuitively the application is consuming, so the consumer configs should apply (not "group.id", but "zookeeper.connect"). So our interface should also require "zookeeper.connect" and not other two. Isnt it?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23906731
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala ---
    @@ -0,0 +1,220 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.reflect.{classTag, ClassTag}
    +
    +import org.apache.spark.{Logging, Partition, SparkContext, SparkException, TaskContext}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.NextIterator
    +
    +import java.util.Properties
    +import kafka.api.{FetchRequestBuilder, FetchResponse}
    +import kafka.common.{ErrorMapping, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +import kafka.message.{MessageAndMetadata, MessageAndOffset}
    +import kafka.serializer.Decoder
    +import kafka.utils.VerifiableProperties
    +
    +/**
    + * 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 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 KafkaRDDPartition 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
    + */
    +private[spark]
    +class KafkaRDD[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag] private[spark] (
    +    sc: SparkContext,
    +    kafkaParams: Map[String, String],
    +    private[spark] val batch: Array[KafkaRDDPartition],
    --- End diff --
    
    Actually, this not the desired way to create RDDs. The partition objects are generated created by the RDD itself, and not provided from outside. Although this is not a written hard rule, it is generally the norm followed by all types of RDDs. Example: 
    https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala#L65


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22362167
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaRDD.scala ---
    @@ -0,0 +1,157 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.reflect.{classTag, ClassTag}
    +
    +import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.NextIterator
    +
    +import java.util.Properties
    +import kafka.api.FetchRequestBuilder
    +import kafka.common.{ErrorMapping, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +import kafka.message.{MessageAndMetadata, MessageAndOffset}
    +import kafka.serializer.Decoder
    +import kafka.utils.VerifiableProperties
    +
    +case class KafkaRDDPartition(
    +  override val index: Int,
    +  topic: String,
    +  partition: Int,
    +  fromOffset: Long,
    +  untilOffset: Long
    +) extends Partition
    +
    +/** A batch-oriented interface for consuming from Kafka.
    +  * Each given Kafka topic/partition corresponds to an RDD partition.
    +  * Starting and ending offsets are specified in advance,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the batch
    +  * @param untilOffsets per-topic/partition Kafka offsets defining the (exclusive)
    +  *  ending point of the batch
    +  * @param messageHandler function for translating each message into the desired type
    +  */
    +class KafkaRDD[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    sc: SparkContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    val untilOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R
    +  ) extends RDD[R](sc, Nil) with Logging {
    +
    +  assert(fromOffsets.keys == untilOffsets.keys,
    +    "Must provide both from and until offsets for each topic/partition")
    +
    +  override def getPartitions: Array[Partition] = fromOffsets.zipWithIndex.map { kvi =>
    +    val ((tp, from), index) = kvi
    +    new KafkaRDDPartition(index, tp.topic, tp.partition, from, untilOffsets(tp))
    +  }.toArray
    +
    +  override def compute(thePart: Partition, context: TaskContext) = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    if (part.fromOffset >= part.untilOffset) {
    +      log.warn("Beginning offset is same or after ending offset " +
    +        s"skipping ${part.topic} ${part.partition}")
    +      Iterator.empty
    +    } else {
    +      new NextIterator[R] {
    +        context.addTaskCompletionListener{ context => closeIfNeeded() }
    +
    +        val kc = new KafkaCluster(kafkaParams)
    +        log.info(s"Computing topic ${part.topic}, partition ${part.partition} " +
    +          s"offsets ${part.fromOffset} -> ${part.untilOffset}")
    +        val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[K]]
    +        val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[V]]
    +        val consumer: SimpleConsumer = kc.connectLeader(part.topic, part.partition).fold(
    +          errs => throw new Exception(
    +            s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " +
    +              errs.mkString("\n")),
    +          consumer => consumer
    +        )
    +        var requestOffset = part.fromOffset
    +        var iter: Iterator[MessageAndOffset] = null
    +
    +        override def close() = consumer.close()
    +
    +        override def getNext: R = {
    +          if (iter == null || !iter.hasNext) {
    +            val req = new FetchRequestBuilder().
    +              addFetch(part.topic, part.partition, requestOffset, kc.config.fetchMessageMaxBytes).
    +              build()
    +            val resp = consumer.fetch(req)
    +            if (resp.hasError) {
    +              val err = resp.errorCode(part.topic, part.partition)
    +              if (err == ErrorMapping.LeaderNotAvailableCode ||
    +                err == ErrorMapping.NotLeaderForPartitionCode) {
    +                log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " +
    +                  s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms")
    +                Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +              }
    +              // Let normal rdd retry sort out reconnect attempts
    +              throw ErrorMapping.exceptionFor(err)
    +            }
    +            iter = resp.messageSet(part.topic, part.partition)
    +              .iterator
    +              .dropWhile(_.offset < requestOffset)
    --- End diff --
    
    Why is there a drop here? Doesnt the response return messages for the requested offset?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72787965
  
    I think the simplest solution is to assign zookeeper.connect. But you are assigning it in KafkaCluster lines 338 - 345. So why is this warning being thrown?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23418148
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala ---
    @@ -0,0 +1,123 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.annotation.tailrec
    +import scala.reflect.{classTag, ClassTag}
    +
    +import kafka.common.TopicAndPartition
    +import kafka.message.MessageAndMetadata
    +import kafka.serializer.Decoder
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.rdd.kafka.{KafkaCluster, KafkaRDD}
    +import org.apache.spark.streaming.{StreamingContext, Time}
    +import org.apache.spark.streaming.dstream._
    +
    +/** A stream of {@link org.apache.spark.rdd.kafka.KafkaRDD} where
    +  * each given Kafka topic/partition corresponds to an RDD partition.
    +  * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages
    +  * per second that each '''partition''' will accept.
    +  * Starting offsets are specified in advance,
    +  * and this DStream is not responsible for committing offsets,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the stream
    +  * @param messageHandler function for translating each message into the desired type
    +  * @param maxRetries maximum number of times in a row to retry getting leaders' offsets
    +  */
    +class DeterministicKafkaInputDStream[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    @transient ssc_ : StreamingContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R,
    +    maxRetries: Int = 1
    +) extends InputDStream[R](ssc_) with Logging {
    +
    +  private val kc = new KafkaCluster(kafkaParams)
    +
    +  private val maxMessagesPerPartition: Option[Long] = {
    +    val ratePerSec = ssc.sparkContext.getConf.getInt("spark.streaming.receiver.maxRate", 0)
    +    if (ratePerSec > 0) {
    +      val secsPerBatch = ssc.graph.batchDuration.milliseconds.toDouble / 1000
    +      Some((secsPerBatch * ratePerSec).toLong)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  // TODO based on the design of InputDStream's lastValidTime, it appears there isn't a
    +  // thread safety concern with private mutable state, but is this certain?
    +  private var currentOffsets = fromOffsets
    +
    +  @tailrec
    +  private def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, Long] = {
    +    val o = kc.getLatestLeaderOffsets(currentOffsets.keys.toSet)
    +    // Either.fold would confuse @tailrec, do it manually
    +    if (o.isLeft) {
    +      val err = o.left.get.toString
    +      if (retries <= 0) {
    +        throw new Exception(err)
    +      } else {
    +        log.error(err)
    +        Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +        latestLeaderOffsets(retries - 1)
    +      }
    +    } else {
    +      o.right.get
    +    }
    +  }
    +
    +  private def clamp(leaderOffsets: Map[TopicAndPartition, Long]): Map[TopicAndPartition, Long] = {
    +    maxMessagesPerPartition.map { mmp =>
    +      leaderOffsets.map { kv =>
    +        val (k, v) = kv
    +        val curr = currentOffsets(k)
    +        val diff = v - curr
    +        if (diff > mmp) (k, curr + mmp) else (k, v)
    +      }
    +    }.getOrElse(leaderOffsets)
    +  }
    +
    +  override def compute(validTime: Time): Option[KafkaRDD[K, V, U, T, R]] = {
    --- End diff --
    
    I like the approach, but like @tdas said above, what happens on driver failure? We'd end up with different batches right? We'd have to persist the batching information somehow to ensure that we can regenerate the original batches.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24019631
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,174 @@ 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    +    val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message)
    +    val kc = new KafkaCluster(kafkaParams)
    +    val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet
    +    val leaders = kc.findLeaders(topics).fold(
    +      errs => throw new SparkException(errs.mkString("\n")),
    +      ok => ok
    +    )
    +    new KafkaRDD[K, V, U, T, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler)
    +  }
    +
    +  /** 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   * @param leaders Kafka leaders for each offset range in batch
    +   * @param messageHandler function for translating each message into the desired type
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange],
    +      leaders: Array[Leader],
    --- End diff --
    
    That's correct on both counts.
    
    If you don't provide a way for clients to supply offset ranges and leaders at the same, you're forcing twice the number of remote calls (because the usual way to get the end of the offset range is to talk to the leader).
    
    Yes, there's no way for people to actually use this currently unless they have their own copy of the functionality provided by KafkaCluster.  In my case, I'm just going to remove SparkException from KafkaCluster, since it's the only spark dependency, and distribute it as a separate jar under a different namespace.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/3798


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22363680
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala ---
    @@ -0,0 +1,123 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.annotation.tailrec
    +import scala.reflect.{classTag, ClassTag}
    +
    +import kafka.common.TopicAndPartition
    +import kafka.message.MessageAndMetadata
    +import kafka.serializer.Decoder
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.rdd.kafka.{KafkaCluster, KafkaRDD}
    +import org.apache.spark.streaming.{StreamingContext, Time}
    +import org.apache.spark.streaming.dstream._
    +
    +/** A stream of {@link org.apache.spark.rdd.kafka.KafkaRDD} where
    +  * each given Kafka topic/partition corresponds to an RDD partition.
    +  * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages
    +  * per second that each '''partition''' will accept.
    +  * Starting offsets are specified in advance,
    +  * and this DStream is not responsible for committing offsets,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the stream
    +  * @param messageHandler function for translating each message into the desired type
    +  * @param maxRetries maximum number of times in a row to retry getting leaders' offsets
    +  */
    +class DeterministicKafkaInputDStream[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    @transient ssc_ : StreamingContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R,
    +    maxRetries: Int = 1
    +) extends InputDStream[R](ssc_) with Logging {
    +
    +  private val kc = new KafkaCluster(kafkaParams)
    +
    +  private val maxMessagesPerPartition: Option[Long] = {
    +    val ratePerSec = ssc.sparkContext.getConf.getInt("spark.streaming.receiver.maxRate", 0)
    +    if (ratePerSec > 0) {
    +      val secsPerBatch = ssc.graph.batchDuration.milliseconds.toDouble / 1000
    +      Some((secsPerBatch * ratePerSec).toLong)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  // TODO based on the design of InputDStream's lastValidTime, it appears there isn't a
    +  // thread safety concern with private mutable state, but is this certain?
    +  private var currentOffsets = fromOffsets
    +
    +  @tailrec
    +  private def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, Long] = {
    +    val o = kc.getLatestLeaderOffsets(currentOffsets.keys.toSet)
    +    // Either.fold would confuse @tailrec, do it manually
    +    if (o.isLeft) {
    +      val err = o.left.get.toString
    +      if (retries <= 0) {
    +        throw new Exception(err)
    +      } else {
    +        log.error(err)
    +        Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +        latestLeaderOffsets(retries - 1)
    +      }
    +    } else {
    +      o.right.get
    +    }
    +  }
    +
    +  private def clamp(leaderOffsets: Map[TopicAndPartition, Long]): Map[TopicAndPartition, Long] = {
    +    maxMessagesPerPartition.map { mmp =>
    +      leaderOffsets.map { kv =>
    +        val (k, v) = kv
    +        val curr = currentOffsets(k)
    +        val diff = v - curr
    +        if (diff > mmp) (k, curr + mmp) else (k, v)
    --- End diff --
    
    a cleaner logic is to use min(curr +mmp, v)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71786989
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/26206/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72592281
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/26593/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23728345
  
    --- Diff: external/kafka/src/test/scala/org/apache/spark/rdd/kafka/KafkaRDDSuite.scala ---
    @@ -0,0 +1,93 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.util.Random
    +
    +import kafka.serializer.StringDecoder
    +import kafka.common.TopicAndPartition
    +import org.scalatest.BeforeAndAfter
    +
    +import org.apache.spark._
    +import org.apache.spark.SparkContext._
    +import org.apache.spark.streaming.kafka.KafkaStreamSuiteBase
    +
    +class KafkaRDDSuite extends KafkaStreamSuiteBase with BeforeAndAfter {
    +  var sc: SparkContext = _
    +  before {
    +    setupKafka()
    +  }
    +
    +  after {
    +    if (sc != null) {
    +      sc.stop
    +      sc = null
    +    }
    +    tearDownKafka()
    +  }
    +
    +  test("Kafka RDD") {
    --- End diff --
    
    Can you explain with inline comments about what this test does? 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by helena <gi...@git.apache.org>.
Github user helena commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22648091
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala ---
    @@ -0,0 +1,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.streaming.kafka
    +
    +import scala.annotation.tailrec
    +import scala.reflect.{classTag, ClassTag}
    +
    +import kafka.common.TopicAndPartition
    +import kafka.message.MessageAndMetadata
    +import kafka.serializer.Decoder
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.rdd.kafka.{KafkaCluster, KafkaRDD}
    +import org.apache.spark.streaming.{StreamingContext, Time}
    +import org.apache.spark.streaming.dstream._
    +
    +/** A stream of {@link org.apache.spark.rdd.kafka.KafkaRDD} where
    +  * each given Kafka topic/partition corresponds to an RDD partition.
    +  * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages
    +  * per second that each '''partition''' will accept.
    +  * Starting offsets are specified in advance,
    +  * and this DStream is not responsible for committing offsets,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the stream
    +  * @param messageHandler function for translating each message into the desired type
    +  * @param maxRetries maximum number of times in a row to retry getting leaders' offsets
    +  */
    +class DeterministicKafkaInputDStream[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    @transient ssc_ : StreamingContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R,
    +    maxRetries: Int = 1
    +) extends InputDStream[R](ssc_) with Logging {
    +
    +  private val kc = new KafkaCluster(kafkaParams)
    +
    +  private val maxMessagesPerPartition: Option[Long] = {
    +    val ratePerSec = context.sparkContext.getConf.getInt("spark.streaming.receiver.maxRate", 0)
    +    if (ratePerSec > 0) {
    +      val secsPerBatch = context.graph.batchDuration.milliseconds.toDouble / 1000
    +      Some((secsPerBatch * ratePerSec).toLong)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  private var currentOffsets = fromOffsets
    +
    +  @tailrec
    +  private def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, Long] = {
    +    val o = kc.getLatestLeaderOffsets(currentOffsets.keys.toSet)
    --- End diff --
    
    currentOffsets.keySet


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23904263
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala ---
    @@ -0,0 +1,150 @@
    +/*
    + * 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.streaming.kafka
    +
    +
    +import scala.annotation.tailrec
    +import scala.collection.mutable
    +import scala.reflect.{classTag, ClassTag}
    +
    +import kafka.common.TopicAndPartition
    +import kafka.message.MessageAndMetadata
    +import kafka.serializer.Decoder
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset
    +import org.apache.spark.streaming.{StreamingContext, Time}
    +import org.apache.spark.streaming.dstream._
    +
    +/** A stream of {@link org.apache.spark.rdd.kafka.KafkaRDD} where
    +  * each given Kafka topic/partition corresponds to an RDD partition.
    +  * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages
    +  * per second that each '''partition''' will accept.
    +  * Starting offsets are specified in advance,
    +  * and this DStream is not responsible for committing offsets,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the stream
    +  * @param messageHandler function for translating each message into the desired type
    +  * @param maxRetries maximum number of times in a row to retry getting leaders' offsets
    +  */
    +private[streaming]
    +class DeterministicKafkaInputDStream[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    @transient ssc_ : StreamingContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R,
    +    maxRetries: Int
    +) extends InputDStream[R](ssc_) with Logging {
    +
    +  protected[streaming] override val checkpointData =
    +    new DeterministicKafkaInputDStreamCheckpointData
    +
    +  protected val kc = new KafkaCluster(kafkaParams)
    +
    +  protected val maxMessagesPerPartition: Option[Long] = {
    +    val ratePerSec = context.sparkContext.getConf.getInt("spark.streaming.receiver.maxRate", 0)
    +    if (ratePerSec > 0) {
    +      val secsPerBatch = context.graph.batchDuration.milliseconds.toDouble / 1000
    +      Some((secsPerBatch * ratePerSec).toLong)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  protected var currentOffsets = fromOffsets
    +
    +  @tailrec
    +  protected final def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, LeaderOffset] = {
    +    val o = kc.getLatestLeaderOffsets(currentOffsets.keySet)
    +    // Either.fold would confuse @tailrec, do it manually
    +    if (o.isLeft) {
    +      val err = o.left.get.toString
    +      if (retries <= 0) {
    +        throw new Exception(err)
    +      } else {
    +        log.error(err)
    +        Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +        latestLeaderOffsets(retries - 1)
    +      }
    +    } else {
    +      o.right.get
    +    }
    +  }
    +
    +  protected def clamp(
    +    leaderOffsets: Map[TopicAndPartition, LeaderOffset]): Map[TopicAndPartition, LeaderOffset] = {
    +    maxMessagesPerPartition.map { mmp =>
    +      leaderOffsets.map { case (tp, lo) =>
    +        tp -> lo.copy(offset = Math.min(currentOffsets(tp) + mmp, lo.offset))
    +      }
    +    }.getOrElse(leaderOffsets)
    +  }
    +
    +  override def compute(validTime: Time): Option[KafkaRDD[K, V, U, T, R]] = {
    +    val untilOffsets = clamp(latestLeaderOffsets(maxRetries))
    +    val rdd = KafkaRDD[K, V, U, T, R](
    +      context.sparkContext, kafkaParams, currentOffsets, untilOffsets, messageHandler)
    +
    +    currentOffsets = untilOffsets.map(kv => kv._1 -> kv._2.offset)
    +    Some(rdd)
    +  }
    +
    +  override def start(): Unit = {
    +  }
    +
    +  def stop(): Unit = {
    +  }
    +
    +  private[streaming]
    +  class DeterministicKafkaInputDStreamCheckpointData extends DStreamCheckpointData(this) {
    +    def batchForTime = data.asInstanceOf[mutable.HashMap[
    +      Time, Array[(Int, String, Int, Long, Long, String, Int)]]]
    --- End diff --
    
    Can we make this type `(Int, String, Int, Long, Long, String, Int)` a specific type name, it is not so straightforward to understand the meaning of this large tuple.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23907051
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,249 @@ 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
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    --- End diff --
    
    Good catch!


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-68092152
  
    Can one of the admins verify this patch?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23726607
  
    --- 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.
    --- End diff --
    
    Docs must first start with what the operation does, not what it does not do :) You have to first specify what the operation does (see createStream in this class). I suggest somethig like this. 
    ```
    Create an input stream that pulls messages exactly-once from Kafka. Compared to `createStream`, stream created by this will guarantee that each record from Kafka is received by the application only once (despite failures, etc.).
    
    Note: This does not guarantee that any output operation will push each record only once. To ensure end-to-end exactly-once semantics (that is, receiving exactly once and outputting exactly once), you have to ensure that the output operation is either idempotent or transactional. See the programming guide for more details.
    
    ```



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72760900
  
    To put it another way, the type you return has to be public.
    
    If you return a public abstract class, what are you going to do when
    someone else subclasses it?
    
    Making it a final concrete class doesn't have that issue.
    On Feb 3, 2015 5:34 PM, "Tathagata Das" <no...@github.com> wrote:
    
    > But of course there can be multiple implementations! For example, there is
    > both KafkaReceiver and ReliableKafkaReceiver. The second was introduced so
    > that the code path for existing uses is not disturbed when we are
    > introducing experimental code paths that are optionally enabled with flags.
    > We never envisioned that happening, but when it occur, we could do this
    > because the KafkaReceiver was not exposed, only the Receiver interface was
    > exposed.
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798#issuecomment-72759859>.
    >


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24021039
  
    --- Diff: external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala ---
    @@ -0,0 +1,99 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.util.Random
    +
    +import kafka.serializer.StringDecoder
    +import kafka.common.TopicAndPartition
    +import org.scalatest.BeforeAndAfter
    +
    +import org.apache.spark._
    +import org.apache.spark.SparkContext._
    +
    +class KafkaRDDSuite extends KafkaStreamSuiteBase with BeforeAndAfter {
    +  var sc: SparkContext = _
    +  before {
    +    setupKafka()
    +  }
    +
    +  after {
    +    if (sc != null) {
    +      sc.stop
    +      sc = null
    +    }
    +    tearDownKafka()
    +  }
    +
    +  test("Kafka RDD") {
    +    val sparkConf = new SparkConf().setMaster("local[4]").setAppName(this.getClass.getSimpleName)
    +    sc = new SparkContext(sparkConf)
    +    val topic = "topic1"
    +    val sent = Map("a" -> 5, "b" -> 3, "c" -> 10)
    +    createTopic(topic)
    +    produceAndSendMessage(topic, sent)
    +
    +    val kafkaParams = Map("metadata.broker.list" -> s"localhost:$brokerPort",
    +      "group.id" -> s"test-consumer-${Random.nextInt(10000)}")
    +
    +    val kc = new KafkaCluster(kafkaParams)
    +
    +    val rdd = getRdd(kc, Set(topic))
    +    // this is the "lots of messages" case
    +    // make sure we get all of them
    +    assert(rdd.isDefined)
    +    assert(rdd.get.count === sent.values.sum)
    +
    +    kc.setConsumerOffsets(
    --- End diff --
    
    See previous answer.
    
    Also, there's nothing inherently wrong with keeping offsets in ZK for the idempotent case.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by dibbhatt <gi...@git.apache.org>.
Github user dibbhatt commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23889636
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala ---
    @@ -0,0 +1,220 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.reflect.{classTag, ClassTag}
    +
    +import org.apache.spark.{Logging, Partition, SparkContext, SparkException, TaskContext}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.NextIterator
    +
    +import java.util.Properties
    +import kafka.api.{FetchRequestBuilder, FetchResponse}
    +import kafka.common.{ErrorMapping, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +import kafka.message.{MessageAndMetadata, MessageAndOffset}
    +import kafka.serializer.Decoder
    +import kafka.utils.VerifiableProperties
    +
    +/**
    + * 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 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 KafkaRDDPartition 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
    + */
    +private[spark]
    +class KafkaRDD[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag] private[spark] (
    +    sc: SparkContext,
    +    kafkaParams: Map[String, String],
    +    private[spark] val batch: Array[KafkaRDDPartition],
    +    messageHandler: MessageAndMetadata[K, V] => R
    +  ) extends RDD[R](sc, Nil) with Logging with HasOffsetRanges {
    +
    +  def offsetRanges: Array[OffsetRange] = batch.asInstanceOf[Array[OffsetRange]]
    +
    +  override def getPartitions: Array[Partition] = batch.asInstanceOf[Array[Partition]]
    +
    +  override def getPreferredLocations(thePart: Partition): Seq[String] = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    // TODO is additional hostname resolution necessary here
    +    Seq(part.host)
    +  }
    +
    +  private def errBeginAfterEnd(part: KafkaRDDPartition): String =
    +    s"Beginning offset ${part.fromOffset} is after the ending offset ${part.untilOffset} " +
    +      s"for topic ${part.topic} partition ${part.partition}. " +
    +      "You either provided an invalid fromOffset, or the Kafka topic has been damaged"
    +
    +  private def errRanOutBeforeEnd(part: KafkaRDDPartition): String =
    +    s"Ran out of messages before reaching ending offset ${part.untilOffset} " +
    +    s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." +
    +    " This should not happen, and indicates that messages may have been lost"
    +
    +  private def errOvershotEnd(itemOffset: Long, part: KafkaRDDPartition): String =
    +    s"Got ${itemOffset} > ending offset ${part.untilOffset} " +
    +    s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." +
    +    " This should not happen, and indicates a message may have been skipped"
    +
    +  override def compute(thePart: Partition, context: TaskContext): Iterator[R] = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    assert(part.fromOffset <= part.untilOffset, errBeginAfterEnd(part))
    +    if (part.fromOffset == part.untilOffset) {
    +      log.warn("Beginning offset ${part.fromOffset} is the same as ending offset " +
    +        s"skipping ${part.topic} ${part.partition}")
    +      Iterator.empty
    +    } else {
    +      new KafkaRDDIterator(part, context)
    +    }
    +  }
    +
    +  private class KafkaRDDIterator(
    +      part: KafkaRDDPartition,
    +      context: TaskContext) extends NextIterator[R] {
    +
    +    context.addTaskCompletionListener{ context => closeIfNeeded() }
    +
    +    log.info(s"Computing topic ${part.topic}, partition ${part.partition} " +
    +      s"offsets ${part.fromOffset} -> ${part.untilOffset}")
    +
    +    val kc = new KafkaCluster(kafkaParams)
    +    val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +      .newInstance(kc.config.props)
    +      .asInstanceOf[Decoder[K]]
    +    val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +      .newInstance(kc.config.props)
    +      .asInstanceOf[Decoder[V]]
    +    val consumer = connectLeader
    +    var requestOffset = part.fromOffset
    +    var iter: Iterator[MessageAndOffset] = null
    +
    +    // The idea is to use the provided preferred host, except on task retry atttempts,
    +    // to minimize number of kafka metadata requests
    +    private def connectLeader: SimpleConsumer = {
    +      if (context.attemptNumber > 0) {
    +        kc.connectLeader(part.topic, part.partition).fold(
    +          errs => throw new SparkException(
    +            s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " +
    +              errs.mkString("\n")),
    +          consumer => consumer
    +        )
    +      } else {
    +        kc.connect(part.host, part.port)
    +      }
    +    }
    +
    +    private def handleFetchErr(resp: FetchResponse) {
    +      if (resp.hasError) {
    +        val err = resp.errorCode(part.topic, part.partition)
    +        if (err == ErrorMapping.LeaderNotAvailableCode ||
    +          err == ErrorMapping.NotLeaderForPartitionCode) {
    +          log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " +
    +            s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms")
    +          Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +        }
    +        // Let normal rdd retry sort out reconnect attempts
    +        throw ErrorMapping.exceptionFor(err)
    +      }
    +    }
    +
    --- End diff --
    
    One of the very common Kafka error may come which is OFFSET_OUT_OF_RANGE . Then Normal RDD retry may not solve this problem. You may need to explicitly either start from earlier offset or latest offset to solve this issue.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71975810
  
      [Test build #26302 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26302/consoleFull) for   PR 3798 at commit [`80fd6ae`](https://github.com/apache/spark/commit/80fd6aefde56299ba7651560e52df20b884c474e).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by helena <gi...@git.apache.org>.
Github user helena commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22662751
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaRDD.scala ---
    @@ -0,0 +1,157 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.reflect.{classTag, ClassTag}
    +
    +import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.NextIterator
    +
    +import java.util.Properties
    +import kafka.api.FetchRequestBuilder
    +import kafka.common.{ErrorMapping, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +import kafka.message.{MessageAndMetadata, MessageAndOffset}
    +import kafka.serializer.Decoder
    +import kafka.utils.VerifiableProperties
    +
    +case class KafkaRDDPartition(
    +  override val index: Int,
    +  topic: String,
    +  partition: Int,
    +  fromOffset: Long,
    +  untilOffset: Long
    +) extends Partition
    +
    +/** A batch-oriented interface for consuming from Kafka.
    +  * Each given Kafka topic/partition corresponds to an RDD partition.
    +  * Starting and ending offsets are specified in advance,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the batch
    +  * @param untilOffsets per-topic/partition Kafka offsets defining the (exclusive)
    +  *  ending point of the batch
    +  * @param messageHandler function for translating each message into the desired type
    +  */
    +class KafkaRDD[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    sc: SparkContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    val untilOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R
    +  ) extends RDD[R](sc, Nil) with Logging {
    +
    +  assert(fromOffsets.keys == untilOffsets.keys,
    +    "Must provide both from and until offsets for each topic/partition")
    +
    +  override def getPartitions: Array[Partition] = fromOffsets.zipWithIndex.map { kvi =>
    +    val ((tp, from), index) = kvi
    +    new KafkaRDDPartition(index, tp.topic, tp.partition, from, untilOffsets(tp))
    +  }.toArray
    +
    +  override def compute(thePart: Partition, context: TaskContext) = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    if (part.fromOffset >= part.untilOffset) {
    +      log.warn("Beginning offset is same or after ending offset " +
    +        s"skipping ${part.topic} ${part.partition}")
    +      Iterator.empty
    +    } else {
    +      new NextIterator[R] {
    +        context.addTaskCompletionListener{ context => closeIfNeeded() }
    +
    +        val kc = new KafkaCluster(kafkaParams)
    +        log.info(s"Computing topic ${part.topic}, partition ${part.partition} " +
    +          s"offsets ${part.fromOffset} -> ${part.untilOffset}")
    +        val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[K]]
    +        val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[V]]
    +        val consumer: SimpleConsumer = kc.connectLeader(part.topic, part.partition).fold(
    +          errs => throw new Exception(
    +            s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " +
    +              errs.mkString("\n")),
    +          consumer => consumer
    +        )
    +        var requestOffset = part.fromOffset
    +        var iter: Iterator[MessageAndOffset] = null
    +
    +        override def close() = consumer.close()
    +
    +        override def getNext: R = {
    +          if (iter == null || !iter.hasNext) {
    +            val req = new FetchRequestBuilder().
    +              addFetch(part.topic, part.partition, requestOffset, kc.config.fetchMessageMaxBytes).
    +              build()
    +            val resp = consumer.fetch(req)
    +            if (resp.hasError) {
    +              val err = resp.errorCode(part.topic, part.partition)
    +              if (err == ErrorMapping.LeaderNotAvailableCode ||
    +                err == ErrorMapping.NotLeaderForPartitionCode) {
    +                log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " +
    +                  s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms")
    +                Thread.sleep(kc.config.refreshLeaderBackoffMs)
    --- End diff --
    
    Not without taking time to review the related code but Thread.sleep in async is a no IMHO and in every project I've worked in for the last 6 years (all scala async envs). But I don't ATM.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72925200
  
    Merging this. Thanks so much Cody! 
    There will be a PR to fix a few things here and there soon. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72754442
  
    Like patrick said, I really don't see any reason not to just expose
    KafkaRDD.  You can still hide its constructor without making a superflous
    abstract class, and you can still make another subclass of KafkaRDD later
    if you need to.
    
    Even if you don't want the static createRDD method to return a KafkaRDD, we
    can just take the "with HasOffsetRanges" off and people who care about
    getting to the offsets can cast it (they'll have to cast it for the stream
    case anyway)
    
    On Tue, Feb 3, 2015 at 4:30 PM, Tathagata Das <no...@github.com>
    wrote:
    
    > @koeninger <https://github.com/koeninger> Thank you very much for all the
    > changes. They are looking good. Unfortunately the issue with createRDD
    > returning RDD[] with OffsetRange (i.e., the issue that @pwendell
    > <https://github.com/pwendell> raised) could be a problem in the future in
    > terms of binary compatibility. Basically, we have not used such things in
    > the rest of Spark to keep things as Java-friendly and binary compatible as
    > possible. Also in the generated Java doc this looks scary. So here is an
    > alternate suggestion that I am trying to implement on top of your PR
    > branch. How about this. We effectively combine KafkaRDD and HasOffsetRanges
    > into a abstract class.
    >
    > abstract class KafkaRDD[T] (val offsetRanges: Array[OffsetRanges], sc: SparkContext)
    >    extends RDD[T](sc, Nil)
    >
    > private[kafka]
    > class KafkaRDDImpl[K, V, KD, VD, R] extends KafkaRDD[R] {
    >   ...
    > }
    >
    > KafkaUtils.createRDD(...simple one without messageHandler...): KafkaRDD[(K, V)] = {
    >    // return KafkaRDDImpl[K, V, KD, VD, (K, V)]
    > }
    >
    > KafkaUtils.createRDD(...simple one WITH messageHandler...): KafkaRDD[R] = {
    >    // return KafkaRDDImpl[K, V, KD, VD, R]
    > }
    >
    >
    > Advantages
    >
    >    - No binary compatibility issues
    >    - Easy to read from Java
    >    - KafkaRDD implementation and constructor all hidden as before
    >    - Can still extend KafkaRDD to expose more methods in future.
    >
    > What do you think?
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798#issuecomment-72749763>.
    >


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23746262
  
    --- 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.
    --- End diff --
    
    Cool, changing it to createNewStream for now


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23418815
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala ---
    @@ -0,0 +1,123 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.annotation.tailrec
    +import scala.reflect.{classTag, ClassTag}
    +
    +import kafka.common.TopicAndPartition
    +import kafka.message.MessageAndMetadata
    +import kafka.serializer.Decoder
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.rdd.kafka.{KafkaCluster, KafkaRDD}
    +import org.apache.spark.streaming.{StreamingContext, Time}
    +import org.apache.spark.streaming.dstream._
    +
    +/** A stream of {@link org.apache.spark.rdd.kafka.KafkaRDD} where
    +  * each given Kafka topic/partition corresponds to an RDD partition.
    +  * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages
    +  * per second that each '''partition''' will accept.
    +  * Starting offsets are specified in advance,
    +  * and this DStream is not responsible for committing offsets,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the stream
    +  * @param messageHandler function for translating each message into the desired type
    +  * @param maxRetries maximum number of times in a row to retry getting leaders' offsets
    +  */
    +class DeterministicKafkaInputDStream[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    @transient ssc_ : StreamingContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R,
    +    maxRetries: Int = 1
    +) extends InputDStream[R](ssc_) with Logging {
    +
    +  private val kc = new KafkaCluster(kafkaParams)
    +
    +  private val maxMessagesPerPartition: Option[Long] = {
    +    val ratePerSec = ssc.sparkContext.getConf.getInt("spark.streaming.receiver.maxRate", 0)
    +    if (ratePerSec > 0) {
    +      val secsPerBatch = ssc.graph.batchDuration.milliseconds.toDouble / 1000
    +      Some((secsPerBatch * ratePerSec).toLong)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  // TODO based on the design of InputDStream's lastValidTime, it appears there isn't a
    +  // thread safety concern with private mutable state, but is this certain?
    +  private var currentOffsets = fromOffsets
    +
    +  @tailrec
    +  private def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, Long] = {
    +    val o = kc.getLatestLeaderOffsets(currentOffsets.keys.toSet)
    +    // Either.fold would confuse @tailrec, do it manually
    +    if (o.isLeft) {
    +      val err = o.left.get.toString
    +      if (retries <= 0) {
    +        throw new Exception(err)
    +      } else {
    +        log.error(err)
    +        Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +        latestLeaderOffsets(retries - 1)
    +      }
    +    } else {
    +      o.right.get
    +    }
    +  }
    +
    +  private def clamp(leaderOffsets: Map[TopicAndPartition, Long]): Map[TopicAndPartition, Long] = {
    +    maxMessagesPerPartition.map { mmp =>
    +      leaderOffsets.map { kv =>
    +        val (k, v) = kv
    +        val curr = currentOffsets(k)
    +        val diff = v - curr
    +        if (diff > mmp) (k, curr + mmp) else (k, v)
    +      }
    +    }.getOrElse(leaderOffsets)
    +  }
    +
    +  override def compute(validTime: Time): Option[KafkaRDD[K, V, U, T, R]] = {
    --- End diff --
    
    Maybe I'm misunderstanding the question, but I already updated the PR a week ago to checkpoint the partitions from generatedRdds.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23972979
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,338 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.util.Random
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    + * Convenience methods for interacting with a Kafka cluster.
    + * @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
    + */
    +private[spark]
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    +  import KafkaCluster.{Err, LeaderOffset}
    +
    +  val seedBrokers: Array[(String, Int)] =
    +    kafkaParams.get("metadata.broker.list")
    +      .orElse(kafkaParams.get("bootstrap.servers"))
    +      .getOrElse(throw new Exception("Must specify metadata.broker.list or bootstrap.servers"))
    +      .split(",").map { hp =>
    +        val hpa = hp.split(":")
    +        (hpa(0), hpa(1).toInt)
    +      }
    +
    +  // ConsumerConfig isn't serializable
    +  @transient private var _config: ConsumerConfig = null
    +
    +  def config: ConsumerConfig = this.synchronized {
    +    if (_config == null) {
    +      _config = KafkaCluster.consumerConfig(kafkaParams)
    +    }
    +    _config
    +  }
    +
    +  def connect(host: String, port: Int): SimpleConsumer =
    +    new SimpleConsumer(host, port, config.socketTimeoutMs,
    +      config.socketReceiveBufferBytes, config.clientId)
    +
    +  def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] =
    +    findLeader(topic, partition).right.map(hp => connect(hp._1, hp._2))
    +
    +  // Metadata api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataAPI
    +  // scalastyle:on
    +
    +  def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, Seq(topic))
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      resp.topicsMetadata.find(_.topic == topic).flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.find(_.partitionId == partition)
    +      }.foreach { pm: PartitionMetadata =>
    +        pm.leader.foreach { leader =>
    +          return Right((leader.host, leader.port))
    +        }
    +      }
    +    }
    +    Left(errs)
    +  }
    +
    +  def findLeaders(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, (String, Int)]] = {
    +    val topics = topicAndPartitions.map(_.topic)
    +    getPartitionMetadata(topics).right.flatMap { tms: Set[TopicMetadata] =>
    +      val result = tms.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.flatMap { pm: PartitionMetadata =>
    +          val tp = TopicAndPartition(tm.topic, pm.partitionId)
    +          if (topicAndPartitions(tp)) {
    +            pm.leader.map { l =>
    +              tp -> (l.host -> l.port)
    +            }
    +          } else {
    +            None
    +          }
    +        }
    +      }.toMap
    +      if (result.keys.size == topicAndPartitions.size) {
    +        Right(result)
    +      } else {
    +        val missing = topicAndPartitions.diff(result.keySet)
    +        val err = new Err
    +        err.append(new Exception(s"Couldn't find leaders for ${missing}"))
    +        Left(err)
    +      }
    +    }
    +  }
    +
    +  def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] =
    +    getPartitionMetadata(topics).right.map { r =>
    +      r.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.map { pm: PartitionMetadata =>
    +          TopicAndPartition(tm.topic, pm.partitionId)
    +        }    
    +      }
    +    }
    +
    +  def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = {
    +    val req = TopicMetadataRequest(
    +      TopicMetadataRequest.CurrentVersion, 0, config.clientId, topics.toSeq)
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      // error codes here indicate missing / just created topic,
    +      // repeating on a different broker wont be useful
    +      return Right(resp.topicsMetadata.toSet)
    +    }
    +    Left(errs)
    +  }
    +
    +  // Leader offset api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI
    +  // scalastyle:on
    +
    +  def getLatestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    --- End diff --
    
    Same indentation comment as before. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23729113
  
    --- 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.
    --- End diff --
    
    It is pretty clear that there is a separation between a stream and a output oepration. Creating a stream does not create an output operation. This oepration creates only a stream that ensures that records are received exactly one in the stream. Whether you want to do the output operation with exactly-once semantics is a distinct questions. Both are necessary for achieving end-to-end exactly-once processing.
    
    So what his name suggests is only creating an exactly-once stream. People are already aware from rest of the documentation in the spark streaming programming guide that output operations are not exactly-once. And if it is not clear, I willing to add whatever that is necessary to make it clear. 
    



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-69651872
  
    The classloader issue was when reading from the checkpoint.
    
    If we want to rely on subclassing, some of the implementation (e.g.
    currentOffsets and latestLeaderOffsets) should probably be made protected.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22366068
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala ---
    @@ -0,0 +1,123 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.annotation.tailrec
    +import scala.reflect.{classTag, ClassTag}
    +
    +import kafka.common.TopicAndPartition
    +import kafka.message.MessageAndMetadata
    +import kafka.serializer.Decoder
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.rdd.kafka.{KafkaCluster, KafkaRDD}
    +import org.apache.spark.streaming.{StreamingContext, Time}
    +import org.apache.spark.streaming.dstream._
    +
    +/** A stream of {@link org.apache.spark.rdd.kafka.KafkaRDD} where
    +  * each given Kafka topic/partition corresponds to an RDD partition.
    +  * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages
    +  * per second that each '''partition''' will accept.
    +  * Starting offsets are specified in advance,
    +  * and this DStream is not responsible for committing offsets,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the stream
    +  * @param messageHandler function for translating each message into the desired type
    +  * @param maxRetries maximum number of times in a row to retry getting leaders' offsets
    +  */
    +class DeterministicKafkaInputDStream[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    @transient ssc_ : StreamingContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R,
    +    maxRetries: Int = 1
    +) extends InputDStream[R](ssc_) with Logging {
    +
    +  private val kc = new KafkaCluster(kafkaParams)
    +
    +  private val maxMessagesPerPartition: Option[Long] = {
    +    val ratePerSec = ssc.sparkContext.getConf.getInt("spark.streaming.receiver.maxRate", 0)
    +    if (ratePerSec > 0) {
    +      val secsPerBatch = ssc.graph.batchDuration.milliseconds.toDouble / 1000
    +      Some((secsPerBatch * ratePerSec).toLong)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  // TODO based on the design of InputDStream's lastValidTime, it appears there isn't a
    +  // thread safety concern with private mutable state, but is this certain?
    +  private var currentOffsets = fromOffsets
    +
    +  @tailrec
    +  private def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, Long] = {
    +    val o = kc.getLatestLeaderOffsets(currentOffsets.keys.toSet)
    +    // Either.fold would confuse @tailrec, do it manually
    +    if (o.isLeft) {
    +      val err = o.left.get.toString
    +      if (retries <= 0) {
    +        throw new Exception(err)
    +      } else {
    +        log.error(err)
    +        Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +        latestLeaderOffsets(retries - 1)
    +      }
    +    } else {
    +      o.right.get
    +    }
    +  }
    +
    +  private def clamp(leaderOffsets: Map[TopicAndPartition, Long]): Map[TopicAndPartition, Long] = {
    +    maxMessagesPerPartition.map { mmp =>
    +      leaderOffsets.map { kv =>
    +        val (k, v) = kv
    +        val curr = currentOffsets(k)
    +        val diff = v - curr
    +        if (diff > mmp) (k, curr + mmp) else (k, v)
    +      }
    +    }.getOrElse(leaderOffsets)
    +  }
    +
    +  override def compute(validTime: Time): Option[KafkaRDD[K, V, U, T, R]] = {
    --- End diff --
    
    Yeah, you're understanding it correctly.
    
    I'm not perfectly happy with the name either - suggestions?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22653905
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaRDD.scala ---
    @@ -0,0 +1,157 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.reflect.{classTag, ClassTag}
    +
    +import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.NextIterator
    +
    +import java.util.Properties
    +import kafka.api.FetchRequestBuilder
    +import kafka.common.{ErrorMapping, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +import kafka.message.{MessageAndMetadata, MessageAndOffset}
    +import kafka.serializer.Decoder
    +import kafka.utils.VerifiableProperties
    +
    +case class KafkaRDDPartition(
    +  override val index: Int,
    +  topic: String,
    +  partition: Int,
    +  fromOffset: Long,
    +  untilOffset: Long
    +) extends Partition
    +
    +/** A batch-oriented interface for consuming from Kafka.
    +  * Each given Kafka topic/partition corresponds to an RDD partition.
    +  * Starting and ending offsets are specified in advance,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the batch
    +  * @param untilOffsets per-topic/partition Kafka offsets defining the (exclusive)
    +  *  ending point of the batch
    +  * @param messageHandler function for translating each message into the desired type
    +  */
    +class KafkaRDD[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    sc: SparkContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    val untilOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R
    +  ) extends RDD[R](sc, Nil) with Logging {
    +
    +  assert(fromOffsets.keys == untilOffsets.keys,
    +    "Must provide both from and until offsets for each topic/partition")
    +
    +  override def getPartitions: Array[Partition] = fromOffsets.zipWithIndex.map { kvi =>
    +    val ((tp, from), index) = kvi
    +    new KafkaRDDPartition(index, tp.topic, tp.partition, from, untilOffsets(tp))
    +  }.toArray
    +
    +  override def compute(thePart: Partition, context: TaskContext) = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    if (part.fromOffset >= part.untilOffset) {
    +      log.warn("Beginning offset is same or after ending offset " +
    +        s"skipping ${part.topic} ${part.partition}")
    +      Iterator.empty
    +    } else {
    +      new NextIterator[R] {
    +        context.addTaskCompletionListener{ context => closeIfNeeded() }
    +
    +        val kc = new KafkaCluster(kafkaParams)
    +        log.info(s"Computing topic ${part.topic}, partition ${part.partition} " +
    +          s"offsets ${part.fromOffset} -> ${part.untilOffset}")
    +        val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[K]]
    +        val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[V]]
    +        val consumer: SimpleConsumer = kc.connectLeader(part.topic, part.partition).fold(
    +          errs => throw new Exception(
    +            s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " +
    +              errs.mkString("\n")),
    +          consumer => consumer
    +        )
    +        var requestOffset = part.fromOffset
    +        var iter: Iterator[MessageAndOffset] = null
    +
    +        override def close() = consumer.close()
    +
    +        override def getNext: R = {
    +          if (iter == null || !iter.hasNext) {
    +            val req = new FetchRequestBuilder().
    +              addFetch(part.topic, part.partition, requestOffset, kc.config.fetchMessageMaxBytes).
    +              build()
    +            val resp = consumer.fetch(req)
    +            if (resp.hasError) {
    +              val err = resp.errorCode(part.topic, part.partition)
    +              if (err == ErrorMapping.LeaderNotAvailableCode ||
    +                err == ErrorMapping.NotLeaderForPartitionCode) {
    +                log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " +
    +                  s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms")
    +                Thread.sleep(kc.config.refreshLeaderBackoffMs)
    --- End diff --
    
    That's only happening in an error case, and is being done to make sure that you don't burn through the maximum number of failed stages while leader election is happening.
    
    I'm pretty sure the existing spark kafka dstreams are doing the exact same thing, because that's what the kafka project's consumer code does in ConsumerFetcherManager.
    
    Do you have an alternative suggestion?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24029286
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,174 @@ 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    +    val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message)
    +    val kc = new KafkaCluster(kafkaParams)
    +    val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet
    +    val leaders = kc.findLeaders(topics).fold(
    +      errs => throw new SparkException(errs.mkString("\n")),
    +      ok => ok
    +    )
    +    new KafkaRDD[K, V, U, T, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler)
    +  }
    +
    +  /** 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   * @param leaders Kafka leaders for each offset range in batch
    +   * @param messageHandler function for translating each message into the desired type
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange],
    +      leaders: Array[Leader],
    +      messageHandler: MessageAndMetadata[K, V] => R
    +  ): RDD[R] with HasOffsetRanges = {
    +
    +    val leaderMap = leaders
    +      .map(l => TopicAndPartition(l.topic, l.partition) -> (l.host, l.port))
    +      .toMap
    +    new KafkaRDD[K, V, U, T, R](sc, kafkaParams, offsetRanges, leaderMap, messageHandler)
    +  }
    +
    +  /**
    +   * This stream can guarantee that each message from Kafka is included in transformations
    +   * (as opposed to output actions) exactly once, even in most failure situations.
    +   *
    +   * Points to note:
    +   *
    +   * Failure Recovery - You must checkpoint this stream, or save offsets yourself and provide them
    +   * as the fromOffsets parameter on restart.
    +   * Kafka must have sufficient log retention to obtain messages after failure.
    +   *
    +   * Getting offsets from the stream - see programming guide
    +   *
    +.  * Zookeeper - This does not use Zookeeper to store offsets.  For interop with Kafka monitors
    +   * that depend on Zookeeper, you must store offsets in ZK yourself.
    +   *
    +   * End-to-end semantics - This does not guarantee that any output operation will push each record
    +   * exactly once. To ensure end-to-end exactly-once semantics (that is, receiving exactly once and
    +   * outputting exactly once), you have to either ensure that the output operation is
    +   * idempotent, or transactionally store offsets with the output. See the programming guide for
    +   * more details.
    +   *
    +   * @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
    +   */
    +  @Experimental
    +  def createNewStream[
    +    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
    +  ): InputDStream[R] = {
    +    new DeterministicKafkaInputDStream[K, V, U, T, R](
    +      ssc, kafkaParams, fromOffsets, messageHandler)
    +  }
    +
    +  /**
    +   * This stream can guarantee that each message from Kafka is included in transformations
    +   * (as opposed to output actions) exactly once, even in most failure situations.
    --- End diff --
    
    So what about saying it provides exactly once semantics subject to the existing fault recovery mechanisms in streaming (i.e. checkpointing). It just conveys basically no information now, it says it might work and might not.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by helena <gi...@git.apache.org>.
Github user helena commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22650033
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,313 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    +  * Convenience methods for interacting with a Kafka cluster.
    +  * @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
    +  */
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    +  import KafkaCluster.Err
    +
    +  val seedBrokers: Array[(String, Int)] =
    +    kafkaParams.get("metadata.broker.list")
    +      .orElse(kafkaParams.get("bootstrap.servers"))
    +      .getOrElse(throw new Exception("Must specify metadata.broker.list or bootstrap.servers"))
    +      .split(",").map { hp =>
    +        val hpa = hp.split(":")
    +        (hpa(0), hpa(1).toInt)
    +      }
    +
    +  // ConsumerConfig isn't serializable
    +  @transient private var _config: ConsumerConfig = null
    +
    +  def config: ConsumerConfig = this.synchronized {
    +    if (_config == null) {
    +      _config = KafkaCluster.consumerConfig(kafkaParams)
    +    }
    +    _config
    +  }
    +
    +  def connect(host: String, port: Int): SimpleConsumer =
    +    new SimpleConsumer(host, port, config.socketTimeoutMs,
    +      config.socketReceiveBufferBytes, config.clientId)
    +
    +  def connect(hostAndPort: (String, Int)): SimpleConsumer =
    +    connect(hostAndPort._1, hostAndPort._2)
    +
    +  def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] =
    +    findLeader(topic, partition).right.map(connect)
    +
    +  def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, Seq(topic))
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      resp.topicsMetadata.find(_.topic == topic).flatMap { t =>
    +        t.partitionsMetadata.find(_.partitionId == partition)
    +      }.foreach { partitionMeta =>
    +        partitionMeta.leader.foreach { leader =>
    +          return Right((leader.host, leader.port))
    +        }
    +      }
    +    }
    +    Left(errs)
    +  }
    +
    +  def findLeaders(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, (String, Int)]] = {
    +    getPartitionMetadata(topicAndPartitions.map(_.topic)).right.flatMap { tms =>
    +      val result = tms.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.flatMap { pm =>
    +          val tp = TopicAndPartition(tm.topic, pm.partitionId)
    +          if (topicAndPartitions(tp)) {
    +            pm.leader.map { l =>
    +              tp -> (l.host -> l.port)
    +            }
    +          } else {
    +            None
    +          }
    +        }
    +      }.toMap
    +      if (result.keys.size == topicAndPartitions.size) {
    +        Right(result)
    +      } else {
    +        val missing = topicAndPartitions.diff(result.keys.toSet)
    +        val err = new Err
    +        err.append(new Exception(s"Couldn't find leaders for ${missing}"))
    +        Left(err)
    +      }
    +    }
    +  }
    +
    +  def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] =
    +    getPartitionMetadata(topics).right.map { r =>
    +      r.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.map { pm =>
    +          TopicAndPartition(tm.topic, pm.partitionId)
    +        }    
    +      }
    +    }
    +
    +  def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, topics.toSeq)
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      // error codes here indicate missing / just created topic,
    +      // repeating on a different broker wont be useful
    +      return Right(resp.topicsMetadata.toSet)
    +    }
    +    Left(errs)
    +  }
    +
    +  def getLatestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime)
    +
    +  def getEarliestLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime)
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long
    +  ): Either[Err, Map[TopicAndPartition, Long]] =
    +    getLeaderOffsets(topicAndPartitions, before, 1).right.map { r =>
    +      r.map { kv =>
    +        // mapValues isnt serializable, see SI-7005
    +        kv._1 -> kv._2.head
    +      }
    +    }
    +
    +  private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] =
    +    m.groupBy(_._2).map { kv =>
    +      kv._1 -> kv._2.keys.toSeq
    +    }
    +
    +  def getLeaderOffsets(
    +    topicAndPartitions: Set[TopicAndPartition],
    +    before: Long,
    +    maxNumOffsets: Int
    +  ): Either[Err, Map[TopicAndPartition, Seq[Long]]] = {
    +    findLeaders(topicAndPartitions).right.flatMap { tpToLeader =>
    +      val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] = flip(tpToLeader)
    +      val leaders = leaderToTp.keys
    +      var result = Map[TopicAndPartition, Seq[Long]]()
    +      val errs = new Err
    +      withBrokers(leaders, errs) { consumer =>
    +        val needed: Seq[TopicAndPartition] = leaderToTp((consumer.host, consumer.port))
    +        val req = OffsetRequest(
    +          needed.map { tp =>
    +            tp -> PartitionOffsetRequestInfo(before, maxNumOffsets)
    +          }.toMap
    +        )
    +        val resp = consumer.getOffsetsBefore(req)
    +        val respMap = resp.partitionErrorAndOffsets
    +        needed.foreach { tp =>
    +          respMap.get(tp).foreach { errAndOffsets =>
    +            if (errAndOffsets.error == ErrorMapping.NoError) {
    +              if (errAndOffsets.offsets.nonEmpty) {
    +                result += tp -> errAndOffsets.offsets
    +              } else {
    +                errs.append(new Exception(
    +                  s"Empty offsets for ${tp}, is ${before} before log beginning?"))
    +              }
    +            } else {
    +              errs.append(ErrorMapping.exceptionFor(errAndOffsets.error))
    +            }
    +          }
    +        }
    +        if (result.keys.size == topicAndPartitions.size) {
    +          return Right(result)
    +        }
    +      }
    +      val missing = topicAndPartitions.diff(result.keys.toSet)
    +      errs.append(new Exception(s"Couldn't find leader offsets for ${missing}"))
    +      Left(errs)
    +    }
    +  }
    +
    +  def getConsumerOffsets(
    +    groupId: String,
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, Long]] = {
    +    getConsumerOffsetMetadata(groupId, topicAndPartitions).right.map { r =>
    +      r.map { kv =>
    +        kv._1 -> kv._2.offset
    +      }
    +    }
    +  }
    +
    +  def getConsumerOffsetMetadata(
    +    groupId: String,
    +    topicAndPartitions: Set[TopicAndPartition]
    +  ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = {
    +    var result = Map[TopicAndPartition, OffsetMetadataAndError]()
    +    val req = OffsetFetchRequest(groupId, topicAndPartitions.toSeq)
    +    val errs = new Err
    +    withBrokers(seedBrokers, errs) { consumer =>
    +      val resp = consumer.fetchOffsets(req)
    +      val respMap = resp.requestInfo
    +      val needed = topicAndPartitions.diff(result.keys.toSet)
    +      needed.foreach { tp =>
    +        respMap.get(tp).foreach { offsetMeta =>
    +          if (offsetMeta.error == ErrorMapping.NoError) {
    +            result += tp -> offsetMeta
    +          } else {
    +            errs.append(ErrorMapping.exceptionFor(offsetMeta.error))
    +          }
    +        }
    +      }
    +      if (result.keys.size == topicAndPartitions.size) {
    +        return Right(result)
    +      }
    +    }
    +    val missing = topicAndPartitions.diff(result.keys.toSet)
    +    errs.append(new Exception(s"Couldn't find consumer offsets for ${missing}"))
    +    Left(errs)
    +  }
    +
    +  def setConsumerOffsets(
    +    groupId: String,
    +    offsets: Map[TopicAndPartition, Long]
    +  ): Either[Err, Map[TopicAndPartition, Short]] = {
    +    setConsumerOffsetMetadata(groupId, offsets.map { kv =>
    --- End diff --
    
    Do a { case (k,v) => vs accessing the tuples as ._1 etc


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by helena <gi...@git.apache.org>.
Github user helena commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22648712
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaRDD.scala ---
    @@ -0,0 +1,157 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.reflect.{classTag, ClassTag}
    +
    +import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.NextIterator
    +
    +import java.util.Properties
    +import kafka.api.FetchRequestBuilder
    +import kafka.common.{ErrorMapping, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +import kafka.message.{MessageAndMetadata, MessageAndOffset}
    +import kafka.serializer.Decoder
    +import kafka.utils.VerifiableProperties
    +
    +case class KafkaRDDPartition(
    +  override val index: Int,
    +  topic: String,
    +  partition: Int,
    +  fromOffset: Long,
    +  untilOffset: Long
    +) extends Partition
    +
    +/** A batch-oriented interface for consuming from Kafka.
    +  * Each given Kafka topic/partition corresponds to an RDD partition.
    +  * Starting and ending offsets are specified in advance,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the batch
    +  * @param untilOffsets per-topic/partition Kafka offsets defining the (exclusive)
    +  *  ending point of the batch
    +  * @param messageHandler function for translating each message into the desired type
    +  */
    +class KafkaRDD[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    sc: SparkContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    val untilOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R
    +  ) extends RDD[R](sc, Nil) with Logging {
    +
    +  assert(fromOffsets.keys == untilOffsets.keys,
    +    "Must provide both from and until offsets for each topic/partition")
    +
    +  override def getPartitions: Array[Partition] = fromOffsets.zipWithIndex.map { kvi =>
    +    val ((tp, from), index) = kvi
    +    new KafkaRDDPartition(index, tp.topic, tp.partition, from, untilOffsets(tp))
    +  }.toArray
    +
    +  override def compute(thePart: Partition, context: TaskContext) = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    if (part.fromOffset >= part.untilOffset) {
    +      log.warn("Beginning offset is same or after ending offset " +
    +        s"skipping ${part.topic} ${part.partition}")
    +      Iterator.empty
    +    } else {
    +      new NextIterator[R] {
    +        context.addTaskCompletionListener{ context => closeIfNeeded() }
    +
    +        val kc = new KafkaCluster(kafkaParams)
    +        log.info(s"Computing topic ${part.topic}, partition ${part.partition} " +
    +          s"offsets ${part.fromOffset} -> ${part.untilOffset}")
    +        val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[K]]
    +        val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[V]]
    +        val consumer: SimpleConsumer = kc.connectLeader(part.topic, part.partition).fold(
    +          errs => throw new Exception(
    +            s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " +
    +              errs.mkString("\n")),
    +          consumer => consumer
    +        )
    +        var requestOffset = part.fromOffset
    +        var iter: Iterator[MessageAndOffset] = null
    +
    +        override def close() = consumer.close()
    +
    +        override def getNext: R = {
    +          if (iter == null || !iter.hasNext) {
    +            val req = new FetchRequestBuilder().
    +              addFetch(part.topic, part.partition, requestOffset, kc.config.fetchMessageMaxBytes).
    +              build()
    +            val resp = consumer.fetch(req)
    +            if (resp.hasError) {
    +              val err = resp.errorCode(part.topic, part.partition)
    +              if (err == ErrorMapping.LeaderNotAvailableCode ||
    --- End diff --
    
    You have 3 nested if statements here, I'd break something out to it's own private function


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72696610
  
      [Test build #26658 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26658/consoleFull) for   PR 3798 at commit [`0df3ebe`](https://github.com/apache/spark/commit/0df3ebe1eed5b149c03a828db621fbc60e5555df).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71972580
  
    Great!
    The offset range is the one which is still tricky. Lets brainstorm on it a bit more. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-68149432
  
    Hi @jerryshao 
    
    I'd politely ask that anyone with questions read at least KafkaRDD.scala and the example usage linked from the jira ticket (it's only about 50 significant lines of  code):
    https://github.com/koeninger/kafka-exactly-once/blob/master/src/main/scala/example/TransactionalExample.scala
    
    I'll try to address your points.
    
    1.  Yes, each RDD partition maps directly to a Kafka (topic, partition, inclusive starting offset, exclusive ending offset)
    
    2.  It's a pull model, not a receiver push model.  All the InputDStream implementation is doing is checking the leaders' highest offsets and defining an RDD based on that.  When the RDD is run, its iterator makes a connection to kafka and pulls the data.  This is done because it's simpler, and because using existing network receiver code would require dedicating 1 core per kafka partition, which is unacceptable from an ops standpoint.
    
    3. Yes.  The fault tolerance model is that it should be safe for any or all of the spark machines to be completely destroyed at any point in the job, and the job should be able to be safely restarted.  I don't think you can do better than this.  This is achieved because all important state, especially the storage of offsets, are controlled by client code, not spark.  In both the transactional and idempotent client code approaches, offsets aren't stored until data is stored, so restart should be safe.
    
    Regarding your approach that you link, the problem there is (a) it's not a part of the spark distribution so people won't know about it, and (b) it assumes control of kafka offsets and storage in zookeeper, which makes it impossible for client code to control exactly once semantics.
    
    Regarding the possible semantic disconnect between spark streaming and treating kafka as a durable store of data from the past (assuming that's what you meant)...  I agree there is a disconnect there.  But it's a fundamental problem with spark streaming in that it implicitly depends on "now" rather than a time embedded in the data stream.  I don't think we're fixing that with this ticket.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-68407841
  
    @koeninger I like the high-level approach of the Kafka RDD and the DeterministicKafkaInputDStream. However, the stuff is not tolerant to driver failures yet. This is because of the following. If the driver fails and needs to be recovered from the checkpoints (by deserializing the DAG of DStream), then the RDDs of previous batches needs to be recreated (say last batches over previous 5 minutes, if 5 minutes window operations are used). Since the offsets used in the previous batches are not serialized with the DStream, the corresponding RDDs cannot be recreated. This problem can be alleviated pretty easily by actually making the DStream save a map of Time --> offsets. See FileInputDStream to understand how this can be done. I can talk offline about this to explain the stuff. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72174732
  
    Hey I see that you have fixed a lot of the formatting issues. Thanks! I will take a more detailed look tomorrow.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72779120
  
    Hey Cody, I was trying it and I found a odd behavior. It was printing this repeatedly. 
    ```
    15/02/03 18:22:08 WARN VerifiableProperties: Property metadata.broker.list is not valid
    ```
    
    I was using this code. 
    ```
    val kafkaParams = Map[String, String]("metadata.broker.list" -> brokerList)
        val lines = KafkaUtils.createNewStream[String, String, StringDecoder, StringDecoder](
          ssc, kafkaParams, topicsSet)
    ```
    I chose "metadata.broker.list" from the code in KafkaCluster, because without that I was getting exception from the KafkaCluster.
    



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71547061
  
      [Test build #26115 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26115/consoleFull) for   PR 3798 at commit [`d4a7cf7`](https://github.com/apache/spark/commit/d4a7cf7ae3360e73572124567f85b775566ae05e).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23743753
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,320 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.util.Random
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    +  * Convenience methods for interacting with a Kafka cluster.
    +  * @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
    +  */
    +private[spark]
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    --- End diff --
    
    Why does this need to be serializable?



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71121466
  
    OK. 
    
    Just a thought: Do you think there might be a way to avoid the spikes? Once the current RDD is checkpointed, create a "new" pending RDD, which continuously receives data, until the compute method is called. When compute gets called, the last offset we received can be considered to be the upper bound, and the data is now available for transformations. That way, we could spread out network transfers from Kafka over a larger period.
    
    Not sure if there are holes in that algorithm, but it looks almost equivalent to the current model, no?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72586040
  
      [Test build #26590 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26590/consoleFull) for   PR 3798 at commit [`825110f`](https://github.com/apache/spark/commit/825110fc22abc8ba3bb8c0e685a35ed5e939ffb9).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72907295
  
      [Test build #26763 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26763/consoleFull) for   PR 3798 at commit [`1dc2941`](https://github.com/apache/spark/commit/1dc29415e3c0ac23a4207513686dfe5ee5ab2725).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  class DirectKafkaInputDStreamCheckpointData extends DStreamCheckpointData(this) `
      * `class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable `
      * `  case class LeaderOffset(host: String, port: Int, offset: Long)`
      * `class KafkaRDDPartition(`
      * `trait HasOffsetRanges `



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71960172
  
    Jenkins, this is okay to test.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24030347
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,174 @@ 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    +    val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message)
    +    val kc = new KafkaCluster(kafkaParams)
    +    val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet
    +    val leaders = kc.findLeaders(topics).fold(
    +      errs => throw new SparkException(errs.mkString("\n")),
    +      ok => ok
    +    )
    +    new KafkaRDD[K, V, U, T, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler)
    +  }
    +
    +  /** 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   * @param leaders Kafka leaders for each offset range in batch
    +   * @param messageHandler function for translating each message into the desired type
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange],
    +      leaders: Array[Leader],
    +      messageHandler: MessageAndMetadata[K, V] => R
    +  ): RDD[R] with HasOffsetRanges = {
    +
    +    val leaderMap = leaders
    +      .map(l => TopicAndPartition(l.topic, l.partition) -> (l.host, l.port))
    +      .toMap
    +    new KafkaRDD[K, V, U, T, R](sc, kafkaParams, offsetRanges, leaderMap, messageHandler)
    +  }
    +
    +  /**
    +   * This stream can guarantee that each message from Kafka is included in transformations
    +   * (as opposed to output actions) exactly once, even in most failure situations.
    +   *
    +   * Points to note:
    +   *
    +   * Failure Recovery - You must checkpoint this stream, or save offsets yourself and provide them
    +   * as the fromOffsets parameter on restart.
    +   * Kafka must have sufficient log retention to obtain messages after failure.
    +   *
    +   * Getting offsets from the stream - see programming guide
    +   *
    +.  * Zookeeper - This does not use Zookeeper to store offsets.  For interop with Kafka monitors
    +   * that depend on Zookeeper, you must store offsets in ZK yourself.
    +   *
    +   * End-to-end semantics - This does not guarantee that any output operation will push each record
    +   * exactly once. To ensure end-to-end exactly-once semantics (that is, receiving exactly once and
    +   * outputting exactly once), you have to either ensure that the output operation is
    +   * idempotent, or transactionally store offsets with the output. See the programming guide for
    +   * more details.
    +   *
    +   * @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
    +   */
    +  @Experimental
    +  def createNewStream[
    --- End diff --
    
    I think createOffsetRangeStream conveys that it returns a stream based on
    offset ranges, and from which you can obtain offset ranges.
    
    If the thought is that it eventually becomes the recommended method, just
    naming these createStream may be the best option.  The parameter list
    doesn't conflict with any of the existing overloads.
    
    On Tue, Feb 3, 2015 at 12:57 PM, Tathagata Das <no...@github.com>
    wrote:
    
    > In
    > external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
    > <https://github.com/apache/spark/pull/3798#discussion_r24028643>:
    >
    > > +   * exactly once. To ensure end-to-end exactly-once semantics (that is, receiving exactly once and
    > > +   * outputting exactly once), you have to either ensure that the output operation is
    > > +   * idempotent, or transactionally store offsets with the output. See the programming guide for
    > > +   * more details.
    > > +   *
    > > +   * @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
    > > +   */
    > > +  @Experimental
    > > +  def createNewStream[
    >
    > CreateOffsetRangeStream seems like you have give a OffsetRange to create
    > one, whereas you can create a stream without explicitly specifying the
    > offsetrange.
    > I have found all the names to either becoming too long, or not be
    > conveying that it is a receiver-less stream. That includes
    > createDirectStream. But createDirectStream does not convey much, but is
    > definitely not overpromising.
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798/files#r24028643>.
    >


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72587226
  
      [Test build #26593 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26593/consoleFull) for   PR 3798 at commit [`8991017`](https://github.com/apache/spark/commit/8991017cdf6c33527cbabab5dcda29519579692f).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71981959
  
      [Test build #26302 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26302/consoleFull) for   PR 3798 at commit [`80fd6ae`](https://github.com/apache/spark/commit/80fd6aefde56299ba7651560e52df20b884c474e).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72143969
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/26356/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23971905
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala ---
    @@ -0,0 +1,150 @@
    +/*
    + * 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.streaming.kafka
    +
    +
    +import scala.annotation.tailrec
    +import scala.collection.mutable
    +import scala.reflect.{classTag, ClassTag}
    +
    +import kafka.common.TopicAndPartition
    +import kafka.message.MessageAndMetadata
    +import kafka.serializer.Decoder
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset
    +import org.apache.spark.streaming.{StreamingContext, Time}
    +import org.apache.spark.streaming.dstream._
    +
    +/** A stream of {@link org.apache.spark.rdd.kafka.KafkaRDD} where
    +  * each given Kafka topic/partition corresponds to an RDD partition.
    +  * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages
    +  * per second that each '''partition''' will accept.
    +  * Starting offsets are specified in advance,
    +  * and this DStream is not responsible for committing offsets,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the stream
    +  * @param messageHandler function for translating each message into the desired type
    +  * @param maxRetries maximum number of times in a row to retry getting leaders' offsets
    +  */
    +private[streaming]
    +class DeterministicKafkaInputDStream[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    @transient ssc_ : StreamingContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R,
    +    maxRetries: Int
    --- End diff --
    
    This maxRetries should be configurable through SparkConf like maxRate.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72137132
  
      [Test build #26356 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26356/consoleFull) for   PR 3798 at commit [`2b340d8`](https://github.com/apache/spark/commit/2b340d86987507ad5712e4d44114d0810351402d).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23989829
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,174 @@ 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    +    val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message)
    +    val kc = new KafkaCluster(kafkaParams)
    +    val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet
    +    val leaders = kc.findLeaders(topics).fold(
    +      errs => throw new SparkException(errs.mkString("\n")),
    +      ok => ok
    +    )
    +    new KafkaRDD[K, V, U, T, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler)
    +  }
    +
    +  /** 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   * @param leaders Kafka leaders for each offset range in batch
    +   * @param messageHandler function for translating each message into the desired type
    --- End diff --
    
    How is this message handler different than having the user just call a map function on a returned RDD? It seems a little risky because this is exposing a Kafka class in the byte code signature, which they could relocate in a future release in a way that causes this to break for callers.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24017456
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,249 @@ 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
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    --- End diff --
    
    There are 2 methods, one that takes a messagehandler (and thus returns RDD[R], and one that doesnt take a messagehandler as an argument, but provides a default one, so instead returning RDD[(K, V)]


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23972007
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala ---
    @@ -0,0 +1,150 @@
    +/*
    + * 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.streaming.kafka
    +
    +
    +import scala.annotation.tailrec
    +import scala.collection.mutable
    +import scala.reflect.{classTag, ClassTag}
    +
    +import kafka.common.TopicAndPartition
    +import kafka.message.MessageAndMetadata
    +import kafka.serializer.Decoder
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset
    +import org.apache.spark.streaming.{StreamingContext, Time}
    +import org.apache.spark.streaming.dstream._
    +
    +/** A stream of {@link org.apache.spark.rdd.kafka.KafkaRDD} where
    +  * each given Kafka topic/partition corresponds to an RDD partition.
    +  * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages
    +  * per second that each '''partition''' will accept.
    +  * Starting offsets are specified in advance,
    +  * and this DStream is not responsible for committing offsets,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the stream
    +  * @param messageHandler function for translating each message into the desired type
    +  * @param maxRetries maximum number of times in a row to retry getting leaders' offsets
    +  */
    +private[streaming]
    +class DeterministicKafkaInputDStream[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    @transient ssc_ : StreamingContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R,
    +    maxRetries: Int
    +) extends InputDStream[R](ssc_) with Logging {
    +
    +  protected[streaming] override val checkpointData =
    +    new DeterministicKafkaInputDStreamCheckpointData
    +
    +  protected val kc = new KafkaCluster(kafkaParams)
    +
    +  protected val maxMessagesPerPartition: Option[Long] = {
    +    val ratePerSec = context.sparkContext.getConf.getInt("spark.streaming.receiver.maxRate", 0)
    +    if (ratePerSec > 0) {
    +      val secsPerBatch = context.graph.batchDuration.milliseconds.toDouble / 1000
    +      Some((secsPerBatch * ratePerSec).toLong)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  protected var currentOffsets = fromOffsets
    +
    +  @tailrec
    +  protected final def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, LeaderOffset] = {
    +    val o = kc.getLatestLeaderOffsets(currentOffsets.keySet)
    +    // Either.fold would confuse @tailrec, do it manually
    +    if (o.isLeft) {
    +      val err = o.left.get.toString
    +      if (retries <= 0) {
    +        throw new Exception(err)
    +      } else {
    +        log.error(err)
    +        Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +        latestLeaderOffsets(retries - 1)
    +      }
    +    } else {
    +      o.right.get
    +    }
    +  }
    +
    +  protected def clamp(
    --- End diff --
    
    Please add some documentation on what this method does. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71555904
  
    I'm not very knowledgeable about streaming, but from my limited perspective it looks good


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72926364
  
    Yay!


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24029463
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,249 @@ 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
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    --- End diff --
    
    Yeah, makes sense. But the comment here suggests `R` is not used, however, I see `R` being used in the return type. So that was my point.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72617088
  
    I took a pass through the public API. I'm not very familiar with Kafka so it was somewhat slow going. However, some reactions:
    
    1. We should try to tighten, simplify, and clarify the way we name and document everything in this public API. Most of the comments were about this. The most important IMO is coming up with a good name for the new streams returned and clearly explaining how they differ from the old Kafka stream. To me, the main differences seems to be in the way we (a) decide what goes into which batch and (b) actually ingest the data. I proposed javadoc and naming scheme that emphasizing that distinction.
    2. Is there plans to add a Java and Python wrappers here next? Those are straightforward and it would be good to have them. Maybe in a follow on PR?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23728409
  
    --- Diff: external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala ---
    @@ -130,7 +130,7 @@ abstract class KafkaStreamSuiteBase extends FunSuite with Eventually with Loggin
       }
     
       def createTopic(topic: String) {
    -    CreateTopicCommand.createTopic(zkClient, topic, 1, 1, "0")
    --- End diff --
    
    Why was this change necessary?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24018460
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,174 @@ 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    --- End diff --
    
    There is definitely a bytecode difference, try running diff on the class files.  It statically guarantees you can call .hasOffsetRanges on the thing returned from createRDD.  Without it, you'd have to cast at runtime.  If you add e.g. a .chzburger method to KafkaRDD, you wont be able to call it without asInstanceOf.  If you then made a Chzburger interface, implement it on KafkaRDD, change the return type to "RDD with HasOffsetranges with Chzburger", you would.  I hear your concern about binary compatibility.
    
    As far as exposing KafkaRDD instead... that's the way I originally designed things.
    
    The current design is the result of a compromise between TD's desire as a maintainer to hide as much as possible, and my desire as a user to expose what's necessary to get my job done.  You can usually tell it's a good compromise if no one is happy :)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72782334
  
      [Test build #26701 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26701/consoleFull) for   PR 3798 at commit [`8c31855`](https://github.com/apache/spark/commit/8c31855cf6b7327c6b6611e715457ba15bb79355).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  class DeterministicKafkaInputDStreamCheckpointData extends DStreamCheckpointData(this) `
      * `class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable `
      * `  case class LeaderOffset(host: String, port: Int, offset: Long)`
      * `class KafkaRDDPartition(`
      * `trait HasOffsetRanges `



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22364301
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala ---
    @@ -0,0 +1,123 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.annotation.tailrec
    +import scala.reflect.{classTag, ClassTag}
    +
    +import kafka.common.TopicAndPartition
    +import kafka.message.MessageAndMetadata
    +import kafka.serializer.Decoder
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.rdd.kafka.{KafkaCluster, KafkaRDD}
    +import org.apache.spark.streaming.{StreamingContext, Time}
    +import org.apache.spark.streaming.dstream._
    +
    +/** A stream of {@link org.apache.spark.rdd.kafka.KafkaRDD} where
    +  * each given Kafka topic/partition corresponds to an RDD partition.
    +  * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages
    +  * per second that each '''partition''' will accept.
    +  * Starting offsets are specified in advance,
    +  * and this DStream is not responsible for committing offsets,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the stream
    +  * @param messageHandler function for translating each message into the desired type
    +  * @param maxRetries maximum number of times in a row to retry getting leaders' offsets
    +  */
    +class DeterministicKafkaInputDStream[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    @transient ssc_ : StreamingContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R,
    +    maxRetries: Int = 1
    +) extends InputDStream[R](ssc_) with Logging {
    +
    +  private val kc = new KafkaCluster(kafkaParams)
    +
    +  private val maxMessagesPerPartition: Option[Long] = {
    +    val ratePerSec = ssc.sparkContext.getConf.getInt("spark.streaming.receiver.maxRate", 0)
    +    if (ratePerSec > 0) {
    +      val secsPerBatch = ssc.graph.batchDuration.milliseconds.toDouble / 1000
    +      Some((secsPerBatch * ratePerSec).toLong)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  // TODO based on the design of InputDStream's lastValidTime, it appears there isn't a
    +  // thread safety concern with private mutable state, but is this certain?
    +  private var currentOffsets = fromOffsets
    --- End diff --
    
    There isnt a thread concern here. For all these class fields, there is a concern with checkpointing necessary for driver fault recovery. Lets talk about this in the main thread.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72159176
  
      [Test build #26381 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26381/consoleFull) for   PR 3798 at commit [`9a838c2`](https://github.com/apache/spark/commit/9a838c29fa40906c97172d248380f316cdf7183c).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  class DeterministicKafkaInputDStreamCheckpointData extends DStreamCheckpointData(this) `
      * `class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable `
      * `  case class LeaderOffset(host: String, port: Int, offset: Long)`
      * `class KafkaRDDPartition(`
      * `trait Leader `
      * `trait OffsetRange `
      * `trait HasOffsetRanges `



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72575053
  
      [Test build #26571 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26571/consoleFull) for   PR 3798 at commit [`4354bce`](https://github.com/apache/spark/commit/4354bced65a7f37a51bde9081d8d19dc9b9316cd).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23726952
  
    --- 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.
    --- End diff --
    
    I mentioned this in the design docs, but I'm not comfortable with naming something "exactlyOnce" when it doesn't actually guarantee that for the usual meaning of exactly once in a messaging system.  Hence the allcaps warning.
    
    I think a better option would be to rename the method, suggested a couple of possibilities in the design doc.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23990370
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,174 @@ 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    +    val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message)
    +    val kc = new KafkaCluster(kafkaParams)
    +    val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet
    +    val leaders = kc.findLeaders(topics).fold(
    +      errs => throw new SparkException(errs.mkString("\n")),
    +      ok => ok
    +    )
    +    new KafkaRDD[K, V, U, T, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler)
    +  }
    +
    +  /** 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   * @param leaders Kafka leaders for each offset range in batch
    +   * @param messageHandler function for translating each message into the desired type
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange],
    +      leaders: Array[Leader],
    +      messageHandler: MessageAndMetadata[K, V] => R
    +  ): RDD[R] with HasOffsetRanges = {
    +
    +    val leaderMap = leaders
    +      .map(l => TopicAndPartition(l.topic, l.partition) -> (l.host, l.port))
    +      .toMap
    +    new KafkaRDD[K, V, U, T, R](sc, kafkaParams, offsetRanges, leaderMap, messageHandler)
    +  }
    +
    +  /**
    +   * This stream can guarantee that each message from Kafka is included in transformations
    +   * (as opposed to output actions) exactly once, even in most failure situations.
    +   *
    +   * Points to note:
    +   *
    +   * Failure Recovery - You must checkpoint this stream, or save offsets yourself and provide them
    +   * as the fromOffsets parameter on restart.
    +   * Kafka must have sufficient log retention to obtain messages after failure.
    +   *
    +   * Getting offsets from the stream - see programming guide
    +   *
    +.  * Zookeeper - This does not use Zookeeper to store offsets.  For interop with Kafka monitors
    +   * that depend on Zookeeper, you must store offsets in ZK yourself.
    +   *
    +   * End-to-end semantics - This does not guarantee that any output operation will push each record
    +   * exactly once. To ensure end-to-end exactly-once semantics (that is, receiving exactly once and
    +   * outputting exactly once), you have to either ensure that the output operation is
    +   * idempotent, or transactionally store offsets with the output. See the programming guide for
    +   * more details.
    +   *
    +   * @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
    +   */
    +  @Experimental
    +  def createNewStream[
    --- End diff --
    
    What about `createDirectStream` or something that conveys we are reading directly from Kafka rather than going through receivers. The issue with "new" is that in short time this won't be new anymore, in fact it will be the main one we ask people to use, most likely.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22365825
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,305 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    +  * Convenience methods for interacting with a Kafka cluster.
    +  * @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
    +  */
    +class KafkaCluster(val kafkaParams: Map[String, String]) {
    --- End diff --
    
    It would probably be good to make this `private[spark]` and keep it as an internal utility.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22363779
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala ---
    @@ -0,0 +1,123 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.annotation.tailrec
    +import scala.reflect.{classTag, ClassTag}
    +
    +import kafka.common.TopicAndPartition
    +import kafka.message.MessageAndMetadata
    +import kafka.serializer.Decoder
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.rdd.kafka.{KafkaCluster, KafkaRDD}
    +import org.apache.spark.streaming.{StreamingContext, Time}
    +import org.apache.spark.streaming.dstream._
    +
    +/** A stream of {@link org.apache.spark.rdd.kafka.KafkaRDD} where
    +  * each given Kafka topic/partition corresponds to an RDD partition.
    +  * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages
    +  * per second that each '''partition''' will accept.
    +  * Starting offsets are specified in advance,
    +  * and this DStream is not responsible for committing offsets,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the stream
    +  * @param messageHandler function for translating each message into the desired type
    +  * @param maxRetries maximum number of times in a row to retry getting leaders' offsets
    +  */
    +class DeterministicKafkaInputDStream[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    @transient ssc_ : StreamingContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R,
    +    maxRetries: Int = 1
    +) extends InputDStream[R](ssc_) with Logging {
    +
    +  private val kc = new KafkaCluster(kafkaParams)
    +
    +  private val maxMessagesPerPartition: Option[Long] = {
    +    val ratePerSec = ssc.sparkContext.getConf.getInt("spark.streaming.receiver.maxRate", 0)
    --- End diff --
    
    Sorry, that part was obvious. Didnt see the name `perPartition` in the name. However, there in lies the difference with the maxRate defined with Receivers. maxRate for receivers defines the global number of records to be received through a receiver, not per partition. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23904918
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,249 @@ 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
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      batch: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    +    val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message)
    +    val kc = new KafkaCluster(kafkaParams)
    +    val topics = batch.map(o => TopicAndPartition(o.topic, o.partition)).toSet
    +    val leaderMap = kc.findLeaders(topics).fold(
    +      errs => throw new SparkException(errs.mkString("\n")),
    +      ok => ok
    +    )
    +    val rddParts = batch.zipWithIndex.map { case (o, i) =>
    +        val tp = TopicAndPartition(o.topic, o.partition)
    +        val (host, port) = leaderMap(tp)
    +        new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset, host, port)
    +    }.toArray
    +    new KafkaRDD[K, V, U, T, (K, V)](sc, kafkaParams, rddParts, messageHandler)
    +  }
    +
    +  /** 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 leaders Kafka leaders for each offset range in batch
    +   * @param messageHandler function for translating each message into the desired type
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      batch: Array[OffsetRange],
    +      leaders: Array[Leader],
    +      messageHandler: MessageAndMetadata[K, V] => R
    +  ): RDD[R] with HasOffsetRanges = {
    +    val leaderMap = leaders.map(l => (l.topic, l.partition) -> (l.host, l.port)).toMap
    +    val rddParts = batch.zipWithIndex.map { case (o, i) =>
    +        val (host, port) = leaderMap((o.topic, o.partition))
    +        new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset, host, port)
    +    }.toArray
    +
    +    new KafkaRDD[K, V, U, T, R](sc, kafkaParams, rddParts, messageHandler)
    +  }
    +
    +  /**
    +   * This stream can guarantee that each message from Kafka is included in transformations
    +   * (as opposed to output actions) exactly once, even in most failure situations.
    +   *
    +   * Points to note:
    +   *
    +   * Failure Recovery - You must checkpoint this stream, or save offsets yourself and provide them
    +   * as the fromOffsets parameter on restart.
    +   * Kafka must have sufficient log retention to obtain messages after failure.
    +   *
    +   * Getting offsets from the stream - see programming guide
    +   *
    +.  * Zookeeper - This does not use Zookeeper to store offsets.  For interop with Kafka monitors
    +   * that depend on Zookeeper, you must store offsets in ZK yourself.
    +   *
    +   * End-to-end semantics - This does not guarantee that any output operation will push each record
    +   * exactly once. To ensure end-to-end exactly-once semantics (that is, receiving exactly once and
    +   * outputting exactly once), you have to either ensure that the output operation is
    +   * idempotent, or transactionally store offsets with the output. See the programming guide for
    +   * more details.
    +   *
    +   * @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
    +   */
    +  @Experimental
    +  def createNewStream[
    +    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 stream can guarantee that each message from Kafka is included in transformations
    +   * (as opposed to output actions) exactly once, even in most failure situations.
    +   *
    +   * Points to note:
    +   *
    +   * Failure Recovery - You must checkpoint this stream.
    +   * Kafka must have sufficient log retention to obtain messages after failure.
    +   *
    +   * Getting offsets from the stream - see programming guide
    +   *
    +.  * Zookeeper - This does not use Zookeeper to store offsets.  For interop with Kafka monitors
    +   * that depend on Zookeeper, you must store offsets in ZK yourself.
    +   *
    +   * End-to-end semantics - This does not guarantee that any output operation will push each record
    +   * exactly once. To ensure end-to-end exactly-once semantics (that is, receiving exactly once and
    +   * outputting exactly once), you have to ensure that the output operation is idempotent.
    +   *
    +   * @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
    +   */
    +  @Experimental
    +  def createNewStream[
    +    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 {
    --- End diff --
    
    Its the same thing that's used if you start up a Kafka consumer with a new
    consumer group, seems pretty comparable to me.  Its also documented, so its
    not like its a surprise.
    On Feb 1, 2015 8:34 PM, "Saisai Shao" <no...@github.com> wrote:
    
    > In
    > external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
    > <https://github.com/apache/spark/pull/3798#discussion_r23904854>:
    >
    > > +      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 {
    >
    > OK, I got it. So the meaning of "auto.offset.reset" may be different from
    > what Kafka originally means, I think maybe we could give this configuration
    > another name, otherwise user may be confused with it.
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798/files#r23904854>.
    >


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23988318
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,174 @@ 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    --- End diff --
    
    I've never seen a trait mixin in a return type. What does this actually mean? I looked at the compiled byte code and the byte code signature is still RDD.
    
    Can we just return a `KafkaRDD` here? If this is enforced somehow by the scala compiler, returning an interface here ties our hands in the future, because we can't add functionality to the returned type without breaking binary compatibility. For instance, we may want to return an RDD that has additional methods beyond just accessing its offset ranges.
    
    I ran a simple example and I couldn't see any byte code reference to the mixed in trait:
    
    ```
    trait Trait {}
    
    class Class extends Trait {}
    
    object Object {
      def getTrait: Class with Trait = {new Class()}
    }
    
    > javap -v Object
      public static Class getTrait();
        flags: ACC_PUBLIC, ACC_STATIC
        Code:
          stack=1, locals=0, args_size=0
             0: getstatic     #16                 // Field Object$.MODULE$:LObject$;
             3: invokevirtual #18                 // Method Object$.getTrait:()LClass;
             6: areturn       
    ```
    



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23974964
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala ---
    @@ -0,0 +1,150 @@
    +/*
    + * 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.streaming.kafka
    +
    +
    +import scala.annotation.tailrec
    +import scala.collection.mutable
    +import scala.reflect.{classTag, ClassTag}
    +
    +import kafka.common.TopicAndPartition
    +import kafka.message.MessageAndMetadata
    +import kafka.serializer.Decoder
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset
    +import org.apache.spark.streaming.{StreamingContext, Time}
    +import org.apache.spark.streaming.dstream._
    +
    +/** A stream of {@link org.apache.spark.rdd.kafka.KafkaRDD} where
    +  * each given Kafka topic/partition corresponds to an RDD partition.
    +  * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages
    +  * per second that each '''partition''' will accept.
    +  * Starting offsets are specified in advance,
    +  * and this DStream is not responsible for committing offsets,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the stream
    +  * @param messageHandler function for translating each message into the desired type
    +  * @param maxRetries maximum number of times in a row to retry getting leaders' offsets
    +  */
    +private[streaming]
    +class DeterministicKafkaInputDStream[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    @transient ssc_ : StreamingContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R,
    +    maxRetries: Int
    +) extends InputDStream[R](ssc_) with Logging {
    +
    +  protected[streaming] override val checkpointData =
    +    new DeterministicKafkaInputDStreamCheckpointData
    +
    +  protected val kc = new KafkaCluster(kafkaParams)
    +
    +  protected val maxMessagesPerPartition: Option[Long] = {
    +    val ratePerSec = context.sparkContext.getConf.getInt("spark.streaming.receiver.maxRate", 0)
    +    if (ratePerSec > 0) {
    +      val secsPerBatch = context.graph.batchDuration.milliseconds.toDouble / 1000
    +      Some((secsPerBatch * ratePerSec).toLong)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  protected var currentOffsets = fromOffsets
    +
    +  @tailrec
    +  protected final def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, LeaderOffset] = {
    --- End diff --
    
    All of the methods in kafka cluster are currently based on the idea of trying (at most) all of the brokers, then giving up and letting the caller establish an error handling policy.
    
    Sleeping and retrying may not in general be the correct error handling policy.  I know it is for the input dstream's usage right here, but that doesn't make sense to bake into KafkaCluster.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23989786
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,174 @@ 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    +    val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message)
    +    val kc = new KafkaCluster(kafkaParams)
    +    val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet
    +    val leaders = kc.findLeaders(topics).fold(
    +      errs => throw new SparkException(errs.mkString("\n")),
    +      ok => ok
    +    )
    +    new KafkaRDD[K, V, U, T, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler)
    +  }
    +
    +  /** 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   * @param leaders Kafka leaders for each offset range in batch
    +   * @param messageHandler function for translating each message into the desired type
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange],
    +      leaders: Array[Leader],
    --- End diff --
    
    Is this version of the constructor assuming that they caller has their own code for finding the leaders? From what I can tell we've locked down the utility function for doing that.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71124795
  
    Say we start pulling down info in real time, wouldn't it be possible to say get me only "n" messages -- that should take care of the second point. 
    
    I am not sure how the ending offset part is a problem. Wouldn't it make sense to do the commits at the end of the compute call? (You don't actually commit the offsets to Kafka anyway, correct?) 
    



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22365633
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DeterministicKafkaInputDStream.scala ---
    @@ -0,0 +1,123 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.annotation.tailrec
    +import scala.reflect.{classTag, ClassTag}
    +
    +import kafka.common.TopicAndPartition
    +import kafka.message.MessageAndMetadata
    +import kafka.serializer.Decoder
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.rdd.kafka.{KafkaCluster, KafkaRDD}
    +import org.apache.spark.streaming.{StreamingContext, Time}
    +import org.apache.spark.streaming.dstream._
    +
    +/** A stream of {@link org.apache.spark.rdd.kafka.KafkaRDD} where
    +  * each given Kafka topic/partition corresponds to an RDD partition.
    +  * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages
    +  * per second that each '''partition''' will accept.
    +  * Starting offsets are specified in advance,
    +  * and this DStream is not responsible for committing offsets,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the stream
    +  * @param messageHandler function for translating each message into the desired type
    +  * @param maxRetries maximum number of times in a row to retry getting leaders' offsets
    +  */
    +class DeterministicKafkaInputDStream[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    @transient ssc_ : StreamingContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R,
    +    maxRetries: Int = 1
    +) extends InputDStream[R](ssc_) with Logging {
    +
    +  private val kc = new KafkaCluster(kafkaParams)
    +
    +  private val maxMessagesPerPartition: Option[Long] = {
    +    val ratePerSec = ssc.sparkContext.getConf.getInt("spark.streaming.receiver.maxRate", 0)
    +    if (ratePerSec > 0) {
    +      val secsPerBatch = ssc.graph.batchDuration.milliseconds.toDouble / 1000
    +      Some((secsPerBatch * ratePerSec).toLong)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  // TODO based on the design of InputDStream's lastValidTime, it appears there isn't a
    +  // thread safety concern with private mutable state, but is this certain?
    +  private var currentOffsets = fromOffsets
    +
    +  @tailrec
    +  private def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, Long] = {
    +    val o = kc.getLatestLeaderOffsets(currentOffsets.keys.toSet)
    +    // Either.fold would confuse @tailrec, do it manually
    +    if (o.isLeft) {
    +      val err = o.left.get.toString
    +      if (retries <= 0) {
    +        throw new Exception(err)
    +      } else {
    +        log.error(err)
    +        Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +        latestLeaderOffsets(retries - 1)
    +      }
    +    } else {
    +      o.right.get
    +    }
    +  }
    +
    +  private def clamp(leaderOffsets: Map[TopicAndPartition, Long]): Map[TopicAndPartition, Long] = {
    +    maxMessagesPerPartition.map { mmp =>
    +      leaderOffsets.map { kv =>
    +        val (k, v) = kv
    +        val curr = currentOffsets(k)
    +        val diff = v - curr
    +        if (diff > mmp) (k, curr + mmp) else (k, v)
    +      }
    +    }.getOrElse(leaderOffsets)
    +  }
    +
    +  override def compute(validTime: Time): Option[KafkaRDD[K, V, U, T, R]] = {
    --- End diff --
    
    I'm slightly confused about how Kafka messages are assigned to batches in this stream. From what I can tell, I give the stream a start offset, and then the first batch will include everything from that start offset until "now" (i.e. when the program is running) and subsequent batches will include any new messages delivered in that window.
    
    Is that right? If so I find it a bit confusing to call it `DeterministicKafkaInputDStream`, because the assignment of Kafka messages to batches is not actually deterministic (i.e. I could run the same program twice and the batches would be different each time). Maybe I missed something?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-68307147
  
    I got some good feedback from Koert Kuipers at Tresata regarding location awareness, so I'll be doing some refactoring to add that.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24020938
  
    --- Diff: external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala ---
    @@ -0,0 +1,73 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.util.Random
    +
    +import org.scalatest.BeforeAndAfter
    +import kafka.common.TopicAndPartition
    +
    +class KafkaClusterSuite extends KafkaStreamSuiteBase with BeforeAndAfter {
    +  val brokerHost = "localhost"
    +
    +  val kafkaParams = Map("metadata.broker.list" -> s"$brokerHost:$brokerPort")
    +
    +  val kc = new KafkaCluster(kafkaParams)
    +
    +  val topic = "kcsuitetopic" + Random.nextInt(10000)
    +
    +  val topicAndPartition = TopicAndPartition(topic, 0)
    +
    +  before {
    +    setupKafka()
    +    createTopic(topic)
    +    produceAndSendMessage(topic, Map("a" -> 1))
    +  }
    +
    +  after {
    +    tearDownKafka()
    +  }
    +
    +  test("metadata apis") {
    +    val leader = kc.findLeaders(Set(topicAndPartition)).right.get
    +    assert(leader(topicAndPartition) === (brokerHost, brokerPort), "didn't get leader")
    +
    +    val parts = kc.getPartitions(Set(topic)).right.get
    +    assert(parts(topicAndPartition), "didn't get partitions")
    +  }
    +
    +  test("leader offset apis") {
    +    val earliest = kc.getEarliestLeaderOffsets(Set(topicAndPartition)).right.get
    +    assert(earliest(topicAndPartition).offset === 0, "didn't get earliest")
    +
    +    val latest = kc.getLatestLeaderOffsets(Set(topicAndPartition)).right.get
    +    assert(latest(topicAndPartition).offset === 1, "didn't get latest")
    +  }
    +
    +  test("consumer offset apis") {
    +    val group = "kcsuitegroup" + Random.nextInt(10000)
    +
    +    val offset = Random.nextInt(10000)
    +
    +    val set = kc.setConsumerOffsets(group, Map(topicAndPartition -> offset))
    +    assert(set.isRight, "didn't set consumer offsets")
    +
    --- End diff --
    
    I'm not sure exactly what the question here is, but this test is just verifying that the consumer offset apis work.  They aren't publicly exposed, so the question of how people might misuse them is somewhat premature.
    
    That being said, the reason you'd typically want to use this api would be for interop with existing kafka monitoring tools that expect offsets in ZK


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72770451
  
    Okay here are the two options.
    
    1. createRDD returns RDD[(K,V)] or RDD[R], and DStream.foreachRDD uses rdd.asInstanceOf[HasOffsetRanges]
    2. createRDD returns KafkaRDD[(K,V)] or KafkaRDD[R] and DStream.foreachRDD uses rdd.asInstanceOf[KafkaRDD[_]]
    
    I think I am okay with either one. Stepping back, my original concern was returning something that had no binary compatibility issues. Both solution suffices. Between these two, since you feel so strongly against (2), lets go with (1). 
    
    
    



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72709851
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/26658/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by dibbhatt <gi...@git.apache.org>.
Github user dibbhatt commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24015154
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,344 @@
    +/*
    + * 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.streaming.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.util.Random
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +import org.apache.spark.SparkException
    +
    +/**
    + * Convenience methods for interacting with a Kafka cluster.
    + * @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
    + */
    +private[spark]
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    +  import KafkaCluster.{Err, LeaderOffset}
    +
    +  val seedBrokers: Array[(String, Int)] =
    +    kafkaParams.get("metadata.broker.list")
    +      .orElse(kafkaParams.get("bootstrap.servers"))
    +      .getOrElse(throw new SparkException("Must specify metadata.broker.list or bootstrap.servers"))
    +      .split(",").map { hp =>
    +        val hpa = hp.split(":")
    +        (hpa(0), hpa(1).toInt)
    +      }
    +
    +  // ConsumerConfig isn't serializable
    +  @transient private var _config: ConsumerConfig = null
    +
    +  def config: ConsumerConfig = this.synchronized {
    +    if (_config == null) {
    +      _config = KafkaCluster.consumerConfig(kafkaParams)
    +    }
    +    _config
    +  }
    +
    +  def connect(host: String, port: Int): SimpleConsumer =
    +    new SimpleConsumer(host, port, config.socketTimeoutMs,
    +      config.socketReceiveBufferBytes, config.clientId)
    +
    +  def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] =
    +    findLeader(topic, partition).right.map(hp => connect(hp._1, hp._2))
    +
    +  // Metadata api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataAPI
    +  // scalastyle:on
    +
    +  def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = {
    +    val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion,
    +      0, config.clientId, Seq(topic))
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      resp.topicsMetadata.find(_.topic == topic).flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.find(_.partitionId == partition)
    +      }.foreach { pm: PartitionMetadata =>
    +        pm.leader.foreach { leader =>
    +          return Right((leader.host, leader.port))
    +        }
    +      }
    +    }
    +    Left(errs)
    +  }
    +
    +  def findLeaders(
    +      topicAndPartitions: Set[TopicAndPartition]
    +    ): Either[Err, Map[TopicAndPartition, (String, Int)]] = {
    +    val topics = topicAndPartitions.map(_.topic)
    +    val response = getPartitionMetadata(topics).right
    +    val answer = response.flatMap { tms: Set[TopicMetadata] =>
    +      val leaderMap = tms.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.flatMap { pm: PartitionMetadata =>
    +          val tp = TopicAndPartition(tm.topic, pm.partitionId)
    +          if (topicAndPartitions(tp)) {
    +            pm.leader.map { l =>
    +              tp -> (l.host -> l.port)
    +            }
    +          } else {
    +            None
    +          }
    +        }
    +      }.toMap
    +
    +      if (leaderMap.keys.size == topicAndPartitions.size) {
    +        Right(leaderMap)
    +      } else {
    +        val missing = topicAndPartitions.diff(leaderMap.keySet)
    +        val err = new Err
    +        err.append(new SparkException(s"Couldn't find leaders for ${missing}"))
    +        Left(err)
    +      }
    +    }
    +    answer
    +  }
    +
    +  def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] = {
    +    getPartitionMetadata(topics).right.map { r =>
    +      r.flatMap { tm: TopicMetadata =>
    +        tm.partitionsMetadata.map { pm: PartitionMetadata =>
    +          TopicAndPartition(tm.topic, pm.partitionId)
    +        }    
    +      }
    +    }
    +  }
    +
    +  def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = {
    +    val req = TopicMetadataRequest(
    +      TopicMetadataRequest.CurrentVersion, 0, config.clientId, topics.toSeq)
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp: TopicMetadataResponse = consumer.send(req)
    +      // error codes here indicate missing / just created topic,
    +      // repeating on a different broker wont be useful
    +      return Right(resp.topicsMetadata.toSet)
    +    }
    +    Left(errs)
    +  }
    +
    +  // Leader offset api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI
    +  // scalastyle:on
    +
    +  def getLatestLeaderOffsets(
    +      topicAndPartitions: Set[TopicAndPartition]
    +    ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime)
    +
    +  def getEarliestLeaderOffsets(
    +      topicAndPartitions: Set[TopicAndPartition]
    +    ): Either[Err, Map[TopicAndPartition, LeaderOffset]] =
    +    getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime)
    +
    +  def getLeaderOffsets(
    +      topicAndPartitions: Set[TopicAndPartition],
    +      before: Long
    +    ): Either[Err, Map[TopicAndPartition, LeaderOffset]] = {
    +    getLeaderOffsets(topicAndPartitions, before, 1).right.map { r =>
    +      r.map { kv =>
    +        // mapValues isnt serializable, see SI-7005
    +        kv._1 -> kv._2.head
    +      }
    +    }
    +  }
    +
    +  private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] =
    +    m.groupBy(_._2).map { kv =>
    +      kv._1 -> kv._2.keys.toSeq
    +    }
    +
    +  def getLeaderOffsets(
    +      topicAndPartitions: Set[TopicAndPartition],
    +      before: Long,
    +      maxNumOffsets: Int
    +    ): Either[Err, Map[TopicAndPartition, Seq[LeaderOffset]]] = {
    +    findLeaders(topicAndPartitions).right.flatMap { tpToLeader =>
    +      val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] = flip(tpToLeader)
    +      val leaders = leaderToTp.keys
    +      var result = Map[TopicAndPartition, Seq[LeaderOffset]]()
    +      val errs = new Err
    +      withBrokers(leaders, errs) { consumer =>
    +        val partitionsToGetOffsets: Seq[TopicAndPartition] =
    +          leaderToTp((consumer.host, consumer.port))
    +        val reqMap = partitionsToGetOffsets.map { tp: TopicAndPartition =>
    +          tp -> PartitionOffsetRequestInfo(before, maxNumOffsets)
    +        }.toMap
    +        val req = OffsetRequest(reqMap)
    +        val resp = consumer.getOffsetsBefore(req)
    +        val respMap = resp.partitionErrorAndOffsets
    +        partitionsToGetOffsets.foreach { tp: TopicAndPartition =>
    +          respMap.get(tp).foreach { por: PartitionOffsetsResponse =>
    +            if (por.error == ErrorMapping.NoError) {
    +              if (por.offsets.nonEmpty) {
    +                result += tp -> por.offsets.map { off =>
    +                  LeaderOffset(consumer.host, consumer.port, off)
    +                }
    +              } else {
    +                errs.append(new SparkException(
    +                  s"Empty offsets for ${tp}, is ${before} before log beginning?"))
    +              }
    +            } else {
    +              errs.append(ErrorMapping.exceptionFor(por.error))
    +            }
    +          }
    +        }
    +        if (result.keys.size == topicAndPartitions.size) {
    +          return Right(result)
    +        }
    +      }
    +      val missing = topicAndPartitions.diff(result.keySet)
    +      errs.append(new SparkException(s"Couldn't find leader offsets for ${missing}"))
    +      Left(errs)
    +    }
    +  }
    +
    +  // Consumer offset api
    +  // scalastyle:off
    +  // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI
    +  // scalastyle:on
    +
    +  def getConsumerOffsets(
    +      groupId: String,
    +      topicAndPartitions: Set[TopicAndPartition]
    +    ): Either[Err, Map[TopicAndPartition, Long]] = {
    +    getConsumerOffsetMetadata(groupId, topicAndPartitions).right.map { r =>
    +      r.map { kv =>
    +        kv._1 -> kv._2.offset
    +      }
    +    }
    +  }
    +
    +  def getConsumerOffsetMetadata(
    +      groupId: String,
    +      topicAndPartitions: Set[TopicAndPartition]
    +    ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = {
    +    var result = Map[TopicAndPartition, OffsetMetadataAndError]()
    +    val req = OffsetFetchRequest(groupId, topicAndPartitions.toSeq)
    +    val errs = new Err
    +    withBrokers(Random.shuffle(seedBrokers), errs) { consumer =>
    +      val resp = consumer.fetchOffsets(req)
    +      val respMap = resp.requestInfo
    +      val needed = topicAndPartitions.diff(result.keySet)
    +      needed.foreach { tp: TopicAndPartition =>
    +        respMap.get(tp).foreach { ome: OffsetMetadataAndError =>
    +          if (ome.error == ErrorMapping.NoError) {
    +            result += tp -> ome
    +          } else {
    +            errs.append(ErrorMapping.exceptionFor(ome.error))
    +          }
    +        }
    +      }
    +      if (result.keys.size == topicAndPartitions.size) {
    +        return Right(result)
    +      }
    +    }
    +    val missing = topicAndPartitions.diff(result.keySet)
    +    errs.append(new SparkException(s"Couldn't find consumer offsets for ${missing}"))
    +    Left(errs)
    +  }
    +
    +  def setConsumerOffsets(
    +      groupId: String,
    +      offsets: Map[TopicAndPartition, Long]
    +    ): Either[Err, Map[TopicAndPartition, Short]] = {
    +    setConsumerOffsetMetadata(groupId, offsets.map { kv =>
    +      kv._1 -> OffsetMetadataAndError(kv._2)
    +    })
    +  }
    +
    +  def setConsumerOffsetMetadata(
    +      groupId: String,
    +      metadata: Map[TopicAndPartition, OffsetMetadataAndError]
    +    ): Either[Err, Map[TopicAndPartition, Short]] = {
    +    var result = Map[TopicAndPartition, Short]()
    +    val req = OffsetCommitRequest(groupId, metadata)
    --- End diff --
    
    This is what mentioned in the Kafka-1841
    
    "To make this really backward compatible, we have to make sure that version 0 of OffsetCommitRequest only writes to ZK. However, this doesn't quite work together with OffsetFetchRequest since in 0.8.2, it only has one version and it always reads offsets from Kafka. To address this issue, I bumped up the version of OffsetFetchRequest in 0.8.2 (with same wire protocol). Then, version 0 of OffsetFetchRequest will read from ZK and version 1 of OffsetFetchRequest will read from Kafka. This works as long as people are only using released final version. However, since this introduces an incompatible change of OffsetFetchRequest in 0.8.2-beta and trunk, this will create problems for people (assuming that they are using this api) who have a deployment of 0.8.2-beta and want to upgrade to 0.8.2 final, or a deployment from trunk and want to upgrade to a later version of trunk in the future. In either case, the upgrade of the broker will cause the old client to behave differently and incorre
 ctly."
    
    If you want to avoid such confusion, you can using ZK Client API directly to commit offsets to ZK instead using OffsetCommitRequest..that would have ported to all kafka versions seamlessly.
     
    



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by helena <gi...@git.apache.org>.
Github user helena commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r22647846
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaRDD.scala ---
    @@ -0,0 +1,157 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.reflect.{classTag, ClassTag}
    +
    +import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.NextIterator
    +
    +import java.util.Properties
    +import kafka.api.FetchRequestBuilder
    +import kafka.common.{ErrorMapping, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +import kafka.message.{MessageAndMetadata, MessageAndOffset}
    +import kafka.serializer.Decoder
    +import kafka.utils.VerifiableProperties
    +
    +case class KafkaRDDPartition(
    +  override val index: Int,
    +  topic: String,
    +  partition: Int,
    +  fromOffset: Long,
    +  untilOffset: Long
    +) extends Partition
    +
    +/** A batch-oriented interface for consuming from Kafka.
    +  * Each given Kafka topic/partition corresponds to an RDD partition.
    +  * Starting and ending offsets are specified in advance,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 fromOffsets per-topic/partition Kafka offsets defining the (inclusive)
    +  *  starting point of the batch
    +  * @param untilOffsets per-topic/partition Kafka offsets defining the (exclusive)
    +  *  ending point of the batch
    +  * @param messageHandler function for translating each message into the desired type
    +  */
    +class KafkaRDD[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    sc: SparkContext,
    +    val kafkaParams: Map[String, String],
    +    val fromOffsets: Map[TopicAndPartition, Long],
    +    val untilOffsets: Map[TopicAndPartition, Long],
    +    messageHandler: MessageAndMetadata[K, V] => R
    +  ) extends RDD[R](sc, Nil) with Logging {
    +
    +  assert(fromOffsets.keys == untilOffsets.keys,
    +    "Must provide both from and until offsets for each topic/partition")
    +
    +  override def getPartitions: Array[Partition] = fromOffsets.zipWithIndex.map { kvi =>
    +    val ((tp, from), index) = kvi
    +    new KafkaRDDPartition(index, tp.topic, tp.partition, from, untilOffsets(tp))
    +  }.toArray
    +
    +  override def compute(thePart: Partition, context: TaskContext) = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    if (part.fromOffset >= part.untilOffset) {
    +      log.warn("Beginning offset is same or after ending offset " +
    +        s"skipping ${part.topic} ${part.partition}")
    +      Iterator.empty
    +    } else {
    +      new NextIterator[R] {
    +        context.addTaskCompletionListener{ context => closeIfNeeded() }
    +
    +        val kc = new KafkaCluster(kafkaParams)
    +        log.info(s"Computing topic ${part.topic}, partition ${part.partition} " +
    +          s"offsets ${part.fromOffset} -> ${part.untilOffset}")
    +        val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[K]]
    +        val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[V]]
    +        val consumer: SimpleConsumer = kc.connectLeader(part.topic, part.partition).fold(
    +          errs => throw new Exception(
    +            s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " +
    +              errs.mkString("\n")),
    +          consumer => consumer
    +        )
    +        var requestOffset = part.fromOffset
    +        var iter: Iterator[MessageAndOffset] = null
    +
    +        override def close() = consumer.close()
    +
    +        override def getNext: R = {
    +          if (iter == null || !iter.hasNext) {
    +            val req = new FetchRequestBuilder().
    +              addFetch(part.topic, part.partition, requestOffset, kc.config.fetchMessageMaxBytes).
    +              build()
    +            val resp = consumer.fetch(req)
    +            if (resp.hasError) {
    +              val err = resp.errorCode(part.topic, part.partition)
    +              if (err == ErrorMapping.LeaderNotAvailableCode ||
    +                err == ErrorMapping.NotLeaderForPartitionCode) {
    +                log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " +
    +                  s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms")
    +                Thread.sleep(kc.config.refreshLeaderBackoffMs)
    --- End diff --
    
    Why Thread.sleep? I would not want to use that in an async app.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r24033509
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,174 @@ 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange]
    +  ): RDD[(K, V)] with HasOffsetRanges = {
    +    val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message)
    +    val kc = new KafkaCluster(kafkaParams)
    +    val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet
    +    val leaders = kc.findLeaders(topics).fold(
    +      errs => throw new SparkException(errs.mkString("\n")),
    +      ok => ok
    +    )
    +    new KafkaRDD[K, V, U, T, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler)
    +  }
    +
    +  /** 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 offsetRanges Each OffsetRange in the batch corresponds to a
    +   *   range of offsets for a given Kafka topic/partition
    +   * @param leaders Kafka leaders for each offset range in batch
    +   * @param messageHandler function for translating each message into the desired type
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    +      sc: SparkContext,
    +      kafkaParams: Map[String, String],
    +      offsetRanges: Array[OffsetRange],
    +      leaders: Array[Leader],
    +      messageHandler: MessageAndMetadata[K, V] => R
    +  ): RDD[R] with HasOffsetRanges = {
    +
    +    val leaderMap = leaders
    +      .map(l => TopicAndPartition(l.topic, l.partition) -> (l.host, l.port))
    +      .toMap
    +    new KafkaRDD[K, V, U, T, R](sc, kafkaParams, offsetRanges, leaderMap, messageHandler)
    +  }
    +
    +  /**
    +   * This stream can guarantee that each message from Kafka is included in transformations
    --- End diff --
    
    The documentation has already been changed several times since your previous comments.  The current version of it doesn't make any comparison to existing "createStream" calls.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23904301
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala ---
    @@ -144,4 +150,249 @@ 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
    +   */
    +  @Experimental
    +  def createRDD[
    +    K: ClassTag,
    +    V: ClassTag,
    +    U <: Decoder[_]: ClassTag,
    +    T <: Decoder[_]: ClassTag,
    +    R: ClassTag] (
    --- End diff --
    
    I think this `R` is not used in this API.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71546698
  
    ok to test


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71556595
  
    @koeninger  do you mind going through the changes for a pass to fix style violations? Our style checker isn't entirely automated yet. https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide
    
    In particular, there are a few things:
    
    1. Use JavaDoc style, not ScalaDoc.
    2. Indent function parameter definitions with 4 spaces, not 2. 
    
    In addition, you probably want to go through the file to tighten visibility of various methods, and reduce the number of things exposed. Historically, certain parts of Spark have not done a good job with visibility tightening and that has always come back to bite us.
    



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72778614
  
      [Test build #26706 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26706/consoleFull) for   PR 3798 at commit [`59e29f6`](https://github.com/apache/spark/commit/59e29f61cd6a730eeea4e47a5316cbbe47615618).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72791220
  
    Holy crap! Dont bother about this at all. This can wait. I hope everything
    is okay. Take care and all the best!
    On Feb 3, 2015 8:45 PM, "Cody Koeninger" <no...@github.com> wrote:
    
    > The warning is for metadata.broker.list, since its not expected by the
    > existing ConsumerConfig (its used by other config classes)
    >
    > Couldn't get subclassing to work, the verifiedproperties class it uses is
    > very dependent on order of operations during construction.
    >
    > I think the simplest thing is a class that is constructed using
    > kafkaparams, and uses the static defaults from the ConsumerConfig object.
    >
    > I'm currently waiting in an ER with my child with a 105 fever, so won't be
    > getting to it for a few hours to say the least.
    > On Feb 3, 2015 10:15 PM, "Tathagata Das" <no...@github.com> wrote:
    >
    > > I think the simplest solution is to assign zookeeper.connect. But you are
    > > assigning it in KafkaCluster lines 338 - 345. So why is this warning
    > being
    > > thrown?
    > >
    > > —
    > > Reply to this email directly or view it on GitHub
    > > <https://github.com/apache/spark/pull/3798#issuecomment-72787965>.
    > >
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/3798#issuecomment-72790044>.
    >



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23586770
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaRDD.scala ---
    @@ -0,0 +1,199 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.reflect.{classTag, ClassTag}
    +
    +import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.NextIterator
    +
    +import java.util.Properties
    +import kafka.api.{FetchRequestBuilder, FetchResponse}
    +import kafka.common.{ErrorMapping, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +import kafka.message.{MessageAndMetadata, MessageAndOffset}
    +import kafka.serializer.Decoder
    +import kafka.utils.VerifiableProperties
    +
    +/** A batch-oriented interface for consuming from Kafka.
    +  * Starting and ending offsets are specified in advance,
    +  * so that you can control exactly-once semantics.
    +  * For an easy interface to Kafka-managed offsets,
    +  *  see {@link org.apache.spark.rdd.kafka.KafkaCluster}
    +  * @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 KafkaRDDPartition 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
    +  */
    +class KafkaRDD[
    +  K: ClassTag,
    +  V: ClassTag,
    +  U <: Decoder[_]: ClassTag,
    +  T <: Decoder[_]: ClassTag,
    +  R: ClassTag](
    +    sc: SparkContext,
    +    val kafkaParams: Map[String, String],
    +    val batch: Array[KafkaRDDPartition],
    +    messageHandler: MessageAndMetadata[K, V] => R
    +  ) extends RDD[R](sc, Nil) with Logging {
    +
    +  override def getPartitions: Array[Partition] = batch.asInstanceOf[Array[Partition]]
    +
    +  override def getPreferredLocations(thePart: Partition): Seq[String] = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    // TODO is additional hostname resolution necessary here
    +    Seq(part.host)
    +  }
    +
    +  override def compute(thePart: Partition, context: TaskContext): Iterator[R] = {
    +    val part = thePart.asInstanceOf[KafkaRDDPartition]
    +    if (part.fromOffset >= part.untilOffset) {
    +      log.warn("Beginning offset is same or after ending offset " +
    +        s"skipping ${part.topic} ${part.partition}")
    +      Iterator.empty
    +    } else {
    +      new NextIterator[R] {
    +        context.addTaskCompletionListener{ context => closeIfNeeded() }
    +
    +        log.info(s"Computing topic ${part.topic}, partition ${part.partition} " +
    +          s"offsets ${part.fromOffset} -> ${part.untilOffset}")
    +
    +        val kc = new KafkaCluster(kafkaParams)
    +        val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[K]]
    +        val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
    +          .newInstance(kc.config.props)
    +          .asInstanceOf[Decoder[V]]
    +        val consumer = connectLeader
    +        var requestOffset = part.fromOffset
    +        var iter: Iterator[MessageAndOffset] = null
    +
    +        // The idea is to use the provided preferred host, except on task retry atttempts,
    +        // to minimize number of kafka metadata requests
    +        private def connectLeader: SimpleConsumer = {
    +          if (context.attemptNumber > 0) {
    +            kc.connectLeader(part.topic, part.partition).fold(
    +              errs => throw new Exception(
    +                s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " +
    +                  errs.mkString("\n")),
    +              consumer => consumer
    +            )
    +          } else {
    +            kc.connect(part.host, part.port)
    +          }
    +        }
    +
    +        private def handleErr(resp: FetchResponse) {
    +          if (resp.hasError) {
    +            val err = resp.errorCode(part.topic, part.partition)
    +            if (err == ErrorMapping.LeaderNotAvailableCode ||
    +              err == ErrorMapping.NotLeaderForPartitionCode) {
    +              log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " +
    +                s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms")
    +              Thread.sleep(kc.config.refreshLeaderBackoffMs)
    +            }
    +            // Let normal rdd retry sort out reconnect attempts
    +            throw ErrorMapping.exceptionFor(err)
    +          }
    +        }
    +
    +        override def close() = consumer.close()
    +
    +        override def getNext: R = {
    +          if (iter == null || !iter.hasNext) {
    +            val req = new FetchRequestBuilder().
    +              addFetch(part.topic, part.partition, requestOffset, kc.config.fetchMessageMaxBytes).
    +              build()
    +            val resp = consumer.fetch(req)
    +            handleErr(resp)
    +            // kafka may return a batch that starts before the requested offset
    +            iter = resp.messageSet(part.topic, part.partition)
    +              .iterator
    +              .dropWhile(_.offset < requestOffset)
    +          }
    +          if (!iter.hasNext) {
    +            assert(requestOffset == part.untilOffset,
    +              s"ran out of messages before reaching ending offset ${part.untilOffset} " +
    +                s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." +
    +                " This should not happen, and indicates that messages may have been lost")
    +            finished = true
    +            null.asInstanceOf[R]
    +          } else {
    +            val item = iter.next
    +            if (item.offset >= part.untilOffset) {
    +              assert(item.offset == part.untilOffset,
    +                s"got ${item.offset} > ending offset ${part.untilOffset} " +
    +                  s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." +
    +                  " This should not happen, and indicates a message may have been skipped")
    +              finished = true
    +              null.asInstanceOf[R]
    +            } else {
    +              requestOffset = item.nextOffset
    +              messageHandler(new MessageAndMetadata(
    +                part.topic, part.partition, item.message, item.offset, keyDecoder, valueDecoder))
    +            }
    --- End diff --
    
    btw one way you can reduce is to use return to help your control flow. for example, if `part.fromOffset >= part.untilOffset`, you can explicitly return an iterator. That reduces one level. Many other ways you can do this.



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72154913
  
      [Test build #26381 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26381/consoleFull) for   PR 3798 at commit [`9a838c2`](https://github.com/apache/spark/commit/9a838c29fa40906c97172d248380f316cdf7183c).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72789850
  
    Hi @tdas , should we add a example to show users how to use this new Kafka API correctly?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71547298
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/26115/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23727741
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/OffsetRange.scala ---
    @@ -0,0 +1,70 @@
    +/*
    + * 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.rdd.kafka
    +
    +/** Represents a range of offsets from a single Kafka TopicAndPartition */
    +trait OffsetRange {
    --- End diff --
    
    Why is this a trait. Why cant it be a simple class? All we need is a structure. Also this should be Java compatible and Scala traits arent a little Java friendly in terms of compatibility. So please use a simple class OffsetRange (not case class).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23746360
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/KafkaCluster.scala ---
    @@ -0,0 +1,320 @@
    +/*
    + * 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.rdd.kafka
    +
    +import scala.util.control.NonFatal
    +import scala.util.Random
    +import scala.collection.mutable.ArrayBuffer
    +import java.util.Properties
    +import kafka.api._
    +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
    +import kafka.consumer.{ConsumerConfig, SimpleConsumer}
    +
    +/**
    +  * Convenience methods for interacting with a Kafka cluster.
    +  * @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
    +  */
    +private[spark]
    +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
    --- End diff --
    
    As I recall, it's because the stream has a kafka cluster as a member value, and it needs to be able to be checkpointed.  The current design of KafkaCluster is essentially stateless aside from configuration.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-71549667
  
      [Test build #26118 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26118/consoleFull) for   PR 3798 at commit [`bb80bbe`](https://github.com/apache/spark/commit/bb80bbea59d11af73430961c925c80dbb942d056).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by koeninger <gi...@git.apache.org>.
Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3798#discussion_r23771686
  
    --- Diff: external/kafka/src/main/scala/org/apache/spark/rdd/kafka/OffsetRange.scala ---
    @@ -0,0 +1,70 @@
    +/*
    + * 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.rdd.kafka
    +
    +/** Represents a range of offsets from a single Kafka TopicAndPartition */
    +trait OffsetRange {
    --- End diff --
    
    I think I understand your confusion regarding the shuffle if you were looking at the CheckpointedExample.scala example, because it's doing a windowed aggregation.  There shouldn't actually be much shuffle because it's reduceByKey, and it's already partitioned.  At any rate, bad example.
    
    Here are the specific cases for getting OffsetRange s that I'm talking about:
    
    Per rdd, on the driver:
    
    https://github.com/koeninger/kafka-exactly-once/blob/d1641718807fc97f46e729e28acaba96ebc94c33/src/main/scala/example/IdempotentExample.scala#L51
    
    Per partition, on the executor:
    
    https://github.com/koeninger/kafka-exactly-once/blob/d1641718807fc97f46e729e28acaba96ebc94c33/src/main/scala/example/TransactionalExample.scala#L54


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4964] [Streaming] Exactly-once semantic...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/3798#issuecomment-72228209
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/26404/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org