PavithranRick commented on code in PR #14261:
URL: https://github.com/apache/hudi/pull/14261#discussion_r2578855395


##########
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowTimelineProcedure.scala:
##########
@@ -0,0 +1,573 @@
+/*
+ * 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.spark.sql.hudi.command.procedures
+
+import org.apache.hudi.SparkAdapterSupport
+import org.apache.hudi.common.table.HoodieTableMetaClient
+import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, 
StructType}
+
+import java.util.function.Supplier
+
+import scala.collection.JavaConverters._
+
+/**
+ * Spark SQL procedure to show timeline information for a Hudi table.
+ *
+ * This procedure displays comprehensive information about all timeline 
instants including commits,
+ * compactions, clustering, cleaning, and other table operations. It supports 
both active and
+ * archived timelines with time-based filtering and generic SQL filters.
+ *
+ * == Parameters ==
+ * - `table`: Optional. The name of the Hudi table to query (mutually 
exclusive with `path`)
+ * - `path`: Optional. The base path of the Hudi table (mutually exclusive 
with `table`)
+ * - `limit`: Optional. Maximum number of timeline entries to return (default: 
20)
+ * - `showArchived`: Optional. Whether to include archived timeline data 
(default: false)
+ * - `filter`: Optional. SQL expression to filter results (default: empty 
string)
+ * - `startTime`: Optional. Start timestamp for filtering results (format: 
yyyyMMddHHmmss)
+ * - `endTime`: Optional. End timestamp for filtering results (format: 
yyyyMMddHHmmss)
+ *
+ * == Output Schema ==
+ * - `instant_time`: Timestamp when the operation was performed
+ * - `action`: The action type (commit, deltacommit, compaction, clustering, 
clean, rollback, etc.)
+ * - `state`: Current state of the operation (REQUESTED, INFLIGHT, COMPLETED)
+ * - `requested_time`: Formatted date when the operation was requested (MM-dd 
HH:mm:ss format)
+ * - `inflight_time`: Formatted date when the operation became inflight (MM-dd 
HH:mm:ss format)
+ * - `completed_time`: Formatted date when the operation completed (MM-dd 
HH:mm format, "-" if not completed)
+ * - `timeline_type`: Whether the data is from ACTIVE or ARCHIVED timeline
+ * - `rollback_info`: Information about rollback operations (what was rolled 
back or what rolled back this operation)
+ *
+ * == Error Handling ==
+ * - Throws `IllegalArgumentException` for invalid filter expressions
+ * - Throws `HoodieException` for table access issues
+ * - Returns empty result set if no timeline entries match the criteria
+ * - Gracefully handles archived timeline access failures with warning logs
+ *
+ * == Filter Support ==
+ * The `filter` parameter supports SQL expressions for filtering results on 
any output column.
+ * The filter uses Spark SQL syntax and supports various data types and 
operations.
+ *
+ * == Usage Examples ==
+ * {{{
+ * -- Basic usage: Show timeline entries
+ * CALL show_timeline(table => 'hudi_table')
+ *
+ * -- Show timeline with custom limit
+ * CALL show_timeline(table => 'hudi_table', limit => 50)
+ *
+ * -- Include archived timeline data
+ * CALL show_timeline(table => 'hudi_table', showArchived => true)
+ *
+ * -- Filter for specific actions
+ * CALL show_timeline(
+ *   table => 'hudi_table',
+ *   filter => "action = 'commit'"
+ * )
+ *
+ * -- Show timeline with time range and filtering
+ * CALL show_timeline(
+ *   table => 'hudi_table',
+ *   startTime => '20251201000000',
+ *   endTime => '20251231235959',
+ *   filter => "action = 'commit' AND state = 'COMPLETED'"
+ * )
+ * }}}
+ *
+ * @see [[HoodieProcedureFilterUtils]] for detailed filter expression syntax
+ */
+class ShowTimelineProcedure extends BaseProcedure with ProcedureBuilder with 
SparkAdapterSupport with Logging {
+
+  private val PARAMETERS = Array[ProcedureParameter](
+    ProcedureParameter.optional(0, "table", DataTypes.StringType),
+    ProcedureParameter.optional(1, "path", DataTypes.StringType),
+    ProcedureParameter.optional(2, "limit", DataTypes.IntegerType, 20),
+    ProcedureParameter.optional(3, "showArchived", DataTypes.BooleanType, 
false),
+    ProcedureParameter.optional(4, "filter", DataTypes.StringType, ""),
+    ProcedureParameter.optional(5, "startTime", DataTypes.StringType, ""),
+    ProcedureParameter.optional(6, "endTime", DataTypes.StringType, "")
+  )
+
+  private val OUTPUT_TYPE = new StructType(Array[StructField](
+    StructField("instant_time", DataTypes.StringType, nullable = true, 
Metadata.empty),
+    StructField("action", DataTypes.StringType, nullable = true, 
Metadata.empty),
+    StructField("state", DataTypes.StringType, nullable = true, 
Metadata.empty),
+    StructField("requested_time", DataTypes.StringType, nullable = true, 
Metadata.empty),
+    StructField("inflight_time", DataTypes.StringType, nullable = true, 
Metadata.empty),
+    StructField("completed_time", DataTypes.StringType, nullable = true, 
Metadata.empty),
+    StructField("timeline_type", DataTypes.StringType, nullable = true, 
Metadata.empty),
+    StructField("rollback_info", DataTypes.StringType, nullable = true, 
Metadata.empty)
+  ))
+
+  def parameters: Array[ProcedureParameter] = PARAMETERS
+
+  def outputType: StructType = OUTPUT_TYPE
+
+  override def call(args: ProcedureArgs): Seq[Row] = {
+    super.checkArgs(PARAMETERS, args)
+
+    val tableName = getArgValueOrDefault(args, PARAMETERS(0))
+    val tablePath = getArgValueOrDefault(args, PARAMETERS(1))
+    val limit = getArgValueOrDefault(args, PARAMETERS(2)).get.asInstanceOf[Int]
+    val showArchived = getArgValueOrDefault(args, 
PARAMETERS(3)).get.asInstanceOf[Boolean]
+    val filter = getArgValueOrDefault(args, 
PARAMETERS(4)).get.asInstanceOf[String]
+    val startTime = getArgValueOrDefault(args, 
PARAMETERS(5)).get.asInstanceOf[String]
+    val endTime = getArgValueOrDefault(args, 
PARAMETERS(6)).get.asInstanceOf[String]
+
+    validateFilter(filter, outputType)
+
+    val basePath: String = getBasePath(tableName, tablePath)
+    val metaClient = createMetaClient(jsc, basePath)
+
+    val timelineEntries = getTimelineEntries(metaClient, limit, showArchived, 
startTime, endTime)
+
+    applyFilter(timelineEntries, filter, outputType)
+  }
+
+  override def build: Procedure = new ShowTimelineProcedure()
+
+  /**
+   * Retrieves timeline entries from both active and archived timelines based 
on the provided parameters.
+   *
+   * This is the main orchestration method that coordinates the retrieval of 
timeline entries. It:
+   * 1. Builds instant information map from the active timeline (for 
modification time lookups)
+   * 2. Loads rollback information for both active and archived timelines
+   * 3. Optionally loads archived timeline details if `showArchived` is true
+   * 4. Retrieves entries from active timeline
+   * 5. Optionally retrieves entries from archived timeline and combines them
+   * 6. Sorts combined entries by timestamp (descending) and state priority
+   * 7. Applies limit or time range filtering as appropriate
+   *
+   * @param metaClient The Hudi table metadata client for accessing timeline 
data
+   * @param limit Maximum number of entries to return (ignored if time range 
is specified)
+   * @param showArchived Whether to include archived timeline entries in the 
result
+   * @param startTime Optional start timestamp for filtering (format: 
yyyyMMddHHmmss). If empty, no start filtering is applied
+   * @param endTime Optional end timestamp for filtering (format: 
yyyyMMddHHmmss). If empty, no end filtering is applied
+   * @return Sequence of Row objects representing timeline entries, sorted by 
timestamp (descending) and state priority
+   *         (COMPLETED > INFLIGHT > REQUESTED)
+   *
+   * @note When `showArchived` is true, this method will:
+   *       - Reload the archived timeline to ensure it's up-to-date
+   *       - Load completed instant details and compaction details into memory 
for the specified time range or limit
+   *       - Combine active and archived entries, with archived entries marked 
with "ARCHIVED" timeline type
+   *
+   * @note The sorting logic prioritizes:
+   *       1. Timestamp (newer first)
+   *       2. State (COMPLETED > INFLIGHT > REQUESTED)
+   *
+   * @see [[getTimelineEntriesFromTimeline]] for extracting entries from a 
specific timeline
+   * @see [[buildInstantInfoFromTimeline]] for building the instant 
information map
+   */
+  private def getTimelineEntries(metaClient: HoodieTableMetaClient,
+                                 limit: Int,
+                                 showArchived: Boolean,
+                                 startTime: String,
+                                 endTime: String): Seq[Row] = {
+
+    val instantInfoMap = buildInstantInfoFromTimeline(metaClient)
+
+    val activeRollbackInfoMap = 
getRolledBackInstantInfo(metaClient.getActiveTimeline, metaClient)
+
+    // Create archived timeline starting from the maximum instant time in 
active timeline
+    // This way, if all archived instants are older than the active timeline's 
max instant,
+    // the archived timeline will be empty and won't load anything, avoiding 
unnecessary loading.
+    // Instead of getArchivedTimeline() which loads with LoadMode.ACTION, we 
use the startTs
+    // constructor which loads with LoadMode.METADATA, and then load specific 
details (PLAN for compactions).

Review Comment:
   Thanks for the suggestion. made the changes now



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to