From 78e2e1dd456ac63eedfa01bdaca5e602f0f4e29d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=BE=8A=E5=B7=9D?= Date: Thu, 9 Apr 2026 14:32:02 +0800 Subject: [PATCH 1/3] Add spark tiering service --- .../FlussTableLakeSnapshotCommitter.java | 10 +- .../tiering}/TieringCommitterInitContext.java | 2 +- .../tiering}/TieringWriterInitContext.java | 2 +- .../committer/TieringCommitOperator.java | 2 + .../tiering/source/TieringSplitReader.java | 1 + .../enumerator/TieringSourceEnumerator.java | 12 + .../FlussTableLakeSnapshotCommitterTest.java | 1 + .../committer/TieringCommitOperatorTest.java | 1 + .../DvTableReadableSnapshotRetrieverTest.java | 2 +- .../spark/tiering/SparkTieringJobRunner.scala | 371 ++++++++++++++++++ .../spark/tiering/SparkTieringOptions.scala | 58 +++ .../spark/tiering/TieringCommitter.scala | 202 ++++++++++ .../spark/tiering/TieringCoordinator.scala | 234 +++++++++++ .../fluss/spark/tiering/TieringSplit.scala | 78 ++++ .../spark/tiering/TieringSplitGenerator.scala | 302 ++++++++++++++ .../fluss/spark/tiering/TieringTask.scala | 231 +++++++++++ .../spark/tiering/TieringTaskResult.scala | 55 +++ .../apache/fluss/spark/tiering/package.scala | 33 ++ fluss-spark/fluss-spark-tiering/pom.xml | 96 +++++ .../spark/tiering/SparkLakeTiering.scala | 130 ++++++ .../tiering/SparkTieringLogTableTest.scala | 153 ++++++++ .../spark/tiering/SparkTieringTestBase.scala | 105 +++++ fluss-spark/pom.xml | 1 + fluss-test-coverage/pom.xml | 4 +- 24 files changed, 2075 insertions(+), 11 deletions(-) rename {fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer => fluss-client/src/main/java/org/apache/fluss/client/tiering}/FlussTableLakeSnapshotCommitter.java (98%) rename {fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer => fluss-client/src/main/java/org/apache/fluss/client/tiering}/TieringCommitterInitContext.java (97%) rename {fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source => fluss-client/src/main/java/org/apache/fluss/client/tiering}/TieringWriterInitContext.java (97%) create mode 100644 fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/SparkTieringJobRunner.scala create mode 100644 fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/SparkTieringOptions.scala create mode 100644 fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringCommitter.scala create mode 100644 fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringCoordinator.scala create mode 100644 fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringSplit.scala create mode 100644 fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringSplitGenerator.scala create mode 100644 fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringTask.scala create mode 100644 fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringTaskResult.scala create mode 100644 fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/package.scala create mode 100644 fluss-spark/fluss-spark-tiering/pom.xml create mode 100644 fluss-spark/fluss-spark-tiering/src/main/scala/org/apache/fluss/spark/tiering/SparkLakeTiering.scala create mode 100644 fluss-spark/fluss-spark-ut/src/test/scala/org/apache/fluss/spark/tiering/SparkTieringLogTableTest.scala create mode 100644 fluss-spark/fluss-spark-ut/src/test/scala/org/apache/fluss/spark/tiering/SparkTieringTestBase.scala diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshotCommitter.java b/fluss-client/src/main/java/org/apache/fluss/client/tiering/FlussTableLakeSnapshotCommitter.java similarity index 98% rename from fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshotCommitter.java rename to fluss-client/src/main/java/org/apache/fluss/client/tiering/FlussTableLakeSnapshotCommitter.java index 76474a5b26..6b43ad9f2e 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshotCommitter.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/tiering/FlussTableLakeSnapshotCommitter.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.fluss.flink.tiering.committer; +package org.apache.fluss.client.tiering; import org.apache.fluss.annotation.VisibleForTesting; import org.apache.fluss.client.metadata.MetadataUpdater; @@ -197,7 +197,7 @@ public void commit( } } - void commit( + public void commit( long tableId, long snapshotId, String lakeBucketTieredOffsetsPath, @@ -327,8 +327,7 @@ private CommitLakeTableSnapshotRequest toCommitLakeTableSnapshotRequest( return commitLakeTableSnapshotRequest; } - @VisibleForTesting - protected CommitLakeTableSnapshotRequest addLogEndOffsets( + public CommitLakeTableSnapshotRequest addLogEndOffsets( CommitLakeTableSnapshotRequest commitLakeTableSnapshotRequest, long tableId, long snapshotId, @@ -357,8 +356,7 @@ protected CommitLakeTableSnapshotRequest addLogEndOffsets( return commitLakeTableSnapshotRequest; } - @VisibleForTesting - CoordinatorGateway getCoordinatorGateway() { + public CoordinatorGateway getCoordinatorGateway() { return coordinatorGateway; } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/TieringCommitterInitContext.java b/fluss-client/src/main/java/org/apache/fluss/client/tiering/TieringCommitterInitContext.java similarity index 97% rename from fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/TieringCommitterInitContext.java rename to fluss-client/src/main/java/org/apache/fluss/client/tiering/TieringCommitterInitContext.java index 79b7aaeb4f..a4f5934d65 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/TieringCommitterInitContext.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/tiering/TieringCommitterInitContext.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.fluss.flink.tiering.committer; +package org.apache.fluss.client.tiering; import org.apache.fluss.config.Configuration; import org.apache.fluss.lake.committer.CommitterInitContext; diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringWriterInitContext.java b/fluss-client/src/main/java/org/apache/fluss/client/tiering/TieringWriterInitContext.java similarity index 97% rename from fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringWriterInitContext.java rename to fluss-client/src/main/java/org/apache/fluss/client/tiering/TieringWriterInitContext.java index 121c4fb9cb..3a155ebdf9 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringWriterInitContext.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/tiering/TieringWriterInitContext.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.fluss.flink.tiering.source; +package org.apache.fluss.client.tiering; import org.apache.fluss.lake.writer.WriterInitContext; import org.apache.fluss.metadata.TableBucket; diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperator.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperator.java index 2f3a69a7ea..570602b37a 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperator.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperator.java @@ -21,6 +21,8 @@ import org.apache.fluss.client.ConnectionFactory; import org.apache.fluss.client.admin.Admin; import org.apache.fluss.client.metadata.LakeSnapshot; +import org.apache.fluss.client.tiering.FlussTableLakeSnapshotCommitter; +import org.apache.fluss.client.tiering.TieringCommitterInitContext; import org.apache.fluss.config.Configuration; import org.apache.fluss.exception.LakeTableSnapshotNotExistException; import org.apache.fluss.flink.tiering.event.FailedTieringEvent; diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java index d59787e15d..b708c54dc2 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java @@ -23,6 +23,7 @@ import org.apache.fluss.client.table.scanner.ScanRecord; import org.apache.fluss.client.table.scanner.log.LogScanner; import org.apache.fluss.client.table.scanner.log.ScanRecords; +import org.apache.fluss.client.tiering.TieringWriterInitContext; import org.apache.fluss.flink.source.reader.BoundedSplitReader; import org.apache.fluss.flink.source.reader.RecordAndPos; import org.apache.fluss.flink.tiering.source.metrics.TieringMetrics; diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java index 89a10ee898..d27a61b92f 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/enumerator/TieringSourceEnumerator.java @@ -682,5 +682,17 @@ private TieringFinishInfo( this.isForceFinished = isForceFinished; this.stats = stats != null ? stats : TieringStats.UNKNOWN; } + + @Override + public String toString() { + return "TieringFinishInfo{" + + "tieringEpoch=" + + tieringEpoch + + ", isForceFinished=" + + isForceFinished + + ", stats=" + + stats + + '}'; + } } } diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshotCommitterTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshotCommitterTest.java index 51bd3bc1da..c4f8f96f0e 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshotCommitterTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/FlussTableLakeSnapshotCommitterTest.java @@ -18,6 +18,7 @@ package org.apache.fluss.flink.tiering.committer; import org.apache.fluss.client.metadata.LakeSnapshot; +import org.apache.fluss.client.tiering.FlussTableLakeSnapshotCommitter; import org.apache.fluss.exception.LakeTableSnapshotNotExistException; import org.apache.fluss.flink.utils.FlinkTestBase; import org.apache.fluss.lake.committer.LakeCommitResult; diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperatorTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperatorTest.java index 70d86bb2c8..bdac25ef46 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperatorTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperatorTest.java @@ -18,6 +18,7 @@ package org.apache.fluss.flink.tiering.committer; import org.apache.fluss.client.metadata.LakeSnapshot; +import org.apache.fluss.client.tiering.FlussTableLakeSnapshotCommitter; import org.apache.fluss.exception.LakeTableSnapshotNotExistException; import org.apache.fluss.flink.adapter.StreamOperatorParametersAdapter; import org.apache.fluss.flink.tiering.TestingLakeTieringFactory; diff --git a/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/utils/DvTableReadableSnapshotRetrieverTest.java b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/utils/DvTableReadableSnapshotRetrieverTest.java index 0ce3d5b8ef..15926a0f34 100644 --- a/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/utils/DvTableReadableSnapshotRetrieverTest.java +++ b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/utils/DvTableReadableSnapshotRetrieverTest.java @@ -21,10 +21,10 @@ import org.apache.fluss.client.Connection; import org.apache.fluss.client.ConnectionFactory; import org.apache.fluss.client.admin.Admin; +import org.apache.fluss.client.tiering.FlussTableLakeSnapshotCommitter; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.exception.FlussRuntimeException; -import org.apache.fluss.flink.tiering.committer.FlussTableLakeSnapshotCommitter; import org.apache.fluss.lake.committer.LakeCommitResult; import org.apache.fluss.metadata.PartitionInfo; import org.apache.fluss.metadata.PartitionSpec; diff --git a/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/SparkTieringJobRunner.scala b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/SparkTieringJobRunner.scala new file mode 100644 index 0000000000..771f217bba --- /dev/null +++ b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/SparkTieringJobRunner.scala @@ -0,0 +1,371 @@ +/* + * 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.fluss.spark.tiering + +import org.apache.fluss.client.{Connection, ConnectionFactory} +import org.apache.fluss.client.admin.Admin +import org.apache.fluss.client.tiering.FlussTableLakeSnapshotCommitter +import org.apache.fluss.config.Configuration +import org.apache.fluss.lake.committer.TieringStats +import org.apache.fluss.lake.lakestorage.LakeStoragePluginSetUp +import org.apache.fluss.lake.writer.LakeTieringFactory +import org.apache.fluss.metadata.TablePath +import org.apache.fluss.spark.tiering.SparkTieringJobRunner.submit + +import org.apache.spark.SparkContext +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SparkSession + +import java.util.concurrent.{Executors, ScheduledExecutorService, ScheduledFuture, TimeUnit} +import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} + +import scala.concurrent.{Await, ExecutionContext, Future, Promise} +import scala.concurrent.duration.{Duration => ScalaDuration} + +/** + * Main driver loop for the Spark tiering service. + * + * Orchestrates a long-running loop: heartbeat -> request table -> generate splits -> parallel RDD + * job -> commit. Processes one table at a time. The loop runs asynchronously via [[startAsync()]] + * on a dedicated daemon thread and terminates cooperatively when [[stop()]] is called. + * + * The [[LakeTieringFactory]] is created internally from `dataLakeFormat` and `lakeConfig`. Tiering + * options (poll interval, heartbeat interval, poll timeout, max heartbeat failures) are read from + * `flussConfig` via [[SparkTieringOptions]]. + */ +class SparkTieringJobRunner( + spark: SparkSession, + flussConfig: Configuration, + lakeTieringConfig: Configuration, + dataLakeFormat: String, + lakeConfig: Configuration, + coordinatorFactory: Configuration => TieringCoordinator = new TieringCoordinator(_), + snapshotCommitterFactory: Configuration => FlussTableLakeSnapshotCommitter = + new FlussTableLakeSnapshotCommitter(_), + connectionFactory: Configuration => Connection = ConnectionFactory.createConnection, + splitGeneratorFactory: Admin => TieringSplitGenerator = new TieringSplitGenerator(_)) + extends Logging { + + // Read tiering options + private val pollIntervalMs = + flussConfig.get(SparkTieringOptions.POLL_TIERING_TABLE_INTERVAL).toMillis + private val heartbeatIntervalMs = + flussConfig.get(SparkTieringOptions.HEARTBEAT_INTERVAL).toMillis + private val pollTimeoutMs = + flussConfig.get(SparkTieringOptions.POLL_TIMEOUT).toMillis + private val maxHeartbeatFailures = + flussConfig.get(SparkTieringOptions.MAX_HEARTBEAT_FAILURES).intValue() + + private val lakeStoragePlugin = LakeStoragePluginSetUp.fromDataLakeFormat(dataLakeFormat, null) + private val lakeStorage = lakeStoragePlugin.createLakeStorage(lakeConfig) + private val lakeTieringFactory = lakeStorage + .createLakeTieringFactory() + .asInstanceOf[LakeTieringFactory[AnyRef, AnyRef]] + + private val stopped: AtomicBoolean = new AtomicBoolean(false) + private val terminationPromise: Promise[Unit] = Promise[Unit]() + + private val loopExecutionContext: ExecutionContext = ExecutionContext.fromExecutorService( + Executors.newSingleThreadExecutor( + (r: Runnable) => { + val t = new Thread(r, "SparkTiering-MainLoop") + t.setDaemon(true) + t + }) + ) + + /** + * Starts the tiering service loop asynchronously on a dedicated single-thread ExecutionContext. + * + * @return + * a Future that completes when the loop exits normally or fails on exception + */ + def startAsync(): Future[Unit] = { + val f = Future { + run() + }(loopExecutionContext) + terminationPromise.completeWith(f) + terminationPromise.future + } + + /** + * Blocks until the tiering loop finishes or the timeout expires. + * + * @throws java.util.concurrent.TimeoutException + * if the loop has not terminated within the given timeout + */ + def stop(timeout: ScalaDuration = ScalaDuration(60, TimeUnit.SECONDS)): Unit = { + stopped.set(true) + try { + Await.result(terminationPromise.future, timeout) + } catch { + case _: Exception => + logWarning("Timed out waiting for tiering loop to finish.") + } + } + + private def run(): Unit = { + val sc = spark.sparkContext + val coordinator = coordinatorFactory(flussConfig) + val snapshotCommitter = snapshotCommitterFactory(flussConfig) + var connection: Connection = null + var admin: Admin = null + + try { + coordinator.open() + snapshotCommitter.open() + connection = connectionFactory(flussConfig) + admin = connection.getAdmin + + val splitGenerator = splitGeneratorFactory(admin) + val consecutiveHeartbeatFailures = new AtomicInteger(0) + + // Register shutdown hook to report all in-progress tables as failed + val shutdownHook = new Thread("SparkTiering-ShutdownHook") { + override def run(): Unit = { + logInfo("Shutdown hook triggered, stopping tiering service...") + stopped.set(true) + coordinator.close() + } + } + Runtime.getRuntime.addShutdownHook(shutdownHook) + + logInfo("Spark tiering service started. Entering main loop.") + + while (!stopped.get()) { + try { + val tableAssignment = coordinator.heartbeatAndRequestTable() + consecutiveHeartbeatFailures.set(0) + + tableAssignment match { + case Some((tableId, tieringEpoch, tablePath)) => + processTieringTable( + sc, + coordinator, + snapshotCommitter, + admin, + splitGenerator, + consecutiveHeartbeatFailures, + tableId, + tieringEpoch, + tablePath) + + case None => + logInfo(s"No tiering table available, poll later after ${pollIntervalMs}ms.") + if (stopped.get()) { + logInfo("Stop was called while polling for table, exiting loop.") + } + Thread.sleep(pollIntervalMs) + } + } catch { + case _: InterruptedException => + logInfo("Main loop interrupted, stopping.") + stopped.set(true) + case e: Exception => + val failures = consecutiveHeartbeatFailures.incrementAndGet() + logWarning(s"Heartbeat failed ($failures/$maxHeartbeatFailures)", e) + if (failures >= maxHeartbeatFailures) { + logError( + s"Max consecutive heartbeat failures ($maxHeartbeatFailures) exceeded." + + " Failing job.") + throw new IllegalStateException( + s"Spark tiering job failed: exceeded max heartbeat failures ($maxHeartbeatFailures)", + e) + } + Thread.sleep(pollIntervalMs) + } + } + + logInfo("Spark tiering service main loop exited.") + } finally { + closeQuietly(snapshotCommitter, "FlussTableLakeSnapshotCommitter") + closeQuietly(coordinator, "TieringCoordinator") + closeQuietly(admin, "Admin") + closeQuietly(connection, "Connection") + } + } + + private def processTieringTable( + sc: SparkContext, + coordinator: TieringCoordinator, + snapshotCommitter: FlussTableLakeSnapshotCommitter, + admin: Admin, + splitGenerator: TieringSplitGenerator, + consecutiveHeartbeatFailures: AtomicInteger, + tableId: Long, + tieringEpoch: Long, + tablePath: TablePath): Unit = { + logInfo(s"Starting tiering for table: tableId=$tableId, epoch=$tieringEpoch, path=$tablePath") + + try { + // Get table info + val tableInfo = admin.getTableInfo(tablePath).get() + + // Generate splits + val rawSplits = splitGenerator.generateTableSplits(tableInfo) + + if (rawSplits.isEmpty) { + logInfo(s"No splits generated for table $tablePath, marking as finished.") + coordinator.markTableFinished(tableId, tieringEpoch, TieringStats.UNKNOWN) + return + } + + // Populate numberOfSplits on each split + val totalSplits = rawSplits.size + val splits = rawSplits.map { + case s: TieringLogSplit => s.copy(numberOfSplits = totalSplits) + case s: TieringSnapshotSplit => s.copy(numberOfSplits = totalSplits) + } + + // Register tiering in coordinator state + coordinator.registerTiering(tableId, tieringEpoch) + + // Run RDD job with background heartbeat + val results = + runTieringRddJob(sc, coordinator, consecutiveHeartbeatFailures, splits, tablePath, tableId) + + // Check if table was cancelled (dropped/recreated detected by heartbeat thread) + if (coordinator.tableCancelled) { + logWarning(s"Table $tablePath was cancelled during tiering (likely dropped/recreated).") + coordinator.tableCancelled = false + coordinator.markTableFailed(tableId, tieringEpoch) + return + } + + // Commit results + val stats = TieringCommitter.commitAll( + tableId, + tablePath, + results, + lakeTieringFactory, + flussConfig, + lakeTieringConfig, + admin, + snapshotCommitter) + + logInfo(s"Tiering completed for table $tablePath, stats=$stats") + coordinator.markTableFinished(tableId, tieringEpoch, stats) + + } catch { + case e: Exception => + logError(s"Tiering failed for table $tablePath (epoch=$tieringEpoch)", e) + coordinator.markTableFailed(tableId, tieringEpoch) + } + } + + /** + * Runs the tiering RDD job with a background heartbeat thread. + * + * Delegates to [[SparkTieringJobRunner.submit()]] to create the RDD. Configs are captured in the + * closure and serialized to executors. Each executor task re-initializes the + * [[LakeTieringFactory]] from `dataLakeFormat` and `lakeConfig`. A background scheduled thread + * sends keepalive heartbeats to the coordinator during the RDD job; if max failures are exceeded, + * it cancels the Spark job group. + */ + private def runTieringRddJob( + sc: SparkContext, + coordinator: TieringCoordinator, + consecutiveHeartbeatFailures: AtomicInteger, + splits: Seq[TieringSplit], + tablePath: TablePath, + tableId: Long): Seq[SerializedTaskResult] = { + val jobGroupId = s"tiering-${tablePath.getDatabaseName}-${tablePath.getTableName}-$tableId" + sc.setJobGroup(jobGroupId, s"Tiering table $tablePath") + + // Start background heartbeat thread + val heartbeatScheduler = Executors.newSingleThreadScheduledExecutor( + (r: Runnable) => { + val t = new Thread(r, "SparkTiering-Heartbeat") + t.setDaemon(true) + t + }) + val heartbeatFuture = startBackgroundHeartbeat( + heartbeatScheduler, + coordinator, + consecutiveHeartbeatFailures, + sc, + jobGroupId, + tablePath, + tableId) + + try { + submit(sc, splits, flussConfig, dataLakeFormat, lakeConfig, pollTimeoutMs) + .collect() + .toSeq + } finally { + // Stop background heartbeat + heartbeatFuture.cancel(false) + heartbeatScheduler.shutdownNow() + sc.clearJobGroup() + } + } + + /** + * Starts a background scheduled task that sends keepalive heartbeats and checks for table + * drop/recreation during RDD job execution. + */ + private def startBackgroundHeartbeat( + scheduler: ScheduledExecutorService, + coordinator: TieringCoordinator, + consecutiveHeartbeatFailures: AtomicInteger, + sc: SparkContext, + jobGroupId: String, + tablePath: TablePath, + expectedTableId: Long): ScheduledFuture[_] = { + coordinator.tableCancelled = false + + scheduler.scheduleAtFixedRate( + new Runnable { + override def run(): Unit = { + try { + coordinator.sendKeepAliveHeartbeat() + consecutiveHeartbeatFailures.set(0) + } catch { + case e: Exception => + val failures = consecutiveHeartbeatFailures.incrementAndGet() + logWarning(s"Background heartbeat failed ($failures/$maxHeartbeatFailures)", e) + if (failures >= maxHeartbeatFailures) { + logError("Max heartbeat failures exceeded during RDD job. Cancelling job group.") + coordinator.tableCancelled = true + sc.cancelJobGroup(jobGroupId) + return + } + } + } + }, + heartbeatIntervalMs, + heartbeatIntervalMs, + TimeUnit.MILLISECONDS + ) + } +} + +object SparkTieringJobRunner { + def submit( + sparkContext: SparkContext, + splits: Seq[TieringSplit], + flussConfig: Configuration, + dataLakeFormat: String, + lakeConfig: Configuration, + pollTimeoutMs: Long): RDD[SerializedTaskResult] = { + sparkContext.parallelize(splits, splits.size).map { + split => TieringTask.process(split, flussConfig, dataLakeFormat, lakeConfig, pollTimeoutMs) + } + } +} diff --git a/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/SparkTieringOptions.scala b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/SparkTieringOptions.scala new file mode 100644 index 0000000000..937accd363 --- /dev/null +++ b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/SparkTieringOptions.scala @@ -0,0 +1,58 @@ +/* + * 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.fluss.spark.tiering + +import org.apache.fluss.config.{ConfigBuilder, ConfigOption} + +import java.time.Duration + +/** Configuration options for the Spark tiering service. */ +object SparkTieringOptions { + + val POLL_TIERING_TABLE_INTERVAL: ConfigOption[Duration] = + ConfigBuilder + .key("spark.tiering.poll.table.interval") + .durationType() + .defaultValue(Duration.ofSeconds(30)) + .withDescription( + "The fixed interval to request tiering table from Fluss cluster, by default 30 seconds.") + + val HEARTBEAT_INTERVAL: ConfigOption[Duration] = + ConfigBuilder + .key("spark.tiering.heartbeat.interval") + .durationType() + .defaultValue(Duration.ofSeconds(10)) + .withDescription( + "The interval for background heartbeat during RDD execution, by default 10 seconds.") + + val POLL_TIMEOUT: ConfigOption[Duration] = + ConfigBuilder + .key("spark.tiering.poll.timeout") + .durationType() + .defaultValue(Duration.ofSeconds(10)) + .withDescription( + "The timeout for polling log records from Fluss on executors, by default 10 seconds.") + + val MAX_HEARTBEAT_FAILURES: ConfigOption[Integer] = + ConfigBuilder + .key("spark.tiering.heartbeat.max.failures") + .intType() + .defaultValue(10) + .withDescription( + "Maximum number of consecutive heartbeat failures before the Spark job fails.") +} diff --git a/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringCommitter.scala b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringCommitter.scala new file mode 100644 index 0000000000..3613a45c07 --- /dev/null +++ b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringCommitter.scala @@ -0,0 +1,202 @@ +/* + * 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.fluss.spark.tiering + +import org.apache.fluss.client.admin.Admin +import org.apache.fluss.client.metadata.LakeSnapshot +import org.apache.fluss.client.tiering.{FlussTableLakeSnapshotCommitter, TieringCommitterInitContext} +import org.apache.fluss.config.Configuration +import org.apache.fluss.exception.LakeTableSnapshotNotExistException +import org.apache.fluss.lake.committer.{LakeCommitResult, LakeCommitter, TieringStats} +import org.apache.fluss.lake.committer.LakeCommitter.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY +import org.apache.fluss.lake.writer.LakeTieringFactory +import org.apache.fluss.metadata.{TableBucket, TablePath} + +import org.apache.spark.internal.Logging + +import java.util.Collections + +import scala.collection.JavaConverters._ + +// TODO: This logic is duplicated from +// org.apache.fluss.flink.tiering.committer.TieringCommitOperator (commit logic). +// Consider extracting to a shared module (e.g., fluss-tiering-common) in the future. + +/** + * Orchestrates the two-phase commit of tiering write results to the lake and Fluss coordinator. + * + * Runs on the Spark driver after all executor tasks complete. The commit pipeline: + * 1. Deserializes [[SerializedTaskResult]] write results from executors. + * 2. Converts them into a lake committable via [[LakeCommitter.toCommittable()]]. + * 3. Checks for missing lake snapshots (Fluss lags behind lake) and recovers if needed. + * 4. Prepares lake snapshot metadata via + * [[FlussTableLakeSnapshotCommitter.prepareLakeSnapshot()]]. + * 5. Commits to the lake storage (e.g., Paimon/Iceberg) via [[LakeCommitter.commit()]]. + * 6. Commits the snapshot metadata to the Fluss coordinator via + * [[FlussTableLakeSnapshotCommitter.commit()]]. + */ +object TieringCommitter extends Logging { + + def commitAll( + tableId: Long, + tablePath: TablePath, + results: Seq[SerializedTaskResult], + lakeTieringFactory: LakeTieringFactory[AnyRef, AnyRef], + flussConfig: Configuration, + lakeTieringConfig: Configuration, + admin: Admin, + snapshotCommitter: FlussTableLakeSnapshotCommitter): TieringStats = { + // Filter to non-empty results + val nonEmptyResults = results.filter(_.serializedWriteResult != null) + + if (nonEmptyResults.isEmpty) { + logInfo(s"Commit tiering write results is empty for table $tableId, table path $tablePath") + return TieringStats.UNKNOWN + } + + // Verify table was not dropped and recreated during tiering + val currentTableInfo = admin.getTableInfo(tablePath).get() + if (currentTableInfo.getTableId != tableId) { + throw new IllegalStateException( + s"The current table id ${currentTableInfo.getTableId} for table path $tablePath" + + s" is different from the table id $tableId in the committable." + + " This usually happens when a table was dropped and recreated during tiering." + + " Aborting commit to prevent dirty commit.") + } + + val writeResultSerializer = lakeTieringFactory.getWriteResultSerializer + + val lakeCommitter: LakeCommitter[AnyRef, AnyRef] = lakeTieringFactory.createLakeCommitter( + new TieringCommitterInitContext(tablePath, currentTableInfo, lakeTieringConfig, flussConfig)) + + try { + // Deserialize write results + val writeResults = nonEmptyResults.map { + result => + writeResultSerializer.deserialize(result.writeResultVersion, result.serializedWriteResult) + }.asJava + + // Build offset maps + val logEndOffsets = new java.util.HashMap[TableBucket, java.lang.Long]() + val logMaxTieredTimestamps = new java.util.HashMap[TableBucket, java.lang.Long]() + nonEmptyResults.foreach { + result => + logEndOffsets.put(result.tableBucket, result.logEndOffset: java.lang.Long) + logMaxTieredTimestamps.put(result.tableBucket, result.maxTimestamp: java.lang.Long) + } + + // Convert to committable + val committable = lakeCommitter.toCommittable(writeResults) + + // Check for missing lake snapshot before committing + val flussCurrentLakeSnapshot = getLatestLakeSnapshot(admin, tablePath) + checkFlussNotMissingLakeSnapshot( + tablePath, + tableId, + lakeCommitter, + committable, + snapshotCommitter, + if (flussCurrentLakeSnapshot != null) flussCurrentLakeSnapshot.getSnapshotId else null) + + // Prepare lake snapshot (get offsets file path) + val lakeBucketTieredOffsetsFile = + snapshotCommitter.prepareLakeSnapshot(tableId, tablePath, logEndOffsets) + + // Commit to lake with offsets file in snapshot properties + val snapshotProperties = Collections.singletonMap( + FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, + lakeBucketTieredOffsetsFile) + val lakeCommitResult = lakeCommitter.commit(committable, snapshotProperties) + + // Commit to Fluss + snapshotCommitter.commit( + tableId, + tablePath, + lakeCommitResult, + lakeBucketTieredOffsetsFile, + logEndOffsets, + logMaxTieredTimestamps) + + lakeCommitResult.getTieringStats + } finally { + lakeCommitter.close() + } + } + + private def getLatestLakeSnapshot(admin: Admin, tablePath: TablePath): LakeSnapshot = { + try { + admin.getLatestLakeSnapshot(tablePath).get() + } catch { + case e: Exception => + e.getCause match { + case _: LakeTableSnapshotNotExistException => null + case _ => throw e + } + } + } + + private def checkFlussNotMissingLakeSnapshot( + tablePath: TablePath, + tableId: Long, + lakeCommitter: LakeCommitter[AnyRef, AnyRef], + committable: AnyRef, + snapshotCommitter: FlussTableLakeSnapshotCommitter, + flussCurrentLakeSnapshot: java.lang.Long): Unit = { + val missingCommittedSnapshot = + lakeCommitter.getMissingLakeSnapshot(flussCurrentLakeSnapshot) + + if (missingCommittedSnapshot != null) { + val lakeSnapshotOffsetPath = missingCommittedSnapshot.getSnapshotProperties + .get(FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY) + + if (lakeSnapshotOffsetPath == null) { + throw new IllegalStateException( + s"Can't find $FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY field from snapshot property.") + } + + val trimmedPath = lakeSnapshotOffsetPath.trim + if (trimmedPath.contains("{")) { + throw new IllegalStateException( + s"The $FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY field in snapshot property" + + s" is a JSON string (tiered by v0.8), which is not supported to restore." + + s" Snapshot ID: ${missingCommittedSnapshot.getLakeSnapshotId}," + + s" Table: {tablePath=$tablePath, tableId=$tableId}.") + } + + // Commit missing snapshot to Fluss + snapshotCommitter.commit( + tableId, + missingCommittedSnapshot.getLakeSnapshotId, + lakeSnapshotOffsetPath, + null, + Collections.emptyMap(), + Collections.emptyMap(), + LakeCommitResult.KEEP_ALL_PREVIOUS + ) + + // Abort current committable + lakeCommitter.abort(committable) + + throw new IllegalStateException( + s"The current Fluss's lake snapshot $flussCurrentLakeSnapshot is less than" + + s" lake actual snapshot ${missingCommittedSnapshot.getLakeSnapshotId}" + + s" committed by Fluss for table: {tablePath=$tablePath, tableId=$tableId}," + + s" missing snapshot: $missingCommittedSnapshot.") + } + } +} diff --git a/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringCoordinator.scala b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringCoordinator.scala new file mode 100644 index 0000000000..6d1edd0aff --- /dev/null +++ b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringCoordinator.scala @@ -0,0 +1,234 @@ +/* + * 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.fluss.spark.tiering + +import org.apache.fluss.client.{Connection, ConnectionFactory} +import org.apache.fluss.client.metadata.MetadataUpdater +import org.apache.fluss.config.Configuration +import org.apache.fluss.lake.committer.TieringStats +import org.apache.fluss.metadata.TablePath +import org.apache.fluss.metrics.registry.MetricRegistry +import org.apache.fluss.rpc.{GatewayClientProxy, RpcClient} +import org.apache.fluss.rpc.gateway.CoordinatorGateway +import org.apache.fluss.rpc.messages._ +import org.apache.fluss.rpc.metrics.ClientMetricGroup + +import org.apache.spark.internal.Logging + +import java.util.concurrent.TimeUnit + +import scala.collection.JavaConverters._ + +// TODO: This logic is duplicated from +// org.apache.fluss.flink.tiering.source.TieringSourceFunction (heartbeat protocol). +// Consider extracting to a shared module (e.g., fluss-tiering-common) in the future. + +/** Manages heartbeat communication with the Fluss coordinator for the Spark tiering service. */ +class TieringCoordinator(flussConfig: Configuration) extends AutoCloseable with Logging { + + import TieringCoordinator._ + + private var connection: Connection = _ + private var rpcClient: RpcClient = _ + private var coordinatorGateway: CoordinatorGateway = _ + private var coordinatorEpoch: Int = 0 + + // State maps (immutable maps, updated via var) + private var tieringTableEpochs: Map[Long, Long] = Map.empty + private var finishedTables: Map[Long, TieringFinishInfo] = Map.empty + private var failedTableEpochs: Map[Long, Long] = Map.empty + + @volatile var tableCancelled: Boolean = false + + def open(): Unit = { + connection = ConnectionFactory.createConnection(flussConfig) + val metricRegistry = MetricRegistry.create(flussConfig, null) + val clientMetricGroup = new ClientMetricGroup(metricRegistry, "SparkLakeTieringService") + rpcClient = RpcClient.create(flussConfig, clientMetricGroup) + val metadataUpdater = new MetadataUpdater(flussConfig, rpcClient) + coordinatorGateway = GatewayClientProxy.createGatewayProxy( + () => metadataUpdater.getCoordinatorServer, + rpcClient, + classOf[CoordinatorGateway]) + + logInfo("Registering Spark Tiering Service with Fluss Coordinator...") + val response = waitHeartbeatResponse(coordinatorGateway.lakeTieringHeartbeat(basicHeartBeat())) + coordinatorEpoch = response.getCoordinatorEpoch + logInfo(s"Registered with Fluss Coordinator (epoch=$coordinatorEpoch).") + } + + /** + * Sends a heartbeat reporting finished/failed tables and requests a new table assignment. + * + * @return + * Some((tableId, tieringEpoch, tablePath)) if a table is assigned, None otherwise + */ + def heartbeatAndRequestTable(): Option[(Long, Long, TablePath)] = { + val currentFinished = finishedTables + val currentFailed = failedTableEpochs + + val request = + buildTieringHeartBeat(tieringTableEpochs, currentFinished, currentFailed, coordinatorEpoch) + request.setRequestTable(true) + + logInfo( + s"Heartbeat: tiering=$tieringTableEpochs, finished=$currentFinished, " + + s"failed=$currentFailed") + + val response = waitHeartbeatResponse(coordinatorGateway.lakeTieringHeartbeat(request)) + + // Clear reported finished/failed tables + finishedTables = finishedTables -- currentFinished.keys + failedTableEpochs = failedTableEpochs -- currentFailed.keys + + if (response.hasTieringTable) { + val tieringTable = response.getTieringTable + val tablePath = TablePath.of( + tieringTable.getTablePath.getDatabaseName, + tieringTable.getTablePath.getTableName) + val tableId = tieringTable.getTableId + val tieringEpoch = tieringTable.getTieringEpoch + logInfo(s"Assigned tiering table: tableId=$tableId, epoch=$tieringEpoch, path=$tablePath") + Some((tableId, tieringEpoch, tablePath)) + } else { + logDebug("No tiering table available.") + None + } + } + + /** Sends a keepalive heartbeat with currently tiering tables (no table request). */ + def sendKeepAliveHeartbeat(): Unit = { + val request = + buildTieringHeartBeat(tieringTableEpochs, finishedTables, failedTableEpochs, coordinatorEpoch) + waitHeartbeatResponse(coordinatorGateway.lakeTieringHeartbeat(request)) + } + + def registerTiering(tableId: Long, tieringEpoch: Long): Unit = { + tieringTableEpochs = tieringTableEpochs + (tableId -> tieringEpoch) + } + + def markTableFinished(tableId: Long, tieringEpoch: Long, stats: TieringStats): Unit = { + tieringTableEpochs = tieringTableEpochs - tableId + finishedTables = finishedTables + (tableId -> TieringFinishInfo(tieringEpoch, stats)) + } + + def markTableFailed(tableId: Long, tieringEpoch: Long): Unit = { + tieringTableEpochs = tieringTableEpochs - tableId + failedTableEpochs = failedTableEpochs + (tableId -> tieringEpoch) + } + + override def close(): Unit = { + // Report all tiering tables as failed before shutdown + if (tieringTableEpochs.nonEmpty) { + failedTableEpochs = failedTableEpochs ++ tieringTableEpochs + tieringTableEpochs = Map.empty + try { + val request = basicHeartBeat() + addFailedTables(request, failedTableEpochs, coordinatorEpoch) + waitHeartbeatResponse(coordinatorGateway.lakeTieringHeartbeat(request)) + logInfo(s"Reported failed tables on shutdown: $failedTableEpochs") + } catch { + case e: Exception => + logWarning("Failed to report failed tables on shutdown", e) + } + } + + closeQuietly(rpcClient, "RpcClient") + closeQuietly(connection, "Connection") + } +} + +/** Heartbeat helper functions and constants for [[TieringCoordinator]]. */ +object TieringCoordinator extends Logging { + + private val HEARTBEAT_TIMEOUT_MINUTES = 3L + + private[tiering] def basicHeartBeat(): LakeTieringHeartbeatRequest = { + new LakeTieringHeartbeatRequest() + } + + private[tiering] def buildTieringHeartBeat( + tiering: Map[Long, Long], + finished: Map[Long, TieringFinishInfo], + failed: Map[Long, Long], + epoch: Int): LakeTieringHeartbeatRequest = { + val request = basicHeartBeat() + + if (tiering.nonEmpty) { + request.addAllTieringTables(toPbHeartbeatReqForTable(tiering, epoch).asJava) + } + + if (finished.nonEmpty) { + val finishedReqs = finished.map { + case (tableId, finishInfo) => + val req = new PbHeartbeatReqForTable() + .setTableId(tableId) + .setCoordinatorEpoch(epoch) + .setTieringEpoch(finishInfo.tieringEpoch) + val stats = finishInfo.stats + if (stats != null && stats.isAvailableStats) { + val pbStats = new PbLakeTieringStats() + if (stats.getFileSize != null) pbStats.setFileSize(stats.getFileSize) + if (stats.getRecordCount != null) pbStats.setRecordCount(stats.getRecordCount) + req.setLakeTieringStats(pbStats) + } + req + }.toSeq + request.addAllFinishedTables(finishedReqs.asJava) + } + + addFailedTables(request, failed, epoch) + request + } + + private[tiering] def addFailedTables( + request: LakeTieringHeartbeatRequest, + failed: Map[Long, Long], + epoch: Int): Unit = { + if (failed.nonEmpty) { + request.addAllFailedTables(toPbHeartbeatReqForTable(failed, epoch).asJava) + } + } + + private def toPbHeartbeatReqForTable( + tableEpochs: Map[Long, Long], + coordinatorEpoch: Int): Set[PbHeartbeatReqForTable] = { + tableEpochs.map { + case (tableId, tieringEpoch) => + new PbHeartbeatReqForTable() + .setTableId(tableId) + .setCoordinatorEpoch(coordinatorEpoch) + .setTieringEpoch(tieringEpoch) + }.toSet + } + + private[tiering] def waitHeartbeatResponse( + future: java.util.concurrent.CompletableFuture[LakeTieringHeartbeatResponse]) + : LakeTieringHeartbeatResponse = { + try { + future.get(HEARTBEAT_TIMEOUT_MINUTES, TimeUnit.MINUTES) + } catch { + case e: Exception => + logError("Failed to wait heartbeat response", e) + throw new IllegalStateException("Failed to wait heartbeat response", e) + } + } +} + +/** Finish info for a tiering table. */ +private[tiering] case class TieringFinishInfo(tieringEpoch: Long, stats: TieringStats) diff --git a/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringSplit.scala b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringSplit.scala new file mode 100644 index 0000000000..ca7fb4e903 --- /dev/null +++ b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringSplit.scala @@ -0,0 +1,78 @@ +/* + * 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.fluss.spark.tiering + +import org.apache.fluss.metadata.{TableBucket, TablePath} + +/** Sealed trait representing a tiering split for distributing work across Spark executors. */ +sealed trait TieringSplit extends Serializable { + def tablePath: TablePath + def tableBucket: TableBucket + def partitionName: Option[String] + def numberOfSplits: Int +} + +/** + * A split for reading log data from Fluss. + * + * @param tablePath + * the table path + * @param tableBucket + * the table bucket + * @param partitionName + * optional partition name + * @param startingOffset + * the starting log offset (inclusive) + * @param stoppingOffset + * the stopping log offset (exclusive) + * @param numberOfSplits + * total number of splits in this tiering round + */ +case class TieringLogSplit( + tablePath: TablePath, + tableBucket: TableBucket, + partitionName: Option[String], + startingOffset: Long, + stoppingOffset: Long, + numberOfSplits: Int) + extends TieringSplit + +/** + * A split for reading a KV snapshot from Fluss (used for primary key tables on first tiering). + * + * @param tablePath + * the table path + * @param tableBucket + * the table bucket + * @param partitionName + * optional partition name + * @param snapshotId + * the KV snapshot ID to read + * @param logOffsetOfSnapshot + * the log offset corresponding to the snapshot + * @param numberOfSplits + * total number of splits in this tiering round + */ +case class TieringSnapshotSplit( + tablePath: TablePath, + tableBucket: TableBucket, + partitionName: Option[String], + snapshotId: Long, + logOffsetOfSnapshot: Long, + numberOfSplits: Int) + extends TieringSplit diff --git a/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringSplitGenerator.scala b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringSplitGenerator.scala new file mode 100644 index 0000000000..af0496dfa7 --- /dev/null +++ b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringSplitGenerator.scala @@ -0,0 +1,302 @@ +/* + * 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.fluss.spark.tiering + +import org.apache.fluss.client.admin.Admin +import org.apache.fluss.client.initializer.BucketOffsetsRetrieverImpl +import org.apache.fluss.client.metadata.{KvSnapshots, LakeSnapshot} +import org.apache.fluss.client.table.scanner.log.LogScanner.EARLIEST_OFFSET +import org.apache.fluss.exception.LakeTableSnapshotNotExistException +import org.apache.fluss.metadata.{TableBucket, TableInfo, TablePath} +import org.apache.fluss.utils.ExceptionUtils + +import org.apache.spark.internal.Logging + +import scala.collection.JavaConverters._ + +// TODO: This logic is duplicated from +// org.apache.fluss.flink.tiering.source.split.TieringSplitGenerator. +// Consider extracting to a shared module (e.g., fluss-tiering-common) in the future. + +/** + * Generates tiering splits for a table, producing one [[TieringSplit]] per bucket. + * + * For log tables, each split is a [[TieringLogSplit]] bounded by the last committed lake offset and + * the current latest offset. For primary key tables that have never been tiered, a + * [[TieringSnapshotSplit]] is generated to read the KV snapshot; subsequent rounds produce log + * splits for incremental changes. Buckets with no new data are skipped. + * + * Supports both partitioned and non-partitioned tables. + */ +class TieringSplitGenerator(admin: Admin) extends Logging { + + def generateTableSplits(tableInfo: TableInfo): Seq[TieringSplit] = { + val tablePath = tableInfo.getTablePath + val bucketOffsetsRetriever = new BucketOffsetsRetrieverImpl(admin, tablePath) + + // Get the latest lake snapshot (may be null if table was never tiered) + val lakeSnapshot: LakeSnapshot = + try { + admin.getLatestLakeSnapshot(tablePath).get() + } catch { + case e: Exception => + ExceptionUtils.stripExecutionException(e) match { + case _: LakeTableSnapshotNotExistException => + null + case t => + throw new IllegalStateException( + s"Failed to get table snapshot for table $tablePath", + t) + } + } + logInfo(s"Last committed lake table snapshot info is: $lakeSnapshot") + + if (tableInfo.isPartitioned) { + val partitionInfos = admin.listPartitionInfos(tablePath).get().asScala + val partitionNameById: Map[Long, String] = + partitionInfos.map(p => (p.getPartitionId: Long) -> p.getPartitionName).toMap + generatePartitionTableSplits( + tableInfo, + partitionNameById, + bucketOffsetsRetriever, + lakeSnapshot) + } else { + generateNonPartitionedTableSplits(tableInfo, bucketOffsetsRetriever, lakeSnapshot) + } + } + + private def generatePartitionTableSplits( + tableInfo: TableInfo, + partitionNameById: Map[Long, String], + bucketOffsetsRetriever: BucketOffsetsRetrieverImpl, + lakeSnapshot: LakeSnapshot): Seq[TieringSplit] = { + partitionNameById.flatMap { + case (partitionId, partitionName) => + val bucketIds = (0 until tableInfo.getNumBuckets).map(Int.box).toList.asJava + val latestBucketsOffset = + bucketOffsetsRetriever.latestOffsets(partitionName, bucketIds).asScala.toMap + + val latestKvSnapshots: KvSnapshots = if (tableInfo.hasPrimaryKey) { + try { + admin.getLatestKvSnapshots(tableInfo.getTablePath, partitionName).get() + } catch { + case e: Exception => + throw new IllegalStateException( + s"Failed to get table snapshot for table ${tableInfo.getTablePath}" + + s" and partition $partitionName", + ExceptionUtils.stripCompletionException(e)) + } + } else { + null + } + + generateTableSplits( + tableInfo, + Some(partitionId), + Some(partitionName), + lakeSnapshot, + latestKvSnapshots, + latestBucketsOffset) + }.toSeq + } + + private def generateNonPartitionedTableSplits( + tableInfo: TableInfo, + bucketOffsetsRetriever: BucketOffsetsRetrieverImpl, + lakeSnapshot: LakeSnapshot): Seq[TieringSplit] = { + val bucketIds = (0 until tableInfo.getNumBuckets).map(Int.box).toList.asJava + val latestBucketsOffset = + bucketOffsetsRetriever.latestOffsets(null, bucketIds).asScala.toMap + + val latestKvSnapshots: KvSnapshots = if (tableInfo.hasPrimaryKey) { + try { + admin.getLatestKvSnapshots(tableInfo.getTablePath).get() + } catch { + case e: Exception => + throw new IllegalStateException( + s"Failed to get table snapshot for table ${tableInfo.getTablePath}", + ExceptionUtils.stripCompletionException(e)) + } + } else { + null + } + + generateTableSplits(tableInfo, None, None, lakeSnapshot, latestKvSnapshots, latestBucketsOffset) + } + + private def generateTableSplits( + tableInfo: TableInfo, + partitionId: Option[Long], + partitionName: Option[String], + lakeSnapshot: LakeSnapshot, + latestKvSnapshots: KvSnapshots, + latestBucketsOffset: Map[Integer, java.lang.Long]): Seq[TieringSplit] = { + val tablePath = tableInfo.getTablePath + + if (tableInfo.hasPrimaryKey) { + require(latestKvSnapshots != null, "latestKvSnapshots must not be null for PK tables") + (0 until tableInfo.getNumBuckets).flatMap { + bucket => + val tableBucket = + new TableBucket(tableInfo.getTableId, partitionId.map(Long.box).orNull, bucket) + val lastCommittedBucketOffset: java.lang.Long = + if (lakeSnapshot != null) lakeSnapshot.getTableBucketsOffset.get(tableBucket) + else null + val latestSnapshotId: java.lang.Long = + if (latestKvSnapshots.getSnapshotId(bucket).isPresent) + latestKvSnapshots.getSnapshotId(bucket).getAsLong + else null + val offsetOfLatestSnapshot: java.lang.Long = + if (latestKvSnapshots.getSnapshotId(bucket).isPresent) + latestKvSnapshots.getLogOffset(bucket).getAsLong + else null + val latestBucketOffset: Long = latestBucketsOffset(bucket) + + generateSplitForPrimaryKeyTableBucket( + tablePath, + tableBucket, + partitionName, + latestSnapshotId, + offsetOfLatestSnapshot, + lastCommittedBucketOffset, + latestBucketOffset) + } + } else { + (0 until tableInfo.getNumBuckets).flatMap { + bucket => + val tableBucket = + new TableBucket(tableInfo.getTableId, partitionId.map(Long.box).orNull, bucket) + val lastCommittedOffset: java.lang.Long = + if (lakeSnapshot != null) lakeSnapshot.getTableBucketsOffset.get(tableBucket) + else null + val latestBucketOffset: Long = latestBucketsOffset(bucket) + + generateSplitForLogTableBucket( + tablePath, + tableBucket, + partitionName, + lastCommittedOffset, + latestBucketOffset) + } + } + } + + private def generateSplitForPrimaryKeyTableBucket( + tablePath: TablePath, + tableBucket: TableBucket, + partitionName: Option[String], + latestSnapshotId: java.lang.Long, + latestOffsetOfSnapshot: java.lang.Long, + lastCommittedBucketOffset: java.lang.Long, + latestBucketOffset: Long): Option[TieringSplit] = { + if (latestBucketOffset <= 0) { + logDebug( + s"The latestBucketOffset $latestBucketOffset is equals or less than 0," + + s" skip generating split for bucket $tableBucket") + return None + } + + if (lastCommittedBucketOffset == null) { + // Never tiered + if (latestSnapshotId == null) { + // No snapshot, scan log from earliest + Some( + TieringLogSplit( + tablePath, + tableBucket, + partitionName, + EARLIEST_OFFSET, + latestBucketOffset, + 0)) + } else { + // Has snapshot, read KV snapshot + require( + latestOffsetOfSnapshot != null, + "latestOffsetOfSnapshot must not be null when latestSnapshotId is present") + Some( + TieringSnapshotSplit( + tablePath, + tableBucket, + partitionName, + latestSnapshotId, + latestOffsetOfSnapshot, + 0)) + } + } else { + // Previously tiered, read bounded log (snapshot is ignored) + if (lastCommittedBucketOffset < latestBucketOffset) { + Some( + TieringLogSplit( + tablePath, + tableBucket, + partitionName, + lastCommittedBucketOffset, + latestBucketOffset, + 0)) + } else { + logDebug( + s"The lastCommittedBucketOffset $lastCommittedBucketOffset is equals or" + + s" bigger than latestBucketOffset $latestBucketOffset," + + s" skip generating split for bucket $tableBucket") + None + } + } + } + + private def generateSplitForLogTableBucket( + tablePath: TablePath, + tableBucket: TableBucket, + partitionName: Option[String], + lastCommittedBucketOffset: java.lang.Long, + latestBucketOffset: Long): Option[TieringSplit] = { + if (latestBucketOffset <= 0) { + logDebug( + s"The latestBucketOffset $latestBucketOffset is equals or less than 0," + + s" skip generating split for bucket $tableBucket") + return None + } + + if (lastCommittedBucketOffset == null) { + // Never tiered, scan from earliest + Some( + TieringLogSplit( + tablePath, + tableBucket, + partitionName, + EARLIEST_OFFSET, + latestBucketOffset, + 0)) + } else if (lastCommittedBucketOffset < latestBucketOffset) { + // Previously tiered, scan remaining log + Some( + TieringLogSplit( + tablePath, + tableBucket, + partitionName, + lastCommittedBucketOffset, + latestBucketOffset, + 0)) + } else { + logDebug( + s"The lastCommittedBucketOffset $lastCommittedBucketOffset is equals or" + + s" bigger than latestBucketOffset $latestBucketOffset," + + s" skip generating split for bucket $tableBucket") + None + } + } +} diff --git a/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringTask.scala b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringTask.scala new file mode 100644 index 0000000000..d5de7466b0 --- /dev/null +++ b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringTask.scala @@ -0,0 +1,231 @@ +/* + * 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.fluss.spark.tiering + +import org.apache.fluss.client.{Connection, ConnectionFactory} +import org.apache.fluss.client.table.Table +import org.apache.fluss.client.table.scanner.ScanRecord +import org.apache.fluss.client.tiering.TieringWriterInitContext +import org.apache.fluss.config.Configuration +import org.apache.fluss.lake.lakestorage.LakeStoragePluginSetUp +import org.apache.fluss.lake.serializer.SimpleVersionedSerializer +import org.apache.fluss.lake.writer.{LakeTieringFactory, LakeWriter} + +import org.apache.spark.internal.Logging + +import java.time.Duration + +import scala.collection.JavaConverters._ + +/** + * Executor-side processing logic for a single tiering split. + * + * Each Spark task processes one [[TieringSplit]]: reads data from Fluss (log or KV snapshot) and + * writes it to the lake via [[LakeWriter]]. + */ +object TieringTask extends Logging { + + private val UNKNOWN_BUCKET_TIMESTAMP = -1L + + def process( + split: TieringSplit, + flussConfig: Configuration, + dataLakeFormat: String, + lakeConfig: Configuration, + pollTimeoutMs: Long): SerializedTaskResult = { + val tablePath = split.tablePath + val tableBucket = split.tableBucket + val partitionName = split.partitionName + + var connection: Connection = null + var table: Table = null + var lakeWriter: LakeWriter[AnyRef] = null + + try { + connection = ConnectionFactory.createConnection(flussConfig) + table = connection.getTable(tablePath) + + // Verify table ID hasn't changed (table not dropped/recreated) + val currentTableId = table.getTableInfo.getTableId + if (currentTableId != tableBucket.getTableId) { + throw new IllegalStateException( + s"Table ID mismatch for $tablePath: expected ${tableBucket.getTableId}" + + s" but got $currentTableId. Table may have been dropped and recreated.") + } + + // Initialize LakeTieringFactory on executor + val lakeTieringFactory = createLakeTieringFactory(dataLakeFormat, lakeConfig) + val writeResultSerializer = lakeTieringFactory.getWriteResultSerializer + + val writerInitContext = + new TieringWriterInitContext( + tablePath, + tableBucket, + partitionName.orNull, + table.getTableInfo) + lakeWriter = lakeTieringFactory.createLakeWriter(writerInitContext) + + split match { + case logSplit: TieringLogSplit => + processLogSplit(table, lakeWriter, logSplit, pollTimeoutMs, writeResultSerializer) + case snapshotSplit: TieringSnapshotSplit => + processSnapshotSplit( + table, + lakeWriter, + snapshotSplit, + pollTimeoutMs, + writeResultSerializer) + } + } finally { + closeQuietly(lakeWriter, "LakeWriter") + closeQuietly(table, "Table") + closeQuietly(connection, "Connection") + } + } + + /** Creates a [[LakeTieringFactory]] from the data lake format and config. */ + private def createLakeTieringFactory( + dataLakeFormat: String, + lakeConfig: Configuration): LakeTieringFactory[AnyRef, AnyRef] = { + val lakeStoragePlugin = LakeStoragePluginSetUp.fromDataLakeFormat(dataLakeFormat, null) + val lakeStorage = lakeStoragePlugin.createLakeStorage(lakeConfig) + lakeStorage.createLakeTieringFactory().asInstanceOf[LakeTieringFactory[AnyRef, AnyRef]] + } + + private def processLogSplit( + table: Table, + lakeWriter: LakeWriter[AnyRef], + split: TieringLogSplit, + pollTimeoutMs: Long, + writeResultSerializer: SimpleVersionedSerializer[AnyRef]): SerializedTaskResult = { + val logScanner = table.newScan().createLogScanner() + try { + val tableBucket = split.tableBucket + if (tableBucket.getPartitionId != null) { + logScanner.subscribe( + tableBucket.getPartitionId, + tableBucket.getBucket, + split.startingOffset) + } else { + logScanner.subscribe(tableBucket.getBucket, split.startingOffset) + } + + val stoppingOffset = split.stoppingOffset + val pollTimeout = Duration.ofMillis(pollTimeoutMs) + var lastOffset = -1L + var maxTimestamp = UNKNOWN_BUCKET_TIMESTAMP + var finished = false + + while (!finished) { + val scanRecords = logScanner.poll(pollTimeout) + val records = scanRecords.records(tableBucket) + if (records != null) { + records.asScala.foreach { + record => + if (record.logOffset() < stoppingOffset) { + lakeWriter.write(record) + lastOffset = record.logOffset() + if (record.timestamp() > maxTimestamp) { + maxTimestamp = record.timestamp() + } + } + if (record.logOffset() >= stoppingOffset - 1) { + finished = true + } + } + } + } + + val writeResult = lakeWriter.complete() + val serializedBytes = if (writeResult != null) { + writeResultSerializer.serialize(writeResult) + } else { + null + } + + logInfo( + s"Finished tiering log split for bucket $tableBucket," + + s" logEndOffset=$stoppingOffset, maxTimestamp=$maxTimestamp") + + SerializedTaskResult( + split.tablePath, + tableBucket, + split.partitionName, + serializedBytes, + writeResultSerializer.getVersion, + stoppingOffset, + maxTimestamp, + split.numberOfSplits) + } finally { + closeQuietly(logScanner, "LogScanner") + } + } + + private def processSnapshotSplit( + table: Table, + lakeWriter: LakeWriter[AnyRef], + split: TieringSnapshotSplit, + pollTimeoutMs: Long, + writeResultSerializer: SimpleVersionedSerializer[AnyRef]): SerializedTaskResult = { + val tableBucket = split.tableBucket + val batchScanner = table + .newScan() + .createBatchScanner(tableBucket, split.snapshotId) + + try { + val pollTimeout = Duration.ofMillis(pollTimeoutMs) + var batch = batchScanner.pollBatch(pollTimeout) + while (batch != null) { + while (batch.hasNext) { + val row = batch.next() + // Wrap InternalRow as ScanRecord with INSERT change type for snapshot data + val scanRecord = new ScanRecord(row) + lakeWriter.write(scanRecord) + } + batch.close() + batch = batchScanner.pollBatch(pollTimeout) + } + + val writeResult = lakeWriter.complete() + val serializedBytes = if (writeResult != null) { + writeResultSerializer.serialize(writeResult) + } else { + null + } + + val logEndOffset = split.logOffsetOfSnapshot + + logInfo( + s"Finished tiering snapshot split for bucket $tableBucket," + + s" snapshotId=${split.snapshotId}, logEndOffset=$logEndOffset") + + SerializedTaskResult( + split.tablePath, + tableBucket, + split.partitionName, + serializedBytes, + writeResultSerializer.getVersion, + logEndOffset, + UNKNOWN_BUCKET_TIMESTAMP, + split.numberOfSplits + ) + } finally { + closeQuietly(batchScanner, "BatchScanner") + } + } +} diff --git a/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringTaskResult.scala b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringTaskResult.scala new file mode 100644 index 0000000000..fb14c7a7b0 --- /dev/null +++ b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringTaskResult.scala @@ -0,0 +1,55 @@ +/* + * 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.fluss.spark.tiering + +import org.apache.fluss.metadata.{TableBucket, TablePath} + +/** + * Serialized result from a tiering task executed on a Spark executor. + * + * The write result is serialized as a byte array using the factory's + * [[org.apache.fluss.lake.serializer.SimpleVersionedSerializer]] to avoid Spark serialization + * issues with lake-specific types. + * + * @param tablePath + * the table path + * @param tableBucket + * the table bucket + * @param partitionName + * optional partition name + * @param serializedWriteResult + * serialized write result bytes, null if no data was written + * @param writeResultVersion + * serializer version used to serialize the write result + * @param logEndOffset + * the log end offset after processing + * @param maxTimestamp + * the maximum timestamp encountered during processing + * @param numberOfSplits + * total number of splits in this tiering round + */ +case class SerializedTaskResult( + tablePath: TablePath, + tableBucket: TableBucket, + partitionName: Option[String], + serializedWriteResult: Array[Byte], + writeResultVersion: Int, + logEndOffset: Long, + maxTimestamp: Long, + numberOfSplits: Int) + extends Serializable diff --git a/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/package.scala b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/package.scala new file mode 100644 index 0000000000..ddaf4f4da3 --- /dev/null +++ b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/package.scala @@ -0,0 +1,33 @@ +/* + * 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.fluss.spark + +import org.apache.spark.internal.Logging + +package object tiering extends Logging { + + /** Closes an [[AutoCloseable]] resource quietly, logging any exception instead of throwing. */ + private[tiering] def closeQuietly(closeable: AutoCloseable, name: String): Unit = { + if (closeable != null) { + try closeable.close() + catch { + case e: Exception => logWarning(s"Error closing $name", e) + } + } + } +} diff --git a/fluss-spark/fluss-spark-tiering/pom.xml b/fluss-spark/fluss-spark-tiering/pom.xml new file mode 100644 index 0000000000..5704ffe764 --- /dev/null +++ b/fluss-spark/fluss-spark-tiering/pom.xml @@ -0,0 +1,96 @@ + + + + + 4.0.0 + + org.apache.fluss + fluss-spark + 1.0-SNAPSHOT + + + jar + + fluss-spark-tiering_${scala.binary.version} + Fluss : Engine Spark : Tiering + + + + org.apache.fluss + fluss-spark-common_${scala.binary.version} + ${project.version} + provided + + + + org.apache.fluss + fluss-client + ${project.version} + provided + + + + org.apache.fluss + fluss-common + ${project.version} + provided + + + + org.apache.spark + spark-core_${scala.binary.version} + ${spark.version} + provided + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${spark.version} + provided + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-fluss + package + + shade + + + + + org.apache.fluss.spark.tiering.SparkLakeTiering + + + + + + + + + diff --git a/fluss-spark/fluss-spark-tiering/src/main/scala/org/apache/fluss/spark/tiering/SparkLakeTiering.scala b/fluss-spark/fluss-spark-tiering/src/main/scala/org/apache/fluss/spark/tiering/SparkLakeTiering.scala new file mode 100644 index 0000000000..176e5d9c77 --- /dev/null +++ b/fluss-spark/fluss-spark-tiering/src/main/scala/org/apache/fluss/spark/tiering/SparkLakeTiering.scala @@ -0,0 +1,130 @@ +/* + * 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.fluss.spark.tiering + +import org.apache.fluss.config.{ConfigOptions, Configuration} +import org.apache.fluss.utils.PropertiesUtils + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession + +import scala.collection.JavaConverters._ +import scala.concurrent.Await +import scala.concurrent.duration.Duration + +/** + * Entry point for the Spark-based Fluss Lake Tiering Service. + * + * Parses CLI arguments, creates a SparkSession, and starts [[SparkTieringJobRunner]] + * asynchronously. The main thread blocks until the runner's Future completes (either normally via + * [[stop()]] or on failure). + * + * CLI arguments follow the same convention as the Flink entry point: + * - `fluss.*` -> Fluss client config (prefix stripped via `extractAndRemovePrefix`) + * - `datalake.format` -> data lake format identifier + * - `datalake.{format}.*` -> lake config (prefix stripped via `extractAndRemovePrefix`) + * - `lake.tiering.*` -> lake tiering config (prefix retained via `extractPrefix`) + */ +object SparkLakeTiering extends Logging { + + private val FLUSS_CONF_PREFIX = "fluss." + private val DATA_LAKE_CONFIG_PREFIX = "datalake." + private val LAKE_TIERING_CONFIG_PREFIX = "lake.tiering." + + def main(args: Array[String]): Unit = { + val paramsMap = parseArgs(args) + + // Extract fluss config + val flussConfigMap = PropertiesUtils.extractAndRemovePrefix(paramsMap.asJava, FLUSS_CONF_PREFIX) + val bootstrapServers = flussConfigMap.get(ConfigOptions.BOOTSTRAP_SERVERS.key()) + if (bootstrapServers == null) { + throw new IllegalArgumentException( + s"The bootstrap server to Fluss is not configured, please configure " + + s"$FLUSS_CONF_PREFIX${ConfigOptions.BOOTSTRAP_SERVERS.key()}") + } + val flussConfig = Configuration.fromMap(flussConfigMap) + + // Extract data lake format + val dataLakeFormat = paramsMap.get("datalake.format").orNull + if (dataLakeFormat == null) { + throw new IllegalArgumentException( + s"${ConfigOptions.DATALAKE_FORMAT.key()} is not configured") + } + + // Extract lake config + val lakeConfigMap = PropertiesUtils.extractAndRemovePrefix( + paramsMap.asJava, + s"$DATA_LAKE_CONFIG_PREFIX$dataLakeFormat.") + val lakeConfig = Configuration.fromMap(lakeConfigMap) + + // Extract lake tiering config + val lakeTieringConfigMap = + PropertiesUtils.extractPrefix(paramsMap.asJava, LAKE_TIERING_CONFIG_PREFIX) + val lakeTieringConfig = Configuration.fromMap(lakeTieringConfigMap) + + // Load lake storage plugin and create factory + logInfo(s"Loading lake storage plugin for format: $dataLakeFormat") + + // Create SparkSession + val spark = SparkSession + .builder() + .appName(s"Fluss Lake Tiering Service - $dataLakeFormat") + .getOrCreate() + + logInfo(s"Starting Fluss Lake Tiering Service with Spark (format=$dataLakeFormat).") + logInfo(s"Parsed CLI args: ${paramsMap.map { case (k, v) => s"$k=$v" }.mkString(", ")}") + + try { + val runner = new SparkTieringJobRunner( + spark, + flussConfig, + lakeTieringConfig, + dataLakeFormat, + lakeConfig + ) + val future = runner.startAsync() + Await.result(future, Duration.Inf) + } finally { + spark.stop() + } + } + + /** + * Parses command line arguments in the format `--key value` or `--key=value` into a map of + * key-value pairs. + */ + private def parseArgs(args: Array[String]): Map[String, String] = { + var result = Map.empty[String, String] + var i = 0 + while (i < args.length) { + val arg = args(i) + if (arg.startsWith("--")) { + val keyValue = arg.substring(2) + val eqIdx = keyValue.indexOf('=') + if (eqIdx >= 0) { + result = result + (keyValue.substring(0, eqIdx) -> keyValue.substring(eqIdx + 1)) + } else if (i + 1 < args.length) { + result = result + (keyValue -> args(i + 1)) + i += 1 + } + } + i += 1 + } + result + } +} diff --git a/fluss-spark/fluss-spark-ut/src/test/scala/org/apache/fluss/spark/tiering/SparkTieringLogTableTest.scala b/fluss-spark/fluss-spark-ut/src/test/scala/org/apache/fluss/spark/tiering/SparkTieringLogTableTest.scala new file mode 100644 index 0000000000..b3f303f09e --- /dev/null +++ b/fluss-spark/fluss-spark-ut/src/test/scala/org/apache/fluss/spark/tiering/SparkTieringLogTableTest.scala @@ -0,0 +1,153 @@ +/* + * 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.fluss.spark.tiering + +import org.apache.fluss.config.{ConfigOptions, Configuration} +import org.apache.fluss.metadata.DataLakeFormat +import org.apache.fluss.spark.SparkConnectorOptions.BUCKET_NUMBER + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.Row + +import java.nio.file.Files + +/** + * Integration test for the Spark tiering pipeline on log tables. + * + * Uses the Spark tiering components directly (TieringSplitGenerator, TieringTask, TieringCommitter) + * instead of the Flink-based LakeTieringJobBuilder, to validate that the Spark tiering path + * produces correct lake data that can be read back via Spark SQL. + */ +abstract class SparkTieringLogTableTest extends SparkTieringTestBase with Logging { + + test("Spark Tiering: log table tier and read back") { + withTable("t_spark_tier") { + sql(s""" + |CREATE TABLE $DEFAULT_DATABASE.t_spark_tier (id INT, name STRING) + | TBLPROPERTIES ( + | '${ConfigOptions.TABLE_DATALAKE_ENABLED.key()}' = true, + | '${ConfigOptions.TABLE_DATALAKE_FRESHNESS.key()}' = '1s', + | '${BUCKET_NUMBER.key()}' = 1) + |""".stripMargin) + + sql(s""" + |INSERT INTO $DEFAULT_DATABASE.t_spark_tier VALUES + |(1, "alpha"), (2, "beta"), (3, "gamma") + |""".stripMargin) + + tierToLakeViaSpark("t_spark_tier") + + checkAnswer( + sql(s"SELECT * FROM $DEFAULT_DATABASE.t_spark_tier ORDER BY id"), + Row(1, "alpha") :: Row(2, "beta") :: Row(3, "gamma") :: Nil + ) + + checkAnswer( + sql(s"SELECT name FROM $DEFAULT_DATABASE.t_spark_tier ORDER BY name"), + Row("alpha") :: Row("beta") :: Row("gamma") :: Nil + ) + } + } + + test("Spark Tiering: log table union read (lake + log tail)") { + withTable("t_spark_union") { + sql(s""" + |CREATE TABLE $DEFAULT_DATABASE.t_spark_union (id INT, name STRING) + | TBLPROPERTIES ( + | '${ConfigOptions.TABLE_DATALAKE_ENABLED.key()}' = true, + | '${ConfigOptions.TABLE_DATALAKE_FRESHNESS.key()}' = '1s', + | '${BUCKET_NUMBER.key()}' = 1) + |""".stripMargin) + + sql(s""" + |INSERT INTO $DEFAULT_DATABASE.t_spark_union VALUES + |(1, "alpha"), (2, "beta"), (3, "gamma") + |""".stripMargin) + + tierToLakeViaSpark("t_spark_union") + + // Insert more data after tiering (this should appear in the log tail) + sql(s""" + |INSERT INTO $DEFAULT_DATABASE.t_spark_union VALUES + |(4, "delta"), (5, "epsilon") + |""".stripMargin) + + checkAnswer( + sql(s"SELECT * FROM $DEFAULT_DATABASE.t_spark_union ORDER BY id"), + Row(1, "alpha") :: Row(2, "beta") :: Row(3, "gamma") :: + Row(4, "delta") :: Row(5, "epsilon") :: Nil + ) + } + } + + test("Spark Tiering: log table incremental tiering") { + withTable("t_spark_incr") { + sql(s""" + |CREATE TABLE $DEFAULT_DATABASE.t_spark_incr (id INT, name STRING) + | TBLPROPERTIES ( + | '${ConfigOptions.TABLE_DATALAKE_ENABLED.key()}' = true, + | '${ConfigOptions.TABLE_DATALAKE_FRESHNESS.key()}' = '1s', + | '${BUCKET_NUMBER.key()}' = 1) + |""".stripMargin) + + // First batch + sql(s""" + |INSERT INTO $DEFAULT_DATABASE.t_spark_incr VALUES + |(1, "alpha"), (2, "beta") + |""".stripMargin) + + tierToLakeViaSpark("t_spark_incr") + + // Second batch + sql(s""" + |INSERT INTO $DEFAULT_DATABASE.t_spark_incr VALUES + |(3, "gamma"), (4, "delta") + |""".stripMargin) + + tierToLakeViaSpark("t_spark_incr") + + checkAnswer( + sql(s"SELECT * FROM $DEFAULT_DATABASE.t_spark_incr ORDER BY id"), + Row(1, "alpha") :: Row(2, "beta") :: Row(3, "gamma") :: Row(4, "delta") :: Nil + ) + } + } +} + +class SparkTieringPaimonLogTableTest extends SparkTieringLogTableTest { + + override def dataLakeFormat: DataLakeFormat = DataLakeFormat.PAIMON + + override def lakeConfig: Configuration = { + val conf = new Configuration() + conf.setString("metastore", "filesystem") + conf.setString("warehouse", warehousePath) + conf + } + + override def flussConf: Configuration = { + val conf = super.flussConf + conf.setString("datalake.format", dataLakeFormat.toString) + conf.setString("datalake.paimon.metastore", "filesystem") + conf.setString("datalake.paimon.cache-enabled", "false") + warehousePath = + Files.createTempDirectory("fluss-testing-spark-tiering").resolve("warehouse").toString + conf.setString("datalake.paimon.warehouse", warehousePath) + conf + } +} diff --git a/fluss-spark/fluss-spark-ut/src/test/scala/org/apache/fluss/spark/tiering/SparkTieringTestBase.scala b/fluss-spark/fluss-spark-ut/src/test/scala/org/apache/fluss/spark/tiering/SparkTieringTestBase.scala new file mode 100644 index 0000000000..391f720f9c --- /dev/null +++ b/fluss-spark/fluss-spark-ut/src/test/scala/org/apache/fluss/spark/tiering/SparkTieringTestBase.scala @@ -0,0 +1,105 @@ +/* + * 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.fluss.spark.tiering + +import org.apache.fluss.config.Configuration +import org.apache.fluss.metadata.{DataLakeFormat, TableBucket} +import org.apache.fluss.spark.FlussSparkTestBase + +import org.apache.spark.internal.Logging + +import java.time.Duration + +/** + * Integration test for the Spark tiering pipeline on log tables. + * + * Uses the Spark tiering components directly (TieringSplitGenerator, TieringTask, TieringCommitter) + * instead of the Flink-based LakeTieringJobBuilder, to validate that the Spark tiering path + * produces correct lake data that can be read back via Spark SQL. + */ +abstract class SparkTieringTestBase extends FlussSparkTestBase with Logging { + + protected var warehousePath: String = _ + private val SYNC_TIMEOUT: Duration = Duration.ofMinutes(2) + private val SYNC_POLL_INTERVAL_MS = 500L + + protected def dataLakeFormat: DataLakeFormat + + protected def lakeConfig: Configuration + + def tierToLakeViaSpark(tableName: String): Unit = { + val tablePath = createTablePath(tableName) + val tableInfo = admin.getTableInfo(tablePath).get() + + val flussConfig = flussServer.getClientConfig + val lakeTieringConfig = new Configuration() + + val runner = + new SparkTieringJobRunner( + spark, + flussConfig, + lakeTieringConfig, + dataLakeFormat.toString, + lakeConfig) + + try { + runner.startAsync() + waitForLakeSnapshotSync(tableInfo) + } finally { + runner.stop() + } + + } + + /** Polls replicas until all buckets have a non-negative lake snapshot ID. */ + private def waitForLakeSnapshotSync(tableInfo: org.apache.fluss.metadata.TableInfo): Unit = { + val tableId = tableInfo.getTableId + val numBuckets = tableInfo.getNumBuckets + + val tableBuckets = (0 until numBuckets).map(new TableBucket(tableId, _)).toSet + + val deadline = System.currentTimeMillis() + SYNC_TIMEOUT.toMillis + val syncedBuckets = scala.collection.mutable.Set[TableBucket]() + + while (syncedBuckets.size < tableBuckets.size && System.currentTimeMillis() < deadline) { + tableBuckets.foreach { + tableBucket => + if (!syncedBuckets.contains(tableBucket)) { + try { + val replica = flussServer.waitAndGetLeaderReplica(tableBucket) + if (replica.getLogTablet.getLakeTableSnapshotId >= 0) { + syncedBuckets.add(tableBucket) + } + } catch { + case _: Exception => + } + } + } + if (syncedBuckets.size < tableBuckets.size) { + Thread.sleep(SYNC_POLL_INTERVAL_MS) + } + } + + assert( + syncedBuckets.size == tableBuckets.size, + s"Not all buckets synced to lake within $SYNC_TIMEOUT. " + + s"Synced: ${syncedBuckets.size}, Total: ${tableBuckets.size}" + ) + logInfo(s"${tableInfo.getTablePath} synced to lake") + } +} diff --git a/fluss-spark/pom.xml b/fluss-spark/pom.xml index b64b12ed66..241e7917ed 100644 --- a/fluss-spark/pom.xml +++ b/fluss-spark/pom.xml @@ -40,6 +40,7 @@ fluss-spark-ut fluss-spark-3.5 fluss-spark-3.4 + fluss-spark-tiering diff --git a/fluss-test-coverage/pom.xml b/fluss-test-coverage/pom.xml index 1df67094f1..86fc8828fa 100644 --- a/fluss-test-coverage/pom.xml +++ b/fluss-test-coverage/pom.xml @@ -460,7 +460,7 @@ org.apache.fluss.flink.tiering.source.enumerator.TieringSourceEnumerator.HeartBeatHelper - org.apache.fluss.flink.tiering.source.TieringWriterInitContext + org.apache.fluss.client.tiering.TieringWriterInitContext org.apache.fluss.flink.tiering.source.TieringSourceReader org.apache.fluss.flink.tiering.source.TieringSourceFetcherManager @@ -475,7 +475,7 @@ org.apache.fluss.flink.tiering.committer.CommittableMessageTypeInfo* - org.apache.fluss.flink.tiering.committer.TieringCommitterInitContext + org.apache.fluss.client.tiering.TieringCommitterInitContext org.apache.fluss.flink.tiering.LakeTieringJobBuilder From fb472b159f837528142895e203d9c69816ea03bd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=BE=8A=E5=B7=9D?= Date: Mon, 18 May 2026 22:56:11 +0800 Subject: [PATCH 2/3] add more tests --- .../tiering/SparkTieringLogTableTest.scala | 106 ++++++ .../tiering/SparkTieringPKTableTest.scala | 308 ++++++++++++++++++ .../spark/tiering/SparkTieringTestBase.scala | 14 +- 3 files changed, 427 insertions(+), 1 deletion(-) create mode 100644 fluss-spark/fluss-spark-ut/src/test/scala/org/apache/fluss/spark/tiering/SparkTieringPKTableTest.scala diff --git a/fluss-spark/fluss-spark-ut/src/test/scala/org/apache/fluss/spark/tiering/SparkTieringLogTableTest.scala b/fluss-spark/fluss-spark-ut/src/test/scala/org/apache/fluss/spark/tiering/SparkTieringLogTableTest.scala index b3f303f09e..9042649d9f 100644 --- a/fluss-spark/fluss-spark-ut/src/test/scala/org/apache/fluss/spark/tiering/SparkTieringLogTableTest.scala +++ b/fluss-spark/fluss-spark-ut/src/test/scala/org/apache/fluss/spark/tiering/SparkTieringLogTableTest.scala @@ -127,6 +127,112 @@ abstract class SparkTieringLogTableTest extends SparkTieringTestBase with Loggin ) } } + + test("Spark Tiering: partitioned log table tier and read back") { + withTable("t_spark_tier_pt") { + sql(s""" + |CREATE TABLE $DEFAULT_DATABASE.t_spark_tier_pt (id INT, name STRING, dt STRING) + | PARTITIONED BY (dt) + | TBLPROPERTIES ( + | '${ConfigOptions.TABLE_DATALAKE_ENABLED.key()}' = true, + | '${ConfigOptions.TABLE_DATALAKE_FRESHNESS.key()}' = '1s', + | '${BUCKET_NUMBER.key()}' = 1) + |""".stripMargin) + + sql(s""" + |INSERT INTO $DEFAULT_DATABASE.t_spark_tier_pt VALUES + |(1, "alpha", "2026-01-01"), (2, "beta", "2026-01-01"), (3, "gamma", "2026-01-02") + |""".stripMargin) + + tierToLakeViaSpark("t_spark_tier_pt") + + checkAnswer( + sql(s"SELECT * FROM $DEFAULT_DATABASE.t_spark_tier_pt ORDER BY id"), + Row(1, "alpha", "2026-01-01") :: + Row(2, "beta", "2026-01-01") :: + Row(3, "gamma", "2026-01-02") :: Nil + ) + + checkAnswer( + sql(s"SELECT name, dt FROM $DEFAULT_DATABASE.t_spark_tier_pt ORDER BY name"), + Row("alpha", "2026-01-01") :: + Row("beta", "2026-01-01") :: + Row("gamma", "2026-01-02") :: Nil + ) + } + } + + test("Spark Tiering: partitioned log table union read (lake + log tail)") { + withTable("t_spark_union_pt") { + sql(s""" + |CREATE TABLE $DEFAULT_DATABASE.t_spark_union_pt (id INT, name STRING, dt STRING) + | PARTITIONED BY (dt) + | TBLPROPERTIES ( + | '${ConfigOptions.TABLE_DATALAKE_ENABLED.key()}' = true, + | '${ConfigOptions.TABLE_DATALAKE_FRESHNESS.key()}' = '1s', + | '${BUCKET_NUMBER.key()}' = 1) + |""".stripMargin) + + sql(s""" + |INSERT INTO $DEFAULT_DATABASE.t_spark_union_pt VALUES + |(1, "alpha", "2026-01-01"), (2, "beta", "2026-01-01"), (3, "gamma", "2026-01-02") + |""".stripMargin) + + tierToLakeViaSpark("t_spark_union_pt") + + // Insert more data after tiering (log tail in existing and new partition) + sql(s""" + |INSERT INTO $DEFAULT_DATABASE.t_spark_union_pt VALUES + |(4, "delta", "2026-01-01"), (5, "epsilon", "2026-01-03") + |""".stripMargin) + + checkAnswer( + sql(s"SELECT * FROM $DEFAULT_DATABASE.t_spark_union_pt ORDER BY id"), + Row(1, "alpha", "2026-01-01") :: + Row(2, "beta", "2026-01-01") :: + Row(3, "gamma", "2026-01-02") :: + Row(4, "delta", "2026-01-01") :: + Row(5, "epsilon", "2026-01-03") :: Nil + ) + } + } + + test("Spark Tiering: partitioned log table incremental tiering") { + withTable("t_spark_incr_pt") { + sql(s""" + |CREATE TABLE $DEFAULT_DATABASE.t_spark_incr_pt (id INT, name STRING, dt STRING) + | PARTITIONED BY (dt) + | TBLPROPERTIES ( + | '${ConfigOptions.TABLE_DATALAKE_ENABLED.key()}' = true, + | '${ConfigOptions.TABLE_DATALAKE_FRESHNESS.key()}' = '1s', + | '${BUCKET_NUMBER.key()}' = 1) + |""".stripMargin) + + // First batch: partition 2026-01-01 + sql(s""" + |INSERT INTO $DEFAULT_DATABASE.t_spark_incr_pt VALUES + |(1, "alpha", "2026-01-01"), (2, "beta", "2026-01-01") + |""".stripMargin) + + tierToLakeViaSpark("t_spark_incr_pt") + + // Second batch: partition 2026-01-02 + sql(s""" + |INSERT INTO $DEFAULT_DATABASE.t_spark_incr_pt VALUES + |(3, "gamma", "2026-01-02"), (4, "delta", "2026-01-02") + |""".stripMargin) + + tierToLakeViaSpark("t_spark_incr_pt") + + checkAnswer( + sql(s"SELECT * FROM $DEFAULT_DATABASE.t_spark_incr_pt ORDER BY id"), + Row(1, "alpha", "2026-01-01") :: + Row(2, "beta", "2026-01-01") :: + Row(3, "gamma", "2026-01-02") :: + Row(4, "delta", "2026-01-02") :: Nil + ) + } + } } class SparkTieringPaimonLogTableTest extends SparkTieringLogTableTest { diff --git a/fluss-spark/fluss-spark-ut/src/test/scala/org/apache/fluss/spark/tiering/SparkTieringPKTableTest.scala b/fluss-spark/fluss-spark-ut/src/test/scala/org/apache/fluss/spark/tiering/SparkTieringPKTableTest.scala new file mode 100644 index 0000000000..f24e5022d8 --- /dev/null +++ b/fluss-spark/fluss-spark-ut/src/test/scala/org/apache/fluss/spark/tiering/SparkTieringPKTableTest.scala @@ -0,0 +1,308 @@ +/* + * 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.fluss.spark.tiering + +import org.apache.fluss.config.{ConfigOptions, Configuration} +import org.apache.fluss.metadata.DataLakeFormat +import org.apache.fluss.spark.SparkConnectorOptions.{BUCKET_NUMBER, PRIMARY_KEY} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.Row + +import java.nio.file.Files + +/** + * Integration test for the Spark tiering pipeline on primary key (PK) tables. + * + * Validates that the Spark tiering path correctly handles PK table semantics including lake-only + * reads, union reads (lake + kv tail), and update merge for both non-partitioned and partitioned + * tables. + */ +abstract class SparkTieringPKTableTest extends SparkTieringTestBase with Logging { + + test("Spark Tiering: pk table tier and read back") { + withTable("t_pk_tier") { + sql(s""" + |CREATE TABLE $DEFAULT_DATABASE.t_pk_tier (id INT, name STRING, score INT) + | TBLPROPERTIES ( + | '${ConfigOptions.TABLE_DATALAKE_ENABLED.key()}' = true, + | '${ConfigOptions.TABLE_DATALAKE_FRESHNESS.key()}' = '1s', + | '${PRIMARY_KEY.key()}' = 'id', + | '${BUCKET_NUMBER.key()}' = 1) + |""".stripMargin) + + sql(s""" + |INSERT INTO $DEFAULT_DATABASE.t_pk_tier VALUES + |(1, "alice", 90), (2, "bob", 85), (3, "charlie", 95) + |""".stripMargin) + + tierToLakeViaSpark("t_pk_tier") + + checkAnswer( + sql(s"SELECT * FROM $DEFAULT_DATABASE.t_pk_tier ORDER BY id"), + Row(1, "alice", 90) :: Row(2, "bob", 85) :: Row(3, "charlie", 95) :: Nil + ) + + // Column projection + checkAnswer( + sql(s"SELECT name FROM $DEFAULT_DATABASE.t_pk_tier ORDER BY name"), + Row("alice") :: Row("bob") :: Row("charlie") :: Nil + ) + } + } + + test("Spark Tiering: pk table union read (lake + kv tail)") { + withTable("t_pk_union") { + sql(s""" + |CREATE TABLE $DEFAULT_DATABASE.t_pk_union (id INT, name STRING, score INT) + | TBLPROPERTIES ( + | '${ConfigOptions.TABLE_DATALAKE_ENABLED.key()}' = true, + | '${ConfigOptions.TABLE_DATALAKE_FRESHNESS.key()}' = '1s', + | '${PRIMARY_KEY.key()}' = 'id', + | '${BUCKET_NUMBER.key()}' = 1) + |""".stripMargin) + + sql(s""" + |INSERT INTO $DEFAULT_DATABASE.t_pk_union VALUES + |(1, "alice", 90), (2, "bob", 85), (3, "charlie", 95) + |""".stripMargin) + + tierToLakeViaSpark("t_pk_union") + + // Insert new rows after tiering (these live in the kv tail) + sql(s""" + |INSERT INTO $DEFAULT_DATABASE.t_pk_union VALUES + |(4, "david", 88), (5, "eve", 92) + |""".stripMargin) + + checkAnswer( + sql(s"SELECT * FROM $DEFAULT_DATABASE.t_pk_union ORDER BY id"), + Row(1, "alice", 90) :: Row(2, "bob", 85) :: Row(3, "charlie", 95) :: + Row(4, "david", 88) :: Row(5, "eve", 92) :: Nil + ) + } + } + + test("Spark Tiering: pk table union read with updates") { + withTable("t_pk_update") { + sql(s""" + |CREATE TABLE $DEFAULT_DATABASE.t_pk_update (id INT, name STRING, score INT) + | TBLPROPERTIES ( + | '${ConfigOptions.TABLE_DATALAKE_ENABLED.key()}' = true, + | '${ConfigOptions.TABLE_DATALAKE_FRESHNESS.key()}' = '1s', + | '${PRIMARY_KEY.key()}' = 'id', + | '${BUCKET_NUMBER.key()}' = 1) + |""".stripMargin) + + sql(s""" + |INSERT INTO $DEFAULT_DATABASE.t_pk_update VALUES + |(1, "alice", 90), (2, "bob", 85), (3, "charlie", 95) + |""".stripMargin) + + tierToLakeViaSpark("t_pk_update") + + // Update id=2 and insert a new record id=4 + sql(s""" + |INSERT INTO $DEFAULT_DATABASE.t_pk_update VALUES + |(2, "bob_updated", 100), (4, "david", 88) + |""".stripMargin) + + // Union read: updated value for id=2 should come from kv tail + checkAnswer( + sql(s"SELECT * FROM $DEFAULT_DATABASE.t_pk_update ORDER BY id"), + Row(1, "alice", 90) :: Row(2, "bob_updated", 100) :: + Row(3, "charlie", 95) :: Row(4, "david", 88) :: Nil + ) + } + } + + test("Spark Tiering: pk table incremental tiering") { + withTable("t_pk_incr") { + sql(s""" + |CREATE TABLE $DEFAULT_DATABASE.t_pk_incr (id INT, name STRING, score INT) + | TBLPROPERTIES ( + | '${ConfigOptions.TABLE_DATALAKE_ENABLED.key()}' = true, + | '${ConfigOptions.TABLE_DATALAKE_FRESHNESS.key()}' = '1s', + | '${PRIMARY_KEY.key()}' = 'id', + | '${BUCKET_NUMBER.key()}' = 1) + |""".stripMargin) + + // First batch + sql(s""" + |INSERT INTO $DEFAULT_DATABASE.t_pk_incr VALUES + |(1, "alice", 90), (2, "bob", 85) + |""".stripMargin) + + tierToLakeViaSpark("t_pk_incr") + + // Second batch: new records + update existing + sql(s""" + |INSERT INTO $DEFAULT_DATABASE.t_pk_incr VALUES + |(2, "bob_v2", 92), (3, "charlie", 95) + |""".stripMargin) + + tierToLakeViaSpark("t_pk_incr") + + checkAnswer( + sql(s"SELECT * FROM $DEFAULT_DATABASE.t_pk_incr ORDER BY id"), + Row(1, "alice", 90) :: Row(2, "bob_v2", 92) :: Row(3, "charlie", 95) :: Nil + ) + } + } + + test("Spark Tiering: partitioned pk table tier and read back") { + withTable("t_pk_tier_pt") { + sql(s""" + |CREATE TABLE $DEFAULT_DATABASE.t_pk_tier_pt + | (id INT, name STRING, score INT, dt STRING) + | PARTITIONED BY (dt) + | TBLPROPERTIES ( + | '${ConfigOptions.TABLE_DATALAKE_ENABLED.key()}' = true, + | '${ConfigOptions.TABLE_DATALAKE_FRESHNESS.key()}' = '1s', + | '${PRIMARY_KEY.key()}' = 'id,dt', + | '${BUCKET_NUMBER.key()}' = 1) + |""".stripMargin) + + sql(s""" + |INSERT INTO $DEFAULT_DATABASE.t_pk_tier_pt VALUES + |(1, "alice", 90, "2026-01-01"), + |(2, "bob", 85, "2026-01-01"), + |(3, "charlie", 95, "2026-01-02") + |""".stripMargin) + + tierToLakeViaSpark("t_pk_tier_pt") + + checkAnswer( + sql(s"SELECT * FROM $DEFAULT_DATABASE.t_pk_tier_pt ORDER BY id"), + Row(1, "alice", 90, "2026-01-01") :: + Row(2, "bob", 85, "2026-01-01") :: + Row(3, "charlie", 95, "2026-01-02") :: Nil + ) + + // Column projection with different order + checkAnswer( + sql(s"SELECT dt, name, id FROM $DEFAULT_DATABASE.t_pk_tier_pt ORDER BY id"), + Row("2026-01-01", "alice", 1) :: + Row("2026-01-01", "bob", 2) :: + Row("2026-01-02", "charlie", 3) :: Nil + ) + } + } + + test("Spark Tiering: partitioned pk table union read (lake + kv tail)") { + withTable("t_pk_union_pt") { + sql(s""" + |CREATE TABLE $DEFAULT_DATABASE.t_pk_union_pt + | (id INT, name STRING, score INT, dt STRING) + | PARTITIONED BY (dt) + | TBLPROPERTIES ( + | '${ConfigOptions.TABLE_DATALAKE_ENABLED.key()}' = true, + | '${ConfigOptions.TABLE_DATALAKE_FRESHNESS.key()}' = '1s', + | '${PRIMARY_KEY.key()}' = 'id,dt', + | '${BUCKET_NUMBER.key()}' = 1) + |""".stripMargin) + + sql(s""" + |INSERT INTO $DEFAULT_DATABASE.t_pk_union_pt VALUES + |(1, "alice", 90, "2026-01-01"), + |(2, "bob", 85, "2026-01-01"), + |(3, "charlie", 95, "2026-01-02") + |""".stripMargin) + + tierToLakeViaSpark("t_pk_union_pt") + + // Insert more data after tiering: existing partition + new partition + sql(s""" + |INSERT INTO $DEFAULT_DATABASE.t_pk_union_pt VALUES + |(4, "david", 88, "2026-01-01"), + |(5, "eve", 92, "2026-01-03") + |""".stripMargin) + + checkAnswer( + sql(s"SELECT * FROM $DEFAULT_DATABASE.t_pk_union_pt ORDER BY id"), + Row(1, "alice", 90, "2026-01-01") :: + Row(2, "bob", 85, "2026-01-01") :: + Row(3, "charlie", 95, "2026-01-02") :: + Row(4, "david", 88, "2026-01-01") :: + Row(5, "eve", 92, "2026-01-03") :: Nil + ) + } + } + + test("Spark Tiering: partitioned pk table union read with updates") { + withTable("t_pk_update_pt") { + sql(s""" + |CREATE TABLE $DEFAULT_DATABASE.t_pk_update_pt + | (id INT, name STRING, score INT, dt STRING) + | PARTITIONED BY (dt) + | TBLPROPERTIES ( + | '${ConfigOptions.TABLE_DATALAKE_ENABLED.key()}' = true, + | '${ConfigOptions.TABLE_DATALAKE_FRESHNESS.key()}' = '1s', + | '${PRIMARY_KEY.key()}' = 'id,dt', + | '${BUCKET_NUMBER.key()}' = 1) + |""".stripMargin) + + sql(s""" + |INSERT INTO $DEFAULT_DATABASE.t_pk_update_pt VALUES + |(1, "alice", 90, "2026-01-01"), + |(2, "bob", 85, "2026-01-01"), + |(3, "charlie", 95, "2026-01-02") + |""".stripMargin) + + tierToLakeViaSpark("t_pk_update_pt") + + // Update id=2 (same partition) and insert new id=4 (same partition as charlie) + sql(s""" + |INSERT INTO $DEFAULT_DATABASE.t_pk_update_pt VALUES + |(2, "bob_updated", 100, "2026-01-01"), + |(4, "david", 88, "2026-01-02") + |""".stripMargin) + + checkAnswer( + sql(s"SELECT * FROM $DEFAULT_DATABASE.t_pk_update_pt ORDER BY id"), + Row(1, "alice", 90, "2026-01-01") :: + Row(2, "bob_updated", 100, "2026-01-01") :: + Row(3, "charlie", 95, "2026-01-02") :: + Row(4, "david", 88, "2026-01-02") :: Nil + ) + } + } +} + +class SparkTieringPaimonPKTableTest extends SparkTieringPKTableTest { + + override def dataLakeFormat: DataLakeFormat = DataLakeFormat.PAIMON + + override def lakeConfig: Configuration = { + val conf = new Configuration() + conf.setString("metastore", "filesystem") + conf.setString("warehouse", warehousePath) + conf + } + + override def flussConf: Configuration = { + val conf = super.flussConf + conf.setString("datalake.format", dataLakeFormat.toString) + conf.setString("datalake.paimon.metastore", "filesystem") + conf.setString("datalake.paimon.cache-enabled", "false") + warehousePath = + Files.createTempDirectory("fluss-testing-spark-tiering-pk").resolve("warehouse").toString + conf.setString("datalake.paimon.warehouse", warehousePath) + conf + } +} diff --git a/fluss-spark/fluss-spark-ut/src/test/scala/org/apache/fluss/spark/tiering/SparkTieringTestBase.scala b/fluss-spark/fluss-spark-ut/src/test/scala/org/apache/fluss/spark/tiering/SparkTieringTestBase.scala index 391f720f9c..c3eee3af07 100644 --- a/fluss-spark/fluss-spark-ut/src/test/scala/org/apache/fluss/spark/tiering/SparkTieringTestBase.scala +++ b/fluss-spark/fluss-spark-ut/src/test/scala/org/apache/fluss/spark/tiering/SparkTieringTestBase.scala @@ -25,6 +25,8 @@ import org.apache.spark.internal.Logging import java.time.Duration +import scala.collection.JavaConverters._ + /** * Integration test for the Spark tiering pipeline on log tables. * @@ -71,7 +73,17 @@ abstract class SparkTieringTestBase extends FlussSparkTestBase with Logging { val tableId = tableInfo.getTableId val numBuckets = tableInfo.getNumBuckets - val tableBuckets = (0 until numBuckets).map(new TableBucket(tableId, _)).toSet + val tableBuckets = if (tableInfo.isPartitioned) { + val partitionInfos = admin.listPartitionInfos(tableInfo.getTablePath).get() + partitionInfos.asScala.flatMap { + partitionInfo => + (0 until numBuckets).map { + bucket => new TableBucket(tableId, partitionInfo.getPartitionId, bucket) + } + }.toSet + } else { + (0 until numBuckets).map(bucket => new TableBucket(tableId, bucket)).toSet + } val deadline = System.currentTimeMillis() + SYNC_TIMEOUT.toMillis val syncedBuckets = scala.collection.mutable.Set[TableBucket]() From 4daae3f761658f0b58841ce4e43c1032dd6df13c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=BE=8A=E5=B7=9D?= Date: Wed, 20 May 2026 00:41:40 +0800 Subject: [PATCH 3/3] extract TieringCommitter --- .../tiering}/TableBucketWriteResult.java | 2 +- .../client/tiering/TieringCommitResult.java | 38 +++ .../client/tiering/TieringCommitter.java | 239 ++++++++++++++++++ .../committer/CommittableMessageTypeInfo.java | 2 +- .../committer/TieringCommitOperator.java | 225 +---------------- .../TieringCommitOperatorFactory.java | 2 +- .../source/TableBucketWriteResultEmitter.java | 14 +- .../TableBucketWriteResultSerializer.java | 15 +- .../TableBucketWriteResultTypeInfo.java | 35 +-- .../flink/tiering/source/TieringSource.java | 15 +- .../source/TieringSourceFetcherManager.java | 24 +- .../tiering/source/TieringSourceReader.java | 19 +- .../tiering/source/TieringSplitReader.java | 74 ++++-- .../committer/TieringCommitOperatorTest.java | 2 +- .../TableBucketWriteResultSerializerTest.java | 17 +- .../source/TieringSourceReaderTest.java | 45 +++- .../source/TieringSplitReaderTest.java | 31 ++- .../spark/tiering/SparkTieringJobRunner.scala | 20 +- .../spark/tiering/TieringCommitter.scala | 202 --------------- .../fluss/spark/tiering/TieringTask.scala | 45 +--- 20 files changed, 505 insertions(+), 561 deletions(-) rename {fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source => fluss-client/src/main/java/org/apache/fluss/client/tiering}/TableBucketWriteResult.java (98%) create mode 100644 fluss-client/src/main/java/org/apache/fluss/client/tiering/TieringCommitResult.java create mode 100644 fluss-client/src/main/java/org/apache/fluss/client/tiering/TieringCommitter.java delete mode 100644 fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringCommitter.scala diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TableBucketWriteResult.java b/fluss-client/src/main/java/org/apache/fluss/client/tiering/TableBucketWriteResult.java similarity index 98% rename from fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TableBucketWriteResult.java rename to fluss-client/src/main/java/org/apache/fluss/client/tiering/TableBucketWriteResult.java index abec3c6c21..de87f00948 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TableBucketWriteResult.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/tiering/TableBucketWriteResult.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.fluss.flink.tiering.source; +package org.apache.fluss.client.tiering; import org.apache.fluss.lake.writer.LakeWriter; import org.apache.fluss.metadata.TableBucket; diff --git a/fluss-client/src/main/java/org/apache/fluss/client/tiering/TieringCommitResult.java b/fluss-client/src/main/java/org/apache/fluss/client/tiering/TieringCommitResult.java new file mode 100644 index 0000000000..88fc451246 --- /dev/null +++ b/fluss-client/src/main/java/org/apache/fluss/client/tiering/TieringCommitResult.java @@ -0,0 +1,38 @@ +/* + * 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.fluss.client.tiering; + +import org.apache.fluss.lake.committer.TieringStats; + +import javax.annotation.Nullable; + +/** + * The result of one table's commit round, holding the lake committable (nullable for empty commits + * where no data was written) and the associated tiering statistics. + */ +public class TieringCommitResult { + /** The lake committable, or {@code null} if nothing was written in this round. */ + @Nullable public final Committable committable; + /** Per-table tiering statistics collected during this round. */ + @Nullable public final TieringStats stats; + + TieringCommitResult(@Nullable Committable committable, @Nullable TieringStats stats) { + this.committable = committable; + this.stats = stats; + } +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/tiering/TieringCommitter.java b/fluss-client/src/main/java/org/apache/fluss/client/tiering/TieringCommitter.java new file mode 100644 index 0000000000..7e5fc80ced --- /dev/null +++ b/fluss-client/src/main/java/org/apache/fluss/client/tiering/TieringCommitter.java @@ -0,0 +1,239 @@ +/* + * 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.fluss.client.tiering; + +import org.apache.fluss.client.admin.Admin; +import org.apache.fluss.client.metadata.LakeSnapshot; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.exception.LakeTableSnapshotNotExistException; +import org.apache.fluss.lake.committer.CommittedLakeSnapshot; +import org.apache.fluss.lake.committer.LakeCommitResult; +import org.apache.fluss.lake.committer.LakeCommitter; +import org.apache.fluss.lake.committer.TieringStats; +import org.apache.fluss.lake.writer.LakeTieringFactory; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TableInfo; +import org.apache.fluss.metadata.TablePath; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.fluss.lake.committer.LakeCommitter.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; + +/** Orchestrates the two-phase commit of tiering write results to the lake and Fluss coordinator. */ +public class TieringCommitter { + private static final Logger LOG = LoggerFactory.getLogger(TieringCommitter.class); + + /** + * Commits the collected write results for one table to the lake and Fluss. + * + *

Always returns a non-null {@link TieringCommitResult}. When all buckets produced no data + * (empty commit), {@link TieringCommitResult#committable} is {@code null} and stats are {@link + * TieringStats#UNKNOWN}. + */ + public TieringCommitResult commitWriteResults( + Admin admin, + long tableId, + TablePath tablePath, + Configuration flussConf, + Configuration lakeTieringConfig, + LakeTieringFactory lakeTieringFactory, + FlussTableLakeSnapshotCommitter flussTableLakeSnapshotCommitter, + List> committableWriteResults) + throws Exception { + // filter down to buckets that actually produced data + List> nonEmptyResults = + committableWriteResults.stream() + .filter(r -> r.writeResult() != null) + .collect(Collectors.toList()); + + // all buckets were empty — nothing to commit to the lake + if (nonEmptyResults.isEmpty()) { + LOG.info( + "Commit tiering write results is empty for table {}, table path {}", + tableId, + tablePath); + return new TieringCommitResult(null, null); + } + + // Check if the table was dropped and recreated during tiering. + // If the current table id differs from the committable's table id, fail this commit + // to avoid dirty commit to a newly created table. + TableInfo currentTableInfo = admin.getTableInfo(tablePath).get(); + if (currentTableInfo.getTableId() != tableId) { + throw new IllegalStateException( + String.format( + "The current table id %s for table path %s is different from the table id %s in the committable. " + + "This usually happens when a table was dropped and recreated during tiering. " + + "Aborting commit to prevent dirty commit.", + currentTableInfo.getTableId(), tablePath, tableId)); + } + + try (LakeCommitter lakeCommitter = + lakeTieringFactory.createLakeCommitter( + new TieringCommitterInitContext( + tablePath, currentTableInfo, lakeTieringConfig, flussConf))) { + List writeResults = + nonEmptyResults.stream() + .map(TableBucketWriteResult::writeResult) + .collect(Collectors.toList()); + + Map logEndOffsets = new HashMap<>(); + Map logMaxTieredTimestamps = new HashMap<>(); + for (TableBucketWriteResult writeResult : nonEmptyResults) { + TableBucket tableBucket = writeResult.tableBucket(); + logEndOffsets.put(tableBucket, writeResult.logEndOffset()); + logMaxTieredTimestamps.put(tableBucket, writeResult.maxTimestamp()); + } + + // to committable + Committable committable = lakeCommitter.toCommittable(writeResults); + // before commit to lake, check fluss not missing any lake snapshot committed by fluss + LakeSnapshot flussCurrentLakeSnapshot = getLatestLakeSnapshot(admin, tablePath); + checkFlussNotMissingLakeSnapshot( + tablePath, + tableId, + flussTableLakeSnapshotCommitter, + lakeCommitter, + committable, + flussCurrentLakeSnapshot == null + ? null + : flussCurrentLakeSnapshot.getSnapshotId()); + + // get the lake bucket offsets file storing the log end offsets + String lakeBucketTieredOffsetsFile = + flussTableLakeSnapshotCommitter.prepareLakeSnapshot( + tableId, tablePath, logEndOffsets); + + // record the lake snapshot bucket offsets file to snapshot property + Map snapshotProperties = + Collections.singletonMap( + FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, lakeBucketTieredOffsetsFile); + LakeCommitResult lakeCommitResult = + lakeCommitter.commit(committable, snapshotProperties); + // commit to fluss + flussTableLakeSnapshotCommitter.commit( + tableId, + tablePath, + lakeCommitResult, + lakeBucketTieredOffsetsFile, + logEndOffsets, + logMaxTieredTimestamps); + return new TieringCommitResult(committable, lakeCommitResult.getTieringStats()); + } + } + + @Nullable + private LakeSnapshot getLatestLakeSnapshot(Admin admin, TablePath tablePath) throws Exception { + LakeSnapshot flussCurrentLakeSnapshot; + try { + flussCurrentLakeSnapshot = admin.getLatestLakeSnapshot(tablePath).get(); + } catch (Exception e) { + Throwable throwable = e.getCause(); + if (throwable instanceof LakeTableSnapshotNotExistException) { + // do-nothing + flussCurrentLakeSnapshot = null; + } else { + throw e; + } + } + return flussCurrentLakeSnapshot; + } + + private void checkFlussNotMissingLakeSnapshot( + TablePath tablePath, + long tableId, + FlussTableLakeSnapshotCommitter flussTableLakeSnapshotCommitter, + LakeCommitter lakeCommitter, + Committable committable, + Long flussCurrentLakeSnapshot) + throws Exception { + // get Fluss missing lake snapshot in Lake + CommittedLakeSnapshot missingCommittedSnapshot = + lakeCommitter.getMissingLakeSnapshot(flussCurrentLakeSnapshot); + + // fluss's known snapshot is less than lake snapshot committed by fluss + // fail this commit since the data is read from the log end-offset of a invalid fluss + // known lake snapshot, which means the data already has been committed to lake, + // not to commit to lake to avoid data duplicated + if (missingCommittedSnapshot != null) { + String lakeSnapshotOffsetPath = + missingCommittedSnapshot + .getSnapshotProperties() + .get(FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY); + + // should only will happen in v0.7 which won't put offsets info + // to properties + if (lakeSnapshotOffsetPath == null) { + throw new IllegalStateException( + String.format( + "Can't find %s field from snapshot property.", + FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY)); + } + + // the fluss-offsets will be a json string if it's tiered by v0.8, + // since this code path should be rare, we do not consider backward compatibility + // and throw IllegalStateException directly + String trimmedPath = lakeSnapshotOffsetPath.trim(); + if (trimmedPath.contains("{")) { + throw new IllegalStateException( + String.format( + "The %s field in snapshot property is a JSON string (tiered by v0.8), " + + "which is not supported to restore. Snapshot ID: %d, Table: {tablePath=%s, tableId=%d}.", + FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, + missingCommittedSnapshot.getLakeSnapshotId(), + tablePath, + tableId)); + } + + // commit this missing snapshot to fluss + flussTableLakeSnapshotCommitter.commit( + tableId, + missingCommittedSnapshot.getLakeSnapshotId(), + lakeSnapshotOffsetPath, + // don't care readable snapshot and offsets, + null, + // use empty log offsets, log max timestamp, since we can't know that + // in last tiering, it doesn't matter for they are just used to + // report metrics + Collections.emptyMap(), + Collections.emptyMap(), + LakeCommitResult.KEEP_ALL_PREVIOUS); + // abort this committable to delete the written files + lakeCommitter.abort(committable); + throw new IllegalStateException( + String.format( + "The current Fluss's lake snapshot %d is less than" + + " lake actual snapshot %d committed by Fluss for table: {tablePath=%s, tableId=%d}," + + " missing snapshot: %s.", + flussCurrentLakeSnapshot, + missingCommittedSnapshot.getLakeSnapshotId(), + tablePath, + tableId, + missingCommittedSnapshot)); + } + } +} diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/CommittableMessageTypeInfo.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/CommittableMessageTypeInfo.java index d541721d19..0fc76e9b39 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/CommittableMessageTypeInfo.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/CommittableMessageTypeInfo.java @@ -17,8 +17,8 @@ package org.apache.fluss.flink.tiering.committer; +import org.apache.fluss.client.tiering.TableBucketWriteResult; import org.apache.fluss.flink.adapter.TypeInformationAdapter; -import org.apache.fluss.flink.tiering.source.TableBucketWriteResult; import org.apache.fluss.lake.serializer.SimpleVersionedSerializer; import org.apache.flink.api.common.typeinfo.TypeInformation; diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperator.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperator.java index 570602b37a..6624e4b555 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperator.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperator.java @@ -20,24 +20,18 @@ import org.apache.fluss.client.Connection; import org.apache.fluss.client.ConnectionFactory; import org.apache.fluss.client.admin.Admin; -import org.apache.fluss.client.metadata.LakeSnapshot; import org.apache.fluss.client.tiering.FlussTableLakeSnapshotCommitter; -import org.apache.fluss.client.tiering.TieringCommitterInitContext; +import org.apache.fluss.client.tiering.TableBucketWriteResult; +import org.apache.fluss.client.tiering.TieringCommitResult; +import org.apache.fluss.client.tiering.TieringCommitter; import org.apache.fluss.config.Configuration; -import org.apache.fluss.exception.LakeTableSnapshotNotExistException; import org.apache.fluss.flink.tiering.event.FailedTieringEvent; import org.apache.fluss.flink.tiering.event.FinishedTieringEvent; -import org.apache.fluss.flink.tiering.source.TableBucketWriteResult; import org.apache.fluss.flink.tiering.source.TieringSource; -import org.apache.fluss.lake.committer.CommittedLakeSnapshot; -import org.apache.fluss.lake.committer.LakeCommitResult; import org.apache.fluss.lake.committer.LakeCommitter; -import org.apache.fluss.lake.committer.TieringStats; import org.apache.fluss.lake.writer.LakeTieringFactory; import org.apache.fluss.lake.writer.LakeWriter; import org.apache.fluss.metadata.TableBucket; -import org.apache.fluss.metadata.TableInfo; -import org.apache.fluss.metadata.TablePath; import org.apache.fluss.utils.ExceptionUtils; import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; @@ -50,15 +44,12 @@ import javax.annotation.Nullable; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.stream.Collectors; -import static org.apache.fluss.lake.committer.LakeCommitter.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; import static org.apache.fluss.utils.Preconditions.checkState; /** @@ -87,6 +78,7 @@ public class TieringCommitOperator private final Configuration lakeTieringConfig; private final LakeTieringFactory lakeTieringFactory; private final FlussTableLakeSnapshotCommitter flussTableLakeSnapshotCommitter; + private final TieringCommitter tieringCommitter; private Connection connection; private Admin admin; @@ -97,22 +89,6 @@ public class TieringCommitOperator private final Map>> collectedTableBucketWriteResults; - /** - * The result of one table's commit round, holding the lake committable (nullable for empty - * commits where no data was written) and the associated tiering statistics. - */ - private final class CommitResult { - /** The lake committable, or {@code null} if nothing was written in this round. */ - @Nullable final Committable committable; - /** Per-table tiering statistics collected during this round. */ - @Nullable final TieringStats stats; - - CommitResult(@Nullable Committable committable, @Nullable TieringStats stats) { - this.committable = committable; - this.stats = stats; - } - } - public TieringCommitOperator( StreamOperatorParameters> parameters, Configuration flussConf, @@ -120,6 +96,7 @@ public TieringCommitOperator( LakeTieringFactory lakeTieringFactory) { this.lakeTieringFactory = lakeTieringFactory; this.flussTableLakeSnapshotCommitter = new FlussTableLakeSnapshotCommitter(flussConf); + this.tieringCommitter = new TieringCommitter<>(); this.collectedTableBucketWriteResults = new HashMap<>(); this.flussConfig = flussConf; this.lakeTieringConfig = lakeTieringConfig; @@ -154,10 +131,15 @@ public void processElement(StreamRecord> str if (committableWriteResults != null) { try { - CommitResult commitResult = - commitWriteResults( + TieringCommitResult commitResult = + tieringCommitter.commitWriteResults( + admin, tableId, tableBucketWriteResult.tablePath(), + flussConfig, + lakeTieringConfig, + lakeTieringFactory, + flussTableLakeSnapshotCommitter, committableWriteResults); // only emit downstream when actual data was written if (commitResult.committable != null) { @@ -183,189 +165,6 @@ public void processElement(StreamRecord> str } } - /** - * Commits the collected write results for one table to the lake and Fluss. - * - *

Always returns a non-null {@link CommitResult}. When all buckets produced no data (empty - * commit), {@link CommitResult#committable} is {@code null} and stats are {@link - * TieringStats#UNKNOWN}. - */ - private CommitResult commitWriteResults( - long tableId, - TablePath tablePath, - List> committableWriteResults) - throws Exception { - // filter down to buckets that actually produced data - List> nonEmptyResults = - committableWriteResults.stream() - .filter(r -> r.writeResult() != null) - .collect(Collectors.toList()); - - // all buckets were empty — nothing to commit to the lake - if (nonEmptyResults.isEmpty()) { - LOG.info( - "Commit tiering write results is empty for table {}, table path {}", - tableId, - tablePath); - return new CommitResult(null, null); - } - - // Check if the table was dropped and recreated during tiering. - // If the current table id differs from the committable's table id, fail this commit - // to avoid dirty commit to a newly created table. - TableInfo currentTableInfo = admin.getTableInfo(tablePath).get(); - if (currentTableInfo.getTableId() != tableId) { - throw new IllegalStateException( - String.format( - "The current table id %s for table path %s is different from the table id %s in the committable. " - + "This usually happens when a table was dropped and recreated during tiering. " - + "Aborting commit to prevent dirty commit.", - currentTableInfo.getTableId(), tablePath, tableId)); - } - - try (LakeCommitter lakeCommitter = - lakeTieringFactory.createLakeCommitter( - new TieringCommitterInitContext( - tablePath, currentTableInfo, lakeTieringConfig, flussConfig))) { - List writeResults = - nonEmptyResults.stream() - .map(TableBucketWriteResult::writeResult) - .collect(Collectors.toList()); - - Map logEndOffsets = new HashMap<>(); - Map logMaxTieredTimestamps = new HashMap<>(); - for (TableBucketWriteResult writeResult : nonEmptyResults) { - TableBucket tableBucket = writeResult.tableBucket(); - logEndOffsets.put(tableBucket, writeResult.logEndOffset()); - logMaxTieredTimestamps.put(tableBucket, writeResult.maxTimestamp()); - } - - // to committable - Committable committable = lakeCommitter.toCommittable(writeResults); - // before commit to lake, check fluss not missing any lake snapshot committed by fluss - LakeSnapshot flussCurrentLakeSnapshot = getLatestLakeSnapshot(tablePath); - checkFlussNotMissingLakeSnapshot( - tablePath, - tableId, - lakeCommitter, - committable, - flussCurrentLakeSnapshot == null - ? null - : flussCurrentLakeSnapshot.getSnapshotId()); - - // get the lake bucket offsets file storing the log end offsets - String lakeBucketTieredOffsetsFile = - flussTableLakeSnapshotCommitter.prepareLakeSnapshot( - tableId, tablePath, logEndOffsets); - - // record the lake snapshot bucket offsets file to snapshot property - Map snapshotProperties = - Collections.singletonMap( - FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, lakeBucketTieredOffsetsFile); - LakeCommitResult lakeCommitResult = - lakeCommitter.commit(committable, snapshotProperties); - // commit to fluss - flussTableLakeSnapshotCommitter.commit( - tableId, - tablePath, - lakeCommitResult, - lakeBucketTieredOffsetsFile, - logEndOffsets, - logMaxTieredTimestamps); - return new CommitResult(committable, lakeCommitResult.getTieringStats()); - } - } - - @Nullable - private LakeSnapshot getLatestLakeSnapshot(TablePath tablePath) throws Exception { - LakeSnapshot flussCurrentLakeSnapshot; - try { - flussCurrentLakeSnapshot = admin.getLatestLakeSnapshot(tablePath).get(); - } catch (Exception e) { - Throwable throwable = e.getCause(); - if (throwable instanceof LakeTableSnapshotNotExistException) { - // do-nothing - flussCurrentLakeSnapshot = null; - } else { - throw e; - } - } - return flussCurrentLakeSnapshot; - } - - private void checkFlussNotMissingLakeSnapshot( - TablePath tablePath, - long tableId, - LakeCommitter lakeCommitter, - Committable committable, - Long flussCurrentLakeSnapshot) - throws Exception { - // get Fluss missing lake snapshot in Lake - CommittedLakeSnapshot missingCommittedSnapshot = - lakeCommitter.getMissingLakeSnapshot(flussCurrentLakeSnapshot); - - // fluss's known snapshot is less than lake snapshot committed by fluss - // fail this commit since the data is read from the log end-offset of a invalid fluss - // known lake snapshot, which means the data already has been committed to lake, - // not to commit to lake to avoid data duplicated - if (missingCommittedSnapshot != null) { - String lakeSnapshotOffsetPath = - missingCommittedSnapshot - .getSnapshotProperties() - .get(FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY); - - // should only will happen in v0.7 which won't put offsets info - // to properties - if (lakeSnapshotOffsetPath == null) { - throw new IllegalStateException( - String.format( - "Can't find %s field from snapshot property.", - FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY)); - } - - // the fluss-offsets will be a json string if it's tiered by v0.8, - // since this code path should be rare, we do not consider backward compatibility - // and throw IllegalStateException directly - String trimmedPath = lakeSnapshotOffsetPath.trim(); - if (trimmedPath.contains("{")) { - throw new IllegalStateException( - String.format( - "The %s field in snapshot property is a JSON string (tiered by v0.8), " - + "which is not supported to restore. Snapshot ID: %d, Table: {tablePath=%s, tableId=%d}.", - FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, - missingCommittedSnapshot.getLakeSnapshotId(), - tablePath, - tableId)); - } - - // commit this missing snapshot to fluss - flussTableLakeSnapshotCommitter.commit( - tableId, - missingCommittedSnapshot.getLakeSnapshotId(), - lakeSnapshotOffsetPath, - // don't care readable snapshot and offsets, - null, - // use empty log offsets, log max timestamp, since we can't know that - // in last tiering, it doesn't matter for they are just used to - // report metrics - Collections.emptyMap(), - Collections.emptyMap(), - LakeCommitResult.KEEP_ALL_PREVIOUS); - // abort this committable to delete the written files - lakeCommitter.abort(committable); - throw new IllegalStateException( - String.format( - "The current Fluss's lake snapshot %d is less than" - + " lake actual snapshot %d committed by Fluss for table: {tablePath=%s, tableId=%d}," - + " missing snapshot: %s.", - flussCurrentLakeSnapshot, - missingCommittedSnapshot.getLakeSnapshotId(), - tablePath, - tableId, - missingCommittedSnapshot)); - } - } - private void registerTableBucketWriteResult( long tableId, TableBucketWriteResult tableBucketWriteResult) { collectedTableBucketWriteResults diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperatorFactory.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperatorFactory.java index efced7aeab..4c573ee845 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperatorFactory.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperatorFactory.java @@ -17,8 +17,8 @@ package org.apache.fluss.flink.tiering.committer; +import org.apache.fluss.client.tiering.TableBucketWriteResult; import org.apache.fluss.config.Configuration; -import org.apache.fluss.flink.tiering.source.TableBucketWriteResult; import org.apache.fluss.lake.writer.LakeTieringFactory; import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TableBucketWriteResultEmitter.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TableBucketWriteResultEmitter.java index 2b12337f84..27e1325536 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TableBucketWriteResultEmitter.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TableBucketWriteResultEmitter.java @@ -23,16 +23,20 @@ import org.apache.flink.api.connector.source.SourceOutput; import org.apache.flink.connector.base.source.reader.RecordEmitter; -/** The emitter to emit {@link TableBucketWriteResult} to downstream {@link LakeCommitter}. */ +/** + * The emitter to emit {@link org.apache.fluss.client.tiering.TableBucketWriteResult} to downstream + * {@link LakeCommitter}. + */ public class TableBucketWriteResultEmitter implements RecordEmitter< - TableBucketWriteResult, - TableBucketWriteResult, + org.apache.fluss.client.tiering.TableBucketWriteResult, + org.apache.fluss.client.tiering.TableBucketWriteResult, TieringSplitState> { @Override public void emitRecord( - TableBucketWriteResult writeResult, - SourceOutput> sourceOutput, + org.apache.fluss.client.tiering.TableBucketWriteResult writeResult, + SourceOutput> + sourceOutput, TieringSplitState splitState) { sourceOutput.collect(writeResult); } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TableBucketWriteResultSerializer.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TableBucketWriteResultSerializer.java index 3651760955..16ba43df08 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TableBucketWriteResultSerializer.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TableBucketWriteResultSerializer.java @@ -26,9 +26,10 @@ import java.io.IOException; -/** The serializer for {@link TableBucketWriteResult}. */ +/** The serializer for {@link org.apache.fluss.client.tiering.TableBucketWriteResult}. */ public class TableBucketWriteResultSerializer - implements SimpleVersionedSerializer> { + implements SimpleVersionedSerializer< + org.apache.fluss.client.tiering.TableBucketWriteResult> { private static final ThreadLocal SERIALIZER_CACHE = ThreadLocal.withInitial(() -> new DataOutputSerializer(64)); @@ -50,7 +51,9 @@ public int getVersion() { } @Override - public byte[] serialize(TableBucketWriteResult tableBucketWriteResult) + public byte[] serialize( + org.apache.fluss.client.tiering.TableBucketWriteResult + tableBucketWriteResult) throws IOException { final DataOutputSerializer out = SERIALIZER_CACHE.get(); // serialize table path @@ -97,8 +100,8 @@ public byte[] serialize(TableBucketWriteResult tableBucketWriteResu } @Override - public TableBucketWriteResult deserialize(int version, byte[] serialized) - throws IOException { + public org.apache.fluss.client.tiering.TableBucketWriteResult deserialize( + int version, byte[] serialized) throws IOException { if (version != CURRENT_VERSION) { throw new IOException("Unknown version " + version); } @@ -136,7 +139,7 @@ public TableBucketWriteResult deserialize(int version, byte[] seria long maxTimestamp = in.readLong(); // deserialize number of write results int numberOfWriteResults = in.readInt(); - return new TableBucketWriteResult<>( + return new org.apache.fluss.client.tiering.TableBucketWriteResult<>( tablePath, tableBucket, partitionName, diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TableBucketWriteResultTypeInfo.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TableBucketWriteResultTypeInfo.java index 424673c26c..fea8a80412 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TableBucketWriteResultTypeInfo.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TableBucketWriteResultTypeInfo.java @@ -25,9 +25,12 @@ import org.apache.flink.core.io.SimpleVersionedSerializerTypeSerializerProxy; import org.apache.flink.util.function.SerializableSupplier; -/** A {@link TypeInformation} for {@link TableBucketWriteResult} . */ +/** + * A {@link TypeInformation} for {@link org.apache.fluss.client.tiering.TableBucketWriteResult} . + */ public class TableBucketWriteResultTypeInfo - extends TypeInformationAdapter> { + extends TypeInformationAdapter< + org.apache.fluss.client.tiering.TableBucketWriteResult> { private final SerializableSupplier> writeResultSerializerFactory; @@ -38,9 +41,10 @@ private TableBucketWriteResultTypeInfo( this.writeResultSerializerFactory = writeResultSerializerFactory; } - public static TypeInformation> of( - SerializableSupplier> - writeResultSerializerFactory) { + public static + TypeInformation> of( + SerializableSupplier> + writeResultSerializerFactory) { return new TableBucketWriteResultTypeInfo<>(writeResultSerializerFactory); } @@ -66,8 +70,9 @@ public int getTotalFields() { @SuppressWarnings({"unchecked", "rawtypes"}) @Override - public Class> getTypeClass() { - return (Class) TableBucketWriteResult.class; + public Class> + getTypeClass() { + return (Class) org.apache.fluss.client.tiering.TableBucketWriteResult.class; } @Override @@ -76,23 +81,23 @@ public boolean isKeyType() { } @Override - protected TypeSerializer> createSerializer( - TypeSerializerCreator typeSerializerCreator) { + protected TypeSerializer> + createSerializer(TypeSerializerCreator typeSerializerCreator) { // no copy, so that data from lake writer is directly going into lake committer while // chaining return new SimpleVersionedSerializerTypeSerializerProxy< - TableBucketWriteResult>( + org.apache.fluss.client.tiering.TableBucketWriteResult>( () -> new TableBucketWriteResultSerializer<>(writeResultSerializerFactory.get())) { @Override - public TableBucketWriteResult copy( - TableBucketWriteResult from) { + public org.apache.fluss.client.tiering.TableBucketWriteResult copy( + org.apache.fluss.client.tiering.TableBucketWriteResult from) { return from; } @Override - public TableBucketWriteResult copy( - TableBucketWriteResult from, - TableBucketWriteResult reuse) { + public org.apache.fluss.client.tiering.TableBucketWriteResult copy( + org.apache.fluss.client.tiering.TableBucketWriteResult from, + org.apache.fluss.client.tiering.TableBucketWriteResult reuse) { return from; } }; diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSource.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSource.java index 649b758704..fc009eac03 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSource.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSource.java @@ -55,7 +55,9 @@ */ public class TieringSource implements Source< - TableBucketWriteResult, TieringSplit, TieringSourceEnumeratorState> { + org.apache.fluss.client.tiering.TableBucketWriteResult, + TieringSplit, + TieringSourceEnumeratorState> { public static final String TIERING_SOURCE_TRANSFORMATION_UID = "$$fluss_tiering_source_operator$$"; @@ -108,9 +110,14 @@ public SimpleVersionedSerializer getSplitSerializer() { } @Override - public SourceReader, TieringSplit> createReader( - SourceReaderContext sourceReaderContext) { - FutureCompletingBlockingQueue>> + public SourceReader< + org.apache.fluss.client.tiering.TableBucketWriteResult, + TieringSplit> + createReader(SourceReaderContext sourceReaderContext) { + FutureCompletingBlockingQueue< + RecordsWithSplitIds< + org.apache.fluss.client.tiering.TableBucketWriteResult< + WriteResult>>> elementsQueue = new FutureCompletingBlockingQueue<>(); flussConf.set( CLIENT_SCANNER_IO_TMP_DIR, diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSourceFetcherManager.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSourceFetcherManager.java index ac72aad664..3df9420794 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSourceFetcherManager.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSourceFetcherManager.java @@ -40,14 +40,21 @@ */ public class TieringSourceFetcherManager extends SingleThreadFetcherManagerAdapter< - TableBucketWriteResult, TieringSplit> { + org.apache.fluss.client.tiering.TableBucketWriteResult, TieringSplit> { private static final Logger LOG = LoggerFactory.getLogger(TieringSourceFetcherManager.class); public TieringSourceFetcherManager( - FutureCompletingBlockingQueue>> + FutureCompletingBlockingQueue< + RecordsWithSplitIds< + org.apache.fluss.client.tiering.TableBucketWriteResult< + WriteResult>>> elementsQueue, - Supplier, TieringSplit>> + Supplier< + SplitReader< + org.apache.fluss.client.tiering.TableBucketWriteResult< + WriteResult>, + TieringSplit>> splitReaderSupplier, Configuration configuration, Consumer> splitFinishedHook) { @@ -64,8 +71,10 @@ public void markTableReachTieringMaxDuration(long tableId) { enqueueMarkTableReachTieringMaxDurationTask( splitFetcher, tableId)); } else { - SplitFetcher, TieringSplit> splitFetcher = - createSplitFetcher(); + SplitFetcher< + org.apache.fluss.client.tiering.TableBucketWriteResult, + TieringSplit> + splitFetcher = createSplitFetcher(); LOG.info( "fetchers is empty, enqueue marking tiering max duration for table {}", tableId); @@ -75,7 +84,10 @@ public void markTableReachTieringMaxDuration(long tableId) { } private void enqueueMarkTableReachTieringMaxDurationTask( - SplitFetcher, TieringSplit> splitFetcher, + SplitFetcher< + org.apache.fluss.client.tiering.TableBucketWriteResult, + TieringSplit> + splitFetcher, long reachTieringDeadlineTable) { splitFetcher.enqueueTask( new SplitFetcherTask() { diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSourceReader.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSourceReader.java index 6f0fc43b95..b217ae334c 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSourceReader.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSourceReader.java @@ -46,8 +46,8 @@ @Internal public final class TieringSourceReader extends SingleThreadMultiplexSourceReaderBaseAdapter< - TableBucketWriteResult, - TableBucketWriteResult, + org.apache.fluss.client.tiering.TableBucketWriteResult, + org.apache.fluss.client.tiering.TableBucketWriteResult, TieringSplit, TieringSplitState> { @@ -56,7 +56,10 @@ public final class TieringSourceReader private final Connection connection; public TieringSourceReader( - FutureCompletingBlockingQueue>> + FutureCompletingBlockingQueue< + RecordsWithSplitIds< + org.apache.fluss.client.tiering.TableBucketWriteResult< + WriteResult>>> elementsQueue, SourceReaderContext context, Connection connection, @@ -66,7 +69,10 @@ public TieringSourceReader( @VisibleForTesting TieringSourceReader( - FutureCompletingBlockingQueue>> + FutureCompletingBlockingQueue< + RecordsWithSplitIds< + org.apache.fluss.client.tiering.TableBucketWriteResult< + WriteResult>>> elementsQueue, SourceReaderContext context, Connection connection, @@ -83,7 +89,10 @@ public TieringSourceReader( } private static TieringSourceFetcherManager createFetcherManager( - FutureCompletingBlockingQueue>> + FutureCompletingBlockingQueue< + RecordsWithSplitIds< + org.apache.fluss.client.tiering.TableBucketWriteResult< + WriteResult>>> elementsQueue, SourceReaderContext context, Connection connection, diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java index b708c54dc2..ece891d0ac 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/tiering/source/TieringSplitReader.java @@ -64,7 +64,8 @@ /** The {@link SplitReader} implementation which will read Fluss and write to lake. */ public class TieringSplitReader - implements SplitReader, TieringSplit> { + implements SplitReader< + org.apache.fluss.client.tiering.TableBucketWriteResult, TieringSplit> { private static final Logger LOG = LoggerFactory.getLogger(TieringSplitReader.class); @@ -140,7 +141,8 @@ protected TieringSplitReader( } @Override - public RecordsWithSplitIds> fetch() throws IOException { + public RecordsWithSplitIds> + fetch() throws IOException { // check empty splits if (!currentEmptySplits.isEmpty()) { LOG.info("Empty split(s) {} finished.", currentEmptySplits); @@ -295,9 +297,10 @@ private void mayCreateLogScanner() { } } - private RecordsWithSplitIds> + private RecordsWithSplitIds> forceCompleteTieringLogRecords() throws IOException { - Map> writeResults = new HashMap<>(); + Map> + writeResults = new HashMap<>(); Map finishedSplitIds = new HashMap<>(); // force finish all splits @@ -320,9 +323,10 @@ private void mayCreateLogScanner() { logOffsetAndTimestamp == null ? UNKNOWN_BUCKET_TIMESTAMP : logOffsetAndTimestamp.timestamp; - TableBucketWriteResult bucketWriteResult = - completeLakeWriter( - bucket, split.getPartitionName(), logEndOffset, timestamp); + org.apache.fluss.client.tiering.TableBucketWriteResult + bucketWriteResult = + completeLakeWriter( + bucket, split.getPartitionName(), logEndOffset, timestamp); if (logEndOffset == UNKNOWN_BUCKET_OFFSET) { // when the log end offset is unknown, the write result must be @@ -351,9 +355,10 @@ private void mayCreateLogScanner() { return new TableBucketWriteResultWithSplitIds(writeResults, finishedSplitIds); } - private RecordsWithSplitIds> forLogRecords( - ScanRecords scanRecords) throws IOException { - Map> writeResults = new HashMap<>(); + private RecordsWithSplitIds> + forLogRecords(ScanRecords scanRecords) throws IOException { + Map> + writeResults = new HashMap<>(); Map finishedSplitIds = new HashMap<>(); for (TableBucket bucket : scanRecords.buckets()) { @@ -438,7 +443,7 @@ private LakeWriter getOrCreateLakeWriter( return lakeWriter; } - private TableBucketWriteResult completeLakeWriter( + private org.apache.fluss.client.tiering.TableBucketWriteResult completeLakeWriter( TableBucket bucket, @Nullable String partitionName, long logEndOffset, @@ -461,7 +466,8 @@ private TableBucketWriteResult completeLakeWriter( } private TableBucketWriteResultWithSplitIds forEmptySplits(Set emptySplits) { - Map> writeResults = new HashMap<>(); + Map> + writeResults = new HashMap<>(); Map finishedSplitIds = new HashMap<>(); for (TieringSplit tieringSplit : emptySplits) { TableBucket tableBucket = tieringSplit.getTableBucket(); @@ -491,7 +497,7 @@ private TableBucketWriteResultWithSplitIds finishCurrentSnapshotSplit() throws I TableBucket tableBucket = currentSnapshotSplit.getTableBucket(); long logEndOffset = currentSnapshotSplit.getLogOffsetOfSnapshot(); String splitId = currentTableSplitsByBucket.remove(tableBucket).splitId(); - TableBucketWriteResult writeResult = + org.apache.fluss.client.tiering.TableBucketWriteResult writeResult = completeLakeWriter( tableBucket, currentSnapshotSplit.getPartitionName(), @@ -636,15 +642,16 @@ private void subscribeLog(TieringLogSplit logSplit) { stoppingOffset); } - private TableBucketWriteResult toTableBucketWriteResult( - TablePath tablePath, - TableBucket tableBucket, - @Nullable String partitionName, - @Nullable WriteResult writeResult, - long endLogOffset, - long maxTimestamp, - int numberOfSplits) { - return new TableBucketWriteResult<>( + private org.apache.fluss.client.tiering.TableBucketWriteResult + toTableBucketWriteResult( + TablePath tablePath, + TableBucket tableBucket, + @Nullable String partitionName, + @Nullable WriteResult writeResult, + long endLogOffset, + long maxTimestamp, + int numberOfSplits) { + return new org.apache.fluss.client.tiering.TableBucketWriteResult<>( tablePath, tableBucket, partitionName, @@ -655,21 +662,30 @@ private TableBucketWriteResult toTableBucketWriteResult( } private class TableBucketWriteResultWithSplitIds - implements RecordsWithSplitIds> { + implements RecordsWithSplitIds< + org.apache.fluss.client.tiering.TableBucketWriteResult> { private final Iterator bucketIterator; - private final Map> bucketWriteResults; + private final Map< + TableBucket, + org.apache.fluss.client.tiering.TableBucketWriteResult> + bucketWriteResults; private final Map bucketSplits; - @Nullable private TableBucketWriteResult writeResultForCurrentSplit; + @Nullable + private org.apache.fluss.client.tiering.TableBucketWriteResult + writeResultForCurrentSplit; public TableBucketWriteResultWithSplitIds() { this(Collections.emptyMap(), Collections.emptyMap()); } public TableBucketWriteResultWithSplitIds( - Map> bucketWriteResults, + Map< + TableBucket, + org.apache.fluss.client.tiering.TableBucketWriteResult> + bucketWriteResults, Map bucketSplits) { this.bucketIterator = bucketWriteResults.keySet().iterator(); this.bucketWriteResults = bucketWriteResults; @@ -691,9 +707,11 @@ public String nextSplit() { @Nullable @Override - public TableBucketWriteResult nextRecordFromSplit() { + public org.apache.fluss.client.tiering.TableBucketWriteResult + nextRecordFromSplit() { if (writeResultForCurrentSplit != null) { - TableBucketWriteResult bucketWriteResult = writeResultForCurrentSplit; + org.apache.fluss.client.tiering.TableBucketWriteResult + bucketWriteResult = writeResultForCurrentSplit; writeResultForCurrentSplit = null; return bucketWriteResult; } else { diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperatorTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperatorTest.java index bdac25ef46..47a52cf061 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperatorTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/committer/TieringCommitOperatorTest.java @@ -19,13 +19,13 @@ import org.apache.fluss.client.metadata.LakeSnapshot; import org.apache.fluss.client.tiering.FlussTableLakeSnapshotCommitter; +import org.apache.fluss.client.tiering.TableBucketWriteResult; import org.apache.fluss.exception.LakeTableSnapshotNotExistException; import org.apache.fluss.flink.adapter.StreamOperatorParametersAdapter; import org.apache.fluss.flink.tiering.TestingLakeTieringFactory; import org.apache.fluss.flink.tiering.TestingWriteResult; import org.apache.fluss.flink.tiering.event.FailedTieringEvent; import org.apache.fluss.flink.tiering.event.FinishedTieringEvent; -import org.apache.fluss.flink.tiering.source.TableBucketWriteResult; import org.apache.fluss.flink.utils.FlinkTestBase; import org.apache.fluss.lake.committer.CommittedLakeSnapshot; import org.apache.fluss.metadata.TableBucket; diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/TableBucketWriteResultSerializerTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/TableBucketWriteResultSerializerTest.java index dbb40eae17..db43ca12b1 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/TableBucketWriteResultSerializerTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/TableBucketWriteResultSerializerTest.java @@ -42,13 +42,20 @@ void testSerializeAndDeserialize(boolean isPartitioned) throws Exception { TableBucket tableBucket = isPartitioned ? new TableBucket(1, 1000L, 2) : new TableBucket(1, 2); String partitionName = isPartitioned ? "partition1" : null; - TableBucketWriteResult tableBucketWriteResult = - new TableBucketWriteResult<>( - tablePath, tableBucket, partitionName, testingWriteResult, 10, 30L, 20); + org.apache.fluss.client.tiering.TableBucketWriteResult + tableBucketWriteResult = + new org.apache.fluss.client.tiering.TableBucketWriteResult<>( + tablePath, + tableBucket, + partitionName, + testingWriteResult, + 10, + 30L, + 20); // test serialize and deserialize byte[] serialized = tableBucketWriteResultSerializer.serialize(tableBucketWriteResult); - TableBucketWriteResult deserialized = + org.apache.fluss.client.tiering.TableBucketWriteResult deserialized = tableBucketWriteResultSerializer.deserialize( tableBucketWriteResultSerializer.getVersion(), serialized); @@ -63,7 +70,7 @@ void testSerializeAndDeserialize(boolean isPartitioned) throws Exception { // verify when writeResult is null tableBucketWriteResult = - new TableBucketWriteResult<>( + new org.apache.fluss.client.tiering.TableBucketWriteResult<>( tablePath, tableBucket, partitionName, null, 20, 30L, 30); serialized = tableBucketWriteResultSerializer.serialize(tableBucketWriteResult); deserialized = diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/TieringSourceReaderTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/TieringSourceReaderTest.java index 9e9de2c792..140dd8d5b2 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/TieringSourceReaderTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/TieringSourceReaderTest.java @@ -60,7 +60,9 @@ void testHandleTieringReachMaxDurationEvent() throws Exception { ConfigOptions.NoKeyAssigner.ROUND_ROBIN); try (Connection connection = ConnectionFactory.createConnection(conf)) { FutureCompletingBlockingQueue< - RecordsWithSplitIds>> + RecordsWithSplitIds< + org.apache.fluss.client.tiering.TableBucketWriteResult< + TestingWriteResult>>> elementsQueue = new FutureCompletingBlockingQueue<>(16); TestingReaderContext readerContext = new TestingReaderContext(); try (TieringSourceReader reader = @@ -87,13 +89,16 @@ void testHandleTieringReachMaxDurationEvent() throws Exception { retry( Duration.ofMinutes(1), () -> { - TestingReaderOutput> output = - new TestingReaderOutput<>(); + TestingReaderOutput< + org.apache.fluss.client.tiering.TableBucketWriteResult< + TestingWriteResult>> + output = new TestingReaderOutput<>(); // should force to finish, the write result is null reader.pollNext(output); assertThat(output.getEmittedRecords()).hasSize(1); - TableBucketWriteResult result = - output.getEmittedRecords().get(0); + org.apache.fluss.client.tiering.TableBucketWriteResult< + TestingWriteResult> + result = output.getEmittedRecords().get(0); assertThat(result.writeResult()).isNull(); }); @@ -117,7 +122,9 @@ void testHandleTieringReachMaxDurationEvent() throws Exception { // wait to run one round of tiering to do some tiering FutureCompletingBlockingQueue< - RecordsWithSplitIds>> + RecordsWithSplitIds< + org.apache.fluss.client.tiering.TableBucketWriteResult< + TestingWriteResult>>> blockingQueue = getElementsQueue(reader); // wait blockingQueue is not empty to make sure we have one fetch // in tiering source reader @@ -135,14 +142,17 @@ void testHandleTieringReachMaxDurationEvent() throws Exception { retry( Duration.ofMinutes(1), () -> { - TestingReaderOutput> + TestingReaderOutput< + org.apache.fluss.client.tiering.TableBucketWriteResult< + TestingWriteResult>> output1 = new TestingReaderOutput<>(); // should force to finish, the write result isn't null reader.pollNext(output1); assertThat(output1.getEmittedRecords()).hasSize(1); - TableBucketWriteResult result = - output1.getEmittedRecords().get(0); + org.apache.fluss.client.tiering.TableBucketWriteResult< + TestingWriteResult> + result = output1.getEmittedRecords().get(0); TestingWriteResult testingWriteResult = result.writeResult(); assertThat(testingWriteResult).isNotNull(); assertThat(result.logEndOffset()).isEqualTo(1); @@ -162,13 +172,16 @@ void testHandleTieringReachMaxDurationEvent() throws Exception { retry( Duration.ofMinutes(1), () -> { - TestingReaderOutput> + TestingReaderOutput< + org.apache.fluss.client.tiering.TableBucketWriteResult< + TestingWriteResult>> output1 = new TestingReaderOutput<>(); // should force to finish, and the result is null reader.pollNext(output1); assertThat(output1.getEmittedRecords()).hasSize(1); - TableBucketWriteResult result = - output1.getEmittedRecords().get(0); + org.apache.fluss.client.tiering.TableBucketWriteResult< + TestingWriteResult> + result = output1.getEmittedRecords().get(0); assertThat(result.writeResult()).isNull(); }); } @@ -183,7 +196,9 @@ void testHandleTieringReachMaxDurationEvent() throws Exception { */ @SuppressWarnings("unchecked") private FutureCompletingBlockingQueue< - RecordsWithSplitIds>> + RecordsWithSplitIds< + org.apache.fluss.client.tiering.TableBucketWriteResult< + TestingWriteResult>>> getElementsQueue(TieringSourceReader reader) throws Exception { Class clazz = reader.getClass(); while (clazz != null) { @@ -191,7 +206,9 @@ void testHandleTieringReachMaxDurationEvent() throws Exception { Field elementsQueueField = clazz.getDeclaredField("elementsQueue"); elementsQueueField.setAccessible(true); return (FutureCompletingBlockingQueue< - RecordsWithSplitIds>>) + RecordsWithSplitIds< + org.apache.fluss.client.tiering.TableBucketWriteResult< + TestingWriteResult>>>) elementsQueueField.get(reader); } catch (NoSuchFieldException e) { // Try parent class diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/TieringSplitReaderTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/TieringSplitReaderTest.java index 171f521e00..b13864c23e 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/TieringSplitReaderTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/tiering/source/TieringSplitReaderTest.java @@ -88,13 +88,15 @@ void testTieringTable() throws Exception { tieringSplitReader.fetch(); // fetch again to get the fetch result of the splits - RecordsWithSplitIds> fetchResult = - tieringSplitReader.fetch(); + RecordsWithSplitIds< + org.apache.fluss.client.tiering.TableBucketWriteResult< + TestingWriteResult>> + fetchResult = tieringSplitReader.fetch(); for (int i = 0; i < 3; i++) { fetchResult.nextSplit(); // should has result, but the writeResult should be null - TableBucketWriteResult nextRecord = - fetchResult.nextRecordFromSplit(); + org.apache.fluss.client.tiering.TableBucketWriteResult + nextRecord = fetchResult.nextRecordFromSplit(); assertThat(nextRecord).isNotNull(); assertThat(nextRecord.writeResult()).isNull(); } @@ -128,8 +130,8 @@ void testTieringTable() throws Exception { // one fetch to make this snapshot split as finished fetchResult = tieringSplitReader.fetch(); fetchResult.nextSplit(); - TableBucketWriteResult tableBucketWriteResult = - fetchResult.nextRecordFromSplit(); + org.apache.fluss.client.tiering.TableBucketWriteResult + tableBucketWriteResult = fetchResult.nextRecordFromSplit(); assertThat(tableBucketWriteResult).isNotNull(); TestingWriteResult testingWriteResult = tableBucketWriteResult.writeResult(); assertThat(testingWriteResult).isNotNull(); @@ -324,11 +326,14 @@ connection, new ThrowOnEmptyCompleteLakeTieringFactory())) { tieringSplitReader.handleSplitsChanges( new SplitsAddition(Collections.singletonList(tieringLogSplit))); - RecordsWithSplitIds> result = - tieringSplitReader.fetch(); + RecordsWithSplitIds< + org.apache.fluss.client.tiering.TableBucketWriteResult< + TestingWriteResult>> + result = tieringSplitReader.fetch(); assertThat(result.nextSplit()).isEqualTo(tieringLogSplit.splitId()); - TableBucketWriteResult writeResult = result.nextRecordFromSplit(); + org.apache.fluss.client.tiering.TableBucketWriteResult writeResult = + result.nextRecordFromSplit(); assertThat(writeResult).isNotNull(); // expect null write result since no any records written assertThat(writeResult.writeResult()).isNull(); @@ -373,7 +378,9 @@ private void verifyTieringRows( LinkedHashMap> expectTierRows, LinkedHashMap> expectedFinishSplits) throws IOException { - RecordsWithSplitIds> fetchResult; + RecordsWithSplitIds< + org.apache.fluss.client.tiering.TableBucketWriteResult> + fetchResult; for (Map.Entry> expectTieringRowEntry : expectTierRows.entrySet()) { long tableId = expectTieringRowEntry.getKey(); @@ -385,8 +392,8 @@ private void verifyTieringRows( fetchResult = tieringSplitReader.fetch(); actualFinishSplits.addAll(fetchResult.finishedSplits()); while (fetchResult.nextSplit() != null) { - TableBucketWriteResult tableBucketWriteResult = - fetchResult.nextRecordFromSplit(); + org.apache.fluss.client.tiering.TableBucketWriteResult + tableBucketWriteResult = fetchResult.nextRecordFromSplit(); assertThat(tableBucketWriteResult).isNotNull(); TableBucket tableBucket = tableBucketWriteResult.tableBucket(); assertThat(tableBucket.getTableId()).isEqualTo(tableId); diff --git a/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/SparkTieringJobRunner.scala b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/SparkTieringJobRunner.scala index 771f217bba..8188ea940c 100644 --- a/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/SparkTieringJobRunner.scala +++ b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/SparkTieringJobRunner.scala @@ -19,7 +19,7 @@ package org.apache.fluss.spark.tiering import org.apache.fluss.client.{Connection, ConnectionFactory} import org.apache.fluss.client.admin.Admin -import org.apache.fluss.client.tiering.FlussTableLakeSnapshotCommitter +import org.apache.fluss.client.tiering.{FlussTableLakeSnapshotCommitter, TableBucketWriteResult, TieringCommitter} import org.apache.fluss.config.Configuration import org.apache.fluss.lake.committer.TieringStats import org.apache.fluss.lake.lakestorage.LakeStoragePluginSetUp @@ -35,6 +35,7 @@ import org.apache.spark.sql.SparkSession import java.util.concurrent.{Executors, ScheduledExecutorService, ScheduledFuture, TimeUnit} import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} +import scala.collection.JavaConverters._ import scala.concurrent.{Await, ExecutionContext, Future, Promise} import scala.concurrent.duration.{Duration => ScalaDuration} @@ -249,18 +250,19 @@ class SparkTieringJobRunner( } // Commit results - val stats = TieringCommitter.commitAll( + val tieringCommitter = new TieringCommitter[AnyRef, AnyRef]() + val stats = tieringCommitter.commitWriteResults( + admin, tableId, tablePath, - results, - lakeTieringFactory, flussConfig, lakeTieringConfig, - admin, - snapshotCommitter) + lakeTieringFactory, + snapshotCommitter, + results.asJava) logInfo(s"Tiering completed for table $tablePath, stats=$stats") - coordinator.markTableFinished(tableId, tieringEpoch, stats) + coordinator.markTableFinished(tableId, tieringEpoch, stats.stats) } catch { case e: Exception => @@ -284,7 +286,7 @@ class SparkTieringJobRunner( consecutiveHeartbeatFailures: AtomicInteger, splits: Seq[TieringSplit], tablePath: TablePath, - tableId: Long): Seq[SerializedTaskResult] = { + tableId: Long): Seq[TableBucketWriteResult[AnyRef]] = { val jobGroupId = s"tiering-${tablePath.getDatabaseName}-${tablePath.getTableName}-$tableId" sc.setJobGroup(jobGroupId, s"Tiering table $tablePath") @@ -363,7 +365,7 @@ object SparkTieringJobRunner { flussConfig: Configuration, dataLakeFormat: String, lakeConfig: Configuration, - pollTimeoutMs: Long): RDD[SerializedTaskResult] = { + pollTimeoutMs: Long): RDD[TableBucketWriteResult[AnyRef]] = { sparkContext.parallelize(splits, splits.size).map { split => TieringTask.process(split, flussConfig, dataLakeFormat, lakeConfig, pollTimeoutMs) } diff --git a/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringCommitter.scala b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringCommitter.scala deleted file mode 100644 index 3613a45c07..0000000000 --- a/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringCommitter.scala +++ /dev/null @@ -1,202 +0,0 @@ -/* - * 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.fluss.spark.tiering - -import org.apache.fluss.client.admin.Admin -import org.apache.fluss.client.metadata.LakeSnapshot -import org.apache.fluss.client.tiering.{FlussTableLakeSnapshotCommitter, TieringCommitterInitContext} -import org.apache.fluss.config.Configuration -import org.apache.fluss.exception.LakeTableSnapshotNotExistException -import org.apache.fluss.lake.committer.{LakeCommitResult, LakeCommitter, TieringStats} -import org.apache.fluss.lake.committer.LakeCommitter.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY -import org.apache.fluss.lake.writer.LakeTieringFactory -import org.apache.fluss.metadata.{TableBucket, TablePath} - -import org.apache.spark.internal.Logging - -import java.util.Collections - -import scala.collection.JavaConverters._ - -// TODO: This logic is duplicated from -// org.apache.fluss.flink.tiering.committer.TieringCommitOperator (commit logic). -// Consider extracting to a shared module (e.g., fluss-tiering-common) in the future. - -/** - * Orchestrates the two-phase commit of tiering write results to the lake and Fluss coordinator. - * - * Runs on the Spark driver after all executor tasks complete. The commit pipeline: - * 1. Deserializes [[SerializedTaskResult]] write results from executors. - * 2. Converts them into a lake committable via [[LakeCommitter.toCommittable()]]. - * 3. Checks for missing lake snapshots (Fluss lags behind lake) and recovers if needed. - * 4. Prepares lake snapshot metadata via - * [[FlussTableLakeSnapshotCommitter.prepareLakeSnapshot()]]. - * 5. Commits to the lake storage (e.g., Paimon/Iceberg) via [[LakeCommitter.commit()]]. - * 6. Commits the snapshot metadata to the Fluss coordinator via - * [[FlussTableLakeSnapshotCommitter.commit()]]. - */ -object TieringCommitter extends Logging { - - def commitAll( - tableId: Long, - tablePath: TablePath, - results: Seq[SerializedTaskResult], - lakeTieringFactory: LakeTieringFactory[AnyRef, AnyRef], - flussConfig: Configuration, - lakeTieringConfig: Configuration, - admin: Admin, - snapshotCommitter: FlussTableLakeSnapshotCommitter): TieringStats = { - // Filter to non-empty results - val nonEmptyResults = results.filter(_.serializedWriteResult != null) - - if (nonEmptyResults.isEmpty) { - logInfo(s"Commit tiering write results is empty for table $tableId, table path $tablePath") - return TieringStats.UNKNOWN - } - - // Verify table was not dropped and recreated during tiering - val currentTableInfo = admin.getTableInfo(tablePath).get() - if (currentTableInfo.getTableId != tableId) { - throw new IllegalStateException( - s"The current table id ${currentTableInfo.getTableId} for table path $tablePath" + - s" is different from the table id $tableId in the committable." + - " This usually happens when a table was dropped and recreated during tiering." + - " Aborting commit to prevent dirty commit.") - } - - val writeResultSerializer = lakeTieringFactory.getWriteResultSerializer - - val lakeCommitter: LakeCommitter[AnyRef, AnyRef] = lakeTieringFactory.createLakeCommitter( - new TieringCommitterInitContext(tablePath, currentTableInfo, lakeTieringConfig, flussConfig)) - - try { - // Deserialize write results - val writeResults = nonEmptyResults.map { - result => - writeResultSerializer.deserialize(result.writeResultVersion, result.serializedWriteResult) - }.asJava - - // Build offset maps - val logEndOffsets = new java.util.HashMap[TableBucket, java.lang.Long]() - val logMaxTieredTimestamps = new java.util.HashMap[TableBucket, java.lang.Long]() - nonEmptyResults.foreach { - result => - logEndOffsets.put(result.tableBucket, result.logEndOffset: java.lang.Long) - logMaxTieredTimestamps.put(result.tableBucket, result.maxTimestamp: java.lang.Long) - } - - // Convert to committable - val committable = lakeCommitter.toCommittable(writeResults) - - // Check for missing lake snapshot before committing - val flussCurrentLakeSnapshot = getLatestLakeSnapshot(admin, tablePath) - checkFlussNotMissingLakeSnapshot( - tablePath, - tableId, - lakeCommitter, - committable, - snapshotCommitter, - if (flussCurrentLakeSnapshot != null) flussCurrentLakeSnapshot.getSnapshotId else null) - - // Prepare lake snapshot (get offsets file path) - val lakeBucketTieredOffsetsFile = - snapshotCommitter.prepareLakeSnapshot(tableId, tablePath, logEndOffsets) - - // Commit to lake with offsets file in snapshot properties - val snapshotProperties = Collections.singletonMap( - FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY, - lakeBucketTieredOffsetsFile) - val lakeCommitResult = lakeCommitter.commit(committable, snapshotProperties) - - // Commit to Fluss - snapshotCommitter.commit( - tableId, - tablePath, - lakeCommitResult, - lakeBucketTieredOffsetsFile, - logEndOffsets, - logMaxTieredTimestamps) - - lakeCommitResult.getTieringStats - } finally { - lakeCommitter.close() - } - } - - private def getLatestLakeSnapshot(admin: Admin, tablePath: TablePath): LakeSnapshot = { - try { - admin.getLatestLakeSnapshot(tablePath).get() - } catch { - case e: Exception => - e.getCause match { - case _: LakeTableSnapshotNotExistException => null - case _ => throw e - } - } - } - - private def checkFlussNotMissingLakeSnapshot( - tablePath: TablePath, - tableId: Long, - lakeCommitter: LakeCommitter[AnyRef, AnyRef], - committable: AnyRef, - snapshotCommitter: FlussTableLakeSnapshotCommitter, - flussCurrentLakeSnapshot: java.lang.Long): Unit = { - val missingCommittedSnapshot = - lakeCommitter.getMissingLakeSnapshot(flussCurrentLakeSnapshot) - - if (missingCommittedSnapshot != null) { - val lakeSnapshotOffsetPath = missingCommittedSnapshot.getSnapshotProperties - .get(FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY) - - if (lakeSnapshotOffsetPath == null) { - throw new IllegalStateException( - s"Can't find $FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY field from snapshot property.") - } - - val trimmedPath = lakeSnapshotOffsetPath.trim - if (trimmedPath.contains("{")) { - throw new IllegalStateException( - s"The $FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY field in snapshot property" + - s" is a JSON string (tiered by v0.8), which is not supported to restore." + - s" Snapshot ID: ${missingCommittedSnapshot.getLakeSnapshotId}," + - s" Table: {tablePath=$tablePath, tableId=$tableId}.") - } - - // Commit missing snapshot to Fluss - snapshotCommitter.commit( - tableId, - missingCommittedSnapshot.getLakeSnapshotId, - lakeSnapshotOffsetPath, - null, - Collections.emptyMap(), - Collections.emptyMap(), - LakeCommitResult.KEEP_ALL_PREVIOUS - ) - - // Abort current committable - lakeCommitter.abort(committable) - - throw new IllegalStateException( - s"The current Fluss's lake snapshot $flussCurrentLakeSnapshot is less than" + - s" lake actual snapshot ${missingCommittedSnapshot.getLakeSnapshotId}" + - s" committed by Fluss for table: {tablePath=$tablePath, tableId=$tableId}," + - s" missing snapshot: $missingCommittedSnapshot.") - } - } -} diff --git a/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringTask.scala b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringTask.scala index d5de7466b0..8be4638c31 100644 --- a/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringTask.scala +++ b/fluss-spark/fluss-spark-common/src/main/scala/org/apache/fluss/spark/tiering/TieringTask.scala @@ -20,10 +20,9 @@ package org.apache.fluss.spark.tiering import org.apache.fluss.client.{Connection, ConnectionFactory} import org.apache.fluss.client.table.Table import org.apache.fluss.client.table.scanner.ScanRecord -import org.apache.fluss.client.tiering.TieringWriterInitContext +import org.apache.fluss.client.tiering.{TableBucketWriteResult, TieringWriterInitContext} import org.apache.fluss.config.Configuration import org.apache.fluss.lake.lakestorage.LakeStoragePluginSetUp -import org.apache.fluss.lake.serializer.SimpleVersionedSerializer import org.apache.fluss.lake.writer.{LakeTieringFactory, LakeWriter} import org.apache.spark.internal.Logging @@ -47,7 +46,7 @@ object TieringTask extends Logging { flussConfig: Configuration, dataLakeFormat: String, lakeConfig: Configuration, - pollTimeoutMs: Long): SerializedTaskResult = { + pollTimeoutMs: Long): TableBucketWriteResult[AnyRef] = { val tablePath = split.tablePath val tableBucket = split.tableBucket val partitionName = split.partitionName @@ -70,7 +69,6 @@ object TieringTask extends Logging { // Initialize LakeTieringFactory on executor val lakeTieringFactory = createLakeTieringFactory(dataLakeFormat, lakeConfig) - val writeResultSerializer = lakeTieringFactory.getWriteResultSerializer val writerInitContext = new TieringWriterInitContext( @@ -82,14 +80,9 @@ object TieringTask extends Logging { split match { case logSplit: TieringLogSplit => - processLogSplit(table, lakeWriter, logSplit, pollTimeoutMs, writeResultSerializer) + processLogSplit(table, lakeWriter, logSplit, pollTimeoutMs) case snapshotSplit: TieringSnapshotSplit => - processSnapshotSplit( - table, - lakeWriter, - snapshotSplit, - pollTimeoutMs, - writeResultSerializer) + processSnapshotSplit(table, lakeWriter, snapshotSplit, pollTimeoutMs) } } finally { closeQuietly(lakeWriter, "LakeWriter") @@ -111,8 +104,7 @@ object TieringTask extends Logging { table: Table, lakeWriter: LakeWriter[AnyRef], split: TieringLogSplit, - pollTimeoutMs: Long, - writeResultSerializer: SimpleVersionedSerializer[AnyRef]): SerializedTaskResult = { + pollTimeoutMs: Long): TableBucketWriteResult[AnyRef] = { val logScanner = table.newScan().createLogScanner() try { val tableBucket = split.tableBucket @@ -152,22 +144,16 @@ object TieringTask extends Logging { } val writeResult = lakeWriter.complete() - val serializedBytes = if (writeResult != null) { - writeResultSerializer.serialize(writeResult) - } else { - null - } logInfo( s"Finished tiering log split for bucket $tableBucket," + s" logEndOffset=$stoppingOffset, maxTimestamp=$maxTimestamp") - SerializedTaskResult( + new TableBucketWriteResult( split.tablePath, tableBucket, - split.partitionName, - serializedBytes, - writeResultSerializer.getVersion, + split.partitionName.orNull, + writeResult, stoppingOffset, maxTimestamp, split.numberOfSplits) @@ -180,8 +166,7 @@ object TieringTask extends Logging { table: Table, lakeWriter: LakeWriter[AnyRef], split: TieringSnapshotSplit, - pollTimeoutMs: Long, - writeResultSerializer: SimpleVersionedSerializer[AnyRef]): SerializedTaskResult = { + pollTimeoutMs: Long): TableBucketWriteResult[AnyRef] = { val tableBucket = split.tableBucket val batchScanner = table .newScan() @@ -202,11 +187,6 @@ object TieringTask extends Logging { } val writeResult = lakeWriter.complete() - val serializedBytes = if (writeResult != null) { - writeResultSerializer.serialize(writeResult) - } else { - null - } val logEndOffset = split.logOffsetOfSnapshot @@ -214,12 +194,11 @@ object TieringTask extends Logging { s"Finished tiering snapshot split for bucket $tableBucket," + s" snapshotId=${split.snapshotId}, logEndOffset=$logEndOffset") - SerializedTaskResult( + new TableBucketWriteResult( split.tablePath, tableBucket, - split.partitionName, - serializedBytes, - writeResultSerializer.getVersion, + split.partitionName.orNull, + writeResult, logEndOffset, UNKNOWN_BUCKET_TIMESTAMP, split.numberOfSplits