HeartSaVioR commented on a change in pull request #34333:
URL: https://github.com/apache/spark/pull/34333#discussion_r739945169



##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RatePerMicroBatchStream.scala
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.sources
+
+import org.json4s.NoTypeHints
+import org.json4s.jackson.Serialization
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, 
PartitionReaderFactory}
+import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, 
Offset, ReadLimit, ReadMaxRows, SupportsAdmissionControl}
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class RatePerMicroBatchStream(
+    rowsPerBatch: Long,
+    numPartitions: Int,
+    startTimestamp: Long,
+    advanceMsPerBatch: Int,
+    options: CaseInsensitiveStringMap)
+  extends SupportsAdmissionControl with MicroBatchStream with Logging {
+
+  override def initialOffset(): Offset = RatePerMicroBatchStreamOffset(0L, 
startTimestamp)
+
+  override def latestOffset(): Offset = {
+    throw new UnsupportedOperationException(
+      "latestOffset(Offset, ReadLimit) should be called instead of this 
method")
+  }
+
+  override def getDefaultReadLimit: ReadLimit = {
+    ReadLimit.maxRows(rowsPerBatch)
+  }
+
+  private def extractOffsetAndTimestamp(offset: Offset): (Long, Long) = {
+    offset match {
+      case o: RatePerMicroBatchStreamOffset => (o.offset, o.timestamp)
+      case _ => throw new IllegalStateException("The type of Offset should be 
" +
+        "RatePerMicroBatchStreamOffset")
+    }
+  }
+
+  override def latestOffset(startOffset: Offset, limit: ReadLimit): Offset = {

Review comment:
       > We cannot change the limit for this data source
   
   The data source doesn't provide such option, but Spark will pass 
ReadLimit.allAvailable() when the trigger is either Once or AvailableNow.
   
   That said, I think you found the bug. I missed to handle these triggers. Not 
sure what is the desired behavior for these triggers, though. Probably simply 
providing a set of inputs for a single batch is acceptable.
   
   I'll fix it. Thanks!




-- 
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]

Reply via email to