Skip to content
Merged
Show file tree
Hide file tree
Changes from all 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
Original file line number Diff line number Diff line change
Expand Up @@ -137,7 +137,14 @@ private Optional<StartingScanner.Result> applyPushDownLimit() {

long scannedRowCount = 0;
SnapshotReader.Plan plan = ((ScannedResult) result).plan();
List<DataSplit> splits = plan.dataSplits();
List<Split> planSplits = plan.splits();
// Limit pushdown only supports DataSplit. Skip for IncrementalSplit.
if (planSplits.stream().anyMatch(s -> !(s instanceof DataSplit))) {
return Optional.of(result);
}
@SuppressWarnings("unchecked")
List<DataSplit> splits = (List<DataSplit>) (List<?>) planSplits;

LOG.info("Applying limit pushdown. Original splits count: {}", splits.size());
if (splits.isEmpty()) {
return Optional.of(result);
Expand Down Expand Up @@ -193,7 +200,13 @@ private Optional<StartingScanner.Result> applyPushDownTopN() {
}

SnapshotReader.Plan plan = ((ScannedResult) result).plan();
List<DataSplit> splits = plan.dataSplits();
List<Split> planSplits = plan.splits();
// TopN pushdown only supports DataSplit. Skip for IncrementalSplit.
if (planSplits.stream().anyMatch(s -> !(s instanceof DataSplit))) {
return Optional.of(result);
}
@SuppressWarnings("unchecked")
List<DataSplit> splits = (List<DataSplit>) (List<?>) planSplits;
if (splits.isEmpty()) {
return Optional.of(result);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -344,6 +344,27 @@ class TableValuedFunctionsTest extends PaimonHiveTestBase {
}
}

test("incremental query by tag with LIMIT") {
sql("use paimon")
withTable("t") {
spark.sql("""
|CREATE TABLE t (a INT, b INT, c STRING)
|USING paimon
|TBLPROPERTIES ('primary-key'='a,b', 'bucket' = '2')
|PARTITIONED BY (a)
|""".stripMargin)
spark.sql("INSERT INTO t VALUES (1, 1, '1'), (2, 2, '2')")
sql("CALL sys.create_tag('t', 'tag1')")
spark.sql("INSERT INTO t VALUES (1, 3, '3'), (2, 4, '4')")
sql("CALL sys.create_tag('t', 'tag2')")

checkAnswer(
spark.sql(
"SELECT * FROM paimon_incremental_query('t', 'tag1', 'tag2') ORDER BY a, b LIMIT 5"),
Seq(Row(1, 3, "3"), Row(2, 4, "4")))
}
}

private def incrementalDF(tableIdent: String, start: Int, end: Int): DataFrame = {
spark.read
.format("paimon")
Expand Down