Github user tdas commented on a diff in the pull request: https://github.com/apache/spark/pull/16686#discussion_r99201123 --- Diff: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala --- @@ -0,0 +1,255 @@ +/* + * 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.sql.kafka010 + +import java.util.concurrent.atomic.AtomicInteger + +import org.apache.kafka.common.TopicPartition +import org.scalatest.BeforeAndAfter + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.test.SharedSQLContext + +class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLContext { + + import testImplicits._ + + private val topicId = new AtomicInteger(0) + + private var testUtils: KafkaTestUtils = _ + + private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" + + private def assignString(topic: String, partitions: Iterable[Int]): String = { + JsonUtils.partitions(partitions.map(p => new TopicPartition(topic, p))) + } + + override def beforeAll(): Unit = { + super.beforeAll() + testUtils = new KafkaTestUtils + testUtils.setup() + } + + override def afterAll(): Unit = { + if (testUtils != null) { + testUtils.teardown() + testUtils = null + super.afterAll() + } + } + + test("explicit earliest to latest offsets") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 3) + testUtils.sendMessages(topic, (0 to 9).map(_.toString).toArray, Some(0)) + testUtils.sendMessages(topic, (10 to 19).map(_.toString).toArray, Some(1)) + testUtils.sendMessages(topic, Array("20"), Some(2)) + + // Specify explicit earliest and latest offset values + val df = spark + .read + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", topic) + .option("startingOffsets", "earliest") + .option("endingOffsets", "latest") + .load() + .selectExpr("CAST(value AS STRING)") + checkAnswer(df, (0 to 20).map(_.toString).toDF) + + // "latest" should late bind to the current (latest) offset in the df + testUtils.sendMessages(topic, (21 to 29).map(_.toString).toArray, Some(2)) + checkAnswer(df, (0 to 29).map(_.toString).toDF) + } + + test("default starting and ending offsets") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 3) + testUtils.sendMessages(topic, (0 to 9).map(_.toString).toArray, Some(0)) + testUtils.sendMessages(topic, (10 to 19).map(_.toString).toArray, Some(1)) + testUtils.sendMessages(topic, Array("20"), Some(2)) + + // Implicit offset values, should default to earliest and latest + val df = spark + .read + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", topic) + .load() + .selectExpr("CAST(value AS STRING)") + // Test that we default to "earliest" and "latest" + checkAnswer(df, (0 to 20).map(_.toString).toDF) + } + + test("explicit offsets") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 3) + testUtils.sendMessages(topic, (0 to 9).map(_.toString).toArray, Some(0)) + testUtils.sendMessages(topic, (10 to 19).map(_.toString).toArray, Some(1)) + testUtils.sendMessages(topic, Array("20"), Some(2)) + + // Test explicitly specified offsets + val startPartitionOffsets = Map( + new TopicPartition(topic, 0) -> -2L, // -2 => earliest + new TopicPartition(topic, 1) -> -2L, + new TopicPartition(topic, 2) -> 0L // explicit earliest + ) + val startingOffsets = JsonUtils.partitionOffsets(startPartitionOffsets) + + val endPartitionOffsets = Map( + new TopicPartition(topic, 0) -> -1L, // -1 => latest + new TopicPartition(topic, 1) -> -1L, + new TopicPartition(topic, 2) -> 1L // explicit offset happens to = the latest + ) + val endingOffsets = JsonUtils.partitionOffsets(endPartitionOffsets) + val df = spark + .read + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", topic) + .option("startingOffsets", startingOffsets) + .option("endingOffsets", endingOffsets) + .load() + .selectExpr("CAST(value as STRING)") + checkAnswer(df, (0 to 20).map(_.toString).toDF) + + // static offset partition 2, nothing should change + testUtils.sendMessages(topic, (31 to 39).map(_.toString).toArray, Some(2)) + checkAnswer(df, (0 to 20).map(_.toString).toDF) + + // latest offset partition 1, should change + testUtils.sendMessages(topic, (21 to 30).map(_.toString).toArray, Some(1)) + checkAnswer(df, (0 to 30).map(_.toString).toDF) + } + + test("reuse same dataframe in query") { + // This test ensures that we do not cache the Kafka Consumer in KafkaRelation + val topic = newTopic() + testUtils.createTopic(topic, partitions = 1) + testUtils.sendMessages(topic, (0 to 10).map(_.toString).toArray, Some(0)) + + // Specify explicit earliest and latest offset values + val df = spark + .read + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", topic) + .option("startingOffsets", "earliest") + .option("endingOffsets", "latest") + .load() + .selectExpr("CAST(value AS STRING)") + checkAnswer(df.union(df), ((0 to 10) ++ (0 to 10)).map(_.toString).toDF) + } + + test("test late binding start offsets") { + try { + // First, establish a new KafkaUtils instance that will clear + // all messages when cleanupLogs is called. + if (testUtils != null) { + testUtils.teardown() + testUtils = null + } + // The following settings will ensure that all log entries + // are removed following a call to cleanupLogs + val brokerProps = Map[String, Object]( + "log.retention.bytes" -> 1.asInstanceOf[AnyRef], // retain nothing + "log.retention.ms" -> 1.asInstanceOf[AnyRef] // no wait time + ) + testUtils = new KafkaTestUtils(withBrokerProps = brokerProps) + testUtils.setup() + + val topic = newTopic() + testUtils.createTopic(topic, partitions = 1) + testUtils.sendMessages(topic, (0 to 9).map(_.toString).toArray, Some(0)) + // Specify explicit earliest and latest offset values + val df = spark + .read + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", topic) + .option("startingOffsets", "earliest") + .option("endingOffsets", "latest") + .load() + .selectExpr("CAST(value AS STRING)") + checkAnswer(df, (0 to 9).map(_.toString).toDF) + // Blow away current set of messages. + testUtils.cleanupLogs() + // Add some more data, but do not call cleanup + testUtils.sendMessages(topic, (10 to 19).map(_.toString).toArray, Some(0)) + // Ensure that we late bind to the new starting position + checkAnswer(df, (10 to 19).map(_.toString).toDF) + } finally { + if (testUtils != null) { + testUtils.teardown() + } + testUtils = new KafkaTestUtils + testUtils.setup() + } + } + + test("bad source options") { + def testBadOptions(options: (String, String)*)(expectedMsgs: String*): Unit = { + val ex = intercept[IllegalArgumentException] { + val reader = spark + .read + .format("kafka") + options.foreach { case (k, v) => reader.option(k, v) } + reader.load() + } + expectedMsgs.foreach { m => + assert(ex.getMessage.toLowerCase.contains(m.toLowerCase)) + } + } + + // Specifying an ending offset as the starting point + testBadOptions("startingOffsets" -> "latest")("starting relation offset can't be latest") + + // Now do it with an explicit json offset + val startPartitionOffsets = Map( + new TopicPartition("t", 0) -> -1L // specify latest + ) + val startingOffsets = JsonUtils.partitionOffsets(startPartitionOffsets) + testBadOptions("subscribe" -> "t", "startingOffsets" -> startingOffsets)( + "startingoffsets for t-0 can't be latest") + + + // Make sure we catch ending offsets that indicate earliest + testBadOptions("endingOffsets" -> "earliest")("ending relation offset can't be earliest") + + val endPartitionOffsets = Map( --- End diff -- nit: can be single line?
--- 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