Skip to content

Commit

Permalink
[SPARK-12177][Streaming][Kafka] fix package object name and doc links
Browse files Browse the repository at this point in the history
  • Loading branch information
koeninger committed Jul 1, 2016
1 parent d1480e7 commit 22db76f
Show file tree
Hide file tree
Showing 4 changed files with 16 additions and 16 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ import org.apache.spark.annotation.Experimental
@Experimental
abstract class ConsumerStrategy[K, V] {
/**
* Kafka <a href="http://kafka.apache.org/documentation.htmll#newconsumerconfigs">
* Kafka <a href="http://kafka.apache.org/documentation.html#newconsumerconfigs">
* configuration parameters</a> to be used on executors. Requires "bootstrap.servers" to be set
* with Kafka broker(s) specified in host1:port1,host2:port2 form.
*/
Expand All @@ -59,7 +59,7 @@ abstract class ConsumerStrategy[K, V] {
* Subscribe to a collection of topics.
* @param topics collection of topics to subscribe
* @param kafkaParams Kafka
* <a href="http://kafka.apache.org/documentation.htmll#newconsumerconfigs">
* <a href="http://kafka.apache.org/documentation.html#newconsumerconfigs">
* configuration parameters</a> to be used on driver. The same params will be used on executors,
* with minor automatic modifications applied.
* Requires "bootstrap.servers" to be set
Expand Down Expand Up @@ -93,7 +93,7 @@ private case class Subscribe[K, V](
* Assign a fixed collection of TopicPartitions
* @param topicPartitions collection of TopicPartitions to assign
* @param kafkaParams Kafka
* <a href="http://kafka.apache.org/documentation.htmll#newconsumerconfigs">
* <a href="http://kafka.apache.org/documentation.html#newconsumerconfigs">
* configuration parameters</a> to be used on driver. The same params will be used on executors,
* with minor automatic modifications applied.
* Requires "bootstrap.servers" to be set
Expand Down Expand Up @@ -134,7 +134,7 @@ object ConsumerStrategies {
* Subscribe to a collection of topics.
* @param topics collection of topics to subscribe
* @param kafkaParams Kafka
* <a href="http://kafka.apache.org/documentation.htmll#newconsumerconfigs">
* <a href="http://kafka.apache.org/documentation.html#newconsumerconfigs">
* configuration parameters</a> to be used on driver. The same params will be used on executors,
* with minor automatic modifications applied.
* Requires "bootstrap.servers" to be set
Expand All @@ -159,7 +159,7 @@ object ConsumerStrategies {
* Subscribe to a collection of topics.
* @param topics collection of topics to subscribe
* @param kafkaParams Kafka
* <a href="http://kafka.apache.org/documentation.htmll#newconsumerconfigs">
* <a href="http://kafka.apache.org/documentation.html#newconsumerconfigs">
* configuration parameters</a> to be used on driver. The same params will be used on executors,
* with minor automatic modifications applied.
* Requires "bootstrap.servers" to be set
Expand All @@ -180,7 +180,7 @@ object ConsumerStrategies {
* Subscribe to a collection of topics.
* @param topics collection of topics to subscribe
* @param kafkaParams Kafka
* <a href="http://kafka.apache.org/documentation.htmll#newconsumerconfigs">
* <a href="http://kafka.apache.org/documentation.html#newconsumerconfigs">
* configuration parameters</a> to be used on driver. The same params will be used on executors,
* with minor automatic modifications applied.
* Requires "bootstrap.servers" to be set
Expand All @@ -202,7 +202,7 @@ object ConsumerStrategies {
* Subscribe to a collection of topics.
* @param topics collection of topics to subscribe
* @param kafkaParams Kafka
* <a href="http://kafka.apache.org/documentation.htmll#newconsumerconfigs">
* <a href="http://kafka.apache.org/documentation.html#newconsumerconfigs">
* configuration parameters</a> to be used on driver. The same params will be used on executors,
* with minor automatic modifications applied.
* Requires "bootstrap.servers" to be set
Expand All @@ -220,7 +220,7 @@ object ConsumerStrategies {
* Assign a fixed collection of TopicPartitions
* @param topicPartitions collection of TopicPartitions to assign
* @param kafkaParams Kafka
* <a href="http://kafka.apache.org/documentation.htmll#newconsumerconfigs">
* <a href="http://kafka.apache.org/documentation.html#newconsumerconfigs">
* configuration parameters</a> to be used on driver. The same params will be used on executors,
* with minor automatic modifications applied.
* Requires "bootstrap.servers" to be set
Expand All @@ -245,7 +245,7 @@ object ConsumerStrategies {
* Assign a fixed collection of TopicPartitions
* @param topicPartitions collection of TopicPartitions to assign
* @param kafkaParams Kafka
* <a href="http://kafka.apache.org/documentation.htmll#newconsumerconfigs">
* <a href="http://kafka.apache.org/documentation.html#newconsumerconfigs">
* configuration parameters</a> to be used on driver. The same params will be used on executors,
* with minor automatic modifications applied.
* Requires "bootstrap.servers" to be set
Expand All @@ -266,7 +266,7 @@ object ConsumerStrategies {
* Assign a fixed collection of TopicPartitions
* @param topicPartitions collection of TopicPartitions to assign
* @param kafkaParams Kafka
* <a href="http://kafka.apache.org/documentation.htmll#newconsumerconfigs">
* <a href="http://kafka.apache.org/documentation.html#newconsumerconfigs">
* configuration parameters</a> to be used on driver. The same params will be used on executors,
* with minor automatic modifications applied.
* Requires "bootstrap.servers" to be set
Expand All @@ -288,7 +288,7 @@ object ConsumerStrategies {
* Assign a fixed collection of TopicPartitions
* @param topicPartitions collection of TopicPartitions to assign
* @param kafkaParams Kafka
* <a href="http://kafka.apache.org/documentation.htmll#newconsumerconfigs">
* <a href="http://kafka.apache.org/documentation.html#newconsumerconfigs">
* configuration parameters</a> to be used on driver. The same params will be used on executors,
* with minor automatic modifications applied.
* Requires "bootstrap.servers" to be set
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ import org.apache.spark.storage.StorageLevel
* 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.htmll#newconsumerconfigs">
* <a href="http://kafka.apache.org/documentation.html#newconsumerconfigs">
* configuration parameters</a>. Requires "bootstrap.servers" to be set
* with Kafka broker(s) specified in host1:port1,host2:port2 form.
* @param offsetRanges offset ranges that define the Kafka data belonging to this RDD
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ import org.apache.spark.streaming.dstream._

/**
* :: Experimental ::
* Companion object for constructing Kafka streams and RDDs
* object for constructing Kafka streams and RDDs
*/
@Experimental
object KafkaUtils extends Logging {
Expand All @@ -44,7 +44,7 @@ object KafkaUtils extends Logging {
* 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.htmll#newconsumerconfigs">
* <a href="http://kafka.apache.org/documentation.html#newconsumerconfigs">
* configuration parameters</a>. Requires "bootstrap.servers" to be set
* with Kafka broker(s) specified in host1:port1,host2:port2 form.
* @param offsetRanges offset ranges that define the Kafka data belonging to this RDD
Expand Down Expand Up @@ -83,7 +83,7 @@ object KafkaUtils extends Logging {
* @param keyClass Class of the keys in the Kafka records
* @param valueClass Class of the values in the Kafka records
* @param kafkaParams Kafka
* <a href="http://kafka.apache.org/documentation.htmll#newconsumerconfigs">
* <a href="http://kafka.apache.org/documentation.html#newconsumerconfigs">
* configuration parameters</a>. Requires "bootstrap.servers" to be set
* with Kafka broker(s) specified in host1:port1,host2:port2 form.
* @param offsetRanges offset ranges that define the Kafka data belonging to this RDD
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,4 +20,4 @@ package org.apache.spark.streaming
/**
* Spark Integration for Kafka 0.10
*/
package object kafka
package object kafka010 //scalastyle:ignore

0 comments on commit 22db76f

Please sign in to comment.