Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Make SortPreservingMergeStream stable on input stream order #1687

Merged
merged 1 commit into from
Jan 27, 2022

Conversation

alamb
Copy link
Contributor

@alamb alamb commented Jan 26, 2022

Which issue does this PR close?

Closes #1686

Rationale for this change

I would like to have a predicatable output order for rows that have equal sorting keys

What changes are included in this PR?

  1. Change SortKeyCursor to use stream_index if it is otherwise equal
  2. Add docstrings
  3. Test

Are there any user-facing changes?

(more) sorting output (?)

@github-actions github-actions bot added the datafusion Changes in the datafusion crate label Jan 26, 2022
Copy link
Contributor Author

@alamb alamb left a comment

Choose a reason for hiding this comment

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

FYI @yjshen

Ok(Ordering::Equal)
// Break ties using stream_idx to ensure a predictable
// ordering of rows when comparing equal streams.
Ok(self.stream_idx.cmp(&other.stream_idx))
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is the only code change. The rest is comments and tests

// Expect the data to be sorted first by "batch_number" (because
// that was the order it was fed in, even though only "value"
// is in the sort key)
assert_batches_eq!(
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Example test output before this fix:

[
    "+--------------+-------+",
    "| batch_number | value |",
    "+--------------+-------+",
    "| 0            | A     |",
    "| 2            | A     |",
    "| 6            | A     |",
    "| 9            | A     |",
    "| 5            | A     |",
    "| 1            | A     |",
    "| 4            | A     |",
    "| 3            | A     |",
    "| 8            | A     |",
    "| 7            | A     |",
    "| 0            | B     |",
    "| 6            | B     |",
    "| 8            | B     |",
    "| 7            | B     |",
    "| 4            | B     |",
    "| 2            | B     |",
    "| 1            | B     |",
    "| 3            | B     |",
    "| 9            | B     |",
    "| 5            | B     |",
    "+--------------+-------+",
]

///
/// Input Streams Output stream
/// (sorted) (sorted)
/// ```
Copy link
Member

Choose a reason for hiding this comment

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

Nice diagram!

@alamb alamb merged commit 63d24bf into apache:master Jan 27, 2022
/// │ ╔═══╦═══╗ │ │
/// │ ║ B ║ E ║ ... │──┘ │
/// │ ╚═══╩═══╝ │ Note Stable Sort: the merged stream
/// └─────────────────────────┘ places equal rows from stream 1
Copy link
Contributor

Choose a reason for hiding this comment

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

wow!!
Can I know what tool you used to create this diagram?
@alamb
It's clearly for every review.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

😄 -- I used https://monodraw.helftone.com/ which some of the other engineers at InfluxData use -- it is quite good though not free :(

Copy link
Member

Choose a reason for hiding this comment

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

@liukun4515 It also surprised me in a previous issue 😂

@alamb alamb deleted the alamb/stable_sort_real branch January 28, 2022 11:16
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
datafusion Changes in the datafusion crate
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Stable SortPreservingMergeStream
5 participants