backend: add error variant for access denied, handle when diffing

Some backends, like the one we have at Google, can restrict access to
certain files. For such files, if they return a regular
`BackendError::ReadObject`, then that will terminate iteration in many
cases (e.g. when diffing or listing files). This patch adds a new
error variant for them to return instead, plus handling of such errors
in diff output and in the working copy.

In order to test the feature, I added a new commit backend that
returns the new `ReadAccessDenied` error when the caller tries to read
certain objects.
This commit is contained in:
Martin von Zweigbergk 2024-03-19 10:56:31 -07:00 committed by Martin von Zweigbergk
parent fccba76e8b
commit 404f31cbc1
16 changed files with 445 additions and 7 deletions

View file

@ -365,6 +365,7 @@ impl From<DiffRenderError> for CommandError {
match err {
DiffRenderError::DiffGenerate(_) => user_error(err),
DiffRenderError::Backend(err) => err.into(),
DiffRenderError::AccessDenied { .. } => user_error(err),
DiffRenderError::Io(err) => err.into(),
}
}

View file

@ -104,6 +104,13 @@ fn write_tree_entries<P: AsRef<RepoPath>>(
let materialized = materialize_tree_value(repo.store(), path.as_ref(), value).block_on()?;
match materialized {
MaterializedTreeValue::Absent => panic!("absent values should be excluded"),
MaterializedTreeValue::AccessDenied(err) => {
let ui_path = workspace_command.format_file_path(path.as_ref());
writeln!(
ui.warning_default(),
"Path '{ui_path}' exists but access is denied: {err}"
)?;
}
MaterializedTreeValue::File { mut reader, .. } => {
io::copy(&mut reader, &mut ui.stdout_formatter().as_mut())?;
}

View file

@ -81,6 +81,7 @@
"diff removed" = "red"
"diff added" = "green"
"diff modified" = "cyan"
"diff access-denied" = { bg = "red" }
"op_log id" = "blue"
"op_log user" = "yellow"

View file

@ -195,6 +195,11 @@ pub enum DiffRenderError {
DiffGenerate(#[source] DiffGenerateError),
#[error(transparent)]
Backend(#[from] BackendError),
#[error("Access denied to {path}: {source}")]
AccessDenied {
path: String,
source: Box<dyn std::error::Error + Send + Sync>,
},
#[error(transparent)]
Io(#[from] io::Error),
}
@ -437,6 +442,10 @@ fn file_content_for_diff(reader: &mut dyn io::Read) -> io::Result<FileContent> {
fn diff_content(path: &RepoPath, value: MaterializedTreeValue) -> io::Result<FileContent> {
match value {
MaterializedTreeValue::Absent => Ok(FileContent::empty()),
MaterializedTreeValue::AccessDenied(err) => Ok(FileContent {
is_binary: false,
contents: format!("Access denied: {err}").into_bytes(),
}),
MaterializedTreeValue::File { mut reader, .. } => {
file_content_for_diff(&mut reader).map_err(Into::into)
}
@ -469,6 +478,7 @@ fn basic_diff_file_type(value: &MaterializedTreeValue) -> &'static str {
MaterializedTreeValue::Absent => {
panic!("absent path in diff");
}
MaterializedTreeValue::AccessDenied(_) => "access denied",
MaterializedTreeValue::File { executable, .. } => {
if *executable {
"executable file"
@ -496,6 +506,19 @@ pub fn show_color_words_diff(
while let Some((path, diff)) = diff_stream.next().await {
let ui_path = path_converter.format_file_path(&path);
let (left_value, right_value) = diff?;
match (&left_value, &right_value) {
(_, MaterializedTreeValue::AccessDenied(source))
| (MaterializedTreeValue::AccessDenied(source), _) => {
write!(
formatter.labeled("access-denied"),
"Access denied to {ui_path}:"
)?;
writeln!(formatter, " {source}")?;
continue;
}
_ => {}
}
if left_value.is_absent() {
let description = basic_diff_file_type(&right_value);
writeln!(
@ -610,7 +633,10 @@ struct GitDiffPart {
content: Vec<u8>,
}
fn git_diff_part(path: &RepoPath, value: MaterializedTreeValue) -> io::Result<GitDiffPart> {
fn git_diff_part(
path: &RepoPath,
value: MaterializedTreeValue,
) -> Result<GitDiffPart, DiffRenderError> {
let mode;
let hash;
let mut contents: Vec<u8>;
@ -618,6 +644,12 @@ fn git_diff_part(path: &RepoPath, value: MaterializedTreeValue) -> io::Result<Gi
MaterializedTreeValue::Absent => {
panic!("Absent path {path:?} in diff should have been handled by caller");
}
MaterializedTreeValue::AccessDenied(err) => {
return Err(DiffRenderError::AccessDenied {
path: path.as_internal_file_string().to_owned(),
source: err,
});
}
MaterializedTreeValue::File {
id,
executable,

View file

@ -137,6 +137,14 @@ fn read_file_contents(
file_mode: scm_record::FileMode::absent(),
contents: FileContents::Absent,
}),
MaterializedTreeValue::AccessDenied(err) => Ok(FileInfo {
file_mode: scm_record::FileMode(mode::NORMAL),
contents: FileContents::Text {
contents: format!("Access denied: {err}"),
hash: None,
num_bytes: 0,
},
}),
MaterializedTreeValue::File {
id,

View file

@ -9,6 +9,7 @@ fn test_no_forgotten_test_files() {
}
mod test_abandon_command;
mod test_acls;
mod test_advance_branches;
mod test_alias;
mod test_branch_command;

121
cli/tests/test_acls.rs Normal file
View file

@ -0,0 +1,121 @@
// Copyright 2024 The Jujutsu Authors
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// https://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use jj_lib::secret_backend::SecretBackend;
use crate::common::{get_stderr_string, get_stdout_string, TestEnvironment};
#[test]
fn test_diff() {
let test_env = TestEnvironment::default();
test_env.jj_cmd_ok(test_env.env_root(), &["git", "init", "repo"]);
let repo_path = test_env.env_root().join("repo");
std::fs::create_dir(repo_path.join("dir")).unwrap();
std::fs::write(repo_path.join("a-first"), "foo\n").unwrap();
std::fs::write(repo_path.join("deleted-secret"), "foo\n").unwrap();
std::fs::write(repo_path.join("dir").join("secret"), "foo\n").unwrap();
std::fs::write(repo_path.join("modified-secret"), "foo\n").unwrap();
std::fs::write(repo_path.join("z-last"), "foo\n").unwrap();
test_env.jj_cmd_ok(&repo_path, &["new"]);
std::fs::write(repo_path.join("a-first"), "bar\n").unwrap();
std::fs::remove_file(repo_path.join("deleted-secret")).unwrap();
std::fs::write(repo_path.join("added-secret"), "bar\n").unwrap();
std::fs::write(repo_path.join("dir").join("secret"), "bar\n").unwrap();
std::fs::write(repo_path.join("modified-secret"), "bar\n").unwrap();
std::fs::write(repo_path.join("z-last"), "bar\n").unwrap();
SecretBackend::adopt_git_repo(&repo_path);
let stdout = test_env.jj_cmd_success(&repo_path, &["diff", "--color-words"]);
insta::assert_snapshot!(stdout.replace('\\', "/"), @r###"
Modified regular file a-first:
1 1: foobar
Access denied to added-secret: No access
Access denied to deleted-secret: No access
Access denied to dir/secret: No access
Access denied to modified-secret: No access
Modified regular file z-last:
1 1: foobar
"###);
let stdout = test_env.jj_cmd_success(&repo_path, &["diff", "--summary"]);
insta::assert_snapshot!(stdout.replace('\\', "/"), @r###"
M a-first
A added-secret
D deleted-secret
M dir/secret
M modified-secret
M z-last
"###);
let stdout = test_env.jj_cmd_success(&repo_path, &["diff", "--types"]);
insta::assert_snapshot!(stdout.replace('\\', "/"), @r###"
FF a-first
-F added-secret
F- deleted-secret
FF dir/secret
FF modified-secret
FF z-last
"###);
let stdout = test_env.jj_cmd_success(&repo_path, &["diff", "--stat"]);
insta::assert_snapshot!(stdout.replace('\\', "/"), @r###"
a-first | 2 +-
added-secret | 1 +
deleted-secret | 1 -
dir/secret | 0
modified-secret | 0
z-last | 2 +-
6 files changed, 3 insertions(+), 3 deletions(-)
"###);
let assert = test_env
.jj_cmd(&repo_path, &["diff", "--git"])
.assert()
.failure();
insta::assert_snapshot!(get_stdout_string(&assert).replace('\\', "/"), @r###"
diff --git a/a-first b/a-first
index 257cc5642c...5716ca5987 100644
--- a/a-first
+++ b/a-first
@@ -1,1 +1,1 @@
-foo
+bar
"###);
insta::assert_snapshot!(get_stderr_string(&assert), @r###"
Error: Access denied to added-secret: No access
Caused by: No access
"###);
// TODO: Test external tool
}
#[test]
fn test_cat() {
let test_env = TestEnvironment::default();
test_env.jj_cmd_ok(test_env.env_root(), &["git", "init", "repo"]);
let repo_path = test_env.env_root().join("repo");
std::fs::write(repo_path.join("a-first"), "foo\n").unwrap();
std::fs::write(repo_path.join("secret"), "bar\n").unwrap();
std::fs::write(repo_path.join("z-last"), "baz\n").unwrap();
SecretBackend::adopt_git_repo(&repo_path);
let (stdout, stderr) = test_env.jj_cmd_ok(&repo_path, &["cat", "."]);
insta::assert_snapshot!(stdout.replace('\\', "/"), @r###"
foo
baz
"###);
insta::assert_snapshot!(stderr.replace('\\', "/"), @r###"
Warning: Path 'secret' exists but access is denied: No access
"###);
}

View file

@ -193,6 +193,12 @@ pub enum BackendError {
hash: String,
source: Box<dyn std::error::Error + Send + Sync>,
},
#[error("Access denied to read object {hash} of type {object_type}")]
ReadAccessDenied {
object_type: String,
hash: String,
source: Box<dyn std::error::Error + Send + Sync>,
},
#[error("Could not write object of type {object_type}")]
WriteObject {
object_type: &'static str,

View file

@ -21,7 +21,7 @@ use futures::StreamExt;
use itertools::Itertools;
use regex::bytes::Regex;
use crate::backend::{BackendResult, CommitId, FileId, SymlinkId, TreeId, TreeValue};
use crate::backend::{BackendError, BackendResult, CommitId, FileId, SymlinkId, TreeId, TreeValue};
use crate::diff::{find_line_ranges, Diff, DiffHunk};
use crate::files;
use crate::files::{ContentHunk, MergeResult};
@ -127,6 +127,7 @@ pub async fn materialize(
/// e.g. the working copy or in a diff.
pub enum MaterializedTreeValue {
Absent,
AccessDenied(Box<dyn std::error::Error + Send + Sync>),
File {
id: FileId,
executable: bool,
@ -161,6 +162,19 @@ pub async fn materialize_tree_value(
store: &Store,
path: &RepoPath,
value: MergedTreeValue,
) -> BackendResult<MaterializedTreeValue> {
match materialize_tree_value_no_access_denied(store, path, value).await {
Err(BackendError::ReadAccessDenied { source, .. }) => {
Ok(MaterializedTreeValue::AccessDenied(source))
}
result => result,
}
}
async fn materialize_tree_value_no_access_denied(
store: &Store,
path: &RepoPath,
value: MergedTreeValue,
) -> BackendResult<MaterializedTreeValue> {
match value.into_resolved() {
Ok(None) => Ok(MaterializedTreeValue::Absent),

View file

@ -71,6 +71,8 @@ pub mod repo_path;
pub mod revset;
mod revset_parser;
pub mod rewrite;
#[cfg(feature = "testing")]
pub mod secret_backend;
pub mod settings;
pub mod signing;
pub mod simple_op_heads_store;

View file

@ -835,7 +835,7 @@ impl TreeState {
let tree = self.current_tree().unwrap();
let tree_paths: HashSet<_> = tree
.entries_matching(sparse_matcher.as_ref())
.map(|(path, _)| path)
.filter_map(|(path, result)| result.is_ok().then_some(path))
.collect();
let file_states = self.file_states.all();
let state_paths: HashSet<_> = file_states.paths().map(|path| path.to_owned()).collect();
@ -1368,7 +1368,7 @@ impl TreeState {
}
// TODO: Check that the file has not changed before overwriting/removing it.
let file_state = match after {
MaterializedTreeValue::Absent => {
MaterializedTreeValue::Absent | MaterializedTreeValue::AccessDenied(_) => {
let mut parent_dir = disk_path.parent().unwrap();
loop {
if fs::remove_dir(parent_dir).is_err() {

View file

@ -383,6 +383,15 @@ impl Default for StoreFactories {
GitBackend::name(),
Box::new(|settings, store_path| Ok(Box::new(GitBackend::load(settings, store_path)?))),
);
#[cfg(feature = "testing")]
factories.add_backend(
crate::secret_backend::SecretBackend::name(),
Box::new(|settings, store_path| {
Ok(Box::new(crate::secret_backend::SecretBackend::load(
settings, store_path,
)?))
}),
);
// OpStores
factories.add_op_store(

173
lib/src/secret_backend.rs Normal file
View file

@ -0,0 +1,173 @@
// Copyright 2024 The Jujutsu Authors
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// https://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
//! Provides a backend for testing ACLs
use std::any::Any;
use std::io::Read;
use std::path::Path;
use std::time::SystemTime;
use async_trait::async_trait;
use crate::backend::{
Backend, BackendError, BackendLoadError, BackendResult, ChangeId, Commit, CommitId, Conflict,
ConflictId, FileId, SigningFn, SymlinkId, Tree, TreeId,
};
use crate::git_backend::GitBackend;
use crate::index::Index;
use crate::object_id::ObjectId;
use crate::repo_path::RepoPath;
use crate::settings::UserSettings;
const SECRET_CONTENTS_HEX: [&str; 2] = [
"d97c5eada5d8c52079031eef0107a4430a9617c5", // "secret\n"
"536aca34dbae6b2b8af26bebdcba83543c9546f0", // "secret"
];
/// A commit backend that's completely compatible with the Git backend, except
/// that it refuses to read files and symlinks with the word "secret" in the
/// path, or "secret" or "secret\n" in the content.
#[derive(Debug)]
pub struct SecretBackend {
inner: GitBackend,
}
impl SecretBackend {
/// "secret"
pub fn name() -> &'static str {
"secret"
}
/// Loads the backend from the given path.
pub fn load(settings: &UserSettings, store_path: &Path) -> Result<Self, BackendLoadError> {
let inner = GitBackend::load(settings, store_path)?;
Ok(SecretBackend { inner })
}
/// Convert a git repo to using `SecretBackend`
// TODO: Avoid this hack
pub fn adopt_git_repo(workspace_path: &Path) {
std::fs::write(
workspace_path
.join(".jj")
.join("repo")
.join("store")
.join("type"),
Self::name(),
)
.unwrap();
}
}
#[async_trait]
impl Backend for SecretBackend {
fn as_any(&self) -> &dyn Any {
self
}
fn name(&self) -> &str {
SecretBackend::name()
}
fn commit_id_length(&self) -> usize {
self.inner.commit_id_length()
}
fn change_id_length(&self) -> usize {
self.inner.change_id_length()
}
fn root_commit_id(&self) -> &CommitId {
self.inner.root_commit_id()
}
fn root_change_id(&self) -> &ChangeId {
self.inner.root_change_id()
}
fn empty_tree_id(&self) -> &TreeId {
self.inner.empty_tree_id()
}
fn concurrency(&self) -> usize {
1
}
async fn read_file(&self, path: &RepoPath, id: &FileId) -> BackendResult<Box<dyn Read>> {
if path.as_internal_file_string().contains("secret")
|| SECRET_CONTENTS_HEX.contains(&id.hex().as_ref())
{
return Err(BackendError::ReadAccessDenied {
object_type: "file".to_string(),
hash: id.hex(),
source: "No access".into(),
});
}
self.inner.read_file(path, id).await
}
fn write_file(&self, path: &RepoPath, contents: &mut dyn Read) -> BackendResult<FileId> {
self.inner.write_file(path, contents)
}
async fn read_symlink(&self, path: &RepoPath, id: &SymlinkId) -> BackendResult<String> {
if path.as_internal_file_string().contains("secret")
|| SECRET_CONTENTS_HEX.contains(&id.hex().as_ref())
{
return Err(BackendError::ReadAccessDenied {
object_type: "symlink".to_string(),
hash: id.hex(),
source: "No access".into(),
});
}
self.inner.read_symlink(path, id).await
}
fn write_symlink(&self, path: &RepoPath, target: &str) -> BackendResult<SymlinkId> {
self.inner.write_symlink(path, target)
}
async fn read_tree(&self, path: &RepoPath, id: &TreeId) -> BackendResult<Tree> {
self.inner.read_tree(path, id).await
}
fn write_tree(&self, path: &RepoPath, contents: &Tree) -> BackendResult<TreeId> {
self.inner.write_tree(path, contents)
}
fn read_conflict(&self, path: &RepoPath, id: &ConflictId) -> BackendResult<Conflict> {
self.inner.read_conflict(path, id)
}
fn write_conflict(&self, path: &RepoPath, contents: &Conflict) -> BackendResult<ConflictId> {
self.inner.write_conflict(path, contents)
}
async fn read_commit(&self, id: &CommitId) -> BackendResult<Commit> {
self.inner.read_commit(id).await
}
fn write_commit(
&self,
contents: Commit,
sign_with: Option<&mut SigningFn>,
) -> BackendResult<(CommitId, Commit)> {
self.inner.write_commit(contents, sign_with)
}
fn gc(&self, index: &dyn Index, keep_newer: SystemTime) -> BackendResult<()> {
self.inner.gc(index, keep_newer)
}
}

View file

@ -29,12 +29,13 @@ use jj_lib::merged_tree::{MergedTree, MergedTreeBuilder};
use jj_lib::op_store::{OperationId, WorkspaceId};
use jj_lib::repo::{ReadonlyRepo, Repo};
use jj_lib::repo_path::{RepoPath, RepoPathBuf, RepoPathComponent};
use jj_lib::secret_backend::SecretBackend;
use jj_lib::settings::UserSettings;
use jj_lib::working_copy::{CheckoutStats, SnapshotError, SnapshotOptions};
use jj_lib::workspace::LockedWorkspace;
use jj_lib::workspace::{default_working_copy_factories, LockedWorkspace, Workspace};
use test_case::test_case;
use testutils::{
commit_with_tree, create_tree, write_random_commit, TestRepoBackend, TestWorkspace,
commit_with_tree, create_tree, write_random_commit, TestRepo, TestRepoBackend, TestWorkspace,
};
fn to_owned_path_vec(paths: &[&RepoPath]) -> Vec<RepoPathBuf> {
@ -328,6 +329,61 @@ fn test_conflict_subdirectory() {
.unwrap();
}
#[test]
fn test_acl() {
let settings = testutils::user_settings();
let test_workspace = TestWorkspace::init_with_backend(&settings, TestRepoBackend::Git);
let repo = &test_workspace.repo;
let workspace_root = test_workspace.workspace.workspace_root().to_owned();
let secret_modified_path = RepoPath::from_internal_string("secret/modified");
let secret_added_path = RepoPath::from_internal_string("secret/added");
let secret_deleted_path = RepoPath::from_internal_string("secret/deleted");
let became_secret_path = RepoPath::from_internal_string("file1");
let became_public_path = RepoPath::from_internal_string("file2");
let tree1 = create_tree(
repo,
&[
(secret_modified_path, "0"),
(secret_deleted_path, "0"),
(became_secret_path, "public"),
(became_public_path, "secret"),
],
);
let tree2 = create_tree(
repo,
&[
(secret_modified_path, "1"),
(secret_added_path, "1"),
(became_secret_path, "secret"),
(became_public_path, "public"),
],
);
let commit1 = commit_with_tree(repo.store(), tree1.id());
let commit2 = commit_with_tree(repo.store(), tree2.id());
SecretBackend::adopt_git_repo(&workspace_root);
let mut ws = Workspace::load(
&settings,
&workspace_root,
&TestRepo::default_store_factories(),
&default_working_copy_factories(),
)
.unwrap();
ws.check_out(repo.op_id().clone(), None, &commit1).unwrap();
assert!(!secret_modified_path.to_fs_path(&workspace_root).is_file());
assert!(!secret_added_path.to_fs_path(&workspace_root).is_file());
assert!(!secret_deleted_path.to_fs_path(&workspace_root).is_file());
assert!(became_secret_path.to_fs_path(&workspace_root).is_file());
assert!(!became_public_path.to_fs_path(&workspace_root).is_file());
ws.check_out(repo.op_id().clone(), None, &commit2).unwrap();
assert!(!secret_modified_path.to_fs_path(&workspace_root).is_file());
assert!(!secret_added_path.to_fs_path(&workspace_root).is_file());
assert!(!secret_deleted_path.to_fs_path(&workspace_root).is_file());
assert!(!became_secret_path.to_fs_path(&workspace_root).is_file());
assert!(became_public_path.to_fs_path(&workspace_root).is_file());
}
#[test]
fn test_tree_builder_file_directory_transition() {
let settings = testutils::user_settings();

View file

@ -20,6 +20,6 @@ config = { workspace = true }
git2 = { workspace = true }
hex = { workspace = true }
itertools = { workspace = true }
jj-lib = { workspace = true }
jj-lib = { workspace = true, features = ["testing"] }
rand = { workspace = true }
tempfile = { workspace = true }

View file

@ -32,6 +32,7 @@ use jj_lib::merged_tree::MergedTree;
use jj_lib::object_id::ObjectId;
use jj_lib::repo::{MutableRepo, ReadonlyRepo, Repo, RepoLoader, StoreFactories};
use jj_lib::repo_path::{RepoPath, RepoPathBuf};
use jj_lib::secret_backend::SecretBackend;
use jj_lib::settings::UserSettings;
use jj_lib::signing::Signer;
use jj_lib::store::Store;
@ -172,6 +173,12 @@ impl TestRepo {
"test",
Box::new(|_settings, store_path| Ok(Box::new(TestBackend::load(store_path)))),
);
factories.add_backend(
SecretBackend::name(),
Box::new(|settings, store_path| {
Ok(Box::new(SecretBackend::load(settings, store_path)?))
}),
);
factories
}
}