Skip to content

Conversation

@PavithranRick
Copy link
Contributor

@PavithranRick PavithranRick commented Nov 13, 2025

Describe the issue this Pull Request addresses

This PR introduces a new comprehensive show_timeline procedure for Hudi Spark SQL that provides detailed timeline information for all table operations. The procedure displays timeline instants including commits, deltacommits, compactions, clustering, cleaning, and rollback operations with support for both active and archived timelines and completed/pending state instants.

Summary and Changelog

Comprehensive timeline view:
Shows all timeline instants with detailed metadata including state transitions (REQUESTED, INFLIGHT, COMPLETED)

Time-based filtering:
Support for startTime and endTime parameters to filter results within specific time ranges

Archive timeline support:
showArchived parameter to include archived timeline data for complete historical view

Generic SQL filtering:
filter parameter supporting SQL expressions for flexible result filtering

Rich metadata output:
Includes formatted timestamps, rollback information, and table type details

The procedure replaces multiple fragmented timeline-related procedures with a single unified interface that provides both pending and completed instant information with partition-specific metadata support.

Impact-related changelog details:

  • New procedure: show_timeline with parameters (table, path, limit, showArchived, filter, startTime, endTime)
  • Enhanced schema: 8-column output including instant_time, action, state, requested_time, inflight_time, completed_time, timeline_type, rollback_info
  • Backward compatibility: existing timeline procedures remain functional (deprecated with guidance to use the new procedure)

Impact

User-facing Features:

  • Unified timeline interface replacing multiple specialized procedures
  • Advanced filtering capabilities: time-based + SQL expression filtering
  • Historical data access through archive timeline support

Performance Impact:

  • Optimized timeline scanning with proper extension filtering
  • Configurable limits (default 20 entries if no start/end time provided)
  • Archive timeline accessed only when explicitly requested

Risk Level

Low

Verification performed

  • Comprehensive test coverage: 8 focused test cases covering basic functionality, MoR tables, rollback operations, and state transitions, timeline
  • Schema validation: all output fields properly typed and validated
  • Error handling: graceful handling of invalid filters, missing tables, and timeline access failures
  • Timeline consistency: proper handling of both active and archived timelines with correct state mapping

Documentation Update

  • Add show_timeline procedure to Hudi Spark SQL procedures documentation
  • Update timeline management examples to use new procedure
  • Add advanced filtering examples and use cases

Contributor's checklist

  • Read through contributor's guide
  • Enough context is provided in the sections above
  • Adequate tests were added if applicable

vamshikrishnakyatham and others added 7 commits August 28, 2025 17:46
…-for-show_timeline-Procedure-with-appropriate-start-and-end-time-for-both-active-and-archive-timelines
…Support-for-show_timeline-Procedure-with-appropriate-start-and-end-time-for-both-active-and-archive-timelines
@github-actions github-actions bot added the size:L PR with lines of changes in (300, 1000] label Nov 13, 2025

// test show timeline after compaction with archived timeline
val timelineResultAfterCompactionDf = spark.sql(s"call show_timeline(table => '$tableName', showArchived => true)")
timelineResultAfterCompactionDf.show(false)
Copy link
Contributor Author

@PavithranRick PavithranRick Nov 18, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

show timeline result

Image

@github-actions github-actions bot added size:XL PR with lines of changes > 1000 and removed size:L PR with lines of changes in (300, 1000] labels Nov 18, 2025
@PavithranRick PavithranRick changed the title [HUDI-9766] Support for show_timeline Procedure with appropriate start and end time for both active and archive timelines feat: [HUDI-9766] Support for show_timeline Procedure with appropriate start and end time for both active and archive timelines Nov 19, 2025

val activeRollbackInfoMap = getRolledBackInstantInfo(metaClient.getActiveTimeline, metaClient)
val archivedRollbackInfoMap = if (showArchived) {
val archivedTimeline = metaClient.getArchivedTimeline.reload()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this line will triggers the whole timeline instants loading twice already, metaClient.getArchivedTimeline for one time with load mode as LoadMode.ACTION, and .reload() another time with load mode as LoadMode.ACTION, note that for V2, only the FULL mode supports reading the plans(rollback, cleaning and compaciton).

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

maybe we just new an empty timeline there and invokes these APIs.

val limitedInstants = if (startTime.trim.nonEmpty && endTime.trim.nonEmpty) {
sortedInstants
} else {
sortedInstants.take(limit)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if we have a final limit on the timeline instants, do we stil need to add limit to the timeline API?

Pavithran Ravichandiran added 2 commits November 21, 2025 13:02
…rchived timeline partial loading (stop after limit / avoid loading entire archived timeline)
Copy link
Contributor

@nsivabalan nsivabalan left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sharing some feedback for now. will review the rest later today.


void loadCompletedInstantDetailsInMemory();

void loadCompletedInstantDetailsInMemory(String startTs, String endTs);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

do we have UTs for these?

record -> {
Object action = record.get(ACTION_STATE);
return record.get(ACTION_TYPE_KEY).toString().equals(HoodieTimeline.COMPACTION_ACTION)
&& (action == null || org.apache.hudi.common.table.timeline.HoodieInstant.State.INFLIGHT.toString().equals(action.toString()));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is the action expected to be null for a completed entry?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lets add a comment
// Older files don't have action state set.

metaClient, null, Option.empty(), LoadMode.PLAN, commitsFilter, loader);
List<HoodieInstant> collectedInstants = loader.getCollectedInstants();
List<HoodieInstant> newInstants = collectedInstants.stream()
.filter(instant -> !getInstants().contains(instant))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can we do getInstants() once outside the loop and use it here.
currently, this will result in N array lists

return instantsInRange.values()
.stream()
.flatMap(Collection::stream)
.sorted()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

don't we need sorting based on instant times followed by states?
i.e REQEUSTED followed by INFLIGHT followed by COMPLETED

entryList.sort(new ArchiveFileVersionComparator());

for (StoragePathInfo fs : entryList) {
if (stoppable != null && stoppable.shouldStop()) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lets sync f2f on this. I have some questions/clarifications

try {
return LSMTimeline.getMaxInstantTime(fileName);
} catch (Exception e) {
return "";
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why return EMPTY_STRING?

}
}
filteredFiles.parallelStream().forEach(fileName -> {
if (stoppable != null && stoppable.shouldStop()) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if we are doing parallel stream, we can't guarantee latest N with limit N right?
there could be non-continuous entries

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

actually, we can't marry limit and parallel stream.
so, whenever we wanted to do limit, lets avoid parallel processing.

}
}
filteredFiles.parallelStream().forEach(fileName -> {
if (stoppable != null && stoppable.shouldStop()) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

actually, we can't marry limit and parallel stream.
so, whenever we wanted to do limit, lets avoid parallel processing.

)
val combinedEntries = (activeEntries ++ archivedEntries)
.sortWith((a, b) => {
val timePriorityOrder = a.getString(0).compareTo(b.getString(0))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lets use RequestedTimeBasedComparator and not come up w/ our own comparator.

})

if (startTime.trim.nonEmpty && endTime.trim.nonEmpty) {
combinedEntries
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lets try to apply code reuse wherever possible


rollbackInstants.asScala.foreach { rollbackInstant =>
try {
if (rollbackInstant.isInflight) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lets do

if (!rollbackInstant.isComplete...) {

private def getRollbackInfo(instant: HoodieInstant, timeline: HoodieTimeline, rollbackInfoMap: Map[String, List[String]], metaClient: HoodieTableMetaClient): String = {
try {
if (HoodieTimeline.ROLLBACK_ACTION.equalsIgnoreCase(instant.getAction)) {
if (instant.isInflight) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

same comment as above


import org.apache.spark.sql.hudi.common.HoodieSparkSqlTestBase

class TestShowTimelineProcedure extends HoodieSparkSqlTestBase {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lets try to cover all cases listed below

both v1 and v2

  1. limit 50.
  2. both active and archived 20. where active contains 40.
  3. both active and archived 20, where active contains 10.
  4. start within active timeline.
  5. end within active timeline.
  6. start and end w/n active timeline.
  7. start in archived. but arch not explicitly enabled.
  8. "". archived enabled.
  9. start in archived and end in active timeline. archived not enabled.
  10. start in archived and end in active timeline. archived enabled.
  11. start and end in archived. arch not enabled.
  12. start and end in archived. arch enabled.

completed commits.
infight commits.
completed dc (mor)
inflight dc (mor)
clean commits.
clean inflight commits.
completed compaction.
infligth compaction.
completed replace commits. -> clustering
inflight replace commits. -> clustering
completed replace commits. -> insert overwrite
inflight replace commits. -> insert overwrite
completed rollback.
pending rollback.

In active timeline, we should have mix of inflight and completed. but in archived, we can only have completed and its fine.

// 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).
Copy link
Contributor

@danny0405 danny0405 Nov 22, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The LoadMode.METADATA does not include the plan, are you saying the LoadMode.PLAN or LoadMode.FULL ?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Even if we use the constructor ArchivedTimelineV2(HoodieTableMetaClient metaClient, String startTs), the whole timeline would still be loaded with filtering, a better way to avoid the eager loading is adding a new API in TimelineFactory.createArchivedTimeline(HoodieTableMetaClient metaClient, boolean loadingInstants), add in each ArchivedTimeline, we add a new constructor that accepts the meta client but does not load any instants by default.

Because in the use cases of this PR, we always want a lazy loading of archived timeline and the load is triggered as needed.

}

// Sort files in reverse chronological order if needed (newest first for limit queries)
if (stoppable != null && stoppable.needsReverseOrder()) {
Copy link
Contributor

@danny0405 danny0405 Nov 22, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If we do not have good way to plugin the limit logic simply and clean, maybe we just add a separate method in ArchivedTimelineLoader.loadInstants with an explicit param StoppableRecordConsumer, the benefits:

  1. get rid of the null check and instance of check;
  2. always sort the files in reverse chronological order;
  3. read the files in single thread instead of in parallel.

Read with limit is somehow a range query instead of full scan, by doting this, we can freely plugin in the logic required for limit while still keep the basic scan query efficient and clean. We can always comeback to this for better abstraction when it is necessary.

public void loadCompactionDetailsInMemory(int limit) {
loadInstantsWithLimit(limit, true,
record -> {
Object action = record.get(ACTION_STATE);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Here action should be state. We should rename it to be clear.

public void loadCompletedInstantDetailsInMemory(int limit) {
loadInstantsWithLimit(limit, true,
record -> {
Object action = record.get(ACTION_STATE);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same here.

@hudi-bot
Copy link

CI report:

Bot commands @hudi-bot supports the following commands:
  • @hudi-bot run azure re-run the last Azure build

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

size:XL PR with lines of changes > 1000

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants