[GitHub] [spark] HeartSaVioR commented on a diff in pull request #38517: [SPARK-39591][SS] Async Progress Tracking

2022-12-19 Thread GitBox


HeartSaVioR commented on code in PR #38517:
URL: https://github.com/apache/spark/pull/38517#discussion_r1052937189


##
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecutionSuite.scala:
##
@@ -68,11 +64,42 @@ class AsyncProgressTrackingMicroBatchExecutionSuite
 }
   }
 
+  class MemoryStreamCapture[A: Encoder](
+ id: Int,

Review Comment:
   nit: 4 spaces



-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



[GitHub] [spark] HeartSaVioR commented on a diff in pull request #38517: [SPARK-39591][SS] Async Progress Tracking

2022-12-19 Thread GitBox


HeartSaVioR commented on code in PR #38517:
URL: https://github.com/apache/spark/pull/38517#discussion_r1052941322


##
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecutionSuite.scala:
##
@@ -0,0 +1,1865 @@
+/*
+ * 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.execution.streaming
+
+import java.io.{File, OutputStream}
+import java.util.concurrent.{CountDownLatch, Semaphore, ThreadPoolExecutor, 
TimeUnit}
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.hadoop.fs.Path
+import org.scalatest.BeforeAndAfter
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.time.{Seconds, Span}
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.streaming.WriteToStream
+import org.apache.spark.sql.connector.read.streaming
+import 
org.apache.spark.sql.execution.streaming.AsyncProgressTrackingMicroBatchExecution.{ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS,
 ASYNC_PROGRESS_TRACKING_ENABLED, 
ASYNC_PROGRESS_TRACKING_OVERRIDE_SINK_SUPPORT_CHECK}
+import org.apache.spark.sql.functions.{column, window}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.{StreamingQuery, 
StreamingQueryException, StreamTest, Trigger}
+import org.apache.spark.sql.streaming.util.StreamManualClock
+import org.apache.spark.util.{Clock, Utils}
+
+class AsyncProgressTrackingMicroBatchExecutionSuite
+extends StreamTest
+with BeforeAndAfter
+with Matchers {
+
+  import testImplicits._
+
+  after {
+sqlContext.streams.active.foreach(_.stop())
+  }
+
+  def getListOfFiles(dir: String): List[File] = {
+val d = new File(dir)
+if (d.exists && d.isDirectory) {
+  d.listFiles.filter(_.isFile).toList
+} else {
+  List[File]()
+}
+  }
+
+  def waitPendingOffsetWrites(streamExecution: StreamExecution): Unit = {
+
assert(streamExecution.isInstanceOf[AsyncProgressTrackingMicroBatchExecution])
+eventually(timeout(Span(5, Seconds))) {
+  streamExecution
+.asInstanceOf[AsyncProgressTrackingMicroBatchExecution]
+.areWritesPendingOrInProgress() should be(false)
+}
+  }
+
+  def waitPendingPurges(streamExecution: StreamExecution): Unit = {
+
assert(streamExecution.isInstanceOf[AsyncProgressTrackingMicroBatchExecution])
+eventually(timeout(Span(5, Seconds))) {
+  streamExecution
+.asInstanceOf[AsyncProgressTrackingMicroBatchExecution]
+.arePendingAsyncPurge should be(false)
+}
+  }
+
+  // test the basic functionality i.e. happy path
+  test("async WAL commits happy path") {
+val checkpointLocation = Utils.createTempDir(namePrefix = 
"streaming.metadata").getCanonicalPath
+
+val inputData = new MemoryStream[Int](id = 0, sqlContext = sqlContext)
+val ds = inputData.toDF()
+
+val tableName = "test"
+
+def startQuery(): StreamingQuery = {
+  ds.writeStream
+.format("memory")
+.queryName(tableName)
+.option(ASYNC_PROGRESS_TRACKING_ENABLED, true)
+.option(ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS, 0)
+.option("checkpointLocation", checkpointLocation)
+.start()
+}
+val query = startQuery()
+val expected = new ListBuffer[Row]()
+for (j <- 0 until 100) {
+  for (i <- 0 until 10) {
+val v = i + (j * 10)
+inputData.addData({ v })
+expected += Row(v)
+  }
+  query.processAllAvailable()
+}
+
+checkAnswer(
+  spark.table(tableName),
+  expected.toSeq
+)
+  }
+
+  test("async WAL commits recovery") {
+val checkpointLocation = Utils.createTempDir(namePrefix = 
"streaming.metadata").getCanonicalPath
+
+val inputData = new MemoryStream[Int](id = 0, sqlContext = sqlContext)
+val ds = inputData.toDF()
+
+var index = 0
+// to synchronize producing and consuming messages so that
+// we can generate and read the desired number of batches
+var countDownLatch = new CountDownLatch(10)
+val sem = new Semaphore(1)
+val data = new ListBuffer[Int]()
+def startQuery(): StreamingQuery = {
+  ds.writeStream
+.foreachBatch((ds: Dataset[Row], batchId: Long) => {
+  

[GitHub] [spark] HeartSaVioR commented on a diff in pull request #38517: [SPARK-39591][SS] Async Progress Tracking

2022-12-19 Thread GitBox


HeartSaVioR commented on code in PR #38517:
URL: https://github.com/apache/spark/pull/38517#discussion_r1052931956


##
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecution.scala:
##
@@ -275,7 +298,7 @@ object AsyncProgressTrackingMicroBatchExecution {
 "_asyncProgressTrackingOverrideSinkSupportCheck"
 
   private def getAsyncProgressTrackingCheckpointingIntervalMs(
-   extraOptions: 
Map[String, String]): Long = {
+extraOptions: Map[String, String]): Long = {

Review Comment:
   nit: 4 spaces



-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



[GitHub] [spark] HeartSaVioR commented on a diff in pull request #38517: [SPARK-39591][SS] Async Progress Tracking

2022-12-19 Thread GitBox


HeartSaVioR commented on code in PR #38517:
URL: https://github.com/apache/spark/pull/38517#discussion_r1052927996


##
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecutionSuite.scala:
##
@@ -78,7 +72,9 @@ class AsyncProgressTrackingMicroBatchExecutionSuite
   test("async WAL commits happy path") {
 val checkpointLocation = Utils.createTempDir(namePrefix = 
"streaming.metadata").getCanonicalPath
 
-val inputData = new MemoryStream[Int](id = 0, sqlContext = sqlContext)
+//val inputData = new MemoryStream[Int](id = 0, sqlContext = sqlContext)

Review Comment:
   nit: remove comment



##
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecution.scala:
##
@@ -283,3 +264,23 @@ class AsyncProgressTrackingMicroBatchExecution(
 }
   }
 }
+
+object AsyncProgressTrackingMicroBatchExecution {
+  val ASYNC_PROGRESS_TRACKING_ENABLED = "asyncProgressTrackingEnabled"
+  val ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS =
+"asyncProgressTrackingCheckpointIntervalMs"
+
+  // for testing purposes
+  val ASYNC_PROGRESS_TRACKING_OVERRIDE_SINK_SUPPORT_CHECK =
+"_asyncProgressTrackingOverrideSinkSupportCheck"
+
+  private def getAsyncProgressTrackingCheckpointingIntervalMs(
+   extraOptions: 
Map[String, String]): Long = {

Review Comment:
   nit: indentation is off



##
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecution.scala:
##
@@ -157,8 +172,17 @@ class AsyncProgressTrackingMicroBatchExecution(
 watermarkTracker.updateWatermark(lastExecution.executedPlan)
 reportTimeTaken("commitOffsets") {
   // check if current batch there is a async write for the offset log is 
issued for this batch
-  // if so, we should do the same for commit log
-  if (offsetLog.getAsyncOffsetWrite(currentBatchId).nonEmpty) {
+  // if so, we should do the same for commit log.  However, if this is the 
first batch executed
+  // in this run we should always persis to the commit log.  There can be 
situations in which

Review Comment:
   nit: persist



-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



[GitHub] [spark] HeartSaVioR commented on a diff in pull request #38517: [SPARK-39591][SS] Async Progress Tracking

2022-12-19 Thread GitBox


HeartSaVioR commented on code in PR #38517:
URL: https://github.com/apache/spark/pull/38517#discussion_r1052728943


##
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecutionSuite.scala:
##
@@ -0,0 +1,1865 @@
+/*
+ * 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.execution.streaming
+
+import java.io.{File, OutputStream}
+import java.util.concurrent.{CountDownLatch, Semaphore, ThreadPoolExecutor, 
TimeUnit}
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.hadoop.fs.Path
+import org.scalatest.BeforeAndAfter
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.time.{Seconds, Span}
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.streaming.WriteToStream
+import org.apache.spark.sql.connector.read.streaming
+import 
org.apache.spark.sql.execution.streaming.AsyncProgressTrackingMicroBatchExecution.{ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS,
 ASYNC_PROGRESS_TRACKING_ENABLED, 
ASYNC_PROGRESS_TRACKING_OVERRIDE_SINK_SUPPORT_CHECK}
+import org.apache.spark.sql.functions.{column, window}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.{StreamingQuery, 
StreamingQueryException, StreamTest, Trigger}
+import org.apache.spark.sql.streaming.util.StreamManualClock
+import org.apache.spark.util.{Clock, Utils}
+
+class AsyncProgressTrackingMicroBatchExecutionSuite
+extends StreamTest
+with BeforeAndAfter
+with Matchers {
+
+  import testImplicits._
+
+  after {
+sqlContext.streams.active.foreach(_.stop())
+  }
+
+  def getListOfFiles(dir: String): List[File] = {
+val d = new File(dir)
+if (d.exists && d.isDirectory) {
+  d.listFiles.filter(_.isFile).toList
+} else {
+  List[File]()
+}
+  }
+
+  def waitPendingOffsetWrites(streamExecution: StreamExecution): Unit = {
+
assert(streamExecution.isInstanceOf[AsyncProgressTrackingMicroBatchExecution])
+eventually(timeout(Span(5, Seconds))) {
+  streamExecution
+.asInstanceOf[AsyncProgressTrackingMicroBatchExecution]
+.areWritesPendingOrInProgress() should be(false)
+}
+  }
+
+  def waitPendingPurges(streamExecution: StreamExecution): Unit = {
+
assert(streamExecution.isInstanceOf[AsyncProgressTrackingMicroBatchExecution])
+eventually(timeout(Span(5, Seconds))) {
+  streamExecution
+.asInstanceOf[AsyncProgressTrackingMicroBatchExecution]
+.arePendingAsyncPurge should be(false)
+}
+  }
+
+  // test the basic functionality i.e. happy path
+  test("async WAL commits happy path") {
+val checkpointLocation = Utils.createTempDir(namePrefix = 
"streaming.metadata").getCanonicalPath
+
+val inputData = new MemoryStream[Int](id = 0, sqlContext = sqlContext)
+val ds = inputData.toDF()
+
+val tableName = "test"
+
+def startQuery(): StreamingQuery = {
+  ds.writeStream
+.format("memory")
+.queryName(tableName)
+.option(ASYNC_PROGRESS_TRACKING_ENABLED, true)
+.option(ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS, 0)
+.option("checkpointLocation", checkpointLocation)
+.start()
+}
+val query = startQuery()
+val expected = new ListBuffer[Row]()
+for (j <- 0 until 100) {
+  for (i <- 0 until 10) {
+val v = i + (j * 10)
+inputData.addData({ v })
+expected += Row(v)
+  }
+  query.processAllAvailable()
+}
+
+checkAnswer(
+  spark.table(tableName),
+  expected.toSeq
+)
+  }
+
+  test("async WAL commits recovery") {
+val checkpointLocation = Utils.createTempDir(namePrefix = 
"streaming.metadata").getCanonicalPath
+
+val inputData = new MemoryStream[Int](id = 0, sqlContext = sqlContext)
+val ds = inputData.toDF()
+
+var index = 0
+// to synchronize producing and consuming messages so that
+// we can generate and read the desired number of batches
+var countDownLatch = new CountDownLatch(10)
+val sem = new Semaphore(1)
+val data = new ListBuffer[Int]()
+def startQuery(): StreamingQuery = {
+  ds.writeStream
+.foreachBatch((ds: Dataset[Row], batchId: Long) => {
+  

[GitHub] [spark] HeartSaVioR commented on a diff in pull request #38517: [SPARK-39591][SS] Async Progress Tracking

2022-12-19 Thread GitBox


HeartSaVioR commented on code in PR #38517:
URL: https://github.com/apache/spark/pull/38517#discussion_r1052722369


##
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala:
##
@@ -342,17 +342,14 @@ class MicroBatchExecution(
 isCurrentBatchConstructed = true
 availableOffsets = nextOffsets.toStreamProgress(sources)
 /* Initialize committed offsets to a committed batch, which at this
- * is the second latest batch id in the offset log. */
-if (latestBatchId != 0) {
-  val secondLatestOffsets = offsetLog.get(latestBatchId - 1).getOrElse 
{
-logError(s"The offset log for batch ${latestBatchId - 1} doesn't 
exist, " +
-  s"which is required to restart the query from the latest batch 
$latestBatchId " +
-  "from the offset log. Please ensure there are two subsequent 
offset logs " +
-  "available for the latest batch via manually deleting the offset 
file(s). " +
-  "Please also ensure the latest batch for commit log is equal or 
one batch " +
-  "earlier than the latest batch for offset log.")
-throw new IllegalStateException(s"batch ${latestBatchId - 1} 
doesn't exist")
-  }
+ * is the second latest batch id in the offset log.

Review Comment:
   How often we encounter this is not important. The fact we are removing guard 
which prevents breaking fault-tolerance semantic is important.



-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



[GitHub] [spark] HeartSaVioR commented on a diff in pull request #38517: [SPARK-39591][SS] Async Progress Tracking

2022-12-19 Thread GitBox


HeartSaVioR commented on code in PR #38517:
URL: https://github.com/apache/spark/pull/38517#discussion_r1052722369


##
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala:
##
@@ -342,17 +342,14 @@ class MicroBatchExecution(
 isCurrentBatchConstructed = true
 availableOffsets = nextOffsets.toStreamProgress(sources)
 /* Initialize committed offsets to a committed batch, which at this
- * is the second latest batch id in the offset log. */
-if (latestBatchId != 0) {
-  val secondLatestOffsets = offsetLog.get(latestBatchId - 1).getOrElse 
{
-logError(s"The offset log for batch ${latestBatchId - 1} doesn't 
exist, " +
-  s"which is required to restart the query from the latest batch 
$latestBatchId " +
-  "from the offset log. Please ensure there are two subsequent 
offset logs " +
-  "available for the latest batch via manually deleting the offset 
file(s). " +
-  "Please also ensure the latest batch for commit log is equal or 
one batch " +
-  "earlier than the latest batch for offset log.")
-throw new IllegalStateException(s"batch ${latestBatchId - 1} 
doesn't exist")
-  }
+ * is the second latest batch id in the offset log.

Review Comment:
   How often we encounter this is not important. The fact we are removing guard 
to not break fault-tolerance semantic is important.



-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



[GitHub] [spark] HeartSaVioR commented on a diff in pull request #38517: [SPARK-39591][SS] Async Progress Tracking

2022-12-19 Thread GitBox


HeartSaVioR commented on code in PR #38517:
URL: https://github.com/apache/spark/pull/38517#discussion_r1052721653


##
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala:
##
@@ -342,17 +342,14 @@ class MicroBatchExecution(
 isCurrentBatchConstructed = true
 availableOffsets = nextOffsets.toStreamProgress(sources)
 /* Initialize committed offsets to a committed batch, which at this
- * is the second latest batch id in the offset log. */
-if (latestBatchId != 0) {
-  val secondLatestOffsets = offsetLog.get(latestBatchId - 1).getOrElse 
{
-logError(s"The offset log for batch ${latestBatchId - 1} doesn't 
exist, " +
-  s"which is required to restart the query from the latest batch 
$latestBatchId " +
-  "from the offset log. Please ensure there are two subsequent 
offset logs " +
-  "available for the latest batch via manually deleting the offset 
file(s). " +
-  "Please also ensure the latest batch for commit log is equal or 
one batch " +
-  "earlier than the latest batch for offset log.")
-throw new IllegalStateException(s"batch ${latestBatchId - 1} 
doesn't exist")
-  }
+ * is the second latest batch id in the offset log.

Review Comment:
   This logic can affect the offset range of microbatch. As you've added the 
test, even without async progress tracking flag on, normal processing trigger 
can technically roll multiple microbatches back, "with composing these offsets 
into one". This breaks the assumption of exactly-once semantic, every 
microbatch should have planned its offset range before execution, and the range 
must not be changed once planned.
   
   This is why async progress tracking cannot work as it is for Delta sink and 
stateful operator. We blocked this for async progress tracking, but 
accidentally exposing this to "normal" processing trigger.



-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



[GitHub] [spark] HeartSaVioR commented on a diff in pull request #38517: [SPARK-39591][SS] Async Progress Tracking

2022-12-19 Thread GitBox


HeartSaVioR commented on code in PR #38517:
URL: https://github.com/apache/spark/pull/38517#discussion_r1052719778


##
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecution.scala:
##
@@ -0,0 +1,282 @@
+/*
+ * 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.execution.streaming
+
+import java.util.concurrent._
+import java.util.concurrent.atomic.AtomicLong
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.streaming.WriteToStream
+import org.apache.spark.sql.streaming.Trigger
+import org.apache.spark.util.{Clock, ThreadUtils}
+
+object AsyncProgressTrackingMicroBatchExecution {
+  val ASYNC_PROGRESS_TRACKING_ENABLED = "asyncProgressTrackingEnabled"
+  val ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS =
+"asyncProgressTrackingCheckpointIntervalMs"
+
+  // for testing purposes
+  val ASYNC_PROGRESS_TRACKING_OVERRIDE_SINK_SUPPORT_CHECK =
+"_asyncProgressTrackingOverrideSinkSupportCheck"
+
+  private def getAsyncProgressTrackingCheckpointingIntervalMs(
+  extraOptions: Map[String, String]): Long = {
+extraOptions
+  .getOrElse(
+
AsyncProgressTrackingMicroBatchExecution.ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS,
+"1000"
+  )
+  .toLong
+  }
+}
+
+/**
+ * Class to execute micro-batches when async progress tracking is enabled
+ */
+class AsyncProgressTrackingMicroBatchExecution(
+sparkSession: SparkSession,
+trigger: Trigger,
+triggerClock: Clock,
+extraOptions: Map[String, String],
+plan: WriteToStream)
+extends MicroBatchExecution(sparkSession, trigger, triggerClock, 
extraOptions, plan) {
+
+  protected val asyncProgressTrackingCheckpointingIntervalMs: Long
+  = AsyncProgressTrackingMicroBatchExecution
+.getAsyncProgressTrackingCheckpointingIntervalMs(extraOptions)
+
+  // Offsets that are ready to be committed by the source.
+  // This is needed so that we can call source commit in the same thread as 
micro-batch execution
+  // to be thread safe
+  private val sourceCommitQueue = new ConcurrentLinkedQueue[OffsetSeq]()
+
+  // to cache the batch id of the last batch written to storage
+  private val lastBatchPersistedToDurableStorage = new AtomicLong(-1)
+
+  override val triggerExecutor: TriggerExecutor = validateAndGetTrigger()
+
+  private var isFirstBatch: Boolean = true
+
+  // thread pool is only one thread because we want offset
+  // writes to execute in order in a serialized fashion
+  protected val asyncWritesExecutorService
+  = ThreadUtils.newDaemonSingleThreadExecutorWithRejectedExecutionHandler(
+"async-log-write",
+2, // one for offset commit and one for completion commit
+new RejectedExecutionHandler() {
+  override def rejectedExecution(r: Runnable, executor: 
ThreadPoolExecutor): Unit = {
+try {
+  if (!executor.isShutdown) {
+val start = System.currentTimeMillis()
+executor.getQueue.put(r)
+logDebug(
+  s"Async write paused execution for " +
+s"${System.currentTimeMillis() - start} due to task queue 
being full."
+)
+  }
+} catch {
+  case e: InterruptedException =>
+Thread.currentThread.interrupt()
+throw new RejectedExecutionException("Producer interrupted", e)
+  case e: Throwable =>
+logError("Encountered error in async write executor service", e)
+errorNotifier.markError(e)
+}
+  }
+})
+
+  override val offsetLog = new AsyncOffsetSeqLog(
+sparkSession,
+checkpointFile("offsets"),
+asyncWritesExecutorService,
+asyncProgressTrackingCheckpointingIntervalMs,
+clock = triggerClock
+  )
+
+  override val commitLog =
+new AsyncCommitLog(sparkSession, checkpointFile("commits"), 
asyncWritesExecutorService)
+
+  override def markMicroBatchExecutionStart(): Unit = {
+// check if pipeline is stateful
+checkNotStatefulPipeline
+  }
+
+  override def cleanUpLastExecutedMicroBatch(): Unit = {
+// this is a no op for async progress tracking since we only want to 
commit sources only
+// after the 

[GitHub] [spark] HeartSaVioR commented on a diff in pull request #38517: [SPARK-39591][SS] Async Progress Tracking

2022-12-18 Thread GitBox


HeartSaVioR commented on code in PR #38517:
URL: https://github.com/apache/spark/pull/38517#discussion_r1051727091


##
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecutionSuite.scala:
##
@@ -0,0 +1,1865 @@
+/*
+ * 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.execution.streaming
+
+import java.io.{File, OutputStream}
+import java.util.concurrent.{CountDownLatch, Semaphore, ThreadPoolExecutor, 
TimeUnit}
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.hadoop.fs.Path
+import org.scalatest.BeforeAndAfter
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.time.{Seconds, Span}
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.streaming.WriteToStream
+import org.apache.spark.sql.connector.read.streaming
+import 
org.apache.spark.sql.execution.streaming.AsyncProgressTrackingMicroBatchExecution.{ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS,
 ASYNC_PROGRESS_TRACKING_ENABLED, 
ASYNC_PROGRESS_TRACKING_OVERRIDE_SINK_SUPPORT_CHECK}
+import org.apache.spark.sql.functions.{column, window}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.{StreamingQuery, 
StreamingQueryException, StreamTest, Trigger}
+import org.apache.spark.sql.streaming.util.StreamManualClock
+import org.apache.spark.util.{Clock, Utils}
+
+class AsyncProgressTrackingMicroBatchExecutionSuite
+extends StreamTest
+with BeforeAndAfter
+with Matchers {
+
+  import testImplicits._
+
+  after {
+sqlContext.streams.active.foreach(_.stop())
+  }
+
+  def getListOfFiles(dir: String): List[File] = {
+val d = new File(dir)
+if (d.exists && d.isDirectory) {
+  d.listFiles.filter(_.isFile).toList
+} else {
+  List[File]()
+}
+  }
+
+  def waitPendingOffsetWrites(streamExecution: StreamExecution): Unit = {
+
assert(streamExecution.isInstanceOf[AsyncProgressTrackingMicroBatchExecution])
+eventually(timeout(Span(5, Seconds))) {
+  streamExecution
+.asInstanceOf[AsyncProgressTrackingMicroBatchExecution]
+.areWritesPendingOrInProgress() should be(false)
+}
+  }
+
+  def waitPendingPurges(streamExecution: StreamExecution): Unit = {
+
assert(streamExecution.isInstanceOf[AsyncProgressTrackingMicroBatchExecution])
+eventually(timeout(Span(5, Seconds))) {
+  streamExecution
+.asInstanceOf[AsyncProgressTrackingMicroBatchExecution]
+.arePendingAsyncPurge should be(false)
+}
+  }
+
+  // test the basic functionality i.e. happy path
+  test("async WAL commits happy path") {
+val checkpointLocation = Utils.createTempDir(namePrefix = 
"streaming.metadata").getCanonicalPath
+
+val inputData = new MemoryStream[Int](id = 0, sqlContext = sqlContext)
+val ds = inputData.toDF()
+
+val tableName = "test"
+
+def startQuery(): StreamingQuery = {
+  ds.writeStream
+.format("memory")
+.queryName(tableName)
+.option(ASYNC_PROGRESS_TRACKING_ENABLED, true)
+.option(ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS, 0)
+.option("checkpointLocation", checkpointLocation)
+.start()
+}
+val query = startQuery()
+val expected = new ListBuffer[Row]()
+for (j <- 0 until 100) {
+  for (i <- 0 until 10) {
+val v = i + (j * 10)
+inputData.addData({ v })
+expected += Row(v)
+  }
+  query.processAllAvailable()
+}
+
+checkAnswer(
+  spark.table(tableName),
+  expected.toSeq
+)
+  }
+
+  test("async WAL commits recovery") {
+val checkpointLocation = Utils.createTempDir(namePrefix = 
"streaming.metadata").getCanonicalPath
+
+val inputData = new MemoryStream[Int](id = 0, sqlContext = sqlContext)
+val ds = inputData.toDF()
+
+var index = 0
+// to synchronize producing and consuming messages so that
+// we can generate and read the desired number of batches
+var countDownLatch = new CountDownLatch(10)
+val sem = new Semaphore(1)
+val data = new ListBuffer[Int]()
+def startQuery(): StreamingQuery = {
+  ds.writeStream
+.foreachBatch((ds: Dataset[Row], batchId: Long) => {
+  

[GitHub] [spark] HeartSaVioR commented on a diff in pull request #38517: [SPARK-39591][SS] Async Progress Tracking

2022-12-14 Thread GitBox


HeartSaVioR commented on code in PR #38517:
URL: https://github.com/apache/spark/pull/38517#discussion_r1049119941


##
connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala:
##
@@ -195,6 +200,102 @@ abstract class KafkaMicroBatchSourceSuiteBase extends 
KafkaSourceSuiteBase {
 true
   }
 
+  /**
+   * Test async progress tracking capability with Kafka source and sink
+   */
+  test("async progress tracking") {
+val inputTopic = newTopic()
+testUtils.createTopic(inputTopic, partitions = 5)
+
+val dataSent = new ListBuffer[String]()
+testUtils.sendMessages(inputTopic, (0 until 15).map { case x =>
+  val m = s"foo-$x"
+  dataSent += m
+  m
+}.toArray, Some(0))
+
+val outputTopic = newTopic()
+testUtils.createTopic(outputTopic, partitions = 5)
+
+withTempDir { dir =>
+  val reader = spark
+.readStream
+.format("kafka")
+.option("kafka.bootstrap.servers", testUtils.brokerAddress)
+.option("kafka.metadata.max.age.ms", "1")
+.option("maxOffsetsPerTrigger", 5)
+.option("subscribe", inputTopic)
+.option("startingOffsets", "earliest")
+.load()
+
+  def startQuery(): StreamingQuery = {
+reader.writeStream
+  .format("kafka")
+  .option("checkpointLocation", dir.getCanonicalPath)
+  .option("kafka.bootstrap.servers", testUtils.brokerAddress)
+  .option("kafka.max.block.ms", "5000")
+  .option("topic", outputTopic)
+  .option(ASYNC_PROGRESS_TRACKING_ENABLED, true)
+  .option(ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS, 1000)
+  .queryName("kafkaStream")
+  .start()
+  }
+
+  def readResults(): ListBuffer[String] = {

Review Comment:
   The output would be same but the code and actual execution would be much 
simpler in batch query. See below code when we just go with batch query:
   
   ```
   spark.read
 .format("kafka")
 .option("kafka.bootstrap.servers", testUtils.brokerAddress)
 .option("startingOffsets", "earliest")
 .option("subscribe", outputTopic)
 .load()
 .select("CAST(value AS string)")
 .toDS()
 .collect()
 .map(_._1)
   ```
   
   The entire code in the method can be replaced with this query. Haven't gave 
a try but the actual code that could execute won't be much different.



-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



[GitHub] [spark] HeartSaVioR commented on a diff in pull request #38517: [SPARK-39591][SS] Async Progress Tracking

2022-12-14 Thread GitBox


HeartSaVioR commented on code in PR #38517:
URL: https://github.com/apache/spark/pull/38517#discussion_r1049119941


##
connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala:
##
@@ -195,6 +200,102 @@ abstract class KafkaMicroBatchSourceSuiteBase extends 
KafkaSourceSuiteBase {
 true
   }
 
+  /**
+   * Test async progress tracking capability with Kafka source and sink
+   */
+  test("async progress tracking") {
+val inputTopic = newTopic()
+testUtils.createTopic(inputTopic, partitions = 5)
+
+val dataSent = new ListBuffer[String]()
+testUtils.sendMessages(inputTopic, (0 until 15).map { case x =>
+  val m = s"foo-$x"
+  dataSent += m
+  m
+}.toArray, Some(0))
+
+val outputTopic = newTopic()
+testUtils.createTopic(outputTopic, partitions = 5)
+
+withTempDir { dir =>
+  val reader = spark
+.readStream
+.format("kafka")
+.option("kafka.bootstrap.servers", testUtils.brokerAddress)
+.option("kafka.metadata.max.age.ms", "1")
+.option("maxOffsetsPerTrigger", 5)
+.option("subscribe", inputTopic)
+.option("startingOffsets", "earliest")
+.load()
+
+  def startQuery(): StreamingQuery = {
+reader.writeStream
+  .format("kafka")
+  .option("checkpointLocation", dir.getCanonicalPath)
+  .option("kafka.bootstrap.servers", testUtils.brokerAddress)
+  .option("kafka.max.block.ms", "5000")
+  .option("topic", outputTopic)
+  .option(ASYNC_PROGRESS_TRACKING_ENABLED, true)
+  .option(ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS, 1000)
+  .queryName("kafkaStream")
+  .start()
+  }
+
+  def readResults(): ListBuffer[String] = {

Review Comment:
   The output would be same but the code and actual execution would be much 
simpler in batch query. See below code when we just go with batch query:
   
   ```
   spark.read
 .format("kafka")
 .option("kafka.bootstrap.servers", testUtils.brokerAddress)
 .option("startingOffsets", "earliest")
 .option("subscribe", outputTopic)
 .load()
 .select("CAST(value AS string)")
 .toDS()
 .collect()
 .map(_._1)
   ```
   
   The entire code in the method can be replaced with this query. Haven't gave 
a try but the actual code that could execute won't be much different.



-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



[GitHub] [spark] HeartSaVioR commented on a diff in pull request #38517: [SPARK-39591][SS] Async Progress Tracking

2022-12-14 Thread GitBox


HeartSaVioR commented on code in PR #38517:
URL: https://github.com/apache/spark/pull/38517#discussion_r1049119941


##
connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala:
##
@@ -195,6 +200,102 @@ abstract class KafkaMicroBatchSourceSuiteBase extends 
KafkaSourceSuiteBase {
 true
   }
 
+  /**
+   * Test async progress tracking capability with Kafka source and sink
+   */
+  test("async progress tracking") {
+val inputTopic = newTopic()
+testUtils.createTopic(inputTopic, partitions = 5)
+
+val dataSent = new ListBuffer[String]()
+testUtils.sendMessages(inputTopic, (0 until 15).map { case x =>
+  val m = s"foo-$x"
+  dataSent += m
+  m
+}.toArray, Some(0))
+
+val outputTopic = newTopic()
+testUtils.createTopic(outputTopic, partitions = 5)
+
+withTempDir { dir =>
+  val reader = spark
+.readStream
+.format("kafka")
+.option("kafka.bootstrap.servers", testUtils.brokerAddress)
+.option("kafka.metadata.max.age.ms", "1")
+.option("maxOffsetsPerTrigger", 5)
+.option("subscribe", inputTopic)
+.option("startingOffsets", "earliest")
+.load()
+
+  def startQuery(): StreamingQuery = {
+reader.writeStream
+  .format("kafka")
+  .option("checkpointLocation", dir.getCanonicalPath)
+  .option("kafka.bootstrap.servers", testUtils.brokerAddress)
+  .option("kafka.max.block.ms", "5000")
+  .option("topic", outputTopic)
+  .option(ASYNC_PROGRESS_TRACKING_ENABLED, true)
+  .option(ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS, 1000)
+  .queryName("kafkaStream")
+  .start()
+  }
+
+  def readResults(): ListBuffer[String] = {

Review Comment:
   The output would be same but the code and actual execution would be much 
simpler in batch query. See below code when we just go with batch query:
   
   ```
   val data = spark.read.format("kafka")...load().select("CAST(value AS 
string)").toDS().collect().map(_._1)
   ```



-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



[GitHub] [spark] HeartSaVioR commented on a diff in pull request #38517: [SPARK-39591][SS] Async Progress Tracking

2022-12-14 Thread GitBox


HeartSaVioR commented on code in PR #38517:
URL: https://github.com/apache/spark/pull/38517#discussion_r1049083624


##
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecution.scala:
##
@@ -0,0 +1,282 @@
+/*
+ * 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.execution.streaming
+
+import java.util.concurrent._
+import java.util.concurrent.atomic.AtomicLong
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.streaming.WriteToStream
+import org.apache.spark.sql.streaming.Trigger
+import org.apache.spark.util.{Clock, ThreadUtils}
+
+object AsyncProgressTrackingMicroBatchExecution {
+  val ASYNC_PROGRESS_TRACKING_ENABLED = "asyncProgressTrackingEnabled"
+  val ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS =
+"asyncProgressTrackingCheckpointIntervalMs"
+
+  // for testing purposes
+  val ASYNC_PROGRESS_TRACKING_OVERRIDE_SINK_SUPPORT_CHECK =
+"_asyncProgressTrackingOverrideSinkSupportCheck"
+
+  private def getAsyncProgressTrackingCheckpointingIntervalMs(
+  extraOptions: Map[String, String]): Long = {
+extraOptions
+  .getOrElse(
+
AsyncProgressTrackingMicroBatchExecution.ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS,
+"1000"
+  )
+  .toLong
+  }
+}
+
+/**
+ * Class to execute micro-batches when async progress tracking is enabled
+ */
+class AsyncProgressTrackingMicroBatchExecution(
+sparkSession: SparkSession,
+trigger: Trigger,
+triggerClock: Clock,
+extraOptions: Map[String, String],
+plan: WriteToStream)
+extends MicroBatchExecution(sparkSession, trigger, triggerClock, 
extraOptions, plan) {
+
+  protected val asyncProgressTrackingCheckpointingIntervalMs: Long
+  = AsyncProgressTrackingMicroBatchExecution
+.getAsyncProgressTrackingCheckpointingIntervalMs(extraOptions)
+
+  // Offsets that are ready to be committed by the source.
+  // This is needed so that we can call source commit in the same thread as 
micro-batch execution
+  // to be thread safe
+  private val sourceCommitQueue = new ConcurrentLinkedQueue[OffsetSeq]()
+
+  // to cache the batch id of the last batch written to storage
+  private val lastBatchPersistedToDurableStorage = new AtomicLong(-1)
+
+  override val triggerExecutor: TriggerExecutor = validateAndGetTrigger()
+
+  private var isFirstBatch: Boolean = true
+
+  // thread pool is only one thread because we want offset
+  // writes to execute in order in a serialized fashion
+  protected val asyncWritesExecutorService
+  = ThreadUtils.newDaemonSingleThreadExecutorWithRejectedExecutionHandler(
+"async-log-write",
+2, // one for offset commit and one for completion commit
+new RejectedExecutionHandler() {
+  override def rejectedExecution(r: Runnable, executor: 
ThreadPoolExecutor): Unit = {
+try {
+  if (!executor.isShutdown) {
+val start = System.currentTimeMillis()
+executor.getQueue.put(r)
+logDebug(
+  s"Async write paused execution for " +
+s"${System.currentTimeMillis() - start} due to task queue 
being full."
+)
+  }
+} catch {
+  case e: InterruptedException =>
+Thread.currentThread.interrupt()
+throw new RejectedExecutionException("Producer interrupted", e)
+  case e: Throwable =>
+logError("Encountered error in async write executor service", e)
+errorNotifier.markError(e)
+}
+  }
+})
+
+  override val offsetLog = new AsyncOffsetSeqLog(
+sparkSession,
+checkpointFile("offsets"),
+asyncWritesExecutorService,
+asyncProgressTrackingCheckpointingIntervalMs,
+clock = triggerClock
+  )
+
+  override val commitLog =
+new AsyncCommitLog(sparkSession, checkpointFile("commits"), 
asyncWritesExecutorService)
+
+  override def markMicroBatchExecutionStart(): Unit = {
+// check if pipeline is stateful
+checkNotStatefulPipeline
+  }
+
+  override def cleanUpLastExecutedMicroBatch(): Unit = {
+// this is a no op for async progress tracking since we only want to 
commit sources only
+// after the 

[GitHub] [spark] HeartSaVioR commented on a diff in pull request #38517: [SPARK-39591][SS] Async Progress Tracking

2022-12-14 Thread GitBox


HeartSaVioR commented on code in PR #38517:
URL: https://github.com/apache/spark/pull/38517#discussion_r1049083624


##
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecution.scala:
##
@@ -0,0 +1,282 @@
+/*
+ * 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.execution.streaming
+
+import java.util.concurrent._
+import java.util.concurrent.atomic.AtomicLong
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.streaming.WriteToStream
+import org.apache.spark.sql.streaming.Trigger
+import org.apache.spark.util.{Clock, ThreadUtils}
+
+object AsyncProgressTrackingMicroBatchExecution {
+  val ASYNC_PROGRESS_TRACKING_ENABLED = "asyncProgressTrackingEnabled"
+  val ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS =
+"asyncProgressTrackingCheckpointIntervalMs"
+
+  // for testing purposes
+  val ASYNC_PROGRESS_TRACKING_OVERRIDE_SINK_SUPPORT_CHECK =
+"_asyncProgressTrackingOverrideSinkSupportCheck"
+
+  private def getAsyncProgressTrackingCheckpointingIntervalMs(
+  extraOptions: Map[String, String]): Long = {
+extraOptions
+  .getOrElse(
+
AsyncProgressTrackingMicroBatchExecution.ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS,
+"1000"
+  )
+  .toLong
+  }
+}
+
+/**
+ * Class to execute micro-batches when async progress tracking is enabled
+ */
+class AsyncProgressTrackingMicroBatchExecution(
+sparkSession: SparkSession,
+trigger: Trigger,
+triggerClock: Clock,
+extraOptions: Map[String, String],
+plan: WriteToStream)
+extends MicroBatchExecution(sparkSession, trigger, triggerClock, 
extraOptions, plan) {
+
+  protected val asyncProgressTrackingCheckpointingIntervalMs: Long
+  = AsyncProgressTrackingMicroBatchExecution
+.getAsyncProgressTrackingCheckpointingIntervalMs(extraOptions)
+
+  // Offsets that are ready to be committed by the source.
+  // This is needed so that we can call source commit in the same thread as 
micro-batch execution
+  // to be thread safe
+  private val sourceCommitQueue = new ConcurrentLinkedQueue[OffsetSeq]()
+
+  // to cache the batch id of the last batch written to storage
+  private val lastBatchPersistedToDurableStorage = new AtomicLong(-1)
+
+  override val triggerExecutor: TriggerExecutor = validateAndGetTrigger()
+
+  private var isFirstBatch: Boolean = true
+
+  // thread pool is only one thread because we want offset
+  // writes to execute in order in a serialized fashion
+  protected val asyncWritesExecutorService
+  = ThreadUtils.newDaemonSingleThreadExecutorWithRejectedExecutionHandler(
+"async-log-write",
+2, // one for offset commit and one for completion commit
+new RejectedExecutionHandler() {
+  override def rejectedExecution(r: Runnable, executor: 
ThreadPoolExecutor): Unit = {
+try {
+  if (!executor.isShutdown) {
+val start = System.currentTimeMillis()
+executor.getQueue.put(r)
+logDebug(
+  s"Async write paused execution for " +
+s"${System.currentTimeMillis() - start} due to task queue 
being full."
+)
+  }
+} catch {
+  case e: InterruptedException =>
+Thread.currentThread.interrupt()
+throw new RejectedExecutionException("Producer interrupted", e)
+  case e: Throwable =>
+logError("Encountered error in async write executor service", e)
+errorNotifier.markError(e)
+}
+  }
+})
+
+  override val offsetLog = new AsyncOffsetSeqLog(
+sparkSession,
+checkpointFile("offsets"),
+asyncWritesExecutorService,
+asyncProgressTrackingCheckpointingIntervalMs,
+clock = triggerClock
+  )
+
+  override val commitLog =
+new AsyncCommitLog(sparkSession, checkpointFile("commits"), 
asyncWritesExecutorService)
+
+  override def markMicroBatchExecutionStart(): Unit = {
+// check if pipeline is stateful
+checkNotStatefulPipeline
+  }
+
+  override def cleanUpLastExecutedMicroBatch(): Unit = {
+// this is a no op for async progress tracking since we only want to 
commit sources only
+// after the 

[GitHub] [spark] HeartSaVioR commented on a diff in pull request #38517: [SPARK-39591][SS] Async Progress Tracking

2022-12-14 Thread GitBox


HeartSaVioR commented on code in PR #38517:
URL: https://github.com/apache/spark/pull/38517#discussion_r1049070447


##
core/src/main/scala/org/apache/spark/util/ThreadUtils.scala:
##
@@ -167,6 +167,27 @@ private[spark] object ThreadUtils {
 Executors.newFixedThreadPool(1, 
threadFactory).asInstanceOf[ThreadPoolExecutor]
   }
 
+  /**
+   * Wrapper over newSingleThreadExecutor that allows the specification
+   * of a RejectedExecutionHandler
+   */
+  def newDaemonSingleThreadExecutorWithRejectedExecutionHandler(
+threadName: String,

Review Comment:
   Not really. Please note that params in method call and params in method 
definition have different indentation. 2 spaces for former, 4 spaces for latter.
   https://github.com/databricks/scala-style-guide#indent
   



-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



[GitHub] [spark] HeartSaVioR commented on a diff in pull request #38517: [SPARK-39591][SS] Async Progress Tracking

2022-12-08 Thread GitBox


HeartSaVioR commented on code in PR #38517:
URL: https://github.com/apache/spark/pull/38517#discussion_r1044068977


##
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecution.scala:
##
@@ -0,0 +1,282 @@
+/*
+ * 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.execution.streaming
+
+import java.util.concurrent._
+import java.util.concurrent.atomic.AtomicLong
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.streaming.WriteToStream
+import org.apache.spark.sql.streaming.Trigger
+import org.apache.spark.util.{Clock, ThreadUtils}
+
+object AsyncProgressTrackingMicroBatchExecution {
+  val ASYNC_PROGRESS_TRACKING_ENABLED = "asyncProgressTrackingEnabled"
+  val ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS =
+"asyncProgressTrackingCheckpointIntervalMs"
+
+  // for testing purposes
+  val ASYNC_PROGRESS_TRACKING_OVERRIDE_SINK_SUPPORT_CHECK =
+"_asyncProgressTrackingOverrideSinkSupportCheck"
+
+  private def getAsyncProgressTrackingCheckpointingIntervalMs(
+  extraOptions: Map[String, String]): Long = {
+extraOptions
+  .getOrElse(
+
AsyncProgressTrackingMicroBatchExecution.ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS,
+"1000"
+  )
+  .toLong
+  }
+}
+
+/**
+ * Class to execute micro-batches when async progress tracking is enabled
+ */
+class AsyncProgressTrackingMicroBatchExecution(
+sparkSession: SparkSession,
+trigger: Trigger,
+triggerClock: Clock,
+extraOptions: Map[String, String],
+plan: WriteToStream)
+extends MicroBatchExecution(sparkSession, trigger, triggerClock, 
extraOptions, plan) {
+
+  protected val asyncProgressTrackingCheckpointingIntervalMs: Long
+  = AsyncProgressTrackingMicroBatchExecution
+.getAsyncProgressTrackingCheckpointingIntervalMs(extraOptions)
+
+  // Offsets that are ready to be committed by the source.
+  // This is needed so that we can call source commit in the same thread as 
micro-batch execution
+  // to be thread safe
+  private val sourceCommitQueue = new ConcurrentLinkedQueue[OffsetSeq]()
+
+  // to cache the batch id of the last batch written to storage
+  private val lastBatchPersistedToDurableStorage = new AtomicLong(-1)
+
+  override val triggerExecutor: TriggerExecutor = validateAndGetTrigger()
+
+  private var isFirstBatch: Boolean = true
+
+  // thread pool is only one thread because we want offset
+  // writes to execute in order in a serialized fashion
+  protected val asyncWritesExecutorService
+  = ThreadUtils.newDaemonSingleThreadExecutorWithRejectedExecutionHandler(
+"async-log-write",
+2, // one for offset commit and one for completion commit
+new RejectedExecutionHandler() {
+  override def rejectedExecution(r: Runnable, executor: 
ThreadPoolExecutor): Unit = {
+try {
+  if (!executor.isShutdown) {
+val start = System.currentTimeMillis()
+executor.getQueue.put(r)
+logDebug(
+  s"Async write paused execution for " +
+s"${System.currentTimeMillis() - start} due to task queue 
being full."
+)
+  }
+} catch {
+  case e: InterruptedException =>
+Thread.currentThread.interrupt()
+throw new RejectedExecutionException("Producer interrupted", e)
+  case e: Throwable =>
+logError("Encountered error in async write executor service", e)
+errorNotifier.markError(e)
+}
+  }
+})
+
+  override val offsetLog = new AsyncOffsetSeqLog(
+sparkSession,
+checkpointFile("offsets"),
+asyncWritesExecutorService,
+asyncProgressTrackingCheckpointingIntervalMs,
+clock = triggerClock
+  )
+
+  override val commitLog =
+new AsyncCommitLog(sparkSession, checkpointFile("commits"), 
asyncWritesExecutorService)
+
+  override def markMicroBatchExecutionStart(): Unit = {
+// check if pipeline is stateful
+checkNotStatefulPipeline
+  }
+
+  override def cleanUpLastExecutedMicroBatch(): Unit = {
+// this is a no op for async progress tracking since we only want to 
commit sources only
+// after the 

[GitHub] [spark] HeartSaVioR commented on a diff in pull request #38517: [SPARK-39591][SS] Async Progress Tracking

2022-12-08 Thread GitBox


HeartSaVioR commented on code in PR #38517:
URL: https://github.com/apache/spark/pull/38517#discussion_r1044075012


##
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecutionSuite.scala:
##
@@ -0,0 +1,1865 @@
+/*
+ * 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.execution.streaming
+
+import java.io.{File, OutputStream}
+import java.util.concurrent.{CountDownLatch, Semaphore, ThreadPoolExecutor, 
TimeUnit}
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.hadoop.fs.Path
+import org.scalatest.BeforeAndAfter
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.time.{Seconds, Span}
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.streaming.WriteToStream
+import org.apache.spark.sql.connector.read.streaming
+import 
org.apache.spark.sql.execution.streaming.AsyncProgressTrackingMicroBatchExecution.{ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS,
 ASYNC_PROGRESS_TRACKING_ENABLED, 
ASYNC_PROGRESS_TRACKING_OVERRIDE_SINK_SUPPORT_CHECK}
+import org.apache.spark.sql.functions.{column, window}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.{StreamingQuery, 
StreamingQueryException, StreamTest, Trigger}
+import org.apache.spark.sql.streaming.util.StreamManualClock
+import org.apache.spark.util.{Clock, Utils}
+
+class AsyncProgressTrackingMicroBatchExecutionSuite
+extends StreamTest
+with BeforeAndAfter
+with Matchers {
+
+  import testImplicits._
+
+  after {
+sqlContext.streams.active.foreach(_.stop())
+  }
+
+  def getListOfFiles(dir: String): List[File] = {
+val d = new File(dir)
+if (d.exists && d.isDirectory) {
+  d.listFiles.filter(_.isFile).toList
+} else {
+  List[File]()
+}
+  }
+
+  def waitPendingOffsetWrites(streamExecution: StreamExecution): Unit = {
+
assert(streamExecution.isInstanceOf[AsyncProgressTrackingMicroBatchExecution])
+eventually(timeout(Span(5, Seconds))) {
+  streamExecution
+.asInstanceOf[AsyncProgressTrackingMicroBatchExecution]
+.areWritesPendingOrInProgress() should be(false)
+}
+  }
+
+  def waitPendingPurges(streamExecution: StreamExecution): Unit = {
+
assert(streamExecution.isInstanceOf[AsyncProgressTrackingMicroBatchExecution])
+eventually(timeout(Span(5, Seconds))) {
+  streamExecution
+.asInstanceOf[AsyncProgressTrackingMicroBatchExecution]
+.arePendingAsyncPurge should be(false)
+}
+  }
+
+  // test the basic functionality i.e. happy path
+  test("async WAL commits happy path") {

Review Comment:
   Never mind there are test cases.



-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



[GitHub] [spark] HeartSaVioR commented on a diff in pull request #38517: [SPARK-39591][SS] Async Progress Tracking

2022-12-08 Thread GitBox


HeartSaVioR commented on code in PR #38517:
URL: https://github.com/apache/spark/pull/38517#discussion_r1043380933


##
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecution.scala:
##
@@ -0,0 +1,282 @@
+/*
+ * 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.execution.streaming
+
+import java.util.concurrent._
+import java.util.concurrent.atomic.AtomicLong
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.streaming.WriteToStream
+import org.apache.spark.sql.streaming.Trigger
+import org.apache.spark.util.{Clock, ThreadUtils}
+
+object AsyncProgressTrackingMicroBatchExecution {
+  val ASYNC_PROGRESS_TRACKING_ENABLED = "asyncProgressTrackingEnabled"
+  val ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS =
+"asyncProgressTrackingCheckpointIntervalMs"
+
+  // for testing purposes
+  val ASYNC_PROGRESS_TRACKING_OVERRIDE_SINK_SUPPORT_CHECK =
+"_asyncProgressTrackingOverrideSinkSupportCheck"
+
+  private def getAsyncProgressTrackingCheckpointingIntervalMs(
+  extraOptions: Map[String, String]): Long = {
+extraOptions
+  .getOrElse(
+
AsyncProgressTrackingMicroBatchExecution.ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS,
+"1000"
+  )
+  .toLong
+  }
+}
+
+/**
+ * Class to execute micro-batches when async progress tracking is enabled
+ */
+class AsyncProgressTrackingMicroBatchExecution(
+sparkSession: SparkSession,
+trigger: Trigger,
+triggerClock: Clock,
+extraOptions: Map[String, String],
+plan: WriteToStream)
+extends MicroBatchExecution(sparkSession, trigger, triggerClock, 
extraOptions, plan) {
+
+  protected val asyncProgressTrackingCheckpointingIntervalMs: Long
+  = AsyncProgressTrackingMicroBatchExecution
+.getAsyncProgressTrackingCheckpointingIntervalMs(extraOptions)
+
+  // Offsets that are ready to be committed by the source.
+  // This is needed so that we can call source commit in the same thread as 
micro-batch execution
+  // to be thread safe
+  private val sourceCommitQueue = new ConcurrentLinkedQueue[OffsetSeq]()
+
+  // to cache the batch id of the last batch written to storage
+  private val lastBatchPersistedToDurableStorage = new AtomicLong(-1)
+
+  override val triggerExecutor: TriggerExecutor = validateAndGetTrigger()
+
+  private var isFirstBatch: Boolean = true
+
+  // thread pool is only one thread because we want offset
+  // writes to execute in order in a serialized fashion
+  protected val asyncWritesExecutorService
+  = ThreadUtils.newDaemonSingleThreadExecutorWithRejectedExecutionHandler(
+"async-log-write",
+2, // one for offset commit and one for completion commit
+new RejectedExecutionHandler() {
+  override def rejectedExecution(r: Runnable, executor: 
ThreadPoolExecutor): Unit = {
+try {
+  if (!executor.isShutdown) {
+val start = System.currentTimeMillis()
+executor.getQueue.put(r)
+logDebug(
+  s"Async write paused execution for " +
+s"${System.currentTimeMillis() - start} due to task queue 
being full."
+)
+  }
+} catch {
+  case e: InterruptedException =>
+Thread.currentThread.interrupt()
+throw new RejectedExecutionException("Producer interrupted", e)
+  case e: Throwable =>
+logError("Encountered error in async write executor service", e)
+errorNotifier.markError(e)
+}
+  }
+})
+
+  override val offsetLog = new AsyncOffsetSeqLog(
+sparkSession,
+checkpointFile("offsets"),
+asyncWritesExecutorService,
+asyncProgressTrackingCheckpointingIntervalMs,
+clock = triggerClock
+  )
+
+  override val commitLog =
+new AsyncCommitLog(sparkSession, checkpointFile("commits"), 
asyncWritesExecutorService)
+
+  override def markMicroBatchExecutionStart(): Unit = {
+// check if pipeline is stateful

Review Comment:
   There seems to be a term issue.
   
   
https://spark.apache.org/docs/latest/structured-streaming-programming-guide.html
   
   We have zero mention of pipeline in the guide doc. It's just a 

[GitHub] [spark] HeartSaVioR commented on a diff in pull request #38517: [SPARK-39591][SS] Async Progress Tracking

2022-12-08 Thread GitBox


HeartSaVioR commented on code in PR #38517:
URL: https://github.com/apache/spark/pull/38517#discussion_r1043434280


##
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecution.scala:
##
@@ -0,0 +1,282 @@
+/*
+ * 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.execution.streaming
+
+import java.util.concurrent._
+import java.util.concurrent.atomic.AtomicLong
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.streaming.WriteToStream
+import org.apache.spark.sql.streaming.Trigger
+import org.apache.spark.util.{Clock, ThreadUtils}
+
+object AsyncProgressTrackingMicroBatchExecution {
+  val ASYNC_PROGRESS_TRACKING_ENABLED = "asyncProgressTrackingEnabled"
+  val ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS =
+"asyncProgressTrackingCheckpointIntervalMs"
+
+  // for testing purposes
+  val ASYNC_PROGRESS_TRACKING_OVERRIDE_SINK_SUPPORT_CHECK =
+"_asyncProgressTrackingOverrideSinkSupportCheck"
+
+  private def getAsyncProgressTrackingCheckpointingIntervalMs(
+  extraOptions: Map[String, String]): Long = {
+extraOptions
+  .getOrElse(
+
AsyncProgressTrackingMicroBatchExecution.ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS,
+"1000"
+  )
+  .toLong
+  }
+}
+
+/**
+ * Class to execute micro-batches when async progress tracking is enabled
+ */
+class AsyncProgressTrackingMicroBatchExecution(
+sparkSession: SparkSession,
+trigger: Trigger,
+triggerClock: Clock,
+extraOptions: Map[String, String],
+plan: WriteToStream)
+extends MicroBatchExecution(sparkSession, trigger, triggerClock, 
extraOptions, plan) {
+
+  protected val asyncProgressTrackingCheckpointingIntervalMs: Long
+  = AsyncProgressTrackingMicroBatchExecution
+.getAsyncProgressTrackingCheckpointingIntervalMs(extraOptions)
+
+  // Offsets that are ready to be committed by the source.
+  // This is needed so that we can call source commit in the same thread as 
micro-batch execution
+  // to be thread safe
+  private val sourceCommitQueue = new ConcurrentLinkedQueue[OffsetSeq]()
+
+  // to cache the batch id of the last batch written to storage
+  private val lastBatchPersistedToDurableStorage = new AtomicLong(-1)
+
+  override val triggerExecutor: TriggerExecutor = validateAndGetTrigger()
+
+  private var isFirstBatch: Boolean = true
+
+  // thread pool is only one thread because we want offset
+  // writes to execute in order in a serialized fashion
+  protected val asyncWritesExecutorService
+  = ThreadUtils.newDaemonSingleThreadExecutorWithRejectedExecutionHandler(
+"async-log-write",
+2, // one for offset commit and one for completion commit
+new RejectedExecutionHandler() {
+  override def rejectedExecution(r: Runnable, executor: 
ThreadPoolExecutor): Unit = {
+try {
+  if (!executor.isShutdown) {
+val start = System.currentTimeMillis()
+executor.getQueue.put(r)
+logDebug(
+  s"Async write paused execution for " +
+s"${System.currentTimeMillis() - start} due to task queue 
being full."
+)
+  }
+} catch {
+  case e: InterruptedException =>
+Thread.currentThread.interrupt()
+throw new RejectedExecutionException("Producer interrupted", e)
+  case e: Throwable =>
+logError("Encountered error in async write executor service", e)
+errorNotifier.markError(e)
+}
+  }
+})
+
+  override val offsetLog = new AsyncOffsetSeqLog(
+sparkSession,
+checkpointFile("offsets"),
+asyncWritesExecutorService,
+asyncProgressTrackingCheckpointingIntervalMs,
+clock = triggerClock
+  )
+
+  override val commitLog =
+new AsyncCommitLog(sparkSession, checkpointFile("commits"), 
asyncWritesExecutorService)
+
+  override def markMicroBatchExecutionStart(): Unit = {
+// check if pipeline is stateful
+checkNotStatefulPipeline
+  }
+
+  override def cleanUpLastExecutedMicroBatch(): Unit = {
+// this is a no op for async progress tracking since we only want to 
commit sources only
+// after the 

[GitHub] [spark] HeartSaVioR commented on a diff in pull request #38517: [SPARK-39591][SS] Async Progress Tracking

2022-12-08 Thread GitBox


HeartSaVioR commented on code in PR #38517:
URL: https://github.com/apache/spark/pull/38517#discussion_r1043131174


##
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecution.scala:
##
@@ -0,0 +1,282 @@
+/*
+ * 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.execution.streaming
+
+import java.util.concurrent._
+import java.util.concurrent.atomic.AtomicLong
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.streaming.WriteToStream
+import org.apache.spark.sql.streaming.Trigger
+import org.apache.spark.util.{Clock, ThreadUtils}
+
+object AsyncProgressTrackingMicroBatchExecution {
+  val ASYNC_PROGRESS_TRACKING_ENABLED = "asyncProgressTrackingEnabled"
+  val ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS =
+"asyncProgressTrackingCheckpointIntervalMs"
+
+  // for testing purposes
+  val ASYNC_PROGRESS_TRACKING_OVERRIDE_SINK_SUPPORT_CHECK =
+"_asyncProgressTrackingOverrideSinkSupportCheck"
+
+  private def getAsyncProgressTrackingCheckpointingIntervalMs(
+  extraOptions: Map[String, String]): Long = {
+extraOptions
+  .getOrElse(
+
AsyncProgressTrackingMicroBatchExecution.ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS,
+"1000"
+  )
+  .toLong
+  }
+}
+
+/**
+ * Class to execute micro-batches when async progress tracking is enabled
+ */
+class AsyncProgressTrackingMicroBatchExecution(
+sparkSession: SparkSession,
+trigger: Trigger,
+triggerClock: Clock,
+extraOptions: Map[String, String],
+plan: WriteToStream)
+extends MicroBatchExecution(sparkSession, trigger, triggerClock, 
extraOptions, plan) {
+
+  protected val asyncProgressTrackingCheckpointingIntervalMs: Long
+  = AsyncProgressTrackingMicroBatchExecution
+.getAsyncProgressTrackingCheckpointingIntervalMs(extraOptions)
+
+  // Offsets that are ready to be committed by the source.
+  // This is needed so that we can call source commit in the same thread as 
micro-batch execution
+  // to be thread safe
+  private val sourceCommitQueue = new ConcurrentLinkedQueue[OffsetSeq]()
+
+  // to cache the batch id of the last batch written to storage
+  private val lastBatchPersistedToDurableStorage = new AtomicLong(-1)
+
+  override val triggerExecutor: TriggerExecutor = validateAndGetTrigger()
+
+  private var isFirstBatch: Boolean = true
+
+  // thread pool is only one thread because we want offset
+  // writes to execute in order in a serialized fashion
+  protected val asyncWritesExecutorService
+  = ThreadUtils.newDaemonSingleThreadExecutorWithRejectedExecutionHandler(
+"async-log-write",
+2, // one for offset commit and one for completion commit
+new RejectedExecutionHandler() {
+  override def rejectedExecution(r: Runnable, executor: 
ThreadPoolExecutor): Unit = {
+try {
+  if (!executor.isShutdown) {
+val start = System.currentTimeMillis()
+executor.getQueue.put(r)
+logDebug(
+  s"Async write paused execution for " +
+s"${System.currentTimeMillis() - start} due to task queue 
being full."
+)
+  }
+} catch {
+  case e: InterruptedException =>
+Thread.currentThread.interrupt()
+throw new RejectedExecutionException("Producer interrupted", e)
+  case e: Throwable =>
+logError("Encountered error in async write executor service", e)
+errorNotifier.markError(e)
+}
+  }
+})
+
+  override val offsetLog = new AsyncOffsetSeqLog(
+sparkSession,
+checkpointFile("offsets"),
+asyncWritesExecutorService,
+asyncProgressTrackingCheckpointingIntervalMs,
+clock = triggerClock
+  )
+
+  override val commitLog =
+new AsyncCommitLog(sparkSession, checkpointFile("commits"), 
asyncWritesExecutorService)
+
+  override def markMicroBatchExecutionStart(): Unit = {
+// check if pipeline is stateful
+checkNotStatefulPipeline
+  }
+
+  override def cleanUpLastExecutedMicroBatch(): Unit = {
+// this is a no op for async progress tracking since we only want to 
commit sources only
+// after the 

[GitHub] [spark] HeartSaVioR commented on a diff in pull request #38517: [SPARK-39591][SS] Async Progress Tracking

2022-12-08 Thread GitBox


HeartSaVioR commented on code in PR #38517:
URL: https://github.com/apache/spark/pull/38517#discussion_r1043003950


##
core/src/main/scala/org/apache/spark/util/ThreadUtils.scala:
##
@@ -167,6 +167,27 @@ private[spark] object ThreadUtils {
 Executors.newFixedThreadPool(1, 
threadFactory).asInstanceOf[ThreadPoolExecutor]
   }
 
+  /**
+   * Wrapper over newSingleThreadExecutor that allows the specification
+   * of a RejectedExecutionHandler
+   */
+  def newDaemonSingleThreadExecutorWithRejectedExecutionHandler(
+threadName: String,

Review Comment:
   nit: 4 spaces for definition of params



##
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:
##
@@ -2014,7 +2014,6 @@ object SQLConf {
   .booleanConf
   .createWithDefault(true)
 
-

Review Comment:
   nit: unnecessary change



##
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala:
##
@@ -342,17 +342,14 @@ class MicroBatchExecution(
 isCurrentBatchConstructed = true
 availableOffsets = nextOffsets.toStreamProgress(sources)
 /* Initialize committed offsets to a committed batch, which at this
- * is the second latest batch id in the offset log. */
-if (latestBatchId != 0) {
-  val secondLatestOffsets = offsetLog.get(latestBatchId - 1).getOrElse 
{
-logError(s"The offset log for batch ${latestBatchId - 1} doesn't 
exist, " +
-  s"which is required to restart the query from the latest batch 
$latestBatchId " +
-  "from the offset log. Please ensure there are two subsequent 
offset logs " +
-  "available for the latest batch via manually deleting the offset 
file(s). " +
-  "Please also ensure the latest batch for commit log is equal or 
one batch " +
-  "earlier than the latest batch for offset log.")
-throw new IllegalStateException(s"batch ${latestBatchId - 1} 
doesn't exist")
-  }
+ * is the second latest batch id in the offset log.
+ * The offset log may not be contiguous */
+val prevBatchId = offsetLog.getPrevBatchFromStorage(latestBatchId)
+if (latestBatchId != 0 && prevBatchId.isDefined) {
+val secondLatestOffsets = 
offsetLog.get(prevBatchId.get).getOrElse({

Review Comment:
   nit: indentation looks to be off



##
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala:
##
@@ -148,6 +148,24 @@ class HDFSMetadataLog[T <: AnyRef : 
ClassTag](sparkSession: SparkSession, path:
 }
   }
 
+  /**
+   * Get the id of the previous batch from storage

Review Comment:
   nit: We don't require to fill all the form with meaningless info. Please 
remove parts if you don't feel it's helpful or you're just reiterating just to 
fill out.



##
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala:
##
@@ -342,17 +342,14 @@ class MicroBatchExecution(
 isCurrentBatchConstructed = true
 availableOffsets = nextOffsets.toStreamProgress(sources)
 /* Initialize committed offsets to a committed batch, which at this
- * is the second latest batch id in the offset log. */
-if (latestBatchId != 0) {
-  val secondLatestOffsets = offsetLog.get(latestBatchId - 1).getOrElse 
{
-logError(s"The offset log for batch ${latestBatchId - 1} doesn't 
exist, " +
-  s"which is required to restart the query from the latest batch 
$latestBatchId " +
-  "from the offset log. Please ensure there are two subsequent 
offset logs " +
-  "available for the latest batch via manually deleting the offset 
file(s). " +
-  "Please also ensure the latest batch for commit log is equal or 
one batch " +
-  "earlier than the latest batch for offset log.")
-throw new IllegalStateException(s"batch ${latestBatchId - 1} 
doesn't exist")
-  }
+ * is the second latest batch id in the offset log.

Review Comment:
   I'm actually in favor of limiting the change to async progress tracking one, 
like we are adding protected methods for extension.
   
   Do we have a goal to support smooth transition between normal microbatch 
execution and async progress tracking for a single query? If we want to do so, 
we should have a clear explanation on the semantic and behavior during 
transition between two (both directions). Otherwise, I'd rather say let's leave 
normal microbatch execution work for the same.



##
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala:
##
@@ -727,18 +719,56 @@ class MicroBatchExecution(
 
 withProgressLocked {
   sinkCommitProgress = batchSinkProgress
-  watermarkTracker.updateWatermark(lastExecution.executedPlan)
-  reportTimeTaken("commitOffsets") {
-