feat(prepare): move to one-file-per-query for offline mode

Co-authored-by: Jonas Platte <jonas@lumeo.com>
This commit is contained in:
Craig Bester 2023-02-18 23:16:30 +02:00 committed by Austin Bonander
parent 291e79abae
commit 96c8b12485
9 changed files with 411 additions and 391 deletions

1
Cargo.lock generated
View file

@ -2428,6 +2428,7 @@ dependencies = [
"sqlx-postgres",
"sqlx-sqlite",
"syn",
"tempfile",
"tokio",
"url",
]

View file

@ -1,9 +1,11 @@
use anyhow::Result;
use futures::{Future, TryFutureExt};
use sqlx::{AnyConnection, Connection};
use std::io;
use std::time::Duration;
use anyhow::Result;
use futures::{Future, TryFutureExt};
use sqlx::{AnyConnection, Connection};
use crate::opt::{Command, ConnectOpts, DatabaseCommand, MigrateCommand};
mod database;
@ -61,18 +63,11 @@ pub async fn run(opt: Opt) -> Result<()> {
},
Command::Prepare {
check: false,
merged,
args,
check,
workspace,
connect_opts,
} => prepare::run(&connect_opts, merged, args).await?,
Command::Prepare {
check: true,
merged,
args,
connect_opts,
} => prepare::check(&connect_opts, merged, args).await?,
} => prepare::run(check, workspace, connect_opts, args).await?,
};
Ok(())

View file

@ -1,14 +1,16 @@
use anyhow::{Context, Result};
use cargo_metadata::{
Metadata as CargoMetadata, Package as MetadataPackage, PackageId as MetadataId,
};
use std::{
collections::{btree_map, BTreeMap, BTreeSet},
ffi::OsStr,
path::{Path, PathBuf},
process::Command,
str::FromStr,
};
use anyhow::Context;
use cargo_metadata::{
Metadata as CargoMetadata, Package as MetadataPackage, PackageId as MetadataId,
};
/// The minimal amount of package information we care about
///
/// The package's `name` is used to `cargo clean -p` specific crates while the `src_paths` are
@ -50,6 +52,8 @@ pub struct Metadata {
packages: BTreeMap<MetadataId, Package>,
/// All of the crates in the current workspace
workspace_members: Vec<MetadataId>,
/// Workspace root path.
workspace_root: PathBuf,
/// Maps each dependency to its set of dependents
reverse_deps: BTreeMap<MetadataId, BTreeSet<MetadataId>>,
/// The target directory of the project
@ -62,6 +66,19 @@ pub struct Metadata {
}
impl Metadata {
/// Parse the manifest from the current working directory using `cargo metadata`.
pub fn from_current_directory(cargo: &OsStr) -> anyhow::Result<Self> {
let output = Command::new(cargo)
.args(["metadata", "--format-version=1"])
.output()
.context("Could not fetch metadata")?;
std::str::from_utf8(&output.stdout)
.context("Invalid `cargo metadata` output")?
.parse()
.context("Issue parsing `cargo metadata` output - consider manually running it to check for issues")
}
pub fn package(&self, id: &MetadataId) -> Option<&Package> {
self.packages.get(id)
}
@ -74,6 +91,10 @@ impl Metadata {
&self.workspace_members
}
pub fn workspace_root(&self) -> &Path {
&self.workspace_root
}
pub fn target_directory(&self) -> &Path {
&self.target_directory
}
@ -97,7 +118,7 @@ impl Metadata {
if let Some(immediate_dependents) = self.reverse_deps.get(id) {
for immediate_dependent in immediate_dependents {
if dependents.insert(immediate_dependent) {
self.all_dependents_of_helper(&immediate_dependent, dependents);
self.all_dependents_of_helper(immediate_dependent, dependents);
}
}
}
@ -117,6 +138,7 @@ impl FromStr for Metadata {
let CargoMetadata {
packages: metadata_packages,
workspace_members,
workspace_root,
resolve,
target_directory,
..
@ -142,14 +164,35 @@ impl FromStr for Metadata {
}
}
let workspace_root = workspace_root.into_std_path_buf();
let target_directory = target_directory.into_std_path_buf();
Ok(Self {
packages,
workspace_members,
workspace_root,
reverse_deps,
target_directory,
current_package,
})
}
}
/// The absolute path to the directory containing the `Cargo.toml` manifest.
/// Depends on the current working directory.
pub(crate) fn manifest_dir(cargo: &OsStr) -> anyhow::Result<PathBuf> {
let stdout = Command::new(cargo)
.args(["locate-project", "--message-format=plain"])
.output()
.context("could not locate manifest directory")?
.stdout;
let mut manifest_path: PathBuf = std::str::from_utf8(&stdout)
.context("output of `cargo locate-project` was not valid UTF-8")?
// remove trailing newline
.trim()
.into();
manifest_path.pop();
Ok(manifest_path)
}

View file

@ -16,8 +16,8 @@ pub enum Command {
/// Generate query metadata to support offline compile-time verification.
///
/// Saves metadata for all invocations of `query!` and related macros to `sqlx-data.json`
/// in the current directory, overwriting if needed.
/// Saves metadata for all invocations of `query!` and related macros to a `.sqlx` directory
/// in the current directory (or workspace root with `--workspace`), overwriting if needed.
///
/// During project compilation, the absence of the `DATABASE_URL` environment variable or
/// the presence of `SQLX_OFFLINE` (with a value of `true` or `1`) will constrain the
@ -29,9 +29,12 @@ pub enum Command {
#[clap(long)]
check: bool,
/// Generate a single top-level `sqlx-data.json` file when using a cargo workspace.
/// Generate a single workspace-level `.sqlx` folder.
///
/// This option is intended for workspaces where multiple crates use SQLx. If there is only
/// one, it is better to run `cargo sqlx prepare` without this option inside that crate.
#[clap(long)]
merged: bool,
workspace: bool,
/// Arguments to be passed to `cargo rustc ...`.
#[clap(last = true)]

View file

@ -1,148 +1,171 @@
use crate::opt::ConnectOpts;
use anyhow::{bail, Context};
use console::style;
use remove_dir_all::remove_dir_all;
use sqlx::any::{AnyConnectOptions, AnyKind};
use sqlx::Connection;
use std::collections::{BTreeMap, BTreeSet};
use std::fs::File;
use std::io::{BufReader, BufWriter};
use std::collections::{BTreeSet, HashSet};
use std::env;
use std::ffi::{OsStr, OsString};
use std::fs;
use std::path::{Path, PathBuf};
use std::process::Command;
use std::str::FromStr;
use std::{env, fs};
use crate::metadata::Metadata;
use anyhow::{bail, Context};
use console::style;
type QueryData = BTreeMap<String, serde_json::Value>;
type JsonObject = serde_json::Map<String, serde_json::Value>;
use sqlx::Connection;
#[derive(serde::Serialize, serde::Deserialize)]
struct DataFile {
db: String,
#[serde(flatten)]
data: QueryData,
use crate::metadata::{manifest_dir, Metadata};
use crate::opt::ConnectOpts;
pub struct PrepareCtx {
pub workspace: bool,
pub cargo: OsString,
pub cargo_args: Vec<String>,
pub metadata: Metadata,
pub connect_opts: ConnectOpts,
}
impl PrepareCtx {
/// Path to the directory where cached queries should be placed.
fn prepare_dir(&self) -> anyhow::Result<PathBuf> {
if self.workspace {
Ok(self.metadata.workspace_root().join(".sqlx"))
} else {
Ok(manifest_dir(&self.cargo)?.join(".sqlx"))
}
}
}
pub async fn run(
connect_opts: &ConnectOpts,
merge: bool,
check: bool,
workspace: bool,
connect_opts: ConnectOpts,
cargo_args: Vec<String>,
) -> anyhow::Result<()> {
let db = check_backend_and_get_name(connect_opts).await?;
let cargo = env::var_os("CARGO")
.context("failed to get value of `CARGO`; `prepare` subcommand may only be invoked as `cargo sqlx prepare`")?;
let url = &connect_opts.database_url;
anyhow::ensure!(
Path::new("Cargo.toml").exists(),
r#"Failed to read `Cargo.toml`.
hint: This command only works in the manifest directory of a Cargo package or workspace."#
);
let data = run_prepare_step(url, merge, cargo_args)?;
let metadata: Metadata = Metadata::from_current_directory(&cargo)?;
let ctx = PrepareCtx {
workspace,
cargo,
cargo_args,
metadata,
connect_opts,
};
if data.is_empty() {
if check {
prepare_check(&ctx).await
} else {
prepare(&ctx).await
}
}
async fn prepare(ctx: &PrepareCtx) -> anyhow::Result<()> {
check_backend(&ctx.connect_opts).await?;
let prepare_dir = ctx.prepare_dir()?;
run_prepare_step(ctx, &prepare_dir)?;
// Warn if no queries were generated. Glob since the directory may contain unrelated files.
if glob_query_files(prepare_dir)?.is_empty() {
println!("{} no queries found", style("warning:").yellow());
return Ok(());
}
if ctx.workspace {
println!(
"{} no queries found; please ensure that the `offline` feature is enabled in sqlx",
"query data written to .sqlx in the workspace root; \
please check this into version control"
);
} else {
println!(
"query data written to .sqlx in the current directory; \
please check this into version control"
);
}
Ok(())
}
async fn prepare_check(ctx: &PrepareCtx) -> anyhow::Result<()> {
let _ = check_backend(&ctx.connect_opts).await?;
// Re-generate and store the queries in a separate directory from both the prepared
// queries and the ones generated by `cargo check`, to avoid conflicts.
let prepare_dir = ctx.prepare_dir()?;
let cache_dir = ctx.metadata.target_directory().join("sqlx-prepare-check");
run_prepare_step(ctx, &cache_dir)?;
// Compare .sqlx to cache.
let prepare_filenames: HashSet<String> = glob_query_files(&prepare_dir)?
.into_iter()
.filter_map(|path| path.file_name().map(|f| f.to_string_lossy().into_owned()))
.collect();
let cache_filenames: HashSet<String> = glob_query_files(&cache_dir)?
.into_iter()
.filter_map(|path| path.file_name().map(|f| f.to_string_lossy().into_owned()))
.collect();
// Error: files in cache but not .sqlx.
if cache_filenames
.difference(&prepare_filenames)
.next()
.is_some()
{
bail!("prepare check failed: .sqlx is missing one or more queries; you should re-run sqlx prepare");
}
// Warn: files in .sqlx but not cache.
if prepare_filenames
.difference(&cache_filenames)
.next()
.is_some()
{
println!(
"{} potentially unused queries found in .sqlx; you may want to re-run sqlx prepare",
style("warning:").yellow()
);
}
serde_json::to_writer_pretty(
BufWriter::new(
File::create("sqlx-data.json").context("failed to create/open `sqlx-data.json`")?,
),
&DataFile { db, data },
)
.context("failed to write to `sqlx-data.json`")?;
println!(
"query data written to `sqlx-data.json` in the current directory; \
please check this into version control"
);
Ok(())
}
pub async fn check(
connect_opts: &ConnectOpts,
merge: bool,
cargo_args: Vec<String>,
) -> anyhow::Result<()> {
let db = check_backend_and_get_name(connect_opts).await?;
let url = &connect_opts.database_url;
let data = run_prepare_step(url, merge, cargo_args)?;
let data_file = File::open("sqlx-data.json").context(
"failed to open `sqlx-data.json`; you may need to run `cargo sqlx prepare` first",
)?;
let DataFile {
db: expected_db,
data: saved_data,
} = serde_json::from_reader(BufReader::new(data_file))?;
if db != expected_db {
bail!(
"saved prepare data is for {}, not {} (inferred from `DATABASE_URL`)",
expected_db,
db
)
}
if data != saved_data {
bail!("`cargo sqlx prepare` needs to be rerun")
}
Ok(())
}
fn run_prepare_step(url: &str, merge: bool, cargo_args: Vec<String>) -> anyhow::Result<QueryData> {
anyhow::ensure!(
Path::new("Cargo.toml").exists(),
r#"Failed to read `Cargo.toml`.
hint: This command only works in the manifest directory of a Cargo package."#
);
// path to the Cargo executable
let cargo = env::var("CARGO")
.context("`prepare` subcommand may only be invoked as `cargo sqlx prepare`")?;
let output = Command::new(&cargo)
.args(&["metadata", "--format-version=1"])
.output()
.context("Could not fetch metadata")?;
let output_str =
std::str::from_utf8(&output.stdout).context("Invalid `cargo metadata` output")?;
let metadata: Metadata = output_str.parse().context(
"Issue parsing `cargo metadata` output - consider manually running it to check for issues",
)?;
// try removing the target/sqlx directory before running, as stale files
// have repeatedly caused issues in the past.
let _ = remove_dir_all(metadata.target_directory().join("sqlx"));
// Try only triggering a recompile on crates that use `sqlx-macros`, falling back to a full
// clean on error.
match setup_minimal_project_recompile(&cargo, &metadata, merge) {
Ok(()) => {}
Err(err) => {
println!(
"Failed minimal recompile setup. Cleaning entire project. Err: {}",
err
);
let clean_status = Command::new(&cargo).arg("clean").status()?;
if !clean_status.success() {
bail!("`cargo clean` failed with status: {}", clean_status);
}
// Compare file contents as JSON to ignore superficial differences.
// Everything in cache checked to be in .sqlx already.
for filename in cache_filenames {
let prepare_json = load_json_file(prepare_dir.join(&filename))?;
let cache_json = load_json_file(cache_dir.join(&filename))?;
if prepare_json != cache_json {
bail!("prepare check failed: one or more query files differ ({}); you should re-run sqlx prepare", filename);
}
};
}
Ok(())
}
fn run_prepare_step(ctx: &PrepareCtx, cache_dir: &Path) -> anyhow::Result<()> {
// Create and/or clean the directory.
fs::create_dir_all(cache_dir).context(format!(
"Failed to create query cache directory: {:?}",
cache_dir
))?;
// Only delete sqlx-*.json files to avoid accidentally deleting any user data.
for query_file in glob_query_files(cache_dir).context("Failed to read query cache files")? {
fs::remove_file(&query_file)
.with_context(|| format!("Failed to delete query file: {}", query_file.display()))?;
}
// Try only triggering a recompile on crates that use `sqlx-macros` falling back to a full
// clean on error
setup_minimal_project_recompile(&ctx.cargo, &ctx.metadata, ctx.workspace)?;
// Compile the queries.
let check_status = {
let mut check_command = Command::new(&cargo);
let mut check_command = Command::new(&ctx.cargo);
check_command
.arg("check")
.args(cargo_args)
.args(&ctx.cargo_args)
.env("DATABASE_URL", &ctx.connect_opts.database_url)
.env("SQLX_OFFLINE", "false")
.env("DATABASE_URL", url);
.env("SQLX_OFFLINE_DIR", cache_dir);
// `cargo check` recompiles on changed rust flags which can be set either via the env var
// or through the `rustflags` field in `$CARGO_HOME/config` when the env var isn't set.
@ -157,56 +180,7 @@ hint: This command only works in the manifest directory of a Cargo package."#
bail!("`cargo check` failed with status: {}", check_status);
}
// Combine the queries into one file.
let package_dir = if merge {
// Merge queries from all workspace crates.
"**"
} else {
// Use a separate sub-directory for each crate in a workspace. This avoids a race condition
// where `prepare` can pull in queries from multiple crates if they happen to be generated
// simultaneously (e.g. Rust Analyzer building in the background).
metadata
.current_package()
.map(|pkg| pkg.name())
.context("Resolving the crate package for the current working directory failed")?
};
let pattern = metadata
.target_directory()
.join("sqlx")
.join(package_dir)
.join("query-*.json");
let mut data = BTreeMap::new();
for path in glob::glob(
pattern
.to_str()
.context("CARGO_TARGET_DIR not valid UTF-8")?,
)? {
let path = path?;
let contents = fs::read(&*path)?;
let mut query_data: JsonObject = serde_json::from_slice(&contents)?;
// we lift the `hash` key to the outer map
let hash = query_data
.remove("hash")
.context("expected key `hash` in query data")?;
if let serde_json::Value::String(hash) = hash {
data.insert(hash, serde_json::Value::Object(query_data));
} else {
bail!(
"expected key `hash` in query data to be string, was {:?} instead; file: {}",
hash,
path.display()
)
}
// lazily remove the file, we don't care too much if we can't
let _ = fs::remove_file(&path);
}
Ok(data)
Ok(())
}
#[derive(Debug, PartialEq)]
@ -225,32 +199,57 @@ struct ProjectRecompileAction {
///
/// If `workspace` is false, only the current package will have its files' mtimes updated.
fn setup_minimal_project_recompile(
cargo: &str,
cargo: impl AsRef<OsStr>,
metadata: &Metadata,
workspace: bool,
) -> anyhow::Result<()> {
let ProjectRecompileAction {
clean_packages,
touch_paths,
} = if workspace {
minimal_project_recompile_action(metadata)?
let recompile_action: ProjectRecompileAction = if workspace {
minimal_project_recompile_action(metadata)
} else {
// Only touch the current crate.
ProjectRecompileAction {
clean_packages: Vec::new(),
touch_paths: metadata.current_package().context("Failed to get package in current working directory, pass `--merged` if running from a workspace root")?.src_paths().to_vec(),
touch_paths: metadata.current_package()
.context("failed to get package in current working directory, pass `--workspace` if running from a workspace root")?
.src_paths()
.to_vec(),
}
};
if let Err(err) = minimal_project_clean(&cargo, recompile_action) {
println!(
"Failed minimal recompile setup. Cleaning entire project. Err: {}",
err
);
let clean_status = Command::new(&cargo).arg("clean").status()?;
if !clean_status.success() {
bail!("`cargo clean` failed with status: {}", clean_status);
}
}
Ok(())
}
fn minimal_project_clean(
cargo: impl AsRef<OsStr>,
action: ProjectRecompileAction,
) -> anyhow::Result<()> {
let ProjectRecompileAction {
clean_packages,
touch_paths,
} = action;
// Update the modified timestamp of package files to force a selective recompilation.
for file in touch_paths {
let now = filetime::FileTime::now();
filetime::set_file_times(&file, now, now)
.with_context(|| format!("Failed to update mtime for {:?}", file))?;
}
// Clean entire packages.
for pkg_id in &clean_packages {
let clean_status = Command::new(cargo)
.args(&["clean", "-p", pkg_id])
let clean_status = Command::new(&cargo)
.args(["clean", "-p", pkg_id])
.status()?;
if !clean_status.success() {
@ -261,7 +260,7 @@ fn setup_minimal_project_recompile(
Ok(())
}
fn minimal_project_recompile_action(metadata: &Metadata) -> anyhow::Result<ProjectRecompileAction> {
fn minimal_project_recompile_action(metadata: &Metadata) -> ProjectRecompileAction {
// Get all the packages that depend on `sqlx-macros`
let mut sqlx_macros_dependents = BTreeSet::new();
let sqlx_macros_ids: BTreeSet<_> = metadata
@ -279,7 +278,7 @@ fn minimal_project_recompile_action(metadata: &Metadata) -> anyhow::Result<Proje
let mut in_workspace_dependents = Vec::new();
let mut out_of_workspace_dependents = Vec::new();
for dependent in sqlx_macros_dependents {
if metadata.workspace_members().contains(&dependent) {
if metadata.workspace_members().contains(dependent) {
in_workspace_dependents.push(dependent);
} else {
out_of_workspace_dependents.push(dependent);
@ -306,76 +305,45 @@ fn minimal_project_recompile_action(metadata: &Metadata) -> anyhow::Result<Proje
})
.collect();
Ok(ProjectRecompileAction {
ProjectRecompileAction {
clean_packages: packages_to_clean,
touch_paths: files_to_touch,
})
}
}
/// Ensure the database server is available.
///
/// Returns the `Database::NAME` of the backend on success.
async fn check_backend_and_get_name(opts: &ConnectOpts) -> anyhow::Result<String> {
let conn = crate::connect(opts).await?;
let db = conn.backend_name().to_string();
conn.close().await?;
Ok(db)
async fn check_backend(opts: &ConnectOpts) -> anyhow::Result<()> {
crate::connect(opts).await?.close().await?;
Ok(())
}
/// Find all `query-*.json` files in a directory.
fn glob_query_files(path: impl AsRef<Path>) -> anyhow::Result<Vec<PathBuf>> {
let path = path.as_ref();
let pattern = path.join("query-*.json");
glob::glob(
pattern
.to_str()
.context("query cache path is invalid UTF-8")?,
)
.with_context(|| format!("failed to read query cache path: {}", path.display()))?
.collect::<Result<Vec<_>, _>>()
.context("glob failed")
}
/// Load the JSON contents of a query data file.
fn load_json_file(path: impl AsRef<Path>) -> anyhow::Result<serde_json::Value> {
let path = path.as_ref();
let file_bytes =
fs::read(path).with_context(|| format!("failed to load file: {}", path.display()))?;
Ok(serde_json::from_slice(&file_bytes)?)
}
#[cfg(test)]
mod tests {
use super::*;
use serde_json::json;
use std::assert_eq;
#[test]
fn data_file_serialization_works() {
let data_file = DataFile {
db: "mysql".to_owned(),
data: {
let mut data = BTreeMap::new();
data.insert("a".to_owned(), json!({"key1": "value1"}));
data.insert("z".to_owned(), json!({"key2": "value2"}));
data
},
};
let data_file = serde_json::to_string(&data_file).expect("Data file serialized.");
assert_eq!(
data_file,
"{\"db\":\"mysql\",\"a\":{\"key1\":\"value1\"},\"z\":{\"key2\":\"value2\"}}"
);
}
#[test]
fn data_file_deserialization_works() {
let data_file =
"{\"db\":\"mysql\",\"a\":{\"key1\":\"value1\"},\"z\":{\"key2\":\"value2\"}}";
let data_file: DataFile = serde_json::from_str(data_file).expect("Data file deserialized.");
let DataFile { db, data } = data_file;
assert_eq!(db, "mysql");
assert_eq!(data.len(), 2);
assert_eq!(data.get("a"), Some(&json!({"key1": "value1"})));
assert_eq!(data.get("z"), Some(&json!({"key2": "value2"})));
}
#[test]
fn data_file_deserialization_works_for_ordered_keys() {
let data_file =
"{\"a\":{\"key1\":\"value1\"},\"db\":\"mysql\",\"z\":{\"key2\":\"value2\"}}";
let data_file: DataFile = serde_json::from_str(data_file).expect("Data file deserialized.");
let DataFile { db, data } = data_file;
assert_eq!(db, "mysql");
assert_eq!(data.len(), 2);
assert_eq!(data.get("a"), Some(&json!({"key1": "value1"})));
assert_eq!(data.get("z"), Some(&json!({"key2": "value2"})));
}
#[test]
fn minimal_project_recompile_action_works() -> anyhow::Result<()> {
let sample_metadata_path = Path::new("tests")
@ -392,7 +360,7 @@ mod tests {
touch_paths: vec![
"/home/user/problematic/workspace/b_in_workspace_lib/src/lib.rs".into(),
"/home/user/problematic/workspace/c_in_workspace_bin/src/main.rs".into(),
]
],
}
);

View file

@ -55,6 +55,7 @@ serde = { version = "1.0.132", features = ["derive"] }
serde_json = { version = "1.0.73" }
sha2 = { version = "0.10.0" }
syn = { version = "1.0.84", default-features = false, features = ["full", "derive", "parsing", "printing", "clone-impls"] }
tempfile = { version = "3.3.0" }
quote = { version = "1.0.14", default-features = false }
url = { version = "2.2.2", default-features = false }

View file

@ -1,11 +1,12 @@
use std::collections::BTreeMap;
use std::fs::{self, File};
use std::io::BufWriter;
use std::collections::HashMap;
use std::fmt::{Debug, Display, Formatter};
use std::fs;
use std::marker::PhantomData;
use std::path::{Path, PathBuf};
use std::sync::Mutex;
use once_cell::sync::Lazy;
use proc_macro2::Span;
use serde::{Serialize, Serializer};
use sqlx_core::database::Database;
use sqlx_core::describe::Describe;
@ -13,13 +14,11 @@ use sqlx_core::executor::Executor;
use crate::database::DatabaseExt;
#[derive(serde::Deserialize, serde::Serialize)]
#[serde(bound(
serialize = "Describe<DB>: serde::Serialize",
deserialize = "Describe<DB>: serde::de::DeserializeOwned"
))]
#[derive(serde::Serialize)]
#[serde(bound(serialize = "Describe<DB>: serde::Serialize",))]
#[derive(Debug)]
pub struct QueryData<DB: Database> {
db_name: SerializeDbName<DB>,
#[allow(dead_code)]
pub(super) query: String,
pub(super) describe: Describe<DB>,
@ -36,6 +35,7 @@ impl<DB: Database> QueryData<DB> {
pub fn from_describe(query: &str, describe: Describe<DB>) -> Self {
QueryData {
db_name: SerializeDbName::default(),
query: query.into(),
describe,
hash: hash_string(query),
@ -43,92 +43,74 @@ impl<DB: Database> QueryData<DB> {
}
}
static OFFLINE_DATA_CACHE: Lazy<Mutex<BTreeMap<PathBuf, OfflineData>>> =
Lazy::new(|| Mutex::new(BTreeMap::new()));
struct SerializeDbName<DB>(PhantomData<DB>);
#[derive(serde::Deserialize)]
struct BaseQuery {
query: String,
describe: serde_json::Value,
}
#[derive(serde::Deserialize)]
struct OfflineData {
db: String,
#[serde(flatten)]
hash_to_query: BTreeMap<String, BaseQuery>,
}
impl OfflineData {
fn get_query_from_hash(&self, hash: &str) -> Option<DynQueryData> {
self.hash_to_query.get(hash).map(|base_query| DynQueryData {
db_name: self.db.clone(),
query: base_query.query.to_owned(),
describe: base_query.describe.to_owned(),
hash: hash.to_owned(),
})
impl<DB> Default for SerializeDbName<DB> {
fn default() -> Self {
SerializeDbName(PhantomData)
}
}
#[derive(serde::Deserialize)]
impl<DB: Database> Debug for SerializeDbName<DB> {
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
f.debug_tuple("SerializeDbName").field(&DB::NAME).finish()
}
}
impl<DB: Database> Display for SerializeDbName<DB> {
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
f.pad(DB::NAME)
}
}
impl<DB: Database> Serialize for SerializeDbName<DB> {
fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
where
S: Serializer,
{
serializer.serialize_str(DB::NAME)
}
}
static OFFLINE_DATA_CACHE: Lazy<Mutex<HashMap<PathBuf, DynQueryData>>> =
Lazy::new(Default::default);
/// Offline query data
#[derive(Clone, serde::Deserialize)]
pub struct DynQueryData {
#[serde(skip)]
pub db_name: String,
pub query: String,
pub describe: serde_json::Value,
#[serde(skip)]
pub hash: String,
}
impl DynQueryData {
/// Find and deserialize the data table for this query from a shared `sqlx-data.json`
/// file. The expected structure is a JSON map keyed by the SHA-256 hash of queries in hex.
/// Loads a query given the path to its "query-<hash>.json" file. Subsequent calls for the same
/// path are retrieved from an in-memory cache.
pub fn from_data_file(path: impl AsRef<Path>, query: &str) -> crate::Result<Self> {
let path = path.as_ref();
let query_data = {
let mut cache = OFFLINE_DATA_CACHE
.lock()
// Just reset the cache on error
.unwrap_or_else(|posion_err| {
let mut guard = posion_err.into_inner();
*guard = BTreeMap::new();
guard
});
if !cache.contains_key(path) {
let offline_data_contents = fs::read_to_string(path)
.map_err(|e| format!("failed to read path {}: {}", path.display(), e))?;
let offline_data: OfflineData = serde_json::from_str(&offline_data_contents)?;
let _ = cache.insert(path.to_owned(), offline_data);
let mut cache = OFFLINE_DATA_CACHE
.lock()
// Just reset the cache on error
.unwrap_or_else(|posion_err| {
let mut guard = posion_err.into_inner();
*guard = Default::default();
guard
});
if let Some(cached) = cache.get(path).cloned() {
if query != cached.query {
return Err("hash collision for saved query data".into());
}
let offline_data = cache
.get(path)
.expect("Missing data should have just been added");
let query_hash = hash_string(query);
let query_data = offline_data
.get_query_from_hash(&query_hash)
.ok_or_else(|| format!("failed to find data for query {}", query))?;
if query != query_data.query {
return Err(format!(
"hash collision for stored queries:\n{:?}\n{:?}",
query, query_data.query
)
.into());
}
query_data
};
return Ok(cached);
}
#[cfg(procmacr2_semver_exempt)]
{
let path = path.as_ref().canonicalize()?;
let path = path.to_str().ok_or_else(|| {
format!(
"sqlx-data.json path cannot be represented as a string: {:?}",
"query-<hash>.json path cannot be represented as a string: {:?}",
path
)
})?;
@ -136,7 +118,16 @@ impl DynQueryData {
proc_macro::tracked_path::path(path);
}
Ok(query_data)
let offline_data_contents = fs::read_to_string(path)
.map_err(|e| format!("failed to read saved query path {}: {}", path.display(), e))?;
let dyn_data: DynQueryData = serde_json::from_str(&offline_data_contents)?;
if query != dyn_data.query {
return Err("hash collision for saved query data".into());
}
let _ = cache.insert(path.to_owned(), dyn_data.clone());
Ok(dyn_data)
}
}
@ -151,6 +142,7 @@ where
if DB::NAME == dyn_data.db_name {
let describe: Describe<DB> = serde_json::from_value(dyn_data.describe)?;
Ok(QueryData {
db_name: SerializeDbName::default(),
query: dyn_data.query,
describe,
hash: dyn_data.hash,
@ -165,26 +157,23 @@ where
}
}
pub fn save_in(&self, dir: impl AsRef<Path>, input_span: Span) -> crate::Result<()> {
// we save under the hash of the span representation because that should be unique
// per invocation
let path = dir.as_ref().join(format!(
"query-{}.json",
hash_string(&format!("{:?}", input_span))
));
pub(super) fn save_in(&self, dir: impl AsRef<Path>) -> crate::Result<()> {
// Output to a temporary file first, then move it atomically to avoid clobbering
// other invocations trying to write to the same path.
let mut tmp_file = tempfile::NamedTempFile::new()
.map_err(|err| format!("failed to create query file: {:?}", err))?;
serde_json::to_writer_pretty(tmp_file.as_file_mut(), self)
.map_err(|err| format!("failed to serialize query data to file: {:?}", err))?;
serde_json::to_writer_pretty(
BufWriter::new(
File::create(&path)
.map_err(|e| format!("failed to open path {}: {}", path.display(), e))?,
),
self,
)
.map_err(Into::into)
tmp_file
.persist(dir.as_ref().join(format!("query-{}.json", self.hash)))
.map_err(|err| format!("failed to move query file: {:?}", err))?;
Ok(())
}
}
pub fn hash_string(query: &str) -> String {
pub(super) fn hash_string(query: &str) -> String {
// picked `sha2` because it's already in the dependency tree for both MySQL and Postgres
use sha2::{Digest, Sha256};

View file

@ -1,7 +1,6 @@
use std::collections::BTreeMap;
use std::path::PathBuf;
use std::str::FromStr;
use std::sync::{Arc, Mutex};
use std::{fs, io};
use once_cell::sync::Lazy;
use proc_macro2::TokenStream;
@ -14,7 +13,7 @@ use sqlx_core::database::Database;
use sqlx_core::{column::Column, describe::Describe, type_info::TypeInfo};
use crate::database::DatabaseExt;
use crate::query::data::{DynQueryData, QueryData};
use crate::query::data::{hash_string, DynQueryData, QueryData};
use crate::query::input::RecordType;
use either::Either;
use url::Url;
@ -75,8 +74,6 @@ struct Metadata {
manifest_dir: PathBuf,
offline: bool,
database_url: Option<String>,
package_name: String,
target_dir: PathBuf,
workspace_root: Arc<Mutex<Option<PathBuf>>>,
}
@ -117,12 +114,6 @@ static METADATA: Lazy<Metadata> = Lazy::new(|| {
.expect("`CARGO_MANIFEST_DIR` must be set")
.into();
let package_name: String = env("CARGO_PKG_NAME")
.expect("`CARGO_PKG_NAME` must be set")
.into();
let target_dir = env("CARGO_TARGET_DIR").map_or_else(|_| "target".into(), |dir| dir.into());
// If a .env file exists at CARGO_MANIFEST_DIR, load environment variables from this,
// otherwise fallback to default dotenv behaviour.
let env_path = manifest_dir.join(".env");
@ -155,8 +146,6 @@ static METADATA: Lazy<Metadata> = Lazy::new(|| {
manifest_dir,
offline,
database_url,
package_name,
target_dir,
workspace_root: Arc::new(Mutex::new(None)),
}
});
@ -173,21 +162,33 @@ pub fn expand_input<'a>(
} => QueryDataSource::live(db_url)?,
_ => {
let data_file_path = METADATA.manifest_dir.join("sqlx-data.json");
// Try load the cached query metadata file.
let filename = format!("query-{}.json", hash_string(&input.sql));
let data_file_path = if data_file_path.exists() {
data_file_path
// Check SQLX_OFFLINE_DIR, then local .sqlx, then workspace .sqlx.
let data_file_path = if let Some(sqlx_offline_dir_path) = env("SQLX_OFFLINE_DIR")
.ok()
.map(PathBuf::from)
.map(|path| path.join(&filename))
.filter(|path| path.exists())
{
sqlx_offline_dir_path
} else if let Some(local_path) =
Some(METADATA.manifest_dir.join(".sqlx").join(&filename))
.filter(|path| path.exists())
{
local_path
} else if let Some(workspace_path) =
Some(METADATA.workspace_root().join(".sqlx").join(&filename))
.filter(|path| path.exists())
{
workspace_path
} else {
let workspace_data_file_path = METADATA.workspace_root().join("sqlx-data.json");
if workspace_data_file_path.exists() {
workspace_data_file_path
} else {
return Err(
"`DATABASE_URL` must be set, or `cargo sqlx prepare` must have been run \
and sqlx-data.json must exist, to use query macros"
.into(),
);
}
return Err(
"`DATABASE_URL` must be set, or `cargo sqlx prepare` must have been run \
and .sqlx must exist, to use query macros"
.into(),
);
};
QueryDataSource::Cached(DynQueryData::from_data_file(&data_file_path, &input.sql)?)
@ -351,15 +352,34 @@ where
// Store query metadata only if offline support is enabled but the current build is online.
// If the build is offline, the cache is our input so it's pointless to also write data for it.
if !offline {
// Use a separate sub-directory for each crate in a workspace. This avoids a race condition
// where `prepare` can pull in queries from multiple crates if they happen to be generated
// simultaneously (e.g. Rust Analyzer building in the background).
let save_dir = METADATA
.target_dir
.join("sqlx")
.join(&METADATA.package_name);
std::fs::create_dir_all(&save_dir)?;
data.save_in(save_dir, input.src_span)?;
// Only save query metadata if SQLX_OFFLINE_DIR is set manually or by `cargo sqlx prepare`.
// Note: in a cargo workspace this path is relative to the root.
if let Ok(dir) = env("SQLX_OFFLINE_DIR") {
let path = PathBuf::from(&dir);
match fs::metadata(&path) {
Err(e) => {
if e.kind() != io::ErrorKind::NotFound {
// Can't obtain information about .sqlx
return Err(format!("{}: {}", e, dir).into());
}
// .sqlx doesn't exist.
return Err(format!("sqlx offline path does not exist: {}", dir).into());
}
Ok(meta) => {
if !meta.is_dir() {
return Err(format!(
"sqlx offline path exists, but is not a directory: {}",
dir
)
.into());
}
// .sqlx exists and is a directory, store data.
data.save_in(path)?;
}
}
}
}
Ok(ret_tokens)

View file

@ -45,7 +45,7 @@
/// server with the schema that the query string will be checked against. All variants of `query!()`
/// use [dotenv]<sup>1</sup> so this can be in a `.env` file instead.
///
/// * Or, `sqlx-data.json` must exist at the workspace root. See [Offline Mode](#offline-mode-requires-the-offline-feature)
/// * Or, `.sqlx` must exist at the workspace root. See [Offline Mode](#offline-mode-requires-the-offline-feature)
/// below.
///
/// * The query must be a string literal, or concatenation of string literals using `+` (useful
@ -283,14 +283,14 @@
/// * Run `cargo install sqlx-cli`.
/// * In your project with `DATABASE_URL` set (or in a `.env` file) and the database server running,
/// run `cargo sqlx prepare`.
/// * Check the generated `sqlx-data.json` file into version control.
/// * Check the generated `.sqlx` directory into version control.
/// * Don't have `DATABASE_URL` set during compilation.
///
/// Your project can now be built without a database connection (you must omit `DATABASE_URL` or
/// else it will still try to connect). To update the generated file simply run `cargo sqlx prepare`
/// again.
///
/// To ensure that your `sqlx-data.json` file is kept up-to-date, both with the queries in your
/// To ensure that your `.sqlx` directory is kept up-to-date, both with the queries in your
/// project and your database schema itself, run
/// `cargo install sqlx-cli && cargo sqlx prepare --check` in your Continuous Integration script.
///