Show (git) diff signs in gutter (#3890)

* Show (git) diff signs in gutter (#3890)

Avoid string allocation when git diffing

Incrementally diff using changesets

refactor diffs to be provider indepndent and improve git implementation

remove dependency on zlib-ng

switch to asynchronus diffing with similar

Update helix-vcs/Cargo.toml

fix toml formatting

Co-authored-by: Ivan Tham <pickfire@riseup.net>

fix typo in documentation

use ropey reexpors from helix-core

fix crash when creating new file

remove useless use if io::Cursor

fix spelling mistakes

implement suggested improvement to repository loading

improve git test isolation

remove lefover comments

Co-authored-by: univerz <univerz@fu-solution.com>

fixed spelling mistake

minor cosmetic changes

fix: set self.differ to None if decoding the diff_base fails

fixup formatting

Co-authored-by: Ivan Tham <pickfire@riseup.net>

reload diff_base when file is reloaded from disk

switch to imara-diff

Fixup formatting

Co-authored-by: Blaž Hrastnik <blaz@mxxn.io>

Redraw buffer whenever a diff is updated.

Only store hunks instead of changes for individual lines to easily allow
jumping between them

Update to latest gitoxide version

Change default diff gutter position

Only update gutter after timeout

* update diff gutter synchronously, with a timeout

* Apply suggestions from code review

Co-authored-by: Blaž Hrastnik <blaz@mxxn.io>
Co-authored-by: Michael Davis <mcarsondavis@gmail.com>

* address review comments and ensure lock is always aquired

* remove configuration for redraw timeout

Co-authored-by: Blaž Hrastnik <blaz@mxxn.io>
Co-authored-by: Michael Davis <mcarsondavis@gmail.com>
pull/1/head
Pascal Kuthe 1 year ago committed by GitHub
parent 67415e096e
commit 5a3ff74221
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23

878
Cargo.lock generated

File diff suppressed because it is too large Load Diff

@ -7,6 +7,7 @@ members = [
"helix-lsp",
"helix-dap",
"helix-loader",
"helix-vcs",
"xtask",
]

@ -46,7 +46,7 @@ on unix operating systems.
| `line-number` | Line number display: `absolute` simply shows each line's number, while `relative` shows the distance from the current line. When unfocused or in insert mode, `relative` will still show absolute line numbers. | `absolute` |
| `cursorline` | Highlight all lines with a cursor. | `false` |
| `cursorcolumn` | Highlight all columns with a cursor. | `false` |
| `gutters` | Gutters to display: Available are `diagnostics` and `line-numbers` and `spacer`, note that `diagnostics` also includes other features like breakpoints, 1-width padding will be inserted if gutters is non-empty | `["diagnostics", "spacer", "line-numbers"]` |
| `gutters` | Gutters to display: Available are `diagnostics` and `diff` and `line-numbers` and `spacer`, note that `diagnostics` also includes other features like breakpoints, 1-width padding will be inserted if gutters is non-empty | `["diagnostics", "spacer", "line-numbers", "spacer", "diff"]` |
| `auto-completion` | Enable automatic pop up of auto-completion. | `true` |
| `auto-format` | Enable automatic formatting on save. | `true` |
| `auto-save` | Enable automatic saving on focus moving away from Helix. Requires [focus event support](https://github.com/helix-editor/helix/wiki/Terminal-Support) from your terminal. | `false` |

@ -83,7 +83,7 @@ pub fn find_root(root: Option<&str>, root_markers: &[String]) -> std::path::Path
top_marker.map_or(current_dir, |a| a.to_path_buf())
}
pub use ropey::{str_utils, Rope, RopeBuilder, RopeSlice};
pub use ropey::{self, str_utils, Rope, RopeBuilder, RopeSlice};
// pub use tendril::StrTendril as Tendril;
pub use smartstring::SmartString;

@ -17,8 +17,10 @@ build = true
app = true
[features]
default = ["git"]
unicode-lines = ["helix-core/unicode-lines"]
integration = []
git = ["helix-vcs/git"]
[[bin]]
name = "hx"
@ -29,6 +31,7 @@ helix-core = { version = "0.6", path = "../helix-core" }
helix-view = { version = "0.6", path = "../helix-view" }
helix-lsp = { version = "0.6", path = "../helix-lsp" }
helix-dap = { version = "0.6", path = "../helix-dap" }
helix-vcs = { version = "0.6", path = "../helix-vcs" }
helix-loader = { version = "0.6", path = "../helix-loader" }
anyhow = "1"

@ -274,16 +274,27 @@ impl Application {
}
#[cfg(feature = "integration")]
fn render(&mut self) {}
async fn render(&mut self) {}
#[cfg(not(feature = "integration"))]
fn render(&mut self) {
async fn render(&mut self) {
let mut cx = crate::compositor::Context {
editor: &mut self.editor,
jobs: &mut self.jobs,
scroll: None,
};
// Acquire mutable access to the redraw_handle lock
// to ensure that there are no tasks running that want to block rendering
drop(cx.editor.redraw_handle.1.write().await);
cx.editor.needs_redraw = false;
{
// exhaust any leftover redraw notifications
let notify = cx.editor.redraw_handle.0.notified();
tokio::pin!(notify);
notify.enable();
}
let area = self
.terminal
.autoresize()
@ -304,7 +315,7 @@ impl Application {
where
S: Stream<Item = crossterm::Result<crossterm::event::Event>> + Unpin,
{
self.render();
self.render().await;
self.last_render = Instant::now();
loop {
@ -329,18 +340,18 @@ impl Application {
biased;
Some(event) = input_stream.next() => {
self.handle_terminal_events(event);
self.handle_terminal_events(event).await;
}
Some(signal) = self.signals.next() => {
self.handle_signals(signal).await;
}
Some(callback) = self.jobs.futures.next() => {
self.jobs.handle_callback(&mut self.editor, &mut self.compositor, callback);
self.render();
self.render().await;
}
Some(callback) = self.jobs.wait_futures.next() => {
self.jobs.handle_callback(&mut self.editor, &mut self.compositor, callback);
self.render();
self.render().await;
}
event = self.editor.wait_event() => {
let _idle_handled = self.handle_editor_event(event).await;
@ -445,25 +456,25 @@ impl Application {
self.compositor.resize(area);
self.terminal.clear().expect("couldn't clear terminal");
self.render();
self.render().await;
}
signal::SIGUSR1 => {
self.refresh_config();
self.render();
self.render().await;
}
_ => unreachable!(),
}
}
pub fn handle_idle_timeout(&mut self) {
pub async fn handle_idle_timeout(&mut self) {
let mut cx = crate::compositor::Context {
editor: &mut self.editor,
jobs: &mut self.jobs,
scroll: None,
};
let should_render = self.compositor.handle_event(&Event::IdleTimeout, &mut cx);
if should_render {
self.render();
if should_render || self.editor.needs_redraw {
self.render().await;
}
}
@ -536,11 +547,11 @@ impl Application {
match event {
EditorEvent::DocumentSaved(event) => {
self.handle_document_write(event);
self.render();
self.render().await;
}
EditorEvent::ConfigEvent(event) => {
self.handle_config_events(event);
self.render();
self.render().await;
}
EditorEvent::LanguageServerMessage((id, call)) => {
self.handle_language_server_message(call, id).await;
@ -548,19 +559,19 @@ impl Application {
let last = self.editor.language_servers.incoming.is_empty();
if last || self.last_render.elapsed() > LSP_DEADLINE {
self.render();
self.render().await;
self.last_render = Instant::now();
}
}
EditorEvent::DebuggerEvent(payload) => {
let needs_render = self.editor.handle_debugger_message(payload).await;
if needs_render {
self.render();
self.render().await;
}
}
EditorEvent::IdleTimer => {
self.editor.clear_idle_timer();
self.handle_idle_timeout();
self.handle_idle_timeout().await;
#[cfg(feature = "integration")]
{
@ -572,7 +583,10 @@ impl Application {
false
}
pub fn handle_terminal_events(&mut self, event: Result<CrosstermEvent, crossterm::ErrorKind>) {
pub async fn handle_terminal_events(
&mut self,
event: Result<CrosstermEvent, crossterm::ErrorKind>,
) {
let mut cx = crate::compositor::Context {
editor: &mut self.editor,
jobs: &mut self.jobs,
@ -596,7 +610,7 @@ impl Application {
};
if should_redraw && !self.editor.should_close() {
self.render();
self.render().await;
}
}

@ -1028,10 +1028,12 @@ fn reload(
}
let scrolloff = cx.editor.config().scrolloff;
let redraw_handle = cx.editor.redraw_handle.clone();
let (view, doc) = current!(cx.editor);
doc.reload(view).map(|_| {
view.ensure_cursor_in_view(doc, scrolloff);
})
doc.reload(view, &cx.editor.diff_providers, redraw_handle)
.map(|_| {
view.ensure_cursor_in_view(doc, scrolloff);
})
}
fn reload_all(
@ -1066,7 +1068,8 @@ fn reload_all(
// Every doc is guaranteed to have at least 1 view at this point.
let view = view_mut!(cx.editor, view_ids[0]);
doc.reload(view)?;
let redraw_handle = cx.editor.redraw_handle.clone();
doc.reload(view, &cx.editor.diff_providers, redraw_handle)?;
for view_id in view_ids {
let view = view_mut!(cx.editor, view_id);

@ -730,7 +730,7 @@ impl EditorView {
let mut text = String::with_capacity(8);
for gutter_type in view.gutters() {
let gutter = gutter_type.style(editor, doc, view, theme, is_focused);
let mut gutter = gutter_type.style(editor, doc, view, theme, is_focused);
let width = gutter_type.width(view, doc);
text.reserve(width); // ensure there's enough space for the gutter
for (i, line) in (view.offset.row..(last_line + 1)).enumerate() {

@ -0,0 +1,28 @@
[package]
name = "helix-vcs"
version = "0.6.0"
authors = ["Blaž Hrastnik <blaz@mxxn.io>"]
edition = "2021"
license = "MPL-2.0"
categories = ["editor"]
repository = "https://github.com/helix-editor/helix"
homepage = "https://helix-editor.com"
# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html
[dependencies]
helix-core = { version = "0.6", path = "../helix-core" }
tokio = { version = "1", features = ["rt", "rt-multi-thread", "time", "sync", "parking_lot", "macros"] }
parking_lot = "0.12"
git-repository = { version = "0.26", default-features = false , optional = true }
imara-diff = "0.1.5"
log = "0.4"
[features]
git = ["git-repository"]
[dev-dependencies]
tempfile = "3.3"

@ -0,0 +1,198 @@
use std::ops::Range;
use std::sync::Arc;
use helix_core::Rope;
use imara_diff::Algorithm;
use parking_lot::{Mutex, MutexGuard};
use tokio::sync::mpsc::{unbounded_channel, UnboundedSender};
use tokio::sync::{Notify, OwnedRwLockReadGuard, RwLock};
use tokio::task::JoinHandle;
use tokio::time::Instant;
use crate::diff::worker::DiffWorker;
mod line_cache;
mod worker;
type RedrawHandle = (Arc<Notify>, Arc<RwLock<()>>);
/// A rendering lock passed to the differ the prevents redraws from occurring
struct RenderLock {
pub lock: OwnedRwLockReadGuard<()>,
pub timeout: Option<Instant>,
}
struct Event {
text: Rope,
is_base: bool,
render_lock: Option<RenderLock>,
}
#[derive(Clone, Debug)]
pub struct DiffHandle {
channel: UnboundedSender<Event>,
render_lock: Arc<RwLock<()>>,
hunks: Arc<Mutex<Vec<Hunk>>>,
inverted: bool,
}
impl DiffHandle {
pub fn new(diff_base: Rope, doc: Rope, redraw_handle: RedrawHandle) -> DiffHandle {
DiffHandle::new_with_handle(diff_base, doc, redraw_handle).0
}
fn new_with_handle(
diff_base: Rope,
doc: Rope,
redraw_handle: RedrawHandle,
) -> (DiffHandle, JoinHandle<()>) {
let (sender, receiver) = unbounded_channel();
let hunks: Arc<Mutex<Vec<Hunk>>> = Arc::default();
let worker = DiffWorker {
channel: receiver,
hunks: hunks.clone(),
new_hunks: Vec::default(),
redraw_notify: redraw_handle.0,
diff_finished_notify: Arc::default(),
};
let handle = tokio::spawn(worker.run(diff_base, doc));
let differ = DiffHandle {
channel: sender,
hunks,
inverted: false,
render_lock: redraw_handle.1,
};
(differ, handle)
}
pub fn invert(&mut self) {
self.inverted = !self.inverted;
}
pub fn hunks(&self) -> FileHunks {
FileHunks {
hunks: self.hunks.lock(),
inverted: self.inverted,
}
}
/// Updates the document associated with this redraw handle
/// This function is only intended to be called from within the rendering loop
/// if called from elsewhere it may fail to acquire the render lock and panic
pub fn update_document(&self, doc: Rope, block: bool) -> bool {
// unwrap is ok here because the rendering lock is
// only exclusively locked during redraw.
// This function is only intended to be called
// from the core rendering loop where no redraw can happen in parallel
let lock = self.render_lock.clone().try_read_owned().unwrap();
let timeout = if block {
None
} else {
Some(Instant::now() + tokio::time::Duration::from_millis(SYNC_DIFF_TIMEOUT))
};
self.update_document_impl(doc, self.inverted, Some(RenderLock { lock, timeout }))
}
pub fn update_diff_base(&self, diff_base: Rope) -> bool {
self.update_document_impl(diff_base, !self.inverted, None)
}
fn update_document_impl(
&self,
text: Rope,
is_base: bool,
render_lock: Option<RenderLock>,
) -> bool {
let event = Event {
text,
is_base,
render_lock,
};
self.channel.send(event).is_ok()
}
}
/// synchronous debounce value should be low
/// so we can update synchronously most of the time
const DIFF_DEBOUNCE_TIME_SYNC: u64 = 1;
/// maximum time that rendering should be blocked until the diff finishes
const SYNC_DIFF_TIMEOUT: u64 = 12;
const DIFF_DEBOUNCE_TIME_ASYNC: u64 = 96;
const ALGORITHM: Algorithm = Algorithm::Histogram;
const MAX_DIFF_LINES: usize = 64 * u16::MAX as usize;
// cap average line length to 128 for files with MAX_DIFF_LINES
const MAX_DIFF_BYTES: usize = MAX_DIFF_LINES * 128;
/// A single change in a file potentially spanning multiple lines
/// Hunks produced by the differs are always ordered by their position
/// in the file and non-overlapping.
/// Specifically for any two hunks `x` and `y` the following properties hold:
///
/// ``` no_compile
/// assert!(x.before.end <= y.before.start);
/// assert!(x.after.end <= y.after.start);
/// ```
#[derive(PartialEq, Eq, Clone, Debug)]
pub struct Hunk {
pub before: Range<u32>,
pub after: Range<u32>,
}
impl Hunk {
/// Can be used instead of `Option::None` for better performance
/// because lines larger then `i32::MAX` are not supported by `imara-diff` anyways.
/// Has some nice properties where it usually is not necessary to check for `None` separately:
/// Empty ranges fail contains checks and also fails smaller then checks.
pub const NONE: Hunk = Hunk {
before: u32::MAX..u32::MAX,
after: u32::MAX..u32::MAX,
};
/// Inverts a change so that `before`
pub fn invert(&self) -> Hunk {
Hunk {
before: self.after.clone(),
after: self.before.clone(),
}
}
pub fn is_pure_insertion(&self) -> bool {
self.before.is_empty()
}
pub fn is_pure_removal(&self) -> bool {
self.after.is_empty()
}
}
/// A list of changes in a file sorted in ascending
/// non-overlapping order
#[derive(Debug)]
pub struct FileHunks<'a> {
hunks: MutexGuard<'a, Vec<Hunk>>,
inverted: bool,
}
impl FileHunks<'_> {
pub fn is_inverted(&self) -> bool {
self.inverted
}
/// Returns the `Hunk` for the `n`th change in this file.
/// if there is no `n`th change `Hunk::NONE` is returned instead.
pub fn nth_hunk(&self, n: u32) -> Hunk {
match self.hunks.get(n as usize) {
Some(hunk) if self.inverted => hunk.invert(),
Some(hunk) => hunk.clone(),
None => Hunk::NONE,
}
}
pub fn len(&self) -> u32 {
self.hunks.len() as u32
}
pub fn is_empty(&self) -> bool {
self.len() == 0
}
}

@ -0,0 +1,130 @@
//! This modules encapsulates a tiny bit of unsafe code that
//! makes diffing significantly faster and more ergonomic to implement.
//! This code is necessary because diffing requires quick random
//! access to the lines of the text that is being diffed.
//!
//! Therefore it is best to collect the `Rope::lines` iterator into a vec
//! first because access to the vec is `O(1)` where `Rope::line` is `O(log N)`.
//! However this process can allocate a (potentially quite large) vector.
//!
//! To avoid reallocation for every diff, the vector is reused.
//! However the RopeSlice references the original rope and therefore forms a self-referential data structure.
//! A transmute is used to change the lifetime of the slice to static to circumvent that project.
use std::mem::transmute;
use helix_core::{Rope, RopeSlice};
use imara_diff::intern::{InternedInput, Interner};
use super::{MAX_DIFF_BYTES, MAX_DIFF_LINES};
/// A cache that stores the `lines` of a rope as a vector.
/// It allows safely reusing the allocation of the vec when updating the rope
pub(crate) struct InternedRopeLines {
diff_base: Rope,
doc: Rope,
num_tokens_diff_base: u32,
interned: InternedInput<RopeSlice<'static>>,
}
impl InternedRopeLines {
pub fn new(diff_base: Rope, doc: Rope) -> InternedRopeLines {
let mut res = InternedRopeLines {
interned: InternedInput {
before: Vec::with_capacity(diff_base.len_lines()),
after: Vec::with_capacity(doc.len_lines()),
interner: Interner::new(diff_base.len_lines() + doc.len_lines()),
},
diff_base,
doc,
// will be populated by update_diff_base_impl
num_tokens_diff_base: 0,
};
res.update_diff_base_impl();
res
}
/// Updates the `diff_base` and optionally the document if `doc` is not None
pub fn update_diff_base(&mut self, diff_base: Rope, doc: Option<Rope>) {
self.interned.clear();
self.diff_base = diff_base;
if let Some(doc) = doc {
self.doc = doc
}
if !self.is_too_large() {
self.update_diff_base_impl();
}
}
/// Updates the `doc` without reinterning the `diff_base`, this function
/// is therefore significantly faster than `update_diff_base` when only the document changes.
pub fn update_doc(&mut self, doc: Rope) {
// Safety: we clear any tokens that were added after
// the interning of `self.diff_base` finished so
// all lines that refer to `self.doc` have been purged.
self.interned
.interner
.erase_tokens_after(self.num_tokens_diff_base.into());
self.doc = doc;
if self.is_too_large() {
self.interned.after.clear();
} else {
self.update_doc_impl();
}
}
fn update_diff_base_impl(&mut self) {
// Safety: This transmute is safe because it only transmutes a lifetime, which has no effect.
// The backing storage for the RopeSlices referred to by the lifetime is stored in `self.diff_base`.
// Therefore as long as `self.diff_base` is not dropped/replaced this memory remains valid.
// `self.diff_base` is only changed in `self.update_diff_base`, which clears the interner.
// When the interned lines are exposed to consumer in `self.diff_input`, the lifetime is bounded to a reference to self.
// That means that on calls to update there exist no references to `self.interned`.
let before = self
.diff_base
.lines()
.map(|line: RopeSlice| -> RopeSlice<'static> { unsafe { transmute(line) } });
self.interned.update_before(before);
self.num_tokens_diff_base = self.interned.interner.num_tokens();
// the has to be interned again because the interner was fully cleared
self.update_doc_impl()
}
fn update_doc_impl(&mut self) {
// Safety: This transmute is save because it only transmutes a lifetime, which has no effect.
// The backing storage for the RopeSlices referred to by the lifetime is stored in `self.doc`.
// Therefore as long as `self.doc` is not dropped/replaced this memory remains valid.
// `self.doc` is only changed in `self.update_doc`, which clears the interner.
// When the interned lines are exposed to consumer in `self.diff_input`, the lifetime is bounded to a reference to self.
// That means that on calls to update there exist no references to `self.interned`.
let after = self
.doc
.lines()
.map(|line: RopeSlice| -> RopeSlice<'static> { unsafe { transmute(line) } });
self.interned.update_after(after);
}
fn is_too_large(&self) -> bool {
// bound both lines and bytes to avoid huge files with few (but huge) lines
// or huge file with tiny lines. While this makes no difference to
// diff itself (the diff performance only depends on the number of tokens)
// the interning runtime depends mostly on filesize and is actually dominant
// for large files
self.doc.len_lines() > MAX_DIFF_LINES
|| self.diff_base.len_lines() > MAX_DIFF_LINES
|| self.doc.len_bytes() > MAX_DIFF_BYTES
|| self.diff_base.len_bytes() > MAX_DIFF_BYTES
}
/// Returns the `InternedInput` for performing the diff.
/// If `diff_base` or `doc` is so large that performing a diff could slow the editor
/// this function returns `None`.
pub fn interned_lines(&self) -> Option<&InternedInput<RopeSlice>> {
if self.is_too_large() {
None
} else {
Some(&self.interned)
}
}
}

@ -0,0 +1,207 @@
use std::mem::swap;
use std::ops::Range;
use std::sync::Arc;
use helix_core::{Rope, RopeSlice};
use imara_diff::intern::InternedInput;
use parking_lot::Mutex;
use tokio::sync::mpsc::UnboundedReceiver;
use tokio::sync::Notify;
use tokio::time::{timeout, timeout_at, Duration};
use crate::diff::{
Event, RenderLock, ALGORITHM, DIFF_DEBOUNCE_TIME_ASYNC, DIFF_DEBOUNCE_TIME_SYNC,
};
use super::line_cache::InternedRopeLines;
use super::Hunk;
#[cfg(test)]
mod test;
pub(super) struct DiffWorker {
pub channel: UnboundedReceiver<Event>,
pub hunks: Arc<Mutex<Vec<Hunk>>>,
pub new_hunks: Vec<Hunk>,
pub redraw_notify: Arc<Notify>,
pub diff_finished_notify: Arc<Notify>,
}
impl DiffWorker {
async fn accumulate_events(&mut self, event: Event) -> (Option<Rope>, Option<Rope>) {
let mut accumulator = EventAccumulator::new();
accumulator.handle_event(event).await;
accumulator
.accumulate_debounced_events(
&mut self.channel,
self.redraw_notify.clone(),
self.diff_finished_notify.clone(),
)
.await;
(accumulator.doc, accumulator.diff_base)
}
pub async fn run(mut self, diff_base: Rope, doc: Rope) {
let mut interner = InternedRopeLines::new(diff_base, doc);
if let Some(lines) = interner.interned_lines() {
self.perform_diff(lines);
}
self.apply_hunks();
while let Some(event) = self.channel.recv().await {
let (doc, diff_base) = self.accumulate_events(event).await;
let process_accumulated_events = || {
if let Some(new_base) = diff_base {
interner.update_diff_base(new_base, doc)
} else {
interner.update_doc(doc.unwrap())
}
if let Some(lines) = interner.interned_lines() {
self.perform_diff(lines)
}
};
// Calculating diffs is computationally expensive and should
// not run inside an async function to avoid blocking other futures.
// Note: tokio::task::block_in_place does not work during tests
#[cfg(test)]
process_accumulated_events();
#[cfg(not(test))]
tokio::task::block_in_place(process_accumulated_events);
self.apply_hunks();
}
}
/// update the hunks (used by the gutter) by replacing it with `self.new_hunks`.
/// `self.new_hunks` is always empty after this function runs.
/// To improve performance this function tries to reuse the allocation of the old diff previously stored in `self.line_diffs`
fn apply_hunks(&mut self) {
swap(&mut *self.hunks.lock(), &mut self.new_hunks);
self.diff_finished_notify.notify_waiters();
self.new_hunks.clear();
}
fn perform_diff(&mut self, input: &InternedInput<RopeSlice>) {
imara_diff::diff(ALGORITHM, input, |before: Range<u32>, after: Range<u32>| {
self.new_hunks.push(Hunk { before, after })
})
}
}
struct EventAccumulator {
diff_base: Option<Rope>,
doc: Option<Rope>,
render_lock: Option<RenderLock>,
}
impl<'a> EventAccumulator {
fn new() -> EventAccumulator {
EventAccumulator {
diff_base: None,
doc: None,
render_lock: None,
}
}
async fn handle_event(&mut self, event: Event) {
let dst = if event.is_base {
&mut self.diff_base
} else {
&mut self.doc
};
*dst = Some(event.text);
// always prefer the most synchronous requested render mode
if let Some(render_lock) = event.render_lock {
match &mut self.render_lock {
Some(RenderLock { timeout, .. }) => {
// A timeout of `None` means that the render should
// always wait for the diff to complete (so no timeout)
// remove the existing timeout, otherwise keep the previous timeout
// because it will be shorter then the current timeout
if render_lock.timeout.is_none() {
timeout.take();
}
}
None => self.render_lock = Some(render_lock),
}
}
}
async fn accumulate_debounced_events(
&mut self,
channel: &mut UnboundedReceiver<Event>,
redraw_notify: Arc<Notify>,
diff_finished_notify: Arc<Notify>,
) {
let async_debounce = Duration::from_millis(DIFF_DEBOUNCE_TIME_ASYNC);
let sync_debounce = Duration::from_millis(DIFF_DEBOUNCE_TIME_SYNC);
loop {
// if we are not blocking rendering use a much longer timeout
let debounce = if self.render_lock.is_none() {
async_debounce
} else {
sync_debounce
};
if let Ok(Some(event)) = timeout(debounce, channel.recv()).await {
self.handle_event(event).await;
} else {
break;
}
}
// setup task to trigger the rendering
match self.render_lock.take() {
// diff is performed outside of the rendering loop
// request a redraw after the diff is done
None => {
tokio::spawn(async move {
diff_finished_notify.notified().await;
redraw_notify.notify_one();
});
}
// diff is performed inside the rendering loop
// block redraw until the diff is done or the timeout is expired
Some(RenderLock {
lock,
timeout: Some(timeout),
}) => {
tokio::spawn(async move {
let res = {
// Acquire a lock on the redraw handle.
// The lock will block the rendering from occurring while held.
// The rendering waits for the diff if it doesn't time out
timeout_at(timeout, diff_finished_notify.notified()).await
};
// we either reached the timeout or the diff is finished, release the render lock
drop(lock);
if res.is_ok() {
// Diff finished in time we are done.
return;
}
// Diff failed to complete in time log the event
// and wait until the diff occurs to trigger an async redraw
log::warn!("Diff computation timed out, update of diffs might appear delayed");
diff_finished_notify.notified().await;
redraw_notify.notify_one();
});
}
// a blocking diff is performed inside the rendering loop
// block redraw until the diff is done
Some(RenderLock {
lock,
timeout: None,
}) => {
tokio::spawn(async move {
diff_finished_notify.notified().await;
// diff is done release the lock
drop(lock)
});
}
};
}
}

@ -0,0 +1,149 @@
use helix_core::Rope;
use tokio::task::JoinHandle;
use crate::diff::{DiffHandle, Hunk};
impl DiffHandle {
fn new_test(diff_base: &str, doc: &str) -> (DiffHandle, JoinHandle<()>) {
DiffHandle::new_with_handle(
Rope::from_str(diff_base),
Rope::from_str(doc),
Default::default(),
)
}
async fn into_diff(self, handle: JoinHandle<()>) -> Vec<Hunk> {
let hunks = self.hunks;
// dropping the channel terminates the task
drop(self.channel);
handle.await.unwrap();
let hunks = hunks.lock();
Vec::clone(&*hunks)
}
}
#[tokio::test]
async fn append_line() {
let (differ, handle) = DiffHandle::new_test("foo\n", "foo\nbar\n");
let line_diffs = differ.into_diff(handle).await;
assert_eq!(
&line_diffs,
&[Hunk {
before: 1..1,
after: 1..2
}]
)
}
#[tokio::test]
async fn prepend_line() {
let (differ, handle) = DiffHandle::new_test("foo\n", "bar\nfoo\n");
let line_diffs = differ.into_diff(handle).await;
assert_eq!(
&line_diffs,
&[Hunk {
before: 0..0,
after: 0..1
}]
)
}
#[tokio::test]
async fn modify() {
let (differ, handle) = DiffHandle::new_test("foo\nbar\n", "foo bar\nbar\n");
let line_diffs = differ.into_diff(handle).await;
assert_eq!(
&line_diffs,
&[Hunk {
before: 0..1,
after: 0..1
}]
)
}
#[tokio::test]
async fn delete_line() {
let (differ, handle) = DiffHandle::new_test("foo\nfoo bar\nbar\n", "foo\nbar\n");
let line_diffs = differ.into_diff(handle).await;
assert_eq!(
&line_diffs,
&[Hunk {
before: 1..2,
after: 1..1
}]
)
}
#[tokio::test]
async fn delete_line_and_modify() {
let (differ, handle) = DiffHandle::new_test("foo\nbar\ntest\nfoo", "foo\ntest\nfoo bar");
let line_diffs = differ.into_diff(handle).await;
assert_eq!(
&line_diffs,
&[
Hunk {
before: 1..2,
after: 1..1
},
Hunk {
before: 3..4,
after: 2..3
},
]
)
}
#[tokio::test]
async fn add_use() {
let (differ, handle) = DiffHandle::new_test(
"use ropey::Rope;\nuse tokio::task::JoinHandle;\n",
"use ropey::Rope;\nuse ropey::RopeSlice;\nuse tokio::task::JoinHandle;\n",
);
let line_diffs = differ.into_diff(handle).await;
assert_eq!(
&line_diffs,
&[Hunk {
before: 1..1,
after: 1..2
},]
)
}
#[tokio::test]
async fn update_document() {
let (differ, handle) = DiffHandle::new_test("foo\nbar\ntest\nfoo", "foo\nbar\ntest\nfoo");
differ.update_document(Rope::from_str("foo\ntest\nfoo bar"), false);
let line_diffs = differ.into_diff(handle).await;
assert_eq!(
&line_diffs,
&[
Hunk {
before: 1..2,
after: 1..1
},
Hunk {
before: 3..4,
after: 2..3
},
]
)
}
#[tokio::test]
async fn update_base() {
let (differ, handle) = DiffHandle::new_test("foo\ntest\nfoo bar", "foo\ntest\nfoo bar");
differ.update_diff_base(Rope::from_str("foo\nbar\ntest\nfoo"));
let line_diffs = differ.into_diff(handle).await;
assert_eq!(
&line_diffs,
&[
Hunk {
before: 1..2,
after: 1..1
},
Hunk {
before: 3..4,
after: 2..3
},
]
)
}

@ -0,0 +1,80 @@
use std::path::Path;
use git::objs::tree::EntryMode;
use git::sec::trust::DefaultForLevel;
use git::{Commit, ObjectId, Repository, ThreadSafeRepository};
use git_repository as git;
use crate::DiffProvider;
#[cfg(test)]
mod test;
pub struct Git;
impl Git {
fn open_repo(path: &Path, ceiling_dir: Option<&Path>) -> Option<ThreadSafeRepository> {
// custom open options
let mut git_open_opts_map = git::sec::trust::Mapping::<git::open::Options>::default();
// don't use the global git configs (not needed)
let config = git::permissions::Config {
system: false,
git: false,
user: false,
env: true,
includes: true,
git_binary: false,
};
// change options for config permissions without touching anything else
git_open_opts_map.reduced = git_open_opts_map.reduced.permissions(git::Permissions {
config,
..git::Permissions::default_for_level(git::sec::Trust::Reduced)
});
git_open_opts_map.full = git_open_opts_map.full.permissions(git::Permissions {
config,
..git::Permissions::default_for_level(git::sec::Trust::Full)
});
let mut open_options = git::discover::upwards::Options::default();
if let Some(ceiling_dir) = ceiling_dir {
open_options.ceiling_dirs = vec![ceiling_dir.to_owned()];
}
ThreadSafeRepository::discover_with_environment_overrides_opts(
path,
open_options,
git_open_opts_map,
)
.ok()
}
}
impl DiffProvider for Git {
fn get_diff_base(&self, file: &Path) -> Option<Vec<u8>> {
debug_assert!(!file.exists() || file.is_file());
debug_assert!(file.is_absolute());
// TODO cache repository lookup
let repo = Git::open_repo(file.parent()?, None)?.to_thread_local();
let head = repo.head_commit().ok()?;
let file_oid = find_file_in_commit(&repo, &head, file)?;
let file_object = repo.find_object(file_oid).ok()?;
Some(file_object.detach().data)
}
}
/// Finds the object that contains the contents of a file at a specific commit.
fn find_file_in_commit(repo: &Repository, commit: &Commit, file: &Path) -> Option<ObjectId> {
let repo_dir = repo.work_dir()?;
let rel_path = file.strip_prefix(repo_dir).ok()?;
let tree = commit.tree().ok()?;
let tree_entry = tree.lookup_entry_by_path(rel_path).ok()??;
match tree_entry.mode() {
// not a file, everything is new, do not show diff
EntryMode::Tree | EntryMode::Commit | EntryMode::Link => None,
// found a file
EntryMode::Blob | EntryMode::BlobExecutable => Some(tree_entry.object_id()),
}
}

@ -0,0 +1,121 @@
use std::{fs::File, io::Write, path::Path, process::Command};
use tempfile::TempDir;
use crate::{DiffProvider, Git};
fn exec_git_cmd(args: &str, git_dir: &Path) {
let res = Command::new("git")
.arg("-C")
.arg(git_dir) // execute the git command in this directory
.args(args.split_whitespace())
.env_remove("GIT_DIR")
.env_remove("GIT_ASKPASS")
.env_remove("SSH_ASKPASS")
.env("GIT_TERMINAL_PROMPT", "false")
.env("GIT_AUTHOR_DATE", "2000-01-01 00:00:00 +0000")
.env("GIT_AUTHOR_EMAIL", "author@example.com")
.env("GIT_AUTHOR_NAME", "author")
.env("GIT_COMMITTER_DATE", "2000-01-02 00:00:00 +0000")
.env("GIT_COMMITTER_EMAIL", "committer@example.com")
.env("GIT_COMMITTER_NAME", "committer")
.env("GIT_CONFIG_COUNT", "2")
.env("GIT_CONFIG_KEY_0", "commit.gpgsign")
.env("GIT_CONFIG_VALUE_0", "false")
.env("GIT_CONFIG_KEY_1", "init.defaultBranch")
.env("GIT_CONFIG_VALUE_1", "main")
.output()
.unwrap_or_else(|_| panic!("`git {args}` failed"));
if !res.status.success() {
println!("{}", String::from_utf8_lossy(&res.stdout));
eprintln!("{}", String::from_utf8_lossy(&res.stderr));
panic!("`git {args}` failed (see output above)")
}
}
fn create_commit(repo: &Path, add_modified: bool) {
if add_modified {
exec_git_cmd("add -A", repo);
}
exec_git_cmd("commit -m message", repo);
}
fn empty_git_repo() -> TempDir {
let tmp = tempfile::tempdir().expect("create temp dir for git testing");
exec_git_cmd("init", tmp.path());
exec_git_cmd("config user.email test@helix.org", tmp.path());
exec_git_cmd("config user.name helix-test", tmp.path());
tmp
}
#[test]
fn missing_file() {
let temp_git = empty_git_repo();
let file = temp_git.path().join("file.txt");
File::create(&file).unwrap().write_all(b"foo").unwrap();
assert_eq!(Git.get_diff_base(&file), None);
}
#[test]
fn unmodified_file() {
let temp_git = empty_git_repo();
let file = temp_git.path().join("file.txt");
let contents = b"foo".as_slice();
File::create(&file).unwrap().write_all(contents).unwrap();
create_commit(temp_git.path(), true);
assert_eq!(Git.get_diff_base(&file), Some(Vec::from(contents)));
}
#[test]
fn modified_file() {
let temp_git = empty_git_repo();
let file = temp_git.path().join("file.txt");
let contents = b"foo".as_slice();
File::create(&file).unwrap().write_all(contents).unwrap();
create_commit(temp_git.path(), true);
File::create(&file).unwrap().write_all(b"bar").unwrap();
assert_eq!(Git.get_diff_base(&file), Some(Vec::from(contents)));
}
/// Test that `get_file_head` does not return content for a directory.
/// This is important to correctly cover cases where a directory is removed and replaced by a file.
/// If the contents of the directory object were returned a diff between a path and the directory children would be produced.
#[test]
fn directory() {
let temp_git = empty_git_repo();
let dir = temp_git.path().join("file.txt");
std::fs::create_dir(&dir).expect("");
let file = dir.join("file.txt");
let contents = b"foo".as_slice();
File::create(&file).unwrap().write_all(contents).unwrap();
create_commit(temp_git.path(), true);
std::fs::remove_dir_all(&dir).unwrap();
File::create(&dir).unwrap().write_all(b"bar").unwrap();
assert_eq!(Git.get_diff_base(&dir), None);
}
/// Test that `get_file_head` does not return content for a symlink.
/// This is important to correctly cover cases where a symlink is removed and replaced by a file.
/// If the contents of the symlink object were returned a diff between a path and the actual file would be produced (bad ui).
#[cfg(any(unix, windows))]
#[test]
fn symlink() {
#[cfg(unix)]
use std::os::unix::fs::symlink;
#[cfg(not(unix))]
use std::os::windows::fs::symlink_file as symlink;
let temp_git = empty_git_repo();
let file = temp_git.path().join("file.txt");
let contents = b"foo".as_slice();
File::create(&file).unwrap().write_all(contents).unwrap();
let file_link = temp_git.path().join("file_link.txt");
symlink("file.txt", &file_link).unwrap();
create_commit(temp_git.path(), true);
assert_eq!(Git.get_diff_base(&file_link), None);
assert_eq!(Git.get_diff_base(&file), Some(Vec::from(contents)));
}

@ -0,0 +1,51 @@
use std::path::Path;
#[cfg(feature = "git")]
pub use git::Git;
#[cfg(not(feature = "git"))]
pub use Dummy as Git;
#[cfg(feature = "git")]
mod git;
mod diff;
pub use diff::{DiffHandle, Hunk};
pub trait DiffProvider {
/// Returns the data that a diff should be computed against
/// if this provider is used.
/// The data is returned as raw byte without any decoding or encoding performed
/// to ensure all file encodings are handled correctly.
fn get_diff_base(&self, file: &Path) -> Option<Vec<u8>>;
}
#[doc(hidden)]
pub struct Dummy;
impl DiffProvider for Dummy {
fn get_diff_base(&self, _file: &Path) -> Option<Vec<u8>> {
None
}
}
pub struct DiffProviderRegistry {
providers: Vec<Box<dyn DiffProvider>>,
}
impl DiffProviderRegistry {
pub fn get_diff_base(&self, file: &Path) -> Option<Vec<u8>> {
self.providers
.iter()
.find_map(|provider| provider.get_diff_base(file))
}
}
impl Default for DiffProviderRegistry {
fn default() -> Self {
// currently only git is supported
// TODO make this configurable when more providers are added
let git: Box<dyn DiffProvider> = Box::new(Git);
let providers = vec![git];
DiffProviderRegistry { providers }
}
}

@ -21,6 +21,7 @@ helix-loader = { version = "0.6", path = "../helix-loader" }
helix-lsp = { version = "0.6", path = "../helix-lsp" }
helix-dap = { version = "0.6", path = "../helix-dap" }
crossterm = { version = "0.25", optional = true }
helix-vcs = { version = "0.6", path = "../helix-vcs" }
# Conversion traits
once_cell = "1.16"
@ -43,6 +44,7 @@ log = "~0.4"
which = "4.2"
[target.'cfg(windows)'.dependencies]
clipboard-win = { version = "4.4", features = ["std"] }

@ -3,6 +3,8 @@ use futures_util::future::BoxFuture;
use futures_util::FutureExt;
use helix_core::auto_pairs::AutoPairs;
use helix_core::Range;
use helix_vcs::{DiffHandle, DiffProviderRegistry};
use serde::de::{self, Deserialize, Deserializer};
use serde::Serialize;
use std::borrow::Cow;
@ -24,6 +26,7 @@ use helix_core::{
DEFAULT_LINE_ENDING,
};
use crate::editor::RedrawHandle;
use crate::{apply_transaction, DocumentId, Editor, View, ViewId};
/// 8kB of buffer space for encoding and decoding `Rope`s.
@ -133,6 +136,8 @@ pub struct Document {
diagnostics: Vec<Diagnostic>,
language_server: Option<Arc<helix_lsp::Client>>,
diff_handle: Option<DiffHandle>,
}
use std::{fmt, mem};
@ -371,6 +376,7 @@ impl Document {
last_saved_revision: 0,
modified_since_accessed: false,
language_server: None,
diff_handle: None,
}
}
@ -624,16 +630,20 @@ impl Document {
}
/// Reload the document from its path.
pub fn reload(&mut self, view: &mut View) -> Result<(), Error> {
pub fn reload(
&mut self,
view: &mut View,
provider_registry: &DiffProviderRegistry,
redraw_handle: RedrawHandle,
) -> Result<(), Error> {
let encoding = &self.encoding;
let path = self.path().filter(|path| path.exists());
// If there is no path or the path no longer exists.
if path.is_none() {
bail!("can't find file to reload from");
}
let path = self
.path()
.filter(|path| path.exists())
.ok_or_else(|| anyhow!("can't find file to reload from"))?
.to_owned();
let mut file = std::fs::File::open(path.unwrap())?;
let mut file = std::fs::File::open(&path)?;
let (rope, ..) = from_reader(&mut file, Some(encoding))?;
// Calculate the difference between the buffer and source text, and apply it.
@ -646,6 +656,11 @@ impl Document {
self.detect_indent_and_line_ending();
match provider_registry.get_diff_base(&path) {
Some(diff_base) => self.set_diff_base(diff_base, redraw_handle),
None => self.diff_handle = None,
}
Ok(())
}
@ -787,6 +802,10 @@ impl Document {
if !transaction.changes().is_empty() {
self.version += 1;
// start computing the diff in parallel
if let Some(diff_handle) = &self.diff_handle {
diff_handle.update_document(self.text.clone(), false);
}
// generate revert to savepoint
if self.savepoint.is_some() {
@ -1046,6 +1065,23 @@ impl Document {
server.is_initialized().then(|| server)
}
pub fn diff_handle(&self) -> Option<&DiffHandle> {
self.diff_handle.as_ref()
}
/// Intialize/updates the differ for this document with a new base.
pub fn set_diff_base(&mut self, diff_base: Vec<u8>, redraw_handle: RedrawHandle) {
if let Ok((diff_base, _)) = from_reader(&mut diff_base.as_slice(), Some(self.encoding)) {
if let Some(differ) = &self.diff_handle {
differ.update_diff_base(diff_base);
return;
}
self.diff_handle = Some(DiffHandle::new(diff_base, self.text.clone(), redraw_handle))
} else {
self.diff_handle = None;
}
}
#[inline]
/// Tree-sitter AST tree
pub fn syntax(&self) -> Option<&Syntax> {

@ -9,6 +9,7 @@ use crate::{
tree::{self, Tree},
Align, Document, DocumentId, View, ViewId,
};
use helix_vcs::DiffProviderRegistry;
use futures_util::stream::select_all::SelectAll;
use futures_util::{future, StreamExt};
@ -26,7 +27,10 @@ use std::{
};
use tokio::{
sync::mpsc::{unbounded_channel, UnboundedReceiver, UnboundedSender},
sync::{
mpsc::{unbounded_channel, UnboundedReceiver, UnboundedSender},
Notify, RwLock,
},
time::{sleep, Duration, Instant, Sleep},
};
@ -454,6 +458,8 @@ pub enum GutterType {
LineNumbers,
/// Show one blank space
Spacer,
/// Highlight local changes
Diff,
}
impl std::str::FromStr for GutterType {
@ -464,6 +470,7 @@ impl std::str::FromStr for GutterType {
"diagnostics" => Ok(Self::Diagnostics),
"spacer" => Ok(Self::Spacer),
"line-numbers" => Ok(Self::LineNumbers),
"diff" => Ok(Self::Diff),
_ => anyhow::bail!("Gutter type can only be `diagnostics` or `line-numbers`."),
}
}
@ -600,6 +607,8 @@ impl Default for Config {
GutterType::Diagnostics,
GutterType::Spacer,
GutterType::LineNumbers,
GutterType::Spacer,
GutterType::Diff,
],
middle_click_paste: true,
auto_pairs: AutoPairConfig::default(),
@ -681,6 +690,7 @@ pub struct Editor {
pub macro_replaying: Vec<char>,
pub language_servers: helix_lsp::Registry,
pub diagnostics: BTreeMap<lsp::Url, Vec<lsp::Diagnostic>>,
pub diff_providers: DiffProviderRegistry,
pub debugger: Option<dap::Client>,
pub debugger_events: SelectAll<UnboundedReceiverStream<dap::Payload>>,
@ -711,8 +721,15 @@ pub struct Editor {
pub exit_code: i32,
pub config_events: (UnboundedSender<ConfigEvent>, UnboundedReceiver<ConfigEvent>),
/// Allows asynchronous tasks to control the rendering
/// The `Notify` allows asynchronous tasks to request the editor to perform a redraw
/// The `RwLock` blocks the editor from performing the render until an exclusive lock can be aquired
pub redraw_handle: RedrawHandle,
pub needs_redraw: bool,
}
pub type RedrawHandle = (Arc<Notify>, Arc<RwLock<()>>);
#[derive(Debug)]
pub enum EditorEvent {
DocumentSaved(DocumentSavedEventResult),
@ -785,6 +802,7 @@ impl Editor {
theme: theme_loader.default(),
language_servers: helix_lsp::Registry::new(),
diagnostics: BTreeMap::new(),
diff_providers: DiffProviderRegistry::default(),
debugger: None,
debugger_events: SelectAll::new(),
breakpoints: HashMap::new(),
@ -803,6 +821,8 @@ impl Editor {
auto_pairs,
exit_code: 0,
config_events: unbounded_channel(),
redraw_handle: Default::default(),
needs_redraw: false,
}
}
@ -1109,7 +1129,9 @@ impl Editor {
let mut doc = Document::open(&path, None, Some(self.syn_loader.clone()))?;
let _ = Self::launch_language_server(&mut self.language_servers, &mut doc);
if let Some(diff_base) = self.diff_providers.get_diff_base(&path) {
doc.set_diff_base(diff_base, self.redraw_handle.clone());
}
self.new_document(doc)
};
@ -1348,24 +1370,39 @@ impl Editor {
}
pub async fn wait_event(&mut self) -> EditorEvent {
tokio::select! {
biased;
// the loop only runs once or twice and would be better implemented with a recursion + const generic
// however due to limitations with async functions that can not be implemented right now
loop {
tokio::select! {
biased;
Some(event) = self.save_queue.next() => {
self.write_count -= 1;
return EditorEvent::DocumentSaved(event)
}
Some(config_event) = self.config_events.1.recv() => {
return EditorEvent::ConfigEvent(config_event)
}
Some(message) = self.language_servers.incoming.next() => {
return EditorEvent::LanguageServerMessage(message)
}
Some(event) = self.debugger_events.next() => {
return EditorEvent::DebuggerEvent(event)
}
Some(event) = self.save_queue.next() => {
self.write_count -= 1;
EditorEvent::DocumentSaved(event)
}
Some(config_event) = self.config_events.1.recv() => {
EditorEvent::ConfigEvent(config_event)
}
Some(message) = self.language_servers.incoming.next() => {
EditorEvent::LanguageServerMessage(message)
}
Some(event) = self.debugger_events.next() => {
EditorEvent::DebuggerEvent(event)
}
_ = &mut self.idle_timer => {
EditorEvent::IdleTimer
_ = self.redraw_handle.0.notified() => {
if !self.needs_redraw{
self.needs_redraw = true;
let timeout = Instant::now() + Duration::from_millis(96);
if timeout < self.idle_timer.deadline(){
self.idle_timer.as_mut().reset(timeout)
}
}
}
_ = &mut self.idle_timer => {
return EditorEvent::IdleTimer
}
}
}
}

@ -12,7 +12,7 @@ fn count_digits(n: usize) -> usize {
std::iter::successors(Some(n), |&n| (n >= 10).then(|| n / 10)).count()
}
pub type GutterFn<'doc> = Box<dyn Fn(usize, bool, &mut String) -> Option<Style> + 'doc>;
pub type GutterFn<'doc> = Box<dyn FnMut(usize, bool, &mut String) -> Option<Style> + 'doc>;
pub type Gutter =
for<'doc> fn(&'doc Editor, &'doc Document, &View, &Theme, bool, usize) -> GutterFn<'doc>;
@ -31,6 +31,7 @@ impl GutterType {
}
GutterType::LineNumbers => line_numbers(editor, doc, view, theme, is_focused),
GutterType::Spacer => padding(editor, doc, view, theme, is_focused),
GutterType::Diff => diff(editor, doc, view, theme, is_focused),
}
}
@ -39,6 +40,7 @@ impl GutterType {
GutterType::Diagnostics => 1,
GutterType::LineNumbers => line_numbers_width(_view, doc),
GutterType::Spacer => 1,
GutterType::Diff => 1,
}
}
}
@ -83,6 +85,53 @@ pub fn diagnostic<'doc>(
})
}
pub fn diff<'doc>(
_editor: &'doc Editor,
doc: &'doc Document,
_view: &View,
theme: &Theme,
_is_focused: bool,
) -> GutterFn<'doc> {
let added = theme.get("diff.plus");
let deleted = theme.get("diff.minus");
let modified = theme.get("diff.delta");
if let Some(diff_handle) = doc.diff_handle() {
let hunks = diff_handle.hunks();
let mut hunk_i = 0;
let mut hunk = hunks.nth_hunk(hunk_i);
Box::new(move |line: usize, _selected: bool, out: &mut String| {
// truncating the line is fine here because we don't compute diffs
// for files with more lines than i32::MAX anyways
// we need to special case removals here
// these technically do not have a range of lines to highlight (`hunk.after.start == hunk.after.end`).
// However we still want to display these hunks correctly we must not yet skip to the next hunk here
while hunk.after.end < line as u32
|| !hunk.is_pure_removal() && line as u32 == hunk.after.end
{
hunk_i += 1;
hunk = hunks.nth_hunk(hunk_i);
}
if hunk.after.start > line as u32 {
return None;
}
let (icon, style) = if hunk.is_pure_insertion() {
("▍", added)
} else if hunk.is_pure_removal() {
("▔", deleted)
} else {
("▍", modified)
};
write!(out, "{}", icon).unwrap();
Some(style)
})
} else {
Box::new(move |_, _, _| None)
}
}
pub fn line_numbers<'doc>(
editor: &'doc Editor,
doc: &'doc Document,
@ -226,8 +275,8 @@ pub fn diagnostics_or_breakpoints<'doc>(
theme: &Theme,
is_focused: bool,
) -> GutterFn<'doc> {
let diagnostics = diagnostic(editor, doc, view, theme, is_focused);
let breakpoints = breakpoints(editor, doc, view, theme, is_focused);
let mut diagnostics = diagnostic(editor, doc, view, theme, is_focused);
let mut breakpoints = breakpoints(editor, doc, view, theme, is_focused);
Box::new(move |line, selected, out| {
breakpoints(line, selected, out).or_else(|| diagnostics(line, selected, out))

@ -158,17 +158,10 @@ impl View {
}
pub fn gutter_offset(&self, doc: &Document) -> u16 {
let mut offset = self
.gutters
self.gutters
.iter()
.map(|gutter| gutter.width(self, doc) as u16)
.sum();
if offset > 0 {
offset += 1
}
offset
.sum()
}
//
@ -392,8 +385,8 @@ impl View {
mod tests {
use super::*;
use helix_core::Rope;
const OFFSET: u16 = 4; // 1 diagnostic + 2 linenr (< 100 lines) + 1 gutter
const OFFSET_WITHOUT_LINE_NUMBERS: u16 = 2; // 1 diagnostic + 1 gutter
const OFFSET: u16 = 3; // 1 diagnostic + 2 linenr (< 100 lines)
const OFFSET_WITHOUT_LINE_NUMBERS: u16 = 1; // 1 diagnostic
// const OFFSET: u16 = GUTTERS.iter().map(|(_, width)| *width as u16).sum();
use crate::document::Document;
use crate::editor::GutterType;

Loading…
Cancel
Save