Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[issue-12] Implement rate limiting in micro batch reader #164

Merged
merged 22 commits into from
Sep 25, 2023
Merged
Show file tree
Hide file tree
Changes from 15 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 3 additions & 1 deletion gradle.properties
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,9 @@ sparkVersion=3.4.0

# Version and base tags can be overridden at build time.
connectorVersion=0.13.0-SNAPSHOT
pravegaVersion=0.13.0-3142.2542966-SNAPSHOT
#pravegaVersion=0.13.0-3142.2542966-SNAPSHOT
pravegaVersion=0.14.0-3257.fd53bec-SNAPSHOT


# flag to indicate if Pravega sub-module should be used instead of the version defined in 'pravegaVersion'
usePravegaVersionSubModule=false
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package io.pravega.connectors.spark

import io.pravega.client.batch.SegmentRange
import io.pravega.client.stream.TruncatedDataException
import io.pravega.client.stream.impl.ByteBufferSerializer
import io.pravega.client.{BatchClientFactory, ClientConfig}
import org.apache.spark.internal.Logging
Expand Down Expand Up @@ -52,13 +53,23 @@ case class PravegaBatchPartitionReader(
override def next(): Boolean = {
if (iterator.hasNext) {
val offset = iterator.getOffset
val event = iterator.next()
nextRow = converter.toUnsafeRow(
event.array,
segmentRange.getScope,
segmentRange.getStreamName,
segmentRange.getSegmentId,
offset)

try
{
val event = iterator.next()
nextRow = converter.toUnsafeRow(
event.array,
segmentRange.getScope,
segmentRange.getStreamName,
segmentRange.getSegmentId,
offset)
}
catch
{
case e: TruncatedDataException =>
log.warn("next: TruncatedDataException while reading data", e)
munish1789 marked this conversation as resolved.
Show resolved Hide resolved
return false
tkaitchuck marked this conversation as resolved.
Show resolved Hide resolved
}
true
} else false
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,12 @@
package io.pravega.connectors.spark

import io.pravega.client.admin.StreamManager
import io.pravega.client.segment.impl.SegmentTruncatedException
import io.pravega.client.stream.{Stream, StreamCut}
import io.pravega.client.{BatchClientFactory, ClientConfig}
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset}
import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset, ReadAllAvailable, ReadLimit, ReadMaxRows, SupportsAdmissionControl}
import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory}
import resource.managed

Expand All @@ -43,12 +44,13 @@ class PravegaMicroBatchStream(
startStreamCut: PravegaStreamCut,
endStreamCut: PravegaStreamCut
)
extends MicroBatchStream with Logging {
extends MicroBatchStream with SupportsAdmissionControl with Logging {

protected var batchStartStreamCut: StreamCut = _
protected var batchEndStreamCut: StreamCut = _
log.info(s"Initializing micro-batch stream: ${this}")

private val batchClientFactory = BatchClientFactory.withScope(scopeName, clientConfig)

private val streamManager = StreamManager.create(clientConfig)

Expand Down Expand Up @@ -129,4 +131,37 @@ class PravegaMicroBatchStream(
s"PravegaMicroBatchStream{clientConfig=${clientConfig}, scopeName=${scopeName}, streamName=${streamName}" +
s" startStreamCut=${startStreamCut}, endStreamCut=${endStreamCut}}"
}

override def latestOffset(start: Offset, readLimit: ReadLimit): Offset = {
val startOffset = Option(start)
.map(_.asInstanceOf[PravegaSourceOffset].streamCut).get
val limits: Seq[ReadLimit] = readLimit match {
case rows => Seq(rows)
}
val nextStreamCut = if (limits.exists(_.isInstanceOf[ReadAllAvailable])) {
PravegaSourceOffset(PravegaUtils.getStreamInfo(streamManager, scopeName, streamName).getTailStreamCut)
} else {
val upperLimit = limits.find(_.isInstanceOf[ReadMaxRows]).map(_.asInstanceOf[ReadMaxRows])
PravegaSourceOffset(batchClientFactory.getNextStreamCut(startOffset, upperLimit.get.maxRows()))
munish1789 marked this conversation as resolved.
Show resolved Hide resolved
try {
PravegaSourceOffset(batchClientFactory.getNextStreamCut(startOffset, upperLimit.get.maxRows()))
munish1789 marked this conversation as resolved.
Show resolved Hide resolved
}
catch
{
case e: SegmentTruncatedException => PravegaSourceOffset(PravegaUtils.getStreamInfo(streamManager, scopeName, streamName).getHeadStreamCut)
}
}
munish1789 marked this conversation as resolved.
Show resolved Hide resolved
log.info(s"nextStreamCut = ${nextStreamCut.streamCut}")
nextStreamCut
}

override def getDefaultReadLimit: ReadLimit = {
val approxBytesPerTrigger = Option(options.get(
PravegaSourceProvider.APPROX_BYTES_PER_TRIGGER)).get.map(_.toLong)
if ( approxBytesPerTrigger.isDefined) {
ReadLimit.maxRows(approxBytesPerTrigger.get)
} else {
super.getDefaultReadLimit
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -96,6 +96,7 @@ object PravegaSourceProvider extends Logging {
private[spark] val DEFAULT_BATCH_TRANSACTION_TIMEOUT_MS: Long = 2 * 60 * 1000 // 2 minutes (maximum allowed by default server)
private[spark] val DEFAULT_TRANSACTION_STATUS_POLL_INTERVAL_MS: Long = 50

private[spark] val APPROX_BYTES_PER_TRIGGER = "approx_bytes_per_trigger"

def buildStreamConfig(caseInsensitiveParams: Map[String, String]): StreamConfiguration = {
var streamConfig = StreamConfiguration.builder
Expand Down Expand Up @@ -201,6 +202,12 @@ object PravegaSourceProvider extends Logging {
if (retentionMilliseconds.isDefined && (Try(retentionMilliseconds.get.toInt).isFailure || retentionMilliseconds.get.toInt < 0)) {
throw new IllegalArgumentException(s"Retention time should be an integer morethan or equal to zero milliseconds")
}

val approxBytesPerTrigger = caseInsensitiveParams.get(PravegaSourceProvider.APPROX_BYTES_PER_TRIGGER)
if (approxBytesPerTrigger.isDefined && (Try(approxBytesPerTrigger.get.toLong).isFailure || approxBytesPerTrigger.get.toLong < 1))
{
throw new IllegalArgumentException(s"Max offset should be of type long more than or equal to one")
}
}

def buildClientConfig(caseInsensitiveParams: Map[String, String]): ClientConfig = {
Expand Down
Loading