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

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

Reply via email to