Yohahaha commented on code in PR #2523:
URL: https://github.com/apache/fluss/pull/2523#discussion_r2754549846
##########
fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/read/FlussUpsertPartitionReader.scala:
##########
@@ -50,39 +90,153 @@ class FlussUpsertPartitionReader(
return false
}
- if (snapshotIterator == null || !snapshotIterator.hasNext) {
- // Try to get next batch from snapshot scanner
- val batch = snapshotScanner.pollBatch(POLL_TIMEOUT)
- if (batch == null) {
- // No more data fetched.
- false
+ if (mergedIterator.hasNext) {
+ currentRow = convertToSparkRow(mergedIterator.next())
+ true
+ } else {
+ false
+ }
+ }
+
+ private def createSortMergeReader(): SortMergeReader = {
+ // Create key encoder for primary keys
+ val keyEncoder =
+ encode.KeyEncoder.ofPrimaryKeyEncoder(
+ rowType,
+ tableInfo.getPhysicalPrimaryKeys,
+ tableInfo.getTableConfig,
+ tableInfo.isDefaultBucketKey)
+
+ // Create comparators based on primary key
+ val comparator = new Comparator[InternalRow] {
+ override def compare(o1: InternalRow, o2: InternalRow): Int = {
+ val key1 = keyEncoder.encodeKey(o1)
+ val key2 = keyEncoder.encodeKey(o2)
+ MemorySegment.wrap(key1).compare(MemorySegment.wrap(key2), 0, 0,
key1.length)
+ }
+ }
+
+ def createLogChangesIterator(): LogChangesIterator = {
+ // Initialize the log scanner
+ logScanner =
table.newScan().project(projectionWithPks).createLogScanner()
+ if (tableBucket.getPartitionId == null) {
+ logScanner.subscribe(tableBucket.getBucket, logStartingOffset)
} else {
- snapshotIterator = batch
- if (snapshotIterator.hasNext) {
- currentRow = convertToSparkRow(snapshotIterator.next())
- true
- } else {
- // Poll a new batch
- next()
+ logScanner.subscribe(tableBucket.getPartitionId,
tableBucket.getBucket, logStartingOffset)
+ }
+
+ // Collect all log records until logStoppingOffset
+ val allLogRecords = mutable.ArrayBuffer[ScanRecord]()
Review Comment:
we need to fetch by size to avoid OOM when log store has huge records.
--
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]