Add `fs::MTime` newtype to encourage `!=` instead of `>` (#20830)

Michael Sloan created

See ["mtime comparison considered
harmful"](https://apenwarr.ca/log/20181113) for details of why
comparators other than equality/inequality should not be used with
mtime.

Release Notes:

- N/A

Change summary

Cargo.lock                                   |   3 
crates/assistant/src/context_store.rs        |   2 
crates/copilot/src/copilot.rs                |   2 
crates/editor/Cargo.toml                     |   1 
crates/editor/src/items.rs                   |  21 +-
crates/editor/src/persistence.rs             |  24 +--
crates/extension_host/src/extension_host.rs  |   5 
crates/fs/Cargo.toml                         |   1 
crates/fs/src/fs.rs                          | 136 ++++++++++++++-------
crates/language/Cargo.toml                   |   1 
crates/language/src/buffer.rs                |  24 +-
crates/semantic_index/src/embedding_index.rs |  14 -
crates/semantic_index/src/summary_backlog.rs |   9 
crates/semantic_index/src/summary_index.rs   |  18 +-
crates/worktree/src/worktree.rs              |   6 
15 files changed, 155 insertions(+), 112 deletions(-)

Detailed changes

Cargo.lock 🔗

@@ -3731,6 +3731,7 @@ dependencies = [
  "emojis",
  "env_logger 0.11.5",
  "file_icons",
+ "fs",
  "futures 0.3.31",
  "fuzzy",
  "git",
@@ -4621,6 +4622,7 @@ dependencies = [
  "objc",
  "parking_lot",
  "paths",
+ "proto",
  "rope",
  "serde",
  "serde_json",
@@ -6487,6 +6489,7 @@ dependencies = [
  "ctor",
  "ec4rs",
  "env_logger 0.11.5",
+ "fs",
  "futures 0.3.31",
  "fuzzy",
  "git",

crates/assistant/src/context_store.rs 🔗

@@ -770,7 +770,7 @@ impl ContextStore {
                         contexts.push(SavedContextMetadata {
                             title: title.to_string(),
                             path,
-                            mtime: metadata.mtime.into(),
+                            mtime: metadata.mtime.timestamp_for_user().into(),
                         });
                     }
                 }

crates/copilot/src/copilot.rs 🔗

@@ -1231,7 +1231,7 @@ mod tests {
 
         fn disk_state(&self) -> language::DiskState {
             language::DiskState::Present {
-                mtime: std::time::UNIX_EPOCH,
+                mtime: ::fs::MTime::from_seconds_and_nanos(100, 42),
             }
         }
 

crates/editor/Cargo.toml 🔗

@@ -42,6 +42,7 @@ emojis.workspace = true
 file_icons.workspace = true
 futures.workspace = true
 fuzzy.workspace = true
+fs.workspace = true
 git.workspace = true
 gpui.workspace = true
 http_client.workspace = true

crates/editor/src/items.rs 🔗

@@ -1618,15 +1618,14 @@ fn path_for_file<'a>(
 #[cfg(test)]
 mod tests {
     use crate::editor_tests::init_test;
+    use fs::Fs;
 
     use super::*;
+    use fs::MTime;
     use gpui::{AppContext, VisualTestContext};
     use language::{LanguageMatcher, TestFile};
     use project::FakeFs;
-    use std::{
-        path::{Path, PathBuf},
-        time::SystemTime,
-    };
+    use std::path::{Path, PathBuf};
 
     #[gpui::test]
     fn test_path_for_file(cx: &mut AppContext) {
@@ -1679,9 +1678,7 @@ mod tests {
     async fn test_deserialize(cx: &mut gpui::TestAppContext) {
         init_test(cx, |_| {});
 
-        let now = SystemTime::now();
         let fs = FakeFs::new(cx.executor());
-        fs.set_next_mtime(now);
         fs.insert_file("/file.rs", Default::default()).await;
 
         // Test case 1: Deserialize with path and contents
@@ -1690,12 +1687,18 @@ mod tests {
             let (workspace, cx) = cx.add_window_view(|cx| Workspace::test_new(project.clone(), cx));
             let workspace_id = workspace::WORKSPACE_DB.next_id().await.unwrap();
             let item_id = 1234 as ItemId;
+            let mtime = fs
+                .metadata(Path::new("/file.rs"))
+                .await
+                .unwrap()
+                .unwrap()
+                .mtime;
 
             let serialized_editor = SerializedEditor {
                 abs_path: Some(PathBuf::from("/file.rs")),
                 contents: Some("fn main() {}".to_string()),
                 language: Some("Rust".to_string()),
-                mtime: Some(now),
+                mtime: Some(mtime),
             };
 
             DB.save_serialized_editor(item_id, workspace_id, serialized_editor.clone())
@@ -1792,9 +1795,7 @@ mod tests {
             let workspace_id = workspace::WORKSPACE_DB.next_id().await.unwrap();
 
             let item_id = 9345 as ItemId;
-            let old_mtime = now
-                .checked_sub(std::time::Duration::from_secs(60 * 60 * 24))
-                .unwrap();
+            let old_mtime = MTime::from_seconds_and_nanos(0, 50);
             let serialized_editor = SerializedEditor {
                 abs_path: Some(PathBuf::from("/file.rs")),
                 contents: Some("fn main() {}".to_string()),

crates/editor/src/persistence.rs 🔗

@@ -1,8 +1,8 @@
 use anyhow::Result;
 use db::sqlez::bindable::{Bind, Column, StaticColumnCount};
 use db::sqlez::statement::Statement;
+use fs::MTime;
 use std::path::PathBuf;
-use std::time::{Duration, SystemTime, UNIX_EPOCH};
 
 use db::sqlez_macros::sql;
 use db::{define_connection, query};
@@ -14,7 +14,7 @@ pub(crate) struct SerializedEditor {
     pub(crate) abs_path: Option<PathBuf>,
     pub(crate) contents: Option<String>,
     pub(crate) language: Option<String>,
-    pub(crate) mtime: Option<SystemTime>,
+    pub(crate) mtime: Option<MTime>,
 }
 
 impl StaticColumnCount for SerializedEditor {
@@ -29,16 +29,13 @@ impl Bind for SerializedEditor {
         let start_index = statement.bind(&self.contents, start_index)?;
         let start_index = statement.bind(&self.language, start_index)?;
 
-        let mtime = self.mtime.and_then(|mtime| {
-            mtime
-                .duration_since(UNIX_EPOCH)
-                .ok()
-                .map(|duration| (duration.as_secs() as i64, duration.subsec_nanos() as i32))
-        });
-        let start_index = match mtime {
+        let start_index = match self
+            .mtime
+            .and_then(|mtime| mtime.to_seconds_and_nanos_for_persistence())
+        {
             Some((seconds, nanos)) => {
-                let start_index = statement.bind(&seconds, start_index)?;
-                statement.bind(&nanos, start_index)?
+                let start_index = statement.bind(&(seconds as i64), start_index)?;
+                statement.bind(&(nanos as i32), start_index)?
             }
             None => {
                 let start_index = statement.bind::<Option<i64>>(&None, start_index)?;
@@ -64,7 +61,7 @@ impl Column for SerializedEditor {
 
         let mtime = mtime_seconds
             .zip(mtime_nanos)
-            .map(|(seconds, nanos)| UNIX_EPOCH + Duration::new(seconds as u64, nanos as u32));
+            .map(|(seconds, nanos)| MTime::from_seconds_and_nanos(seconds as u64, nanos as u32));
 
         let editor = Self {
             abs_path,
@@ -280,12 +277,11 @@ mod tests {
         assert_eq!(have, serialized_editor);
 
         // Storing and retrieving mtime
-        let now = SystemTime::now();
         let serialized_editor = SerializedEditor {
             abs_path: None,
             contents: None,
             language: None,
-            mtime: Some(now),
+            mtime: Some(MTime::from_seconds_and_nanos(100, 42)),
         };
 
         DB.save_serialized_editor(1234, workspace_id, serialized_editor.clone())

crates/extension_host/src/extension_host.rs 🔗

@@ -345,7 +345,10 @@ impl ExtensionStore {
                 if let (Ok(Some(index_metadata)), Ok(Some(extensions_metadata))) =
                     (index_metadata, extensions_metadata)
                 {
-                    if index_metadata.mtime > extensions_metadata.mtime {
+                    if index_metadata
+                        .mtime
+                        .bad_is_greater_than(extensions_metadata.mtime)
+                    {
                         extension_index_needs_rebuild = false;
                     }
                 }

crates/fs/Cargo.toml 🔗

@@ -24,6 +24,7 @@ libc.workspace = true
 parking_lot.workspace = true
 paths.workspace = true
 rope.workspace = true
+proto.workspace = true
 serde.workspace = true
 serde_json.workspace = true
 smol.workspace = true

crates/fs/src/fs.rs 🔗

@@ -27,13 +27,14 @@ use futures::{future::BoxFuture, AsyncRead, Stream, StreamExt};
 use git::repository::{GitRepository, RealGitRepository};
 use gpui::{AppContext, Global, ReadGlobal};
 use rope::Rope;
+use serde::{Deserialize, Serialize};
 use smol::io::AsyncWriteExt;
 use std::{
     io::{self, Write},
     path::{Component, Path, PathBuf},
     pin::Pin,
     sync::Arc,
-    time::{Duration, SystemTime},
+    time::{Duration, SystemTime, UNIX_EPOCH},
 };
 use tempfile::{NamedTempFile, TempDir};
 use text::LineEnding;
@@ -179,13 +180,62 @@ pub struct RemoveOptions {
 #[derive(Copy, Clone, Debug)]
 pub struct Metadata {
     pub inode: u64,
-    pub mtime: SystemTime,
+    pub mtime: MTime,
     pub is_symlink: bool,
     pub is_dir: bool,
     pub len: u64,
     pub is_fifo: bool,
 }
 
+/// Filesystem modification time. The purpose of this newtype is to discourage use of operations
+/// that do not make sense for mtimes. In particular, it is not always valid to compare mtimes using
+/// `<` or `>`, as there are many things that can cause the mtime of a file to be earlier than it
+/// was. See ["mtime comparison considered harmful" - apenwarr](https://apenwarr.ca/log/20181113).
+///
+/// Do not derive Ord, PartialOrd, or arithmetic operation traits.
+#[derive(Copy, Clone, Debug, PartialEq, Eq, Hash, Deserialize, Serialize)]
+#[serde(transparent)]
+pub struct MTime(SystemTime);
+
+impl MTime {
+    /// Conversion intended for persistence and testing.
+    pub fn from_seconds_and_nanos(secs: u64, nanos: u32) -> Self {
+        MTime(UNIX_EPOCH + Duration::new(secs, nanos))
+    }
+
+    /// Conversion intended for persistence.
+    pub fn to_seconds_and_nanos_for_persistence(self) -> Option<(u64, u32)> {
+        self.0
+            .duration_since(UNIX_EPOCH)
+            .ok()
+            .map(|duration| (duration.as_secs(), duration.subsec_nanos()))
+    }
+
+    /// Returns the value wrapped by this `MTime`, for presentation to the user. The name including
+    /// "_for_user" is to discourage misuse - this method should not be used when making decisions
+    /// about file dirtiness.
+    pub fn timestamp_for_user(self) -> SystemTime {
+        self.0
+    }
+
+    /// Temporary method to split out the behavior changes from introduction of this newtype.
+    pub fn bad_is_greater_than(self, other: MTime) -> bool {
+        self.0 > other.0
+    }
+}
+
+impl From<proto::Timestamp> for MTime {
+    fn from(timestamp: proto::Timestamp) -> Self {
+        MTime(timestamp.into())
+    }
+}
+
+impl From<MTime> for proto::Timestamp {
+    fn from(mtime: MTime) -> Self {
+        mtime.0.into()
+    }
+}
+
 #[derive(Default)]
 pub struct RealFs {
     git_hosting_provider_registry: Arc<GitHostingProviderRegistry>,
@@ -558,7 +608,7 @@ impl Fs for RealFs {
 
         Ok(Some(Metadata {
             inode,
-            mtime: metadata.modified().unwrap(),
+            mtime: MTime(metadata.modified().unwrap()),
             len: metadata.len(),
             is_symlink,
             is_dir: metadata.file_type().is_dir(),
@@ -818,13 +868,13 @@ struct FakeFsState {
 enum FakeFsEntry {
     File {
         inode: u64,
-        mtime: SystemTime,
+        mtime: MTime,
         len: u64,
         content: Vec<u8>,
     },
     Dir {
         inode: u64,
-        mtime: SystemTime,
+        mtime: MTime,
         len: u64,
         entries: BTreeMap<String, Arc<Mutex<FakeFsEntry>>>,
         git_repo_state: Option<Arc<Mutex<git::repository::FakeGitRepositoryState>>>,
@@ -836,6 +886,18 @@ enum FakeFsEntry {
 
 #[cfg(any(test, feature = "test-support"))]
 impl FakeFsState {
+    fn get_and_increment_mtime(&mut self) -> MTime {
+        let mtime = self.next_mtime;
+        self.next_mtime += FakeFs::SYSTEMTIME_INTERVAL;
+        MTime(mtime)
+    }
+
+    fn get_and_increment_inode(&mut self) -> u64 {
+        let inode = self.next_inode;
+        self.next_inode += 1;
+        inode
+    }
+
     fn read_path(&self, target: &Path) -> Result<Arc<Mutex<FakeFsEntry>>> {
         Ok(self
             .try_read_path(target, true)
@@ -959,7 +1021,7 @@ pub static FS_DOT_GIT: std::sync::LazyLock<&'static OsStr> =
 impl FakeFs {
     /// We need to use something large enough for Windows and Unix to consider this a new file.
     /// https://doc.rust-lang.org/nightly/std/time/struct.SystemTime.html#platform-specific-behavior
-    const SYSTEMTIME_INTERVAL: u64 = 100;
+    const SYSTEMTIME_INTERVAL: Duration = Duration::from_nanos(100);
 
     pub fn new(executor: gpui::BackgroundExecutor) -> Arc<Self> {
         let (tx, mut rx) = smol::channel::bounded::<PathBuf>(10);
@@ -969,13 +1031,13 @@ impl FakeFs {
             state: Mutex::new(FakeFsState {
                 root: Arc::new(Mutex::new(FakeFsEntry::Dir {
                     inode: 0,
-                    mtime: SystemTime::UNIX_EPOCH,
+                    mtime: MTime(UNIX_EPOCH),
                     len: 0,
                     entries: Default::default(),
                     git_repo_state: None,
                 })),
                 git_event_tx: tx,
-                next_mtime: SystemTime::UNIX_EPOCH,
+                next_mtime: UNIX_EPOCH + Self::SYSTEMTIME_INTERVAL,
                 next_inode: 1,
                 event_txs: Default::default(),
                 buffered_events: Vec::new(),
@@ -1007,13 +1069,16 @@ impl FakeFs {
         state.next_mtime = next_mtime;
     }
 
+    pub fn get_and_increment_mtime(&self) -> MTime {
+        let mut state = self.state.lock();
+        state.get_and_increment_mtime()
+    }
+
     pub async fn touch_path(&self, path: impl AsRef<Path>) {
         let mut state = self.state.lock();
         let path = path.as_ref();
-        let new_mtime = state.next_mtime;
-        let new_inode = state.next_inode;
-        state.next_inode += 1;
-        state.next_mtime += Duration::from_nanos(Self::SYSTEMTIME_INTERVAL);
+        let new_mtime = state.get_and_increment_mtime();
+        let new_inode = state.get_and_increment_inode();
         state
             .write_path(path, move |entry| {
                 match entry {
@@ -1062,19 +1127,14 @@ impl FakeFs {
 
     fn write_file_internal(&self, path: impl AsRef<Path>, content: Vec<u8>) -> Result<()> {
         let mut state = self.state.lock();
-        let path = path.as_ref();
-        let inode = state.next_inode;
-        let mtime = state.next_mtime;
-        state.next_inode += 1;
-        state.next_mtime += Duration::from_nanos(Self::SYSTEMTIME_INTERVAL);
         let file = Arc::new(Mutex::new(FakeFsEntry::File {
-            inode,
-            mtime,
+            inode: state.get_and_increment_inode(),
+            mtime: state.get_and_increment_mtime(),
             len: content.len() as u64,
             content,
         }));
         let mut kind = None;
-        state.write_path(path, {
+        state.write_path(path.as_ref(), {
             let kind = &mut kind;
             move |entry| {
                 match entry {
@@ -1090,7 +1150,7 @@ impl FakeFs {
                 Ok(())
             }
         })?;
-        state.emit_event([(path, kind)]);
+        state.emit_event([(path.as_ref(), kind)]);
         Ok(())
     }
 
@@ -1383,16 +1443,6 @@ impl FakeFsEntry {
         }
     }
 
-    fn set_file_content(&mut self, path: &Path, new_content: Vec<u8>) -> Result<()> {
-        if let Self::File { content, mtime, .. } = self {
-            *mtime = SystemTime::now();
-            *content = new_content;
-            Ok(())
-        } else {
-            Err(anyhow!("not a file: {}", path.display()))
-        }
-    }
-
     fn dir_entries(
         &mut self,
         path: &Path,
@@ -1456,10 +1506,8 @@ impl Fs for FakeFs {
             }
             let mut state = self.state.lock();
 
-            let inode = state.next_inode;
-            let mtime = state.next_mtime;
-            state.next_mtime += Duration::from_nanos(Self::SYSTEMTIME_INTERVAL);
-            state.next_inode += 1;
+            let inode = state.get_and_increment_inode();
+            let mtime = state.get_and_increment_mtime();
             state.write_path(&cur_path, |entry| {
                 entry.or_insert_with(|| {
                     created_dirs.push((cur_path.clone(), Some(PathEventKind::Created)));
@@ -1482,10 +1530,8 @@ impl Fs for FakeFs {
     async fn create_file(&self, path: &Path, options: CreateOptions) -> Result<()> {
         self.simulate_random_delay().await;
         let mut state = self.state.lock();
-        let inode = state.next_inode;
-        let mtime = state.next_mtime;
-        state.next_mtime += Duration::from_nanos(Self::SYSTEMTIME_INTERVAL);
-        state.next_inode += 1;
+        let inode = state.get_and_increment_inode();
+        let mtime = state.get_and_increment_mtime();
         let file = Arc::new(Mutex::new(FakeFsEntry::File {
             inode,
             mtime,
@@ -1625,13 +1671,12 @@ impl Fs for FakeFs {
         let source = normalize_path(source);
         let target = normalize_path(target);
         let mut state = self.state.lock();
-        let mtime = state.next_mtime;
-        let inode = util::post_inc(&mut state.next_inode);
-        state.next_mtime += Duration::from_nanos(Self::SYSTEMTIME_INTERVAL);
+        let mtime = state.get_and_increment_mtime();
+        let inode = state.get_and_increment_inode();
         let source_entry = state.read_path(&source)?;
         let content = source_entry.lock().file_content(&source)?.clone();
         let mut kind = Some(PathEventKind::Created);
-        let entry = state.write_path(&target, |e| match e {
+        state.write_path(&target, |e| match e {
             btree_map::Entry::Occupied(e) => {
                 if options.overwrite {
                     kind = Some(PathEventKind::Changed);
@@ -1647,14 +1692,11 @@ impl Fs for FakeFs {
                     inode,
                     mtime,
                     len: content.len() as u64,
-                    content: Vec::new(),
+                    content,
                 })))
                 .clone(),
             )),
         })?;
-        if let Some(entry) = entry {
-            entry.lock().set_file_content(&target, content)?;
-        }
         state.emit_event([(target, kind)]);
         Ok(())
     }

crates/language/Cargo.toml 🔗

@@ -31,6 +31,7 @@ async-watch.workspace = true
 clock.workspace = true
 collections.workspace = true
 ec4rs.workspace = true
+fs.workspace = true
 futures.workspace = true
 fuzzy.workspace = true
 git.workspace = true

crates/language/src/buffer.rs 🔗

@@ -21,6 +21,7 @@ use async_watch as watch;
 use clock::Lamport;
 pub use clock::ReplicaId;
 use collections::HashMap;
+use fs::MTime;
 use futures::channel::oneshot;
 use gpui::{
     AnyElement, AppContext, Context as _, EventEmitter, HighlightStyle, Model, ModelContext,
@@ -51,7 +52,7 @@ use std::{
     path::{Path, PathBuf},
     str,
     sync::{Arc, LazyLock},
-    time::{Duration, Instant, SystemTime},
+    time::{Duration, Instant},
     vec,
 };
 use sum_tree::TreeMap;
@@ -108,7 +109,7 @@ pub struct Buffer {
     file: Option<Arc<dyn File>>,
     /// The mtime of the file when this buffer was last loaded from
     /// or saved to disk.
-    saved_mtime: Option<SystemTime>,
+    saved_mtime: Option<MTime>,
     /// The version vector when this buffer was last loaded from
     /// or saved to disk.
     saved_version: clock::Global,
@@ -406,22 +407,19 @@ pub trait File: Send + Sync {
 /// modified. In the case where the file is not stored, it can be either `New` or `Deleted`. In the
 /// UI these two states are distinguished. For example, the buffer tab does not display a deletion
 /// indicator for new files.
-#[derive(Clone, Copy, Debug, PartialEq)]
+#[derive(Copy, Clone, Debug, PartialEq)]
 pub enum DiskState {
     /// File created in Zed that has not been saved.
     New,
     /// File present on the filesystem.
-    Present {
-        /// Last known mtime (modification time).
-        mtime: SystemTime,
-    },
+    Present { mtime: MTime },
     /// Deleted file that was previously present.
     Deleted,
 }
 
 impl DiskState {
     /// Returns the file's last known modification time on disk.
-    pub fn mtime(self) -> Option<SystemTime> {
+    pub fn mtime(self) -> Option<MTime> {
         match self {
             DiskState::New => None,
             DiskState::Present { mtime } => Some(mtime),
@@ -976,7 +974,7 @@ impl Buffer {
     }
 
     /// The mtime of the buffer's file when the buffer was last saved or reloaded from disk.
-    pub fn saved_mtime(&self) -> Option<SystemTime> {
+    pub fn saved_mtime(&self) -> Option<MTime> {
         self.saved_mtime
     }
 
@@ -1011,7 +1009,7 @@ impl Buffer {
     pub fn did_save(
         &mut self,
         version: clock::Global,
-        mtime: Option<SystemTime>,
+        mtime: Option<MTime>,
         cx: &mut ModelContext<Self>,
     ) {
         self.saved_version = version;
@@ -1077,7 +1075,7 @@ impl Buffer {
         &mut self,
         version: clock::Global,
         line_ending: LineEnding,
-        mtime: Option<SystemTime>,
+        mtime: Option<MTime>,
         cx: &mut ModelContext<Self>,
     ) {
         self.saved_version = version;
@@ -1777,7 +1775,9 @@ impl Buffer {
         match file.disk_state() {
             DiskState::New => false,
             DiskState::Present { mtime } => match self.saved_mtime {
-                Some(saved_mtime) => mtime > saved_mtime && self.has_unsaved_edits(),
+                Some(saved_mtime) => {
+                    mtime.bad_is_greater_than(saved_mtime) && self.has_unsaved_edits()
+                }
                 None => true,
             },
             DiskState::Deleted => true,

crates/semantic_index/src/embedding_index.rs 🔗

@@ -7,6 +7,7 @@ use anyhow::{anyhow, Context as _, Result};
 use collections::Bound;
 use feature_flags::FeatureFlagAppExt;
 use fs::Fs;
+use fs::MTime;
 use futures::stream::StreamExt;
 use futures_batch::ChunksTimeoutStreamExt;
 use gpui::{AppContext, Model, Task};
@@ -17,14 +18,7 @@ use project::{Entry, UpdatedEntriesSet, Worktree};
 use serde::{Deserialize, Serialize};
 use smol::channel;
 use smol::future::FutureExt;
-use std::{
-    cmp::Ordering,
-    future::Future,
-    iter,
-    path::Path,
-    sync::Arc,
-    time::{Duration, SystemTime},
-};
+use std::{cmp::Ordering, future::Future, iter, path::Path, sync::Arc, time::Duration};
 use util::ResultExt;
 use worktree::Snapshot;
 
@@ -451,7 +445,7 @@ struct ChunkFiles {
 
 pub struct ChunkedFile {
     pub path: Arc<Path>,
-    pub mtime: Option<SystemTime>,
+    pub mtime: Option<MTime>,
     pub handle: IndexingEntryHandle,
     pub text: String,
     pub chunks: Vec<Chunk>,
@@ -465,7 +459,7 @@ pub struct EmbedFiles {
 #[derive(Debug, Serialize, Deserialize)]
 pub struct EmbeddedFile {
     pub path: Arc<Path>,
-    pub mtime: Option<SystemTime>,
+    pub mtime: Option<MTime>,
     pub chunks: Vec<EmbeddedChunk>,
 }
 

crates/semantic_index/src/summary_backlog.rs 🔗

@@ -1,5 +1,6 @@
 use collections::HashMap;
-use std::{path::Path, sync::Arc, time::SystemTime};
+use fs::MTime;
+use std::{path::Path, sync::Arc};
 
 const MAX_FILES_BEFORE_RESUMMARIZE: usize = 4;
 const MAX_BYTES_BEFORE_RESUMMARIZE: u64 = 1_000_000; // 1 MB
@@ -7,14 +8,14 @@ const MAX_BYTES_BEFORE_RESUMMARIZE: u64 = 1_000_000; // 1 MB
 #[derive(Default, Debug)]
 pub struct SummaryBacklog {
     /// Key: path to a file that needs summarization, but that we haven't summarized yet. Value: that file's size on disk, in bytes, and its mtime.
-    files: HashMap<Arc<Path>, (u64, Option<SystemTime>)>,
+    files: HashMap<Arc<Path>, (u64, Option<MTime>)>,
     /// Cache of the sum of all values in `files`, so we don't have to traverse the whole map to check if we're over the byte limit.
     total_bytes: u64,
 }
 
 impl SummaryBacklog {
     /// Store the given path in the backlog, along with how many bytes are in it.
-    pub fn insert(&mut self, path: Arc<Path>, bytes_on_disk: u64, mtime: Option<SystemTime>) {
+    pub fn insert(&mut self, path: Arc<Path>, bytes_on_disk: u64, mtime: Option<MTime>) {
         let (prev_bytes, _) = self
             .files
             .insert(path, (bytes_on_disk, mtime))
@@ -34,7 +35,7 @@ impl SummaryBacklog {
 
     /// Remove all the entries in the backlog and return the file paths as an iterator.
     #[allow(clippy::needless_lifetimes)] // Clippy thinks this 'a can be elided, but eliding it gives a compile error
-    pub fn drain<'a>(&'a mut self) -> impl Iterator<Item = (Arc<Path>, Option<SystemTime>)> + 'a {
+    pub fn drain<'a>(&'a mut self) -> impl Iterator<Item = (Arc<Path>, Option<MTime>)> + 'a {
         self.total_bytes = 0;
 
         self.files

crates/semantic_index/src/summary_index.rs 🔗

@@ -1,6 +1,6 @@
 use anyhow::{anyhow, Context as _, Result};
 use arrayvec::ArrayString;
-use fs::Fs;
+use fs::{Fs, MTime};
 use futures::{stream::StreamExt, TryFutureExt};
 use futures_batch::ChunksTimeoutStreamExt;
 use gpui::{AppContext, Model, Task};
@@ -21,7 +21,7 @@ use std::{
     future::Future,
     path::Path,
     sync::Arc,
-    time::{Duration, Instant, SystemTime},
+    time::{Duration, Instant},
 };
 use util::ResultExt;
 use worktree::Snapshot;
@@ -39,7 +39,7 @@ struct UnsummarizedFile {
     // Path to the file on disk
     path: Arc<Path>,
     // The mtime of the file on disk
-    mtime: Option<SystemTime>,
+    mtime: Option<MTime>,
     // BLAKE3 hash of the source file's contents
     digest: Blake3Digest,
     // The source file's contents
@@ -51,7 +51,7 @@ struct SummarizedFile {
     // Path to the file on disk
     path: String,
     // The mtime of the file on disk
-    mtime: Option<SystemTime>,
+    mtime: Option<MTime>,
     // BLAKE3 hash of the source file's contents
     digest: Blake3Digest,
     // The LLM's summary of the file's contents
@@ -63,7 +63,7 @@ pub type Blake3Digest = ArrayString<{ blake3::OUT_LEN * 2 }>;
 
 #[derive(Debug, Serialize, Deserialize)]
 pub struct FileDigest {
-    pub mtime: Option<SystemTime>,
+    pub mtime: Option<MTime>,
     pub digest: Blake3Digest,
 }
 
@@ -88,7 +88,7 @@ pub struct SummaryIndex {
 }
 
 struct Backlogged {
-    paths_to_digest: channel::Receiver<Vec<(Arc<Path>, Option<SystemTime>)>>,
+    paths_to_digest: channel::Receiver<Vec<(Arc<Path>, Option<MTime>)>>,
     task: Task<Result<()>>,
 }
 
@@ -319,7 +319,7 @@ impl SummaryIndex {
         digest_db: heed::Database<Str, SerdeBincode<FileDigest>>,
         txn: &RoTxn<'_>,
         entry: &Entry,
-    ) -> Vec<(Arc<Path>, Option<SystemTime>)> {
+    ) -> Vec<(Arc<Path>, Option<MTime>)> {
         let entry_db_key = db_key_for_path(&entry.path);
 
         match digest_db.get(&txn, &entry_db_key) {
@@ -414,7 +414,7 @@ impl SummaryIndex {
 
     fn digest_files(
         &self,
-        paths: channel::Receiver<Vec<(Arc<Path>, Option<SystemTime>)>>,
+        paths: channel::Receiver<Vec<(Arc<Path>, Option<MTime>)>>,
         worktree_abs_path: Arc<Path>,
         cx: &AppContext,
     ) -> MightNeedSummaryFiles {
@@ -646,7 +646,7 @@ impl SummaryIndex {
         let start = Instant::now();
         let backlogged = {
             let (tx, rx) = channel::bounded(512);
-            let needs_summary: Vec<(Arc<Path>, Option<SystemTime>)> = {
+            let needs_summary: Vec<(Arc<Path>, Option<MTime>)> = {
                 let mut backlog = self.backlog.lock();
 
                 backlog.drain().collect()

crates/worktree/src/worktree.rs 🔗

@@ -7,7 +7,7 @@ use ::ignore::gitignore::{Gitignore, GitignoreBuilder};
 use anyhow::{anyhow, Context as _, Result};
 use clock::ReplicaId;
 use collections::{HashMap, HashSet, VecDeque};
-use fs::{copy_recursive, Fs, PathEvent, RemoveOptions, Watcher};
+use fs::{copy_recursive, Fs, MTime, PathEvent, RemoveOptions, Watcher};
 use futures::{
     channel::{
         mpsc::{self, UnboundedSender},
@@ -61,7 +61,7 @@ use std::{
         atomic::{AtomicUsize, Ordering::SeqCst},
         Arc,
     },
-    time::{Duration, Instant, SystemTime},
+    time::{Duration, Instant},
 };
 use sum_tree::{Bias, Edit, SeekTarget, SumTree, TreeMap, TreeSet};
 use text::{LineEnding, Rope};
@@ -3395,7 +3395,7 @@ pub struct Entry {
     pub kind: EntryKind,
     pub path: Arc<Path>,
     pub inode: u64,
-    pub mtime: Option<SystemTime>,
+    pub mtime: Option<MTime>,
 
     pub canonical_path: Option<Box<Path>>,
     /// Whether this entry is ignored by Git.