Make powdr generic over the pgo method (#3188)

Introduce `PgoAdapter` which wraps `Adapter` and can call apc
generation. Removes the match statements over the pgo method. Breaking
change. Now the client can pick some pgo implementations like
`pgo::CellPgo` and use it like
`CellPgo::<MyAdapter>::new(pgo_data).generate_apcs(blocks)`. The client
can also implement `PgoAdapter<A: Adapter>` for its own pgo
implementation.
This commit is contained in:
Thibaut Schaeffer
2025-08-22 15:59:23 +02:00
committed by GitHub
parent 4adeccd914
commit 86c31acd9a
14 changed files with 515 additions and 374 deletions

View File

@@ -6,13 +6,62 @@ use powdr_number::FieldElement;
use serde::{Deserialize, Serialize};
use crate::{
blocks::{BasicBlock, Candidate, Instruction, Program},
blocks::{BasicBlock, Instruction, Program},
constraint_optimizer::IsBusStateful,
memory_optimizer::MemoryBusInteraction,
range_constraint_optimizer::RangeConstraintHandler,
Apc, InstructionHandler, VmConfig,
Apc, InstructionHandler, PowdrConfig, VmConfig,
};
pub struct ApcWithStats<F, I, S> {
apc: Apc<F, I>,
stats: Option<S>,
}
impl<F, I, S> ApcWithStats<F, I, S> {
pub fn with_stats(mut self, stats: S) -> Self {
self.stats = Some(stats);
self
}
pub fn into_parts(self) -> (Apc<F, I>, Option<S>) {
(self.apc, self.stats)
}
}
impl<F, I, S> From<Apc<F, I>> for ApcWithStats<F, I, S> {
fn from(apc: Apc<F, I>) -> Self {
Self { apc, stats: None }
}
}
pub trait PgoAdapter {
type Adapter: Adapter;
fn filter_blocks_and_create_apcs_with_pgo(
&self,
blocks: Vec<BasicBlock<<Self::Adapter as Adapter>::Instruction>>,
config: &PowdrConfig,
vm_config: AdapterVmConfig<Self::Adapter>,
) -> Vec<AdapterApcWithStats<Self::Adapter>> {
let filtered_blocks = blocks
.into_iter()
.filter(|block| !Self::Adapter::should_skip_block(block))
.collect();
self.create_apcs_with_pgo(filtered_blocks, config, vm_config)
}
fn create_apcs_with_pgo(
&self,
blocks: Vec<BasicBlock<<Self::Adapter as Adapter>::Instruction>>,
config: &PowdrConfig,
vm_config: AdapterVmConfig<Self::Adapter>,
) -> Vec<AdapterApcWithStats<Self::Adapter>>;
fn pc_execution_count(&self, _pc: u64) -> Option<u32> {
None
}
}
pub trait Adapter: Sized {
type Field: Serialize + for<'de> Deserialize<'de> + Send + Clone;
type PowdrField: FieldElement;
@@ -22,7 +71,6 @@ pub trait Adapter: Sized {
+ IsBusStateful<Self::PowdrField>
+ RangeConstraintHandler<Self::PowdrField>
+ Sync;
type Candidate: Candidate<Self> + Send;
type Program: Program<Self::Instruction> + Send;
type Instruction: Instruction<Self::Field> + Serialize + for<'de> Deserialize<'de> + Send;
type MemoryBusInteraction<V: Ord + Clone + Eq + Display + Hash>: MemoryBusInteraction<
@@ -30,6 +78,7 @@ pub trait Adapter: Sized {
V,
>;
type CustomBusTypes: Clone + Display + Sync + Eq + PartialEq;
type ApcStats: Send + Sync;
fn into_field(e: Self::PowdrField) -> Self::Field;
@@ -40,7 +89,9 @@ pub trait Adapter: Sized {
}
}
pub type ApcStats<A> = <<A as Adapter>::Candidate as Candidate<A>>::ApcStats;
pub type AdapterApcWithStats<A> =
ApcWithStats<<A as Adapter>::Field, <A as Adapter>::Instruction, <A as Adapter>::ApcStats>;
pub type ApcStats<A> = <A as Adapter>::ApcStats;
pub type AdapterApc<A> = Apc<<A as Adapter>::Field, <A as Adapter>::Instruction>;
pub type AdapterVmConfig<'a, A> = VmConfig<
'a,

View File

@@ -3,15 +3,8 @@ use serde::{Deserialize, Serialize};
/// Tools to detect basic blocks in a program
mod detection;
/// Tools to generate autoprecompiles using different PGO strategies
mod pgo;
/// Tools to select autoprecompiles using a knapsack-like algorithm
mod selection;
pub use detection::collect_basic_blocks;
pub use pgo::{generate_apcs_with_pgo, ApcCandidateJsonExport, Candidate};
pub use pgo::{pgo_config, PgoConfig, PgoType};
pub use selection::KnapsackItem;
#[derive(Debug, Serialize, Deserialize, Clone)]
pub struct BasicBlock<I> {

View File

@@ -1,319 +0,0 @@
use std::{
collections::HashMap,
io::BufWriter,
path::Path,
sync::{Arc, Mutex},
};
use rayon::iter::{IndexedParallelIterator, IntoParallelIterator, ParallelIterator};
use serde::{Deserialize, Serialize};
use strum::{Display, EnumString};
use crate::{
adapter::{Adapter, AdapterApc, AdapterVmConfig, ApcStats},
blocks::selection::{parallel_fractional_knapsack, KnapsackItem},
evaluation::EvaluationResult,
BasicBlock, PowdrConfig,
};
/// Three modes for profiler guided optimization with different cost functions to sort the basic blocks by descending cost and select the most costly ones to accelerate.
/// The inner HashMap contains number of time a pc is executed.
#[derive(Default)]
pub enum PgoConfig {
/// value = cells saved per apc * times executed
/// cost = number of columns in the apc
/// constraint of max total columns
Cell(HashMap<u64, u32>, Option<usize>),
/// value = instruction per apc * times executed
Instruction(HashMap<u64, u32>),
/// value = instruction per apc
#[default]
None,
}
impl PgoConfig {
/// Returns the number of times a certain pc was executed in the profile.
pub fn pc_execution_count(&self, pc: u64) -> Option<u32> {
match self {
PgoConfig::Cell(pc_count, _) | PgoConfig::Instruction(pc_count) => {
pc_count.get(&pc).copied()
}
PgoConfig::None => None,
}
}
}
/// CLI enum for PGO mode
#[derive(Copy, Clone, Debug, EnumString, Display, Default)]
#[strum(serialize_all = "lowercase")]
pub enum PgoType {
/// cost = cells saved per apc * times executed
#[default]
Cell,
/// cost = instruction per apc * times executed
Instruction,
/// cost = instruction per apc
None,
}
pub fn pgo_config(
pgo: PgoType,
max_columns: Option<usize>,
execution_profile: HashMap<u64, u32>,
) -> PgoConfig {
match pgo {
PgoType::Cell => PgoConfig::Cell(execution_profile, max_columns),
PgoType::Instruction => PgoConfig::Instruction(execution_profile),
PgoType::None => PgoConfig::None,
}
}
/// Trait for autoprecompile candidates.
/// Implementors of this trait wrap an APC with additional data used by the `KnapsackItem` trait to select the most cost-effective APCs.
pub trait Candidate<A: Adapter>: Sized + KnapsackItem {
type ApcStats;
/// Try to create an autoprecompile candidate from a block.
fn create(
apc: AdapterApc<A>,
pgo_program_pc_count: &HashMap<u64, u32>,
vm_config: AdapterVmConfig<A>,
max_degree: usize,
) -> Self;
/// Return a JSON export of the APC candidate.
fn to_json_export(
&self,
apc_candidates_dir_path: &Path,
) -> ApcCandidateJsonExport<A::Instruction>;
/// Convert the candidate into an autoprecompile and its statistics.
fn into_apc_and_stats(self) -> (AdapterApc<A>, Self::ApcStats);
}
#[derive(Serialize, Deserialize)]
pub struct ApcCandidateJsonExport<I> {
// execution_frequency
pub execution_frequency: usize,
// original instructions
pub original_block: BasicBlock<I>,
// before and after optimization stats
pub stats: EvaluationResult,
// width before optimisation, used for software version cells in effectiveness plot
pub width_before: usize,
// value used in ranking of candidates
pub value: usize,
// cost before optimisation, used for effectiveness calculation
pub cost_before: f64,
// cost after optimization, used for effectiveness calculation and ranking of candidates
pub cost_after: f64,
// path to the apc candidate file
pub apc_candidate_file: String,
}
// Note: This function can lead to OOM since it generates the apc for many blocks.
fn create_apcs_with_cell_pgo<A: Adapter>(
mut blocks: Vec<BasicBlock<A::Instruction>>,
pgo_program_pc_count: HashMap<u64, u32>,
config: &PowdrConfig,
max_total_apc_columns: Option<usize>,
vm_config: AdapterVmConfig<A>,
) -> Vec<(AdapterApc<A>, ApcStats<A>)> {
if config.autoprecompiles == 0 {
return vec![];
}
// drop any block whose start index cannot be found in pc_idx_count,
// because a basic block might not be executed at all.
// Also only keep basic blocks with more than one original instruction.
blocks.retain(|b| pgo_program_pc_count.contains_key(&b.start_pc) && b.statements.len() > 1);
tracing::debug!(
"Retained {} basic blocks after filtering by pc_idx_count",
blocks.len()
);
// generate apc for all basic blocks and only cache the ones we eventually use
// calculate number of trace cells saved per row for each basic block to sort them by descending cost
let max_cache = (config.autoprecompiles + config.skip_autoprecompiles) as usize;
tracing::info!(
"Generating autoprecompiles for all ({}) basic blocks in parallel and caching costliest {}",
blocks.len(),
max_cache,
);
let apc_candidates = Arc::new(Mutex::new(vec![]));
// mapreduce over blocks into a single BinaryHeap<ApcCandidate<P>> capped at max_cache
let res = parallel_fractional_knapsack(
blocks.into_par_iter().filter_map(|block| {
let apc = crate::build::<A>(
block.clone(),
vm_config.clone(),
config.degree_bound,
config.apc_candidates_dir_path.as_deref(),
)
.ok()?;
let candidate = A::Candidate::create(
apc,
&pgo_program_pc_count,
vm_config.clone(),
config.degree_bound.identities,
);
if let Some(apc_candidates_dir_path) = &config.apc_candidates_dir_path {
let json_export = candidate.to_json_export(apc_candidates_dir_path);
apc_candidates.lock().unwrap().push(json_export);
}
Some(candidate)
}),
max_cache,
max_total_apc_columns,
)
.skip(config.skip_autoprecompiles as usize)
.map(A::Candidate::into_apc_and_stats)
.collect();
// Write the APC candidates JSON to disk if the directory is specified.
if let Some(apc_candidates_dir_path) = &config.apc_candidates_dir_path {
let apc_candidates_json_file = apc_candidates.lock().unwrap();
let json_path = apc_candidates_dir_path.join("apc_candidates.json");
let file = std::fs::File::create(&json_path)
.expect("Failed to create file for APC candidates JSON");
serde_json::to_writer(BufWriter::new(file), &*apc_candidates_json_file)
.expect("Failed to write APC candidates JSON to file");
}
res
}
fn create_apcs_with_instruction_pgo<A: Adapter>(
mut blocks: Vec<BasicBlock<A::Instruction>>,
pgo_program_pc_count: HashMap<u64, u32>,
config: &PowdrConfig,
vm_config: AdapterVmConfig<A>,
) -> Vec<AdapterApc<A>> {
// drop any block whose start index cannot be found in pc_idx_count,
// because a basic block might not be executed at all.
// Also only keep basic blocks with more than one original instruction.
blocks.retain(|b| pgo_program_pc_count.contains_key(&b.start_pc) && b.statements.len() > 1);
tracing::debug!(
"Retained {} basic blocks after filtering by pc_idx_count",
blocks.len()
);
// cost = cells_saved_per_row
blocks.sort_by(|a, b| {
let a_cnt = pgo_program_pc_count[&a.start_pc];
let b_cnt = pgo_program_pc_count[&b.start_pc];
(b_cnt * (b.statements.len() as u32)).cmp(&(a_cnt * (a.statements.len() as u32)))
});
// Debug print blocks by descending cost
for block in &blocks {
let frequency = pgo_program_pc_count[&block.start_pc];
let number_of_instructions = block.statements.len();
let value = frequency * number_of_instructions as u32;
tracing::debug!(
"Basic block start_pc: {start_pc}, value: {value}, frequency: {frequency}, number_of_instructions: {number_of_instructions}",
start_pc = block.start_pc,
);
}
create_apcs_for_all_blocks::<A>(blocks, config, vm_config)
}
fn create_apcs_with_no_pgo<A: Adapter>(
mut blocks: Vec<BasicBlock<A::Instruction>>,
config: &PowdrConfig,
vm_config: AdapterVmConfig<A>,
) -> Vec<AdapterApc<A>> {
// cost = number_of_original_instructions
blocks.sort_by(|a, b| b.statements.len().cmp(&a.statements.len()));
// Debug print blocks by descending cost
for block in &blocks {
tracing::debug!(
"Basic block start_pc: {}, number_of_instructions: {}",
block.start_pc,
block.statements.len(),
);
}
create_apcs_for_all_blocks::<A>(blocks, config, vm_config)
}
pub fn generate_apcs_with_pgo<A: Adapter>(
mut blocks: Vec<BasicBlock<A::Instruction>>,
config: &PowdrConfig,
max_total_apc_columns: Option<usize>,
pgo_config: PgoConfig,
vm_config: AdapterVmConfig<A>,
) -> Vec<(AdapterApc<A>, Option<ApcStats<A>>)> {
// filter out blocks that should be skipped according to the adapter
blocks.retain(|block| !A::should_skip_block(block));
// sort basic blocks by:
// 1. if PgoConfig::Cell, cost = frequency * cells_saved_per_row
// 2. if PgoConfig::Instruction, cost = frequency * number_of_instructions
// 3. if PgoConfig::None, cost = number_of_instructions
let res: Vec<_> = match pgo_config {
PgoConfig::Cell(pgo_program_idx_count, _) => create_apcs_with_cell_pgo::<A>(
blocks,
pgo_program_idx_count,
config,
max_total_apc_columns,
vm_config,
)
.into_iter()
.map(|(apc, apc_stats)| (apc, Some(apc_stats)))
.collect(),
PgoConfig::Instruction(pgo_program_idx_count) => {
create_apcs_with_instruction_pgo::<A>(blocks, pgo_program_idx_count, config, vm_config)
.into_iter()
.map(|apc| (apc, None))
.collect()
}
PgoConfig::None => create_apcs_with_no_pgo::<A>(blocks, config, vm_config)
.into_iter()
.map(|apc| (apc, None))
.collect(),
};
assert!(res.len() <= config.autoprecompiles as usize);
res
}
// Only used for PgoConfig::Instruction and PgoConfig::None,
// because PgoConfig::Cell caches all APCs in sorting stage.
fn create_apcs_for_all_blocks<A: Adapter>(
blocks: Vec<BasicBlock<A::Instruction>>,
config: &PowdrConfig,
vm_config: AdapterVmConfig<A>,
) -> Vec<AdapterApc<A>> {
let n_acc = config.autoprecompiles as usize;
tracing::info!("Generating {n_acc} autoprecompiles in parallel");
blocks
.into_par_iter()
.skip(config.skip_autoprecompiles as usize)
.take(n_acc)
.map(|block| {
tracing::debug!(
"Accelerating block of length {} and start pc {}",
block.statements.len(),
block.start_pc
);
crate::build::<A>(
block,
vm_config.clone(),
config.degree_bound,
config.apc_candidates_dir_path.as_deref(),
)
.unwrap()
})
.collect()
}

View File

@@ -1,9 +1,9 @@
use crate::adapter::{Adapter, AdapterApc, AdapterVmConfig};
use crate::blocks::BasicBlock;
use crate::bus_map::{BusMap, BusType};
use crate::evaluation::AirStats;
use crate::expression_conversion::algebraic_to_grouped_expression;
use crate::symbolic_machine_generator::convert_machine;
pub use blocks::{pgo_config, BasicBlock, PgoConfig, PgoType};
use expression::{AlgebraicExpression, AlgebraicReference};
use itertools::Itertools;
use powdr::UniqueReferences;
@@ -32,10 +32,12 @@ pub mod expression_conversion;
pub mod low_degree_bus_interaction_optimizer;
pub mod memory_optimizer;
pub mod optimizer;
pub mod pgo;
pub mod powdr;
pub mod range_constraint_optimizer;
mod stats_logger;
pub mod symbolic_machine_generator;
pub use pgo::{PgoConfig, PgoType};
pub use powdr_constraint_solver::inliner::DegreeBound;
#[derive(Clone)]

View File

@@ -0,0 +1,167 @@
use std::{
collections::HashMap,
io::BufWriter,
path::Path,
sync::{Arc, Mutex},
};
use rayon::iter::{IntoParallelIterator, ParallelIterator};
use serde::{Deserialize, Serialize};
use crate::{
adapter::{Adapter, AdapterApc, AdapterApcWithStats, AdapterVmConfig, PgoAdapter},
blocks::BasicBlock,
evaluation::EvaluationResult,
pgo::cell::selection::parallel_fractional_knapsack,
PowdrConfig,
};
mod selection;
pub use selection::KnapsackItem;
/// Trait for autoprecompile candidates.
/// Implementors of this trait wrap an APC with additional data used by the `KnapsackItem` trait to select the most cost-effective APCs.
pub trait Candidate<A: Adapter>: Sized + KnapsackItem {
/// Try to create an autoprecompile candidate from a block.
fn create(
apc: AdapterApc<A>,
pgo_program_pc_count: &HashMap<u64, u32>,
vm_config: AdapterVmConfig<A>,
max_degree: usize,
) -> Self;
/// Return a JSON export of the APC candidate.
fn to_json_export(
&self,
apc_candidates_dir_path: &Path,
) -> ApcCandidateJsonExport<A::Instruction>;
/// Convert the candidate into an autoprecompile and its statistics.
fn into_apc_and_stats(self) -> AdapterApcWithStats<A>;
}
#[derive(Serialize, Deserialize)]
pub struct ApcCandidateJsonExport<I> {
// execution_frequency
pub execution_frequency: usize,
// original instructions
pub original_block: BasicBlock<I>,
// before and after optimization stats
pub stats: EvaluationResult,
// width before optimisation, used for software version cells in effectiveness plot
pub width_before: usize,
// value used in ranking of candidates
pub value: usize,
// cost before optimisation, used for effectiveness calculation
pub cost_before: f64,
// cost after optimization, used for effectiveness calculation and ranking of candidates
pub cost_after: f64,
// path to the apc candidate file
pub apc_candidate_file: String,
}
pub struct CellPgo<A, C> {
_marker: std::marker::PhantomData<(A, C)>,
data: HashMap<u64, u32>,
max_total_apc_columns: Option<usize>,
}
impl<A, C> CellPgo<A, C> {
pub fn with_pgo_data_and_max_columns(
data: HashMap<u64, u32>,
max_total_apc_columns: Option<usize>,
) -> Self {
Self {
_marker: std::marker::PhantomData,
data,
max_total_apc_columns,
}
}
}
impl<A: Adapter + Send + Sync, C: Candidate<A> + Send + Sync> PgoAdapter for CellPgo<A, C> {
type Adapter = A;
fn create_apcs_with_pgo(
&self,
mut blocks: Vec<BasicBlock<<Self::Adapter as Adapter>::Instruction>>,
config: &PowdrConfig,
vm_config: AdapterVmConfig<Self::Adapter>,
) -> Vec<AdapterApcWithStats<Self::Adapter>> {
tracing::info!(
"Generating autoprecompiles with cell PGO for {} blocks",
blocks.len()
);
if config.autoprecompiles == 0 {
return vec![];
}
// drop any block whose start index cannot be found in pc_idx_count,
// because a basic block might not be executed at all.
// Also only keep basic blocks with more than one original instruction.
blocks.retain(|b| self.data.contains_key(&b.start_pc) && b.statements.len() > 1);
tracing::debug!(
"Retained {} basic blocks after filtering by pc_idx_count",
blocks.len()
);
// generate apc for all basic blocks and only cache the ones we eventually use
// calculate number of trace cells saved per row for each basic block to sort them by descending cost
let max_cache = (config.autoprecompiles + config.skip_autoprecompiles) as usize;
tracing::info!(
"Generating autoprecompiles for all ({}) basic blocks in parallel and caching costliest {}",
blocks.len(),
max_cache,
);
let apc_candidates = Arc::new(Mutex::new(vec![]));
// mapreduce over blocks into a single BinaryHeap<ApcCandidate<P>> capped at max_cache
let res = parallel_fractional_knapsack(
blocks.into_par_iter().filter_map(|block| {
let apc = crate::build::<A>(
block.clone(),
vm_config.clone(),
config.degree_bound,
config.apc_candidates_dir_path.as_deref(),
)
.ok()?;
let candidate = C::create(
apc,
&self.data,
vm_config.clone(),
config.degree_bound.identities,
);
if let Some(apc_candidates_dir_path) = &config.apc_candidates_dir_path {
let json_export = candidate.to_json_export(apc_candidates_dir_path);
apc_candidates.lock().unwrap().push(json_export);
}
Some(candidate)
}),
max_cache,
self.max_total_apc_columns,
)
.skip(config.skip_autoprecompiles as usize)
.map(C::into_apc_and_stats)
.collect();
// Write the APC candidates JSON to disk if the directory is specified.
if let Some(apc_candidates_dir_path) = &config.apc_candidates_dir_path {
let apc_candidates_json_file = apc_candidates.lock().unwrap();
let json_path = apc_candidates_dir_path.join("apc_candidates.json");
let file = std::fs::File::create(&json_path)
.expect("Failed to create file for APC candidates JSON");
serde_json::to_writer(BufWriter::new(file), &*apc_candidates_json_file)
.expect("Failed to write APC candidates JSON to file");
}
res
}
fn pc_execution_count(&self, pc: u64) -> Option<u32> {
self.data.get(&pc).cloned()
}
}

View File

@@ -0,0 +1,78 @@
use std::collections::HashMap;
use crate::{
adapter::{Adapter, AdapterApcWithStats, AdapterVmConfig, PgoAdapter},
blocks::BasicBlock,
pgo::create_apcs_for_all_blocks,
PowdrConfig,
};
pub struct InstructionPgo<A> {
_marker: std::marker::PhantomData<A>,
data: HashMap<u64, u32>,
}
impl<A> InstructionPgo<A> {
pub fn with_pgo_data(data: HashMap<u64, u32>) -> Self {
Self {
_marker: std::marker::PhantomData,
data,
}
}
}
impl<A: Adapter> PgoAdapter for InstructionPgo<A> {
type Adapter = A;
fn create_apcs_with_pgo(
&self,
mut blocks: Vec<BasicBlock<<Self::Adapter as Adapter>::Instruction>>,
config: &PowdrConfig,
vm_config: AdapterVmConfig<Self::Adapter>,
) -> Vec<AdapterApcWithStats<Self::Adapter>> {
tracing::info!(
"Generating autoprecompiles with instruction PGO for {} blocks",
blocks.len()
);
if config.autoprecompiles == 0 {
return vec![];
}
let pgo_program_pc_count: &HashMap<u64, u32> = &self.data;
// drop any block whose start index cannot be found in pc_idx_count,
// because a basic block might not be executed at all.
// Also only keep basic blocks with more than one original instruction.
blocks.retain(|b| pgo_program_pc_count.contains_key(&b.start_pc) && b.statements.len() > 1);
tracing::debug!(
"Retained {} basic blocks after filtering by pc_idx_count",
blocks.len()
);
// cost = cells_saved_per_row
blocks.sort_by(|a, b| {
let a_cnt = pgo_program_pc_count[&a.start_pc];
let b_cnt = pgo_program_pc_count[&b.start_pc];
(b_cnt * (b.statements.len() as u32)).cmp(&(a_cnt * (a.statements.len() as u32)))
});
// Debug print blocks by descending cost
for block in &blocks {
let frequency = pgo_program_pc_count[&block.start_pc];
let number_of_instructions = block.statements.len();
let value = frequency * number_of_instructions as u32;
tracing::debug!(
"Basic block start_pc: {start_pc}, value: {value}, frequency: {frequency}, number_of_instructions: {number_of_instructions}",
start_pc = block.start_pc,
);
}
create_apcs_for_all_blocks::<Self::Adapter>(blocks, config, vm_config)
}
fn pc_execution_count(&self, pc: u64) -> Option<u32> {
self.data.get(&pc).cloned()
}
}

View File

@@ -0,0 +1,105 @@
use std::collections::HashMap;
use rayon::iter::{IndexedParallelIterator, IntoParallelIterator, ParallelIterator};
use strum::{Display, EnumString};
use crate::{
adapter::{Adapter, AdapterApcWithStats, AdapterVmConfig, ApcWithStats},
blocks::BasicBlock,
PowdrConfig,
};
mod cell;
mod instruction;
mod none;
pub use {
cell::{ApcCandidateJsonExport, Candidate, CellPgo, KnapsackItem},
instruction::InstructionPgo,
none::NonePgo,
};
/// Three modes for profiler guided optimization with different cost functions to sort the basic blocks by descending cost and select the most costly ones to accelerate.
/// The inner HashMap contains number of time a pc is executed.
#[derive(Default)]
pub enum PgoConfig {
/// value = cells saved per apc * times executed
/// cost = number of columns in the apc
/// constraint of max total columns
Cell(HashMap<u64, u32>, Option<usize>),
/// value = instruction per apc * times executed
Instruction(HashMap<u64, u32>),
/// value = instruction per apc
#[default]
None,
}
impl PgoConfig {
/// Returns the number of times a certain pc was executed in the profile.
pub fn pc_execution_count(&self, pc: u64) -> Option<u32> {
match self {
PgoConfig::Cell(pc_count, _) | PgoConfig::Instruction(pc_count) => {
pc_count.get(&pc).copied()
}
PgoConfig::None => None,
}
}
}
/// CLI enum for PGO mode
#[derive(Copy, Clone, Debug, EnumString, Display, Default)]
#[strum(serialize_all = "lowercase")]
pub enum PgoType {
/// cost = cells saved per apc * times executed
#[default]
Cell,
/// cost = instruction per apc * times executed
Instruction,
/// cost = instruction per apc
None,
}
pub fn pgo_config(
pgo: PgoType,
max_columns: Option<usize>,
execution_profile: HashMap<u64, u32>,
) -> PgoConfig {
match pgo {
PgoType::Cell => PgoConfig::Cell(execution_profile, max_columns),
PgoType::Instruction => PgoConfig::Instruction(execution_profile),
PgoType::None => PgoConfig::None,
}
}
// Only used for PgoConfig::Instruction and PgoConfig::None,
// because PgoConfig::Cell caches all APCs in sorting stage.
fn create_apcs_for_all_blocks<A: Adapter>(
blocks: Vec<BasicBlock<A::Instruction>>,
config: &PowdrConfig,
vm_config: AdapterVmConfig<A>,
) -> Vec<AdapterApcWithStats<A>> {
let n_acc = config.autoprecompiles as usize;
tracing::info!("Generating {n_acc} autoprecompiles in parallel");
blocks
.into_par_iter()
.skip(config.skip_autoprecompiles as usize)
.take(n_acc)
.map(|block| {
tracing::debug!(
"Accelerating block of length {} and start pc {}",
block.statements.len(),
block.start_pc
);
crate::build::<A>(
block,
vm_config.clone(),
config.degree_bound,
config.apc_candidates_dir_path.as_deref(),
)
.unwrap()
})
.map(ApcWithStats::from)
.collect()
}

View File

@@ -0,0 +1,44 @@
use crate::{
adapter::{Adapter, AdapterApcWithStats, AdapterVmConfig, PgoAdapter},
blocks::BasicBlock,
pgo::create_apcs_for_all_blocks,
PowdrConfig,
};
pub struct NonePgo<A> {
_marker: std::marker::PhantomData<A>,
}
// TODO: derive with explicit bounds
impl<A> Default for NonePgo<A> {
fn default() -> Self {
Self {
_marker: std::marker::PhantomData,
}
}
}
impl<A: Adapter> PgoAdapter for NonePgo<A> {
type Adapter = A;
fn create_apcs_with_pgo(
&self,
mut blocks: Vec<BasicBlock<<Self::Adapter as Adapter>::Instruction>>,
config: &PowdrConfig,
vm_config: AdapterVmConfig<Self::Adapter>,
) -> Vec<AdapterApcWithStats<Self::Adapter>> {
// cost = number_of_original_instructions
blocks.sort_by(|a, b| b.statements.len().cmp(&a.statements.len()));
// Debug print blocks by descending cost
for block in &blocks {
tracing::debug!(
"Basic block start_pc: {}, number_of_instructions: {}",
block.start_pc,
block.statements.len(),
);
}
create_apcs_for_all_blocks::<Self::Adapter>(blocks, config, vm_config)
}
}

View File

@@ -3,8 +3,10 @@ use powdr_expression::AlgebraicBinaryOperation;
use powdr_number::FieldElement;
use crate::{
adapter::Adapter, blocks::Instruction, expression::AlgebraicExpression, powdr, BasicBlock,
BusMap, BusType, InstructionHandler, SymbolicBusInteraction, SymbolicConstraint,
adapter::Adapter,
blocks::{BasicBlock, Instruction},
expression::AlgebraicExpression,
powdr, BusMap, BusType, InstructionHandler, SymbolicBusInteraction, SymbolicConstraint,
SymbolicMachine,
};

View File

@@ -3,7 +3,7 @@ use metrics_tracing_context::{MetricsLayer, TracingContextLayer};
use metrics_util::{debugging::DebuggingRecorder, layers::Layer};
use openvm_sdk::StdIn;
use openvm_stark_sdk::bench::serialize_metric_snapshot;
use powdr_autoprecompiles::{pgo_config, PgoType};
use powdr_autoprecompiles::pgo::{pgo_config, PgoType};
use powdr_openvm::{
default_powdr_openvm_config, CompiledProgram, GuestOptions, PrecompileImplementation,
};

View File

@@ -25,13 +25,13 @@ use openvm_stark_backend::{
p3_field::{FieldAlgebra, PrimeField32},
};
use openvm_stark_sdk::p3_baby_bear::BabyBear;
use powdr_autoprecompiles::adapter::{Adapter, AdapterApc, AdapterVmConfig};
use powdr_autoprecompiles::blocks::{
collect_basic_blocks, ApcCandidateJsonExport, Instruction, Program,
use powdr_autoprecompiles::adapter::{
Adapter, AdapterApc, AdapterApcWithStats, AdapterVmConfig, ApcWithStats, PgoAdapter,
};
use powdr_autoprecompiles::blocks::{generate_apcs_with_pgo, Candidate, KnapsackItem, PgoConfig};
use powdr_autoprecompiles::blocks::{collect_basic_blocks, Instruction, Program};
use powdr_autoprecompiles::evaluation::{evaluate_apc, EvaluationResult};
use powdr_autoprecompiles::expression::try_convert;
use powdr_autoprecompiles::pgo::{ApcCandidateJsonExport, Candidate, KnapsackItem};
use powdr_autoprecompiles::SymbolicBusInteraction;
use powdr_autoprecompiles::VmConfig;
use powdr_autoprecompiles::{Apc, PowdrConfig};
@@ -70,12 +70,12 @@ impl<'a> Adapter for BabyBearOpenVmApcAdapter<'a> {
type Field = BabyBear;
type InstructionHandler = OriginalAirs<Self::Field>;
type BusInteractionHandler = OpenVmBusInteractionHandler<Self::PowdrField>;
type Candidate = OpenVmApcCandidate<Self::Field, Instr<Self::Field>>;
type Program = Prog<'a, Self::Field>;
type Instruction = Instr<Self::Field>;
type MemoryBusInteraction<V: Ord + Clone + Eq + Display + Hash> =
OpenVmMemoryBusInteraction<Self::PowdrField, V>;
type CustomBusTypes = OpenVmBusType;
type ApcStats = OvmApcStats;
fn into_field(e: Self::PowdrField) -> Self::Field {
openvm_stark_sdk::p3_baby_bear::BabyBear::from_canonical_u32(
@@ -145,13 +145,13 @@ impl<'a, F: PrimeField32> Program<Instr<F>> for Prog<'a, F> {
}
}
pub fn customize(
pub fn customize<'a, P: PgoAdapter<Adapter = BabyBearOpenVmApcAdapter<'a>>>(
OriginalCompiledProgram { mut exe, vm_config }: OriginalCompiledProgram,
labels: &BTreeSet<u32>,
debug_info: &DebugInfo,
config: PowdrConfig,
implementation: PrecompileImplementation,
pgo_config: PgoConfig,
pgo: P,
) -> CompiledProgram {
let original_config = OriginalVmConfig::new(vm_config.clone());
let airs = original_config.airs(config.degree_bound.identities).expect("Failed to convert the AIR of an OpenVM instruction, even after filtering by the blacklist!");
@@ -180,18 +180,6 @@ pub fn customize(
bus_map: bus_map.clone(),
};
let max_total_apc_columns: Option<usize> = match pgo_config {
PgoConfig::Cell(_, max_total_columns) => max_total_columns.map(|max_total_columns| {
let total_non_apc_columns = original_config
.chip_inventory_air_metrics(config.degree_bound.identities)
.values()
.map(|m| m.total_width())
.sum::<usize>();
max_total_columns - total_non_apc_columns
}),
PgoConfig::Instruction(_) | PgoConfig::None => None,
};
// Convert the jump destinations to u64 for compatibility with the `collect_basic_blocks` function.
let jumpdest_set = jumpdest_set
.iter()
@@ -207,7 +195,7 @@ pub fn customize(
tracing::debug!("Basic blocks sorted by execution count (top 10):");
for (count, block) in blocks
.iter()
.filter_map(|block| Some((pgo_config.pc_execution_count(block.start_pc)?, block)))
.filter_map(|block| Some((pgo.pc_execution_count(block.start_pc)?, block)))
.sorted_by_key(|(count, _)| *count)
.rev()
.take(10)
@@ -225,13 +213,7 @@ pub fn customize(
}
let start = std::time::Instant::now();
let apcs = generate_apcs_with_pgo::<BabyBearOpenVmApcAdapter>(
blocks,
&config,
max_total_apc_columns,
pgo_config,
vm_config,
);
let apcs = pgo.filter_blocks_and_create_apcs_with_pgo(blocks, &config, vm_config);
metrics::gauge!("total_apc_gen_time_ms").set(start.elapsed().as_millis() as f64);
let pc_base = exe.program.pc_base;
@@ -242,6 +224,7 @@ pub fn customize(
let extensions = apcs
.into_iter()
.map(ApcWithStats::into_parts)
.enumerate()
.map(|(i, (apc, apc_stats))| {
let Apc {
@@ -358,8 +341,6 @@ impl OvmApcStats {
}
impl<'a> Candidate<BabyBearOpenVmApcAdapter<'a>> for OpenVmApcCandidate<BabyBear, Instr<BabyBear>> {
type ApcStats = OvmApcStats;
fn create(
apc: AdapterApc<BabyBearOpenVmApcAdapter<'a>>,
pgo_program_pc_count: &HashMap<u64, u32>,
@@ -420,8 +401,8 @@ impl<'a> Candidate<BabyBearOpenVmApcAdapter<'a>> for OpenVmApcCandidate<BabyBear
}
}
fn into_apc_and_stats(self) -> (AdapterApc<BabyBearOpenVmApcAdapter<'a>>, Self::ApcStats) {
(self.apc, OvmApcStats::new(self.widths))
fn into_apc_and_stats(self) -> AdapterApcWithStats<BabyBearOpenVmApcAdapter<'a>> {
ApcWithStats::from(self.apc).with_stats(OvmApcStats::new(self.widths))
}
}

View File

@@ -29,6 +29,7 @@ use openvm_stark_sdk::engine::StarkFriEngine;
use openvm_stark_sdk::openvm_stark_backend::p3_field::PrimeField32;
use openvm_stark_sdk::p3_baby_bear::BabyBear;
use powdr_autoprecompiles::evaluation::AirStats;
use powdr_autoprecompiles::pgo::{CellPgo, InstructionPgo, NonePgo};
use powdr_autoprecompiles::{execution_profile::execution_profile, PowdrConfig};
use powdr_extension::{PowdrExecutor, PowdrExtension, PowdrPeriphery};
use powdr_openvm_hints_circuit::{HintsExecutor, HintsExtension, HintsPeriphery};
@@ -45,6 +46,7 @@ use std::{
sync::Arc,
};
use crate::customize_exe::OpenVmApcCandidate;
pub use crate::customize_exe::Prog;
use tracing::Level;
@@ -375,14 +377,48 @@ pub fn compile_exe_with_elf(
pgo_config: PgoConfig,
) -> Result<CompiledProgram, Box<dyn std::error::Error>> {
let elf = powdr_riscv_elf::load_elf_from_buffer(elf);
let compiled = customize(
original_program,
elf.text_labels(),
elf.debug_info(),
config,
implementation,
pgo_config,
);
let compiled = match pgo_config {
PgoConfig::Cell(pgo_data, max_total_columns) => {
let max_total_apc_columns: Option<usize> = max_total_columns.map(|max_total_columns| {
let original_config = OriginalVmConfig::new(original_program.vm_config.clone());
let total_non_apc_columns = original_config
.chip_inventory_air_metrics(config.degree_bound.identities)
.values()
.map(|m| m.total_width())
.sum::<usize>();
max_total_columns - total_non_apc_columns
});
customize(
original_program,
elf.text_labels(),
elf.debug_info(),
config,
implementation,
CellPgo::<_, OpenVmApcCandidate<_, _>>::with_pgo_data_and_max_columns(
pgo_data,
max_total_apc_columns,
),
)
}
PgoConfig::Instruction(pgo_data) => customize(
original_program,
elf.text_labels(),
elf.debug_info(),
config,
implementation,
InstructionPgo::with_pgo_data(pgo_data),
),
PgoConfig::None => customize(
original_program,
elf.text_labels(),
elf.debug_info(),
config,
implementation,
NonePgo::default(),
),
};
// Export the compiled program to a PIL file for debugging purposes.
export_pil(
&mut BufWriter::new(File::create("debug.pil").unwrap()),

View File

@@ -1,8 +1,9 @@
use openvm_instructions::instruction::Instruction;
use openvm_sdk::config::SdkVmConfig;
use openvm_stark_sdk::p3_baby_bear::BabyBear;
use powdr_autoprecompiles::blocks::BasicBlock;
use powdr_autoprecompiles::evaluation::evaluate_apc;
use powdr_autoprecompiles::{build, BasicBlock, VmConfig};
use powdr_autoprecompiles::{build, VmConfig};
use powdr_number::BabyBearField;
use powdr_openvm::bus_interaction_handler::OpenVmBusInteractionHandler;
use powdr_openvm::extraction_utils::OriginalVmConfig;