sign: Define signing backend API and integrate it

Finished everything except actual signing backend implementation(s) and
the UI.
This commit is contained in:
Anton Bulakh 2023-11-24 23:08:16 +02:00 committed by Anton Bulakh
parent 076b49b610
commit d7229a3f90
15 changed files with 442 additions and 19 deletions

View file

@ -57,6 +57,7 @@ use jj_lib::revset::{
};
use jj_lib::rewrite::restore_tree;
use jj_lib::settings::{ConfigResultExt as _, UserSettings};
use jj_lib::signing::SignInitError;
use jj_lib::str_util::{StringPattern, StringPatternParseError};
use jj_lib::transaction::Transaction;
use jj_lib::tree::TreeMergeError;
@ -194,6 +195,10 @@ impl From<WorkspaceInitError> for CommandError {
WorkspaceInitError::WorkingCopyState(err) => {
CommandError::InternalError(format!("Failed to access the repository: {err}"))
}
WorkspaceInitError::SignInit(err @ SignInitError::UnknownBackend(_)) => {
user_error(format!("{err}"))
}
WorkspaceInitError::SignInit(err) => CommandError::InternalError(format!("{err}")),
}
}
}
@ -1655,6 +1660,10 @@ jj init --git-repo=.",
) => CommandError::InternalError(format!(
"The repository appears broken or inaccessible: {err}"
)),
WorkspaceLoadError::StoreLoadError(StoreLoadError::Signing(
err @ SignInitError::UnknownBackend(_),
)) => user_error(format!("{err}")),
WorkspaceLoadError::StoreLoadError(err) => CommandError::InternalError(format!("{err}")),
}
}

View file

@ -355,6 +355,31 @@
"default": "1MiB"
}
}
},
"signing": {
"type": "object",
"description": "Settings for verifying and creating cryptographic commit signatures",
"properties": {
"backend": {
"type": "string",
"description": "Which backend to use to create commit signatures"
},
"key": {
"type": "string",
"description": "The key parameter to pass to the signing backend. Overridden by `jj sign` parameter or by the global `--sign-with` option"
},
"sign-all": {
"type": "boolean",
"description": "Whether to sign all commits by default. Overridden by global `--no-sign` option",
"default": false
},
"backends": {
"type": "object",
"description": "Tables of options to pass to specific signing backends",
"properties": {},
"additionalProperties": true
}
}
}
}
}

View file

@ -27,6 +27,7 @@ use thiserror::Error;
use crate::content_hash::ContentHash;
use crate::merge::Merge;
use crate::repo_path::{RepoPath, RepoPathComponent, RepoPathComponentBuf};
use crate::signing::SignResult;
pub trait ObjectId {
fn new(value: Vec<u8>) -> Self;
@ -147,7 +148,7 @@ content_hash! {
}
}
pub type SigningFn = Box<dyn FnMut(&[u8]) -> BackendResult<Vec<u8>>>;
pub type SigningFn = Box<dyn FnMut(&[u8]) -> SignResult<Vec<u8>>>;
/// Identifies a single legacy tree, which may have path-level conflicts, or a
/// merge of multiple trees, where the individual trees do not have conflicts.

View file

@ -22,6 +22,7 @@ use std::sync::Arc;
use crate::backend;
use crate::backend::{BackendError, ChangeId, CommitId, MergedTreeId, Signature};
use crate::merged_tree::MergedTree;
use crate::signing::{SignResult, Verification};
use crate::store::Store;
#[derive(Clone)]
@ -146,6 +147,20 @@ impl Commit {
}
false
}
/// A quick way to just check if a signature is present.
pub fn is_signed(&self) -> bool {
self.data.secure_sig.is_some()
}
/// A slow (but cached) way to get the full verification.
pub fn verification(&self) -> SignResult<Option<Verification>> {
self.data
.secure_sig
.as_ref()
.map(|sig| self.store.signer().verify(&self.id, &sig.data, &sig.sig))
.transpose()
}
}
/// Wrapper to sort `Commit` by committer timestamp.

View file

@ -16,10 +16,11 @@
use std::sync::Arc;
use crate::backend::{self, BackendResult, ChangeId, CommitId, MergedTreeId, Signature};
use crate::backend::{self, BackendResult, ChangeId, CommitId, MergedTreeId, Signature, SigningFn};
use crate::commit::Commit;
use crate::repo::{MutableRepo, Repo};
use crate::settings::{JJRng, UserSettings};
use crate::settings::{JJRng, SignSettings, UserSettings};
use crate::signing::SignBehavior;
#[must_use]
pub struct CommitBuilder<'repo> {
@ -27,6 +28,7 @@ pub struct CommitBuilder<'repo> {
rng: Arc<JJRng>,
commit: backend::Commit,
rewrite_source: Option<Commit>,
sign_settings: SignSettings,
}
impl CommitBuilder<'_> {
@ -55,6 +57,7 @@ impl CommitBuilder<'_> {
rng,
commit,
rewrite_source: None,
sign_settings: settings.sign_settings(),
}
}
@ -83,6 +86,7 @@ impl CommitBuilder<'_> {
commit,
rng: settings.get_rng(),
rewrite_source: Some(predecessor.clone()),
sign_settings: settings.sign_settings(),
}
}
@ -157,14 +161,44 @@ impl CommitBuilder<'_> {
self
}
pub fn write(self) -> BackendResult<Commit> {
pub fn sign_settings(&self) -> &SignSettings {
&self.sign_settings
}
pub fn set_sign_behavior(mut self, sign_behavior: SignBehavior) -> Self {
self.sign_settings.behavior = sign_behavior;
self
}
pub fn set_sign_key(mut self, sign_key: Option<String>) -> Self {
self.sign_settings.key = sign_key;
self
}
pub fn write(mut self) -> BackendResult<Commit> {
let mut rewrite_source_id = None;
if let Some(rewrite_source) = self.rewrite_source {
if *rewrite_source.change_id() == self.commit.change_id {
rewrite_source_id.replace(rewrite_source.id().clone());
}
}
let commit = self.mut_repo.write_commit(self.commit)?;
let sign_settings = self.sign_settings;
let store = self.mut_repo.store();
let signing_fn = (store.signer().can_sign() && sign_settings.should_sign(&self.commit))
.then(|| {
let store = store.clone();
Box::new(move |data: &_| store.signer().sign(data, sign_settings.key.as_deref()))
as SigningFn
});
// Commit backend doesn't use secure_sig for writing and enforces it with an
// assert, but sign_settings.should_sign check above will want to know
// if we're rewriting a signed commit
self.commit.secure_sig = None;
let commit = self.mut_repo.write_commit(self.commit, signing_fn)?;
if let Some(rewrite_source_id) = rewrite_source_id {
self.mut_repo
.record_rewritten_commit(rewrite_source_id, commit.id().clone())

View file

@ -967,7 +967,10 @@ impl Backend for GitBackend {
let mut data = Vec::with_capacity(512);
commit.write_to(&mut data).unwrap();
let sig = sign(&data)?;
let sig = sign(&data).map_err(|err| BackendError::WriteObject {
object_type: "commit",
source: Box::new(err),
})?;
commit
.extra_headers
.push(("gpgsig".into(), sig.clone().into()));

View file

@ -58,6 +58,7 @@ pub mod revset;
pub mod revset_graph;
pub mod rewrite;
pub mod settings;
pub mod signing;
pub mod simple_op_heads_store;
pub mod simple_op_store;
pub mod stacked_table;

View file

@ -281,7 +281,7 @@ impl Backend for LocalBackend {
let mut proto = commit_to_proto(&commit);
if let Some(mut sign) = sign_with {
let data = proto.encode_to_vec();
let sig = sign(&data)?;
let sig = sign(&data).map_err(to_other_err)?;
proto.secure_sig = Some(sig.clone());
commit.secure_sig = Some(SecureSig { data, sig });
}

View file

@ -31,7 +31,7 @@ use tracing::instrument;
use self::dirty_cell::DirtyCell;
use crate::backend::{
Backend, BackendError, BackendInitError, BackendLoadError, BackendResult, ChangeId, CommitId,
MergedTreeId, ObjectId,
MergedTreeId, ObjectId, SigningFn,
};
use crate::commit::{Commit, CommitByCommitterTimestamp};
use crate::commit_builder::CommitBuilder;
@ -52,6 +52,7 @@ use crate::refs::{
use crate::revset::{self, ChangeIdIndex, Revset, RevsetExpression};
use crate::rewrite::{DescendantRebaser, RebaseOptions};
use crate::settings::{RepoSettings, UserSettings};
use crate::signing::{SignInitError, Signer};
use crate::simple_op_heads_store::SimpleOpHeadsStore;
use crate::simple_op_store::SimpleOpStore;
use crate::store::Store;
@ -118,6 +119,8 @@ pub enum RepoInitError {
Backend(#[from] BackendInitError),
#[error(transparent)]
Path(#[from] PathError),
#[error(transparent)]
SignInit(#[from] SignInitError),
}
impl ReadonlyRepo {
@ -156,7 +159,8 @@ impl ReadonlyRepo {
let backend = backend_initializer(user_settings, &store_path)?;
let backend_path = store_path.join("type");
fs::write(&backend_path, backend.name()).context(&backend_path)?;
let store = Store::new(backend, user_settings.use_tree_conflict_format());
let signer = Signer::from_settings(user_settings)?;
let store = Store::new(backend, signer, user_settings.use_tree_conflict_format());
let repo_settings = user_settings.with_repo(&repo_path).unwrap();
let op_store_path = repo_path.join("op_store");
@ -418,6 +422,8 @@ pub enum StoreLoadError {
},
#[error(transparent)]
Backend(#[from] BackendLoadError),
#[error(transparent)]
Signing(#[from] SignInitError),
}
impl StoreFactories {
@ -608,6 +614,7 @@ impl RepoLoader {
) -> Result<Self, StoreLoadError> {
let store = Store::new(
store_factories.load_backend(user_settings, &repo_path.join("store"))?,
Signer::from_settings(user_settings)?,
user_settings.use_tree_conflict_format(),
);
let repo_settings = user_settings.with_repo(repo_path).unwrap();
@ -792,8 +799,12 @@ impl MutableRepo {
CommitBuilder::for_rewrite_from(self, settings, predecessor)
}
pub fn write_commit(&mut self, commit: backend::Commit) -> BackendResult<Commit> {
let commit = self.store().write_commit(commit)?;
pub fn write_commit(
&mut self,
commit: backend::Commit,
sign_with: Option<SigningFn>,
) -> BackendResult<Commit> {
let commit = self.store().write_commit(commit, sign_with)?;
self.add_head(&commit);
Ok(commit)
}

View file

@ -21,9 +21,10 @@ use chrono::DateTime;
use rand::prelude::*;
use rand_chacha::ChaCha20Rng;
use crate::backend::{ChangeId, ObjectId, Signature, Timestamp};
use crate::backend::{ChangeId, Commit, ObjectId, Signature, Timestamp};
use crate::fmt_util::binary_prefix;
use crate::fsmonitor::FsmonitorKind;
use crate::signing::SignBehavior;
#[derive(Debug, Clone)]
pub struct UserSettings {
@ -63,6 +64,50 @@ impl Default for GitSettings {
}
}
/// Commit signing settings, describes how to and if to sign commits.
#[derive(Debug, Clone, Default)]
pub struct SignSettings {
/// What to actually do, see [SignBehavior].
pub behavior: SignBehavior,
/// The email address to compare against the commit author when determining
/// if the existing signature is "our own" in terms of the sign behavior.
pub user_email: String,
/// The signing backend specific key, to be passed to the signing backend.
pub key: Option<String>,
}
impl SignSettings {
/// Load the signing settings from the config.
pub fn from_settings(settings: &UserSettings) -> Self {
let sign_all = settings
.config()
.get_bool("signing.sign-all")
.unwrap_or(false);
Self {
behavior: if sign_all {
SignBehavior::Own
} else {
SignBehavior::Keep
},
user_email: settings.user_email(),
key: settings.config().get_string("signing.key").ok(),
}
}
/// Check if a commit should be signed according to the configured behavior
/// and email.
pub fn should_sign(&self, commit: &Commit) -> bool {
match self.behavior {
SignBehavior::Drop => false,
SignBehavior::Keep => {
commit.secure_sig.is_some() && commit.author.email == self.user_email
}
SignBehavior::Own => commit.author.email == self.user_email,
SignBehavior::Force => true,
}
}
}
fn get_timestamp_config(config: &config::Config, key: &str) -> Option<Timestamp> {
match config.get_string(key) {
Ok(timestamp_str) => match DateTime::parse_from_rfc3339(&timestamp_str) {
@ -215,6 +260,16 @@ impl UserSettings {
e @ Err(_) => e,
}
}
// separate from sign_settings as those two are needed in pretty different
// places
pub fn signing_backend(&self) -> Option<String> {
self.config.get_string("signing.backend").ok()
}
pub fn sign_settings(&self) -> SignSettings {
SignSettings::from_settings(self)
}
}
/// This Rng uses interior mutability to allow generating random values using an

249
lib/src/signing.rs Normal file
View file

@ -0,0 +1,249 @@
// Copyright 2023 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.
//! Generic APIs to work with cryptographic signatures created and verified by
//! various backends.
use std::collections::HashMap;
use std::fmt::Debug;
use std::sync::RwLock;
use thiserror::Error;
use crate::backend::CommitId;
use crate::settings::UserSettings;
/// A status of the signature, part of the [Verification] type.
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
pub enum SigStatus {
/// Valid signature that matches the data.
Good,
/// Valid signature that could not be verified (e.g. due to an unknown key).
Unknown,
/// Valid signature that does not match the signed data.
Bad,
}
/// The result of a signature verification.
/// Key and display are optional additional info that backends can or can not
/// provide to add additional information for the templater to potentially show.
#[derive(Debug, Clone, PartialEq, Eq)]
pub struct Verification {
/// The status of the signature.
pub status: SigStatus,
/// The key id representation, if available. For GPG, this will be the key
/// fingerprint.
pub key: Option<String>,
/// A display string, if available. For GPG, this will be formatted primary
/// user ID.
pub display: Option<String>,
}
impl Verification {
/// A shortcut to create an `Unknown` verification with no additional
/// metadata.
pub fn unknown() -> Self {
Self {
status: SigStatus::Unknown,
key: None,
display: None,
}
}
}
/// The backend for signing and verifying cryptographic signatures.
///
/// This allows using different signers, such as GPG or SSH, or different
/// versions of them.
pub trait SigningBackend: Debug + Send + Sync {
/// Name of the backend, used in the config and for display.
fn name(&self) -> &str;
/// Check if the signature can be read and verified by this backend.
///
/// Should check the signature format, usually just looks at the prefix.
fn can_read(&self, signature: &[u8]) -> bool;
/// Create a signature for arbitrary data.
///
/// The `key` parameter is what `jj sign` receives as key argument, or what
/// is configured in the `signing.key` config.
fn sign(&self, data: &[u8], key: Option<&str>) -> SignResult<Vec<u8>>;
/// Verify a signature. Should be reflexive with `sign`:
/// ```rust,ignore
/// verify(data, sign(data)?)?.status == SigStatus::Good
/// ```
fn verify(&self, data: &[u8], signature: &[u8]) -> SignResult<Verification>;
}
/// An error type for the signing/verifying operations
#[derive(Debug, Error)]
pub enum SignError {
/// The verification failed because the signature *format* was invalid.
#[error("Invalid signature")]
InvalidSignatureFormat,
/// A generic error from the backend impl.
#[error("Signing error: {0}")]
Backend(Box<dyn std::error::Error + Send + Sync>),
}
/// A result type for the signing/verifying operations
pub type SignResult<T> = Result<T, SignError>;
/// An error type for the signing backend initialization.
#[derive(Debug, Error)]
pub enum SignInitError {
/// If the backend name specified in the config is not known.
#[error("Unknown signing backend configured: {0}")]
UnknownBackend(String),
/// A generic error from the backend impl.
#[error("Failed to initialize signing: {0}")]
Backend(Box<dyn std::error::Error + Send + Sync>),
}
/// A enum that describes if a created/rewritten commit should be signed or not.
#[derive(Debug, Clone, Copy, PartialEq, Eq, Default)]
pub enum SignBehavior {
/// Drop existing signatures.
/// This is what jj did before signing support or does now when a signing
/// backend is not configured.
#[default]
Drop,
/// Only sign commits that were authored by self and already signed,
/// "preserving" the signature across rewrites.
/// This is what jj does when a signing backend is configured.
Keep,
/// Sign/re-sign commits that were authored by self and drop them for
/// others. This is what jj does when configured to always sign.
Own,
/// Always sign commits, regardless of who authored or signed them before.
/// This is what jj does on `jj sign -f`.
Force,
}
/// Wraps low-level signing backends and adds caching, similar to `Store`.
#[derive(Debug, Default)]
pub struct Signer {
/// The backend that is used for signing commits.
/// Optional because signing might not be configured.
main_backend: Option<Box<dyn SigningBackend>>,
/// All known backends without the main one - used for verification.
/// Main backend is also used for verification, but it's not in this list
/// for ownership reasons.
backends: Vec<Box<dyn SigningBackend>>,
cache: RwLock<HashMap<CommitId, Verification>>,
}
impl Signer {
/// Creates a signer based on user settings. Uses all known backends, and
/// chooses one of them to be used for signing depending on the config.
pub fn from_settings(settings: &UserSettings) -> Result<Self, SignInitError> {
let mut backends: Vec<Box<dyn SigningBackend>> = vec![
// Box::new(GpgBackend::from_settings(settings)?),
// Box::new(SshBackend::from_settings(settings)?),
// Box::new(X509Backend::from_settings(settings)?),
];
let main_backend = settings
.signing_backend()
.map(|backend| {
backends
.iter()
.position(|b| b.name() == backend)
.map(|i| backends.remove(i))
.ok_or(SignInitError::UnknownBackend(backend))
})
.transpose()?;
Ok(Self::new(main_backend, backends))
}
/// Creates a signer with the given backends.
pub fn new(
main_backend: Option<Box<dyn SigningBackend>>,
other_backends: Vec<Box<dyn SigningBackend>>,
) -> Self {
Self {
main_backend,
backends: other_backends,
cache: Default::default(),
}
}
/// Checks if the signer can sign, i.e. if a main backend is configured.
pub fn can_sign(&self) -> bool {
self.main_backend.is_some()
}
/// This is just a pass-through to the main backend that unconditionally
/// creates a signature.
pub fn sign(&self, data: &[u8], key: Option<&str>) -> SignResult<Vec<u8>> {
self.main_backend
.as_ref()
.expect("tried to sign without checking can_sign first")
.sign(data, key)
}
/// Looks for backend that can verify the signature and returns the result
/// of its verification.
pub fn verify(
&self,
commit_id: &CommitId,
data: &[u8],
signature: &[u8],
) -> SignResult<Verification> {
let cached = self.cache.read().unwrap().get(commit_id).cloned();
if let Some(check) = cached {
return Ok(check);
}
let verification = self
.main_backend
.iter()
.chain(self.backends.iter())
.filter(|b| b.can_read(signature))
// skip unknown and invalid sigs to allow other backends that can read to try
// for example, we might have gpg and sq, both of which could read a PGP signature
.find_map(|backend| match backend.verify(data, signature) {
Ok(check) if check.status == SigStatus::Unknown => None,
Err(SignError::InvalidSignatureFormat) => None,
e => Some(e),
})
.transpose()?;
if let Some(verification) = verification {
// a key might get imported before next call?.
// realistically this is unlikely, but technically
// it's correct to not cache unknowns here
if verification.status != SigStatus::Unknown {
self.cache
.write()
.unwrap()
.insert(commit_id.clone(), verification.clone());
}
Ok(verification)
} else {
// now here it's correct to cache unknowns, as we don't
// have a backend that knows how to handle this signature
//
// not sure about how much of an optimization this is
self.cache
.write()
.unwrap()
.insert(commit_id.clone(), Verification::unknown());
Ok(Verification::unknown())
}
}
}

View file

@ -22,14 +22,15 @@ use std::sync::{Arc, RwLock};
use pollster::FutureExt;
use crate::backend;
use crate::backend::{
Backend, BackendResult, ChangeId, CommitId, ConflictId, FileId, MergedTreeId, SymlinkId, TreeId,
self, Backend, BackendResult, ChangeId, CommitId, ConflictId, FileId, MergedTreeId, SigningFn,
SymlinkId, TreeId,
};
use crate::commit::Commit;
use crate::merge::{Merge, MergedTreeValue};
use crate::merged_tree::MergedTree;
use crate::repo_path::{RepoPath, RepoPathBuf};
use crate::signing::Signer;
use crate::tree::Tree;
use crate::tree_builder::TreeBuilder;
@ -37,6 +38,7 @@ use crate::tree_builder::TreeBuilder;
/// adds caching.
pub struct Store {
backend: Box<dyn Backend>,
signer: Signer,
commit_cache: RwLock<HashMap<CommitId, Arc<backend::Commit>>>,
tree_cache: RwLock<HashMap<(RepoPathBuf, TreeId), Arc<backend::Tree>>>,
use_tree_conflict_format: bool,
@ -51,9 +53,14 @@ impl Debug for Store {
}
impl Store {
pub fn new(backend: Box<dyn Backend>, use_tree_conflict_format: bool) -> Arc<Self> {
pub fn new(
backend: Box<dyn Backend>,
signer: Signer,
use_tree_conflict_format: bool,
) -> Arc<Self> {
Arc::new(Store {
backend,
signer,
commit_cache: Default::default(),
tree_cache: Default::default(),
use_tree_conflict_format,
@ -64,6 +71,10 @@ impl Store {
self.backend.as_any()
}
pub fn signer(&self) -> &Signer {
&self.signer
}
/// Whether new tree should be written using the tree-level format.
pub fn use_tree_conflict_format(&self) -> bool {
self.use_tree_conflict_format
@ -124,9 +135,14 @@ impl Store {
Ok(data)
}
pub fn write_commit(self: &Arc<Self>, commit: backend::Commit) -> BackendResult<Commit> {
pub fn write_commit(
self: &Arc<Self>,
commit: backend::Commit,
sign_with: Option<SigningFn>,
) -> BackendResult<Commit> {
assert!(!commit.parents.is_empty());
let (commit_id, commit) = self.backend.write_commit(commit, None)?;
let (commit_id, commit) = self.backend.write_commit(commit, sign_with)?;
let data = Arc::new(commit);
{
let mut write_locked_cache = self.commit_cache.write().unwrap();

View file

@ -36,6 +36,7 @@ use crate::repo::{
StoreFactories, StoreLoadError, SubmoduleStoreInitializer,
};
use crate::settings::UserSettings;
use crate::signing::SignInitError;
use crate::store::Store;
use crate::working_copy::{
CheckoutError, CheckoutStats, LockedWorkingCopy, WorkingCopy, WorkingCopyStateError,
@ -55,6 +56,8 @@ pub enum WorkspaceInitError {
Path(#[from] PathError),
#[error(transparent)]
Backend(#[from] BackendInitError),
#[error(transparent)]
SignInit(#[from] SignInitError),
}
#[derive(Error, Debug)]
@ -249,6 +252,7 @@ impl Workspace {
.map_err(|repo_init_err| match repo_init_err {
RepoInitError::Backend(err) => WorkspaceInitError::Backend(err),
RepoInitError::Path(err) => WorkspaceInitError::Path(err),
RepoInitError::SignInit(err) => WorkspaceInitError::SignInit(err),
})?;
let (working_copy, repo) = init_working_copy(
user_settings,

View file

@ -336,7 +336,7 @@ pub fn commit_with_tree(store: &Arc<Store>, tree_id: MergedTreeId) -> Commit {
committer: signature,
secure_sig: None,
};
store.write_commit(commit).unwrap()
store.write_commit(commit, None).unwrap()
}
pub fn dump_tree(store: &Arc<Store>, tree_id: &MergedTreeId) -> String {

View file

@ -282,7 +282,7 @@ impl Backend for TestBackend {
if let Some(sign) = &mut sign_with {
let data = format!("{contents:?}").into_bytes();
let sig = sign(&data)?;
let sig = sign(&data).map_err(|err| BackendError::Other(Box::new(err)))?;
contents.secure_sig = Some(SecureSig { data, sig });
}