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

[spark] Fallback to spark except query if increment query with rescale bucket #4989

Merged
merged 4 commits into from
Jan 23, 2025
Merged
Show file tree
Hide file tree
Changes from 2 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 @@ -57,8 +57,8 @@ public IncrementalTagStartingScanner(
snapshotManager.fileIO(),
snapshotManager.tablePath(),
snapshotManager.branch()),
start.schemaId(),
end.schemaId());
start,
end);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -128,20 +128,47 @@ private static Snapshot resolveSnapshotByTagName(
}

public static void checkRescaleBucketForIncrementalTagQuery(
SchemaManager schemaManager, long schemaId1, long schemaId2) {
if (schemaId1 != schemaId2) {
int bucketNumber1 = bucketNumber(schemaManager, schemaId1);
int bucketNumber2 = bucketNumber(schemaManager, schemaId2);
checkArgument(
bucketNumber1 == bucketNumber2,
"The bucket number of two tags are different (%s, %s), which is not supported in incremental tag query.",
bucketNumber1,
bucketNumber2);
SchemaManager schemaManager, Snapshot start, Snapshot end) {
if (start.schemaId() != end.schemaId()) {
int startBucketNumber = bucketNumber(schemaManager, start.schemaId());
int endBucketNumber = bucketNumber(schemaManager, end.schemaId());
if (startBucketNumber != endBucketNumber) {
throw new InconsistentTagBucketException(
start.id(),
end.id(),
String.format(
"The bucket number of two tags are different (%s, %s), which is not supported in incremental tag query.",
startBucketNumber, endBucketNumber));
}
}
}

private static int bucketNumber(SchemaManager schemaManager, long schemaId) {
TableSchema schema = schemaManager.schema(schemaId);
return CoreOptions.fromMap(schema.options()).bucket();
}

/**
* Exception thrown when the bucket number of two tags are different in incremental tag query.
*/
public static class InconsistentTagBucketException extends RuntimeException {

private final long startSnapshotId;
private final long endSnapshotId;

public InconsistentTagBucketException(
long startSnapshotId, long endSnapshotId, String message) {
super(message);
this.startSnapshotId = startSnapshotId;
this.endSnapshotId = endSnapshotId;
}

public long startSnapshotId() {
return startSnapshotId;
}

public long endSnapshotId() {
return endSnapshotId;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,15 +19,19 @@
package org.apache.paimon.spark.catalyst.plans.logical

import org.apache.paimon.CoreOptions
import org.apache.paimon.spark.SparkTable
import org.apache.paimon.spark.catalyst.plans.logical.PaimonTableValuedFunctions._
import org.apache.paimon.table.FileStoreTable
import org.apache.paimon.table.source.snapshot.TimeTravelUtil.InconsistentTagBucketException

import org.apache.spark.sql.PaimonUtils.createDataset
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.FunctionIdentifier
import org.apache.spark.sql.catalyst.analysis.FunctionRegistryBase
import org.apache.spark.sql.catalyst.analysis.TableFunctionRegistry.TableFunctionBuilder
import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, ExpressionInfo}
import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan}
import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog}
import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog}
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
import org.apache.spark.sql.util.CaseInsensitiveStringMap

Expand Down Expand Up @@ -83,11 +87,65 @@ object PaimonTableValuedFunctions {
val sparkTable = sparkCatalog.loadTable(ident)
val options = tvf.parseArgs(args.tail)

DataSourceV2Relation.create(
sparkTable,
Some(sparkCatalog),
Some(ident),
new CaseInsensitiveStringMap(options.asJava))
usingSparkIncrementQuery(tvf, sparkTable, options) match {
case Some(snapshotIdPair: (Long, Long)) =>
sparkIncrementQuery(spark, sparkTable, sparkCatalog, ident, options, snapshotIdPair)
case _ =>
DataSourceV2Relation.create(
sparkTable,
Some(sparkCatalog),
Some(ident),
new CaseInsensitiveStringMap(options.asJava))
}
}

private def usingSparkIncrementQuery(
tvf: PaimonTableValueFunction,
sparkTable: Table,
options: Map[String, String]): Option[(Long, Long)] = {
tvf.fnName match {
case INCREMENTAL_QUERY | INCREMENTAL_TO_AUTO_TAG =>
sparkTable match {
case SparkTable(fileStoreTable: FileStoreTable) =>
try {
fileStoreTable.copy(options.asJava).newScan().plan()
None
} catch {
case e: InconsistentTagBucketException =>
Some((e.startSnapshotId, e.endSnapshotId))
}
case _ => None
}
case _ => None
}
}

private def sparkIncrementQuery(
spark: SparkSession,
sparkTable: Table,
sparkCatalog: TableCatalog,
ident: Identifier,
options: Map[String, String],
snapshotIdPair: (Long, Long)): LogicalPlan = {
val filteredOptions =
options - CoreOptions.INCREMENTAL_BETWEEN.key - CoreOptions.INCREMENTAL_TO_AUTO_TAG.key

def datasetOfSnapshot(snapshotId: Long) = {
val updatedOptions = filteredOptions + (CoreOptions.SCAN_VERSION.key() -> snapshotId.toString)
createDataset(
spark,
DataSourceV2Relation.create(
sparkTable,
Some(sparkCatalog),
Some(ident),
new CaseInsensitiveStringMap(updatedOptions.asJava)
))
}

datasetOfSnapshot(snapshotIdPair._2)
.except(datasetOfSnapshot(snapshotIdPair._1))
.queryExecution
.logical
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -189,6 +189,33 @@ class TableValuedFunctionsTest extends PaimonHiveTestBase {
}
}

test("Table Valued Functions: incremental query with inconsistent tag bucket") {
withTable("t") {
sql("""
|CREATE TABLE t (a INT, b INT) USING paimon
|TBLPROPERTIES ('primary-key'='a', 'bucket' = '1')
|""".stripMargin)

val table = loadTable("t")

sql("INSERT INTO t VALUES (1, 11), (2, 22)")
sql("ALTER TABLE t SET TBLPROPERTIES ('bucket' = '2')")
sql("INSERT OVERWRITE t SELECT * FROM t")
sql("INSERT INTO t VALUES (3, 33)")

table.createTag("2024-01-01", 1)
table.createTag("2024-01-02", 3)

checkAnswer(
sql(
"SELECT * FROM paimon_incremental_query('t', '2024-01-01', '2024-01-02') ORDER BY a, b"),
Seq(Row(3, 33)))
checkAnswer(
sql("SELECT * FROM paimon_incremental_to_auto_tag('t', '2024-01-02') ORDER BY a, b"),
Seq(Row(3, 33)))
}
}

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