HeartSaVioR commented on a change in pull request #33763: URL: https://github.com/apache/spark/pull/33763#discussion_r693735699
########## File path: sql/core/src/test/scala/org/apache/spark/sql/streaming/TriggerAvailableNowSuite.scala ########## @@ -0,0 +1,170 @@ +/* + * 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.streaming + +import java.io.File + +import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.catalyst.plans.logical.Range +import org.apache.spark.sql.catalyst.util.stringToFile +import org.apache.spark.sql.connector.read.streaming +import org.apache.spark.sql.connector.read.streaming.{ReadLimit, SupportsAdmissionControl} +import org.apache.spark.sql.execution.streaming.{LongOffset, MemoryStream, Offset, SerializedOffset, Source, StreamingExecutionRelation} +import org.apache.spark.sql.types.{LongType, StructType} + +class TriggerAvailableNowSuite extends FileStreamSourceTest { + + import testImplicits._ + + abstract class TestDataFrameProvider { + @volatile var currentOffset = 0L + + def toDF: DataFrame + + def incrementAvailableOffset(numNewRows: Int): Unit + } + + class TestSource extends TestDataFrameProvider with Source { + override def getOffset: Option[Offset] = { + if (currentOffset <= 0) None else Some(LongOffset(currentOffset)) + } + + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + if (currentOffset == 0) currentOffset = getOffsetValue(end) + val plan = Range( + start.map(getOffsetValue).getOrElse(0L) + 1L, getOffsetValue(end) + 1L, 1, None, + isStreaming = true) + Dataset.ofRows(spark, plan) + } + + override def incrementAvailableOffset(numNewRows: Int): Unit = { + currentOffset += numNewRows + } + + override def toDF: DataFrame = + Dataset.ofRows(spark, StreamingExecutionRelation(this, spark)) + + override def schema: StructType = new StructType().add("value", LongType) + + override def stop(): Unit = {} + + private def getOffsetValue(offset: Offset): Long = { + offset match { + case s: SerializedOffset => LongOffset(s).offset + case l: LongOffset => l.offset + case _ => throw new IllegalArgumentException("incorrect offset type: " + offset) + } + } + } + + class TestSourceWithAdmissionControl extends TestSource with SupportsAdmissionControl { + override def getDefaultReadLimit: ReadLimit = ReadLimit.maxRows(1) // this will be overridden + + override def latestOffset(startOffset: streaming.Offset, limit: ReadLimit): streaming.Offset = { + val currentOffset = getOffset + assert(currentOffset.nonEmpty, + "the latestOffset should be called after incrementAvailableOffset") + currentOffset.get + } + } + + class TestMicroBatchStream extends TestDataFrameProvider { + + private lazy val memoryStream = MemoryStream[Long](0, spark.sqlContext) + + override def toDF: DataFrame = memoryStream.toDF() + + override def incrementAvailableOffset(numNewRows: Int): Unit = { + for (_ <- 1 to numNewRows) { + currentOffset += 1 + memoryStream.addData(currentOffset) + } + } + } + + Seq( + new TestSource, + new TestSourceWithAdmissionControl, + new TestMicroBatchStream + ).foreach { testSource => + test(s"TriggerAvailableNow for multiple sources with ${testSource.getClass}") { + withTempDirs { (src, target) => + val checkpoint = new File(target, "chk").getCanonicalPath + val targetDir = new File(target, "data").getCanonicalPath + var lastFileModTime: Option[Long] = None + + /** Create a text file with a single data item */ + def createFile(data: Int): File = { + val file = stringToFile(new File(src, s"$data.txt"), data.toString) + if (lastFileModTime.nonEmpty) file.setLastModified(lastFileModTime.get + 1000) + lastFileModTime = Some(file.lastModified) + file + } + + // Set up a query to read text files one at a time + val df1 = spark + .readStream + .option("maxFilesPerTrigger", 1) + .text(src.getCanonicalPath) + + val df2 = testSource.toDF + + def startQuery(): StreamingQuery = { + df1.union(df2).writeStream + .format("parquet") + .trigger(Trigger.AvailableNow) + .option("checkpointLocation", checkpoint) + .start(targetDir) + } + + testSource.incrementAvailableOffset(3) + createFile(7) + createFile(8) + createFile(9) + + val q = startQuery() + + try { + assert(q.awaitTermination(streamingTimeout.toMillis)) + // only one batch has data in both sources, thus counted, see SPARK-24050 Review comment: (Beyond the scope of the PR) it would be ideal if we can revisit and fix it. ########## File path: sql/core/src/test/scala/org/apache/spark/sql/streaming/TriggerAvailableNowSuite.scala ########## @@ -0,0 +1,170 @@ +/* + * 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.streaming + +import java.io.File + +import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.catalyst.plans.logical.Range +import org.apache.spark.sql.catalyst.util.stringToFile +import org.apache.spark.sql.connector.read.streaming +import org.apache.spark.sql.connector.read.streaming.{ReadLimit, SupportsAdmissionControl} +import org.apache.spark.sql.execution.streaming.{LongOffset, MemoryStream, Offset, SerializedOffset, Source, StreamingExecutionRelation} +import org.apache.spark.sql.types.{LongType, StructType} + +class TriggerAvailableNowSuite extends FileStreamSourceTest { + + import testImplicits._ + + abstract class TestDataFrameProvider { + @volatile var currentOffset = 0L + + def toDF: DataFrame + + def incrementAvailableOffset(numNewRows: Int): Unit + } + + class TestSource extends TestDataFrameProvider with Source { + override def getOffset: Option[Offset] = { + if (currentOffset <= 0) None else Some(LongOffset(currentOffset)) + } + + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + if (currentOffset == 0) currentOffset = getOffsetValue(end) + val plan = Range( + start.map(getOffsetValue).getOrElse(0L) + 1L, getOffsetValue(end) + 1L, 1, None, + isStreaming = true) + Dataset.ofRows(spark, plan) + } + + override def incrementAvailableOffset(numNewRows: Int): Unit = { + currentOffset += numNewRows + } + + override def toDF: DataFrame = + Dataset.ofRows(spark, StreamingExecutionRelation(this, spark)) + + override def schema: StructType = new StructType().add("value", LongType) + + override def stop(): Unit = {} + + private def getOffsetValue(offset: Offset): Long = { + offset match { + case s: SerializedOffset => LongOffset(s).offset + case l: LongOffset => l.offset + case _ => throw new IllegalArgumentException("incorrect offset type: " + offset) + } + } + } + + class TestSourceWithAdmissionControl extends TestSource with SupportsAdmissionControl { + override def getDefaultReadLimit: ReadLimit = ReadLimit.maxRows(1) // this will be overridden + + override def latestOffset(startOffset: streaming.Offset, limit: ReadLimit): streaming.Offset = { + val currentOffset = getOffset + assert(currentOffset.nonEmpty, + "the latestOffset should be called after incrementAvailableOffset") + currentOffset.get + } + } + + class TestMicroBatchStream extends TestDataFrameProvider { + + private lazy val memoryStream = MemoryStream[Long](0, spark.sqlContext) + + override def toDF: DataFrame = memoryStream.toDF() + + override def incrementAvailableOffset(numNewRows: Int): Unit = { + for (_ <- 1 to numNewRows) { + currentOffset += 1 + memoryStream.addData(currentOffset) + } + } + } + + Seq( + new TestSource, + new TestSourceWithAdmissionControl, + new TestMicroBatchStream + ).foreach { testSource => + test(s"TriggerAvailableNow for multiple sources with ${testSource.getClass}") { + withTempDirs { (src, target) => + val checkpoint = new File(target, "chk").getCanonicalPath + val targetDir = new File(target, "data").getCanonicalPath + var lastFileModTime: Option[Long] = None + + /** Create a text file with a single data item */ + def createFile(data: Int): File = { + val file = stringToFile(new File(src, s"$data.txt"), data.toString) + if (lastFileModTime.nonEmpty) file.setLastModified(lastFileModTime.get + 1000) + lastFileModTime = Some(file.lastModified) + file + } + + // Set up a query to read text files one at a time + val df1 = spark + .readStream + .option("maxFilesPerTrigger", 1) + .text(src.getCanonicalPath) + + val df2 = testSource.toDF + + def startQuery(): StreamingQuery = { + df1.union(df2).writeStream + .format("parquet") + .trigger(Trigger.AvailableNow) + .option("checkpointLocation", checkpoint) + .start(targetDir) + } + + testSource.incrementAvailableOffset(3) + createFile(7) + createFile(8) + createFile(9) + + val q = startQuery() + + try { + assert(q.awaitTermination(streamingTimeout.toMillis)) + // only one batch has data in both sources, thus counted, see SPARK-24050 Review comment: (Beyond the scope of the PR) it would be ideal if we can revisit and fix it later. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
