move column family constants into db module (#1955)

This commit is contained in:
Robert Habermeier 2016-08-18 18:24:49 +02:00 committed by Gav Wood
parent 8159d743ef
commit 9adf77581c
11 changed files with 83 additions and 80 deletions

View File

@ -29,8 +29,7 @@ use blockchain::best_block::BestBlock;
use types::tree_route::TreeRoute;
use blockchain::update::ExtrasUpdate;
use blockchain::{CacheSize, ImportRoute, Config};
use db::{Writable, Readable, CacheUpdatePolicy};
use client::{DB_COL_EXTRA, DB_COL_HEADERS, DB_COL_BODIES};
use db::{self, Writable, Readable, CacheUpdatePolicy};
use cache_manager::CacheManager;
const LOG_BLOOMS_LEVELS: usize = 3;
@ -133,7 +132,7 @@ enum CacheID {
impl bc::group::BloomGroupDatabase for BlockChain {
fn blooms_at(&self, position: &bc::group::GroupPosition) -> Option<bc::group::BloomGroup> {
let position = LogGroupPosition::from(position.clone());
let result = self.db.read_with_cache(DB_COL_EXTRA, &self.blocks_blooms, &position).map(Into::into);
let result = self.db.read_with_cache(db::COL_EXTRA, &self.blocks_blooms, &position).map(Into::into);
self.cache_man.lock().note_used(CacheID::BlocksBlooms(position));
result
}
@ -172,7 +171,7 @@ impl BlockProvider for BlockChain {
/// Returns true if the given block is known
/// (though not necessarily a part of the canon chain).
fn is_known(&self, hash: &H256) -> bool {
self.db.exists_with_cache(DB_COL_EXTRA, &self.block_details, hash)
self.db.exists_with_cache(db::COL_EXTRA, &self.block_details, hash)
}
/// Get raw block data
@ -209,7 +208,7 @@ impl BlockProvider for BlockChain {
}
// Read from DB and populate cache
let opt = self.db.get(DB_COL_HEADERS, hash)
let opt = self.db.get(db::COL_HEADERS, hash)
.expect("Low level database error. Some issue with disk?");
let result = match opt {
@ -245,7 +244,7 @@ impl BlockProvider for BlockChain {
}
// Read from DB and populate cache
let opt = self.db.get(DB_COL_BODIES, hash)
let opt = self.db.get(db::COL_BODIES, hash)
.expect("Low level database error. Some issue with disk?");
let result = match opt {
@ -265,28 +264,28 @@ impl BlockProvider for BlockChain {
/// Get the familial details concerning a block.
fn block_details(&self, hash: &H256) -> Option<BlockDetails> {
let result = self.db.read_with_cache(DB_COL_EXTRA, &self.block_details, hash);
let result = self.db.read_with_cache(db::COL_EXTRA, &self.block_details, hash);
self.cache_man.lock().note_used(CacheID::BlockDetails(hash.clone()));
result
}
/// Get the hash of given block's number.
fn block_hash(&self, index: BlockNumber) -> Option<H256> {
let result = self.db.read_with_cache(DB_COL_EXTRA, &self.block_hashes, &index);
let result = self.db.read_with_cache(db::COL_EXTRA, &self.block_hashes, &index);
self.cache_man.lock().note_used(CacheID::BlockHashes(index));
result
}
/// Get the address of transaction with given hash.
fn transaction_address(&self, hash: &H256) -> Option<TransactionAddress> {
let result = self.db.read_with_cache(DB_COL_EXTRA, &self.transaction_addresses, hash);
let result = self.db.read_with_cache(db::COL_EXTRA, &self.transaction_addresses, hash);
self.cache_man.lock().note_used(CacheID::TransactionAddresses(hash.clone()));
result
}
/// Get receipts of block with given hash.
fn block_receipts(&self, hash: &H256) -> Option<BlockReceipts> {
let result = self.db.read_with_cache(DB_COL_EXTRA, &self.block_receipts, hash);
let result = self.db.read_with_cache(db::COL_EXTRA, &self.block_receipts, hash);
self.cache_man.lock().note_used(CacheID::BlockReceipts(hash.clone()));
result
}
@ -347,7 +346,7 @@ impl BlockChain {
};
// load best block
let best_block_hash = match bc.db.get(DB_COL_EXTRA, b"best").unwrap() {
let best_block_hash = match bc.db.get(db::COL_EXTRA, b"best").unwrap() {
Some(best) => {
H256::from_slice(&best)
}
@ -366,12 +365,12 @@ impl BlockChain {
};
let batch = DBTransaction::new(&db);
batch.put(DB_COL_HEADERS, &hash, block.header_rlp().as_raw());
batch.put(DB_COL_BODIES, &hash, &Self::block_to_body(genesis));
batch.put(db::COL_HEADERS, &hash, block.header_rlp().as_raw());
batch.put(db::COL_BODIES, &hash, &Self::block_to_body(genesis));
batch.write(DB_COL_EXTRA, &hash, &details);
batch.write(DB_COL_EXTRA, &header.number(), &hash);
batch.put(DB_COL_EXTRA, b"best", &hash);
batch.write(db::COL_EXTRA, &hash, &details);
batch.write(db::COL_EXTRA, &header.number(), &hash);
batch.put(db::COL_EXTRA, b"best", &hash);
bc.db.write(batch).expect("Low level database error. Some issue with disk?");
hash
}
@ -399,7 +398,7 @@ impl BlockChain {
/// Returns true if the given parent block has given child
/// (though not necessarily a part of the canon chain).
fn is_known_child(&self, parent: &H256, hash: &H256) -> bool {
self.db.read_with_cache(DB_COL_EXTRA, &self.block_details, parent).map_or(false, |d| d.children.contains(hash))
self.db.read_with_cache(db::COL_EXTRA, &self.block_details, parent).map_or(false, |d| d.children.contains(hash))
}
/// Rewind to a previous block
@ -408,22 +407,22 @@ impl BlockChain {
use db::Key;
let batch = self.db.transaction();
// track back to the best block we have in the blocks database
if let Some(best_block_hash) = self.db.get(DB_COL_EXTRA, b"best").unwrap() {
if let Some(best_block_hash) = self.db.get(db::COL_EXTRA, b"best").unwrap() {
let best_block_hash = H256::from_slice(&best_block_hash);
if best_block_hash == self.genesis_hash() {
return None;
}
if let Some(extras) = self.db.read(DB_COL_EXTRA, &best_block_hash) as Option<BlockDetails> {
if let Some(extras) = self.db.read(db::COL_EXTRA, &best_block_hash) as Option<BlockDetails> {
type DetailsKey = Key<BlockDetails, Target=H264>;
batch.delete(DB_COL_EXTRA, &(DetailsKey::key(&best_block_hash)));
batch.delete(db::COL_EXTRA, &(DetailsKey::key(&best_block_hash)));
let hash = extras.parent;
let range = extras.number as bc::Number .. extras.number as bc::Number;
let chain = bc::group::BloomGroupChain::new(self.blooms_config, self);
let changes = chain.replace(&range, vec![]);
for (k, v) in changes.into_iter() {
batch.write(DB_COL_EXTRA, &LogGroupPosition::from(k), &BloomGroup::from(v));
batch.write(db::COL_EXTRA, &LogGroupPosition::from(k), &BloomGroup::from(v));
}
batch.put(DB_COL_EXTRA, b"best", &hash);
batch.put(db::COL_EXTRA, b"best", &hash);
let best_block_total_difficulty = self.block_details(&hash).unwrap().total_difficulty;
let best_block_rlp = self.block(&hash).unwrap();
@ -436,9 +435,9 @@ impl BlockChain {
block: best_block_rlp,
};
// update parent extras
if let Some(mut details) = self.db.read(DB_COL_EXTRA, &hash) as Option<BlockDetails> {
if let Some(mut details) = self.db.read(db::COL_EXTRA, &hash) as Option<BlockDetails> {
details.children.clear();
batch.write(DB_COL_EXTRA, &hash, &details);
batch.write(db::COL_EXTRA, &hash, &details);
}
self.db.write(batch).expect("Writing to db failed");
self.block_details.write().clear();
@ -566,8 +565,8 @@ impl BlockChain {
let compressed_body = UntrustedRlp::new(&Self::block_to_body(bytes)).compress(RlpType::Blocks);
// store block in db
batch.put(DB_COL_HEADERS, &hash, &compressed_header);
batch.put(DB_COL_BODIES, &hash, &compressed_body);
batch.put(db::COL_HEADERS, &hash, &compressed_header);
batch.put(db::COL_BODIES, &hash, &compressed_body);
let maybe_parent = self.block_details(&header.parent_hash());
@ -645,7 +644,7 @@ impl BlockChain {
let mut write_details = self.block_details.write();
batch.extend_with_cache(DB_COL_EXTRA, &mut *write_details, update, CacheUpdatePolicy::Overwrite);
batch.extend_with_cache(db::COL_EXTRA, &mut *write_details, update, CacheUpdatePolicy::Overwrite);
self.cache_man.lock().note_used(CacheID::BlockDetails(block_hash));
@ -669,8 +668,8 @@ impl BlockChain {
assert!(self.pending_best_block.read().is_none());
// store block in db
batch.put_compressed(DB_COL_HEADERS, &hash, block.header_rlp().as_raw().to_vec());
batch.put_compressed(DB_COL_BODIES, &hash, Self::block_to_body(bytes));
batch.put_compressed(db::COL_HEADERS, &hash, block.header_rlp().as_raw().to_vec());
batch.put_compressed(db::COL_BODIES, &hash, Self::block_to_body(bytes));
let info = self.block_info(&header);
@ -742,7 +741,7 @@ impl BlockChain {
let block_hashes: Vec<_> = update.block_details.keys().cloned().collect();
let mut write_details = self.block_details.write();
batch.extend_with_cache(DB_COL_EXTRA, &mut *write_details, update.block_details, CacheUpdatePolicy::Overwrite);
batch.extend_with_cache(db::COL_EXTRA, &mut *write_details, update.block_details, CacheUpdatePolicy::Overwrite);
let mut cache_man = self.cache_man.lock();
for hash in block_hashes {
@ -752,12 +751,12 @@ impl BlockChain {
{
let mut write_receipts = self.block_receipts.write();
batch.extend_with_cache(DB_COL_EXTRA, &mut *write_receipts, update.block_receipts, CacheUpdatePolicy::Remove);
batch.extend_with_cache(db::COL_EXTRA, &mut *write_receipts, update.block_receipts, CacheUpdatePolicy::Remove);
}
{
let mut write_blocks_blooms = self.blocks_blooms.write();
batch.extend_with_cache(DB_COL_EXTRA, &mut *write_blocks_blooms, update.blocks_blooms, CacheUpdatePolicy::Remove);
batch.extend_with_cache(db::COL_EXTRA, &mut *write_blocks_blooms, update.blocks_blooms, CacheUpdatePolicy::Remove);
}
// These cached values must be updated last with all three locks taken to avoid
@ -768,7 +767,7 @@ impl BlockChain {
match update.info.location {
BlockLocation::Branch => (),
_ => if is_best {
batch.put(DB_COL_EXTRA, b"best", &update.info.hash);
batch.put(db::COL_EXTRA, b"best", &update.info.hash);
*best_block = Some(BestBlock {
hash: update.info.hash,
number: update.info.number,
@ -780,8 +779,8 @@ impl BlockChain {
let mut write_hashes = self.pending_block_hashes.write();
let mut write_txs = self.pending_transaction_addresses.write();
batch.extend_with_cache(DB_COL_EXTRA, &mut *write_hashes, update.block_hashes, CacheUpdatePolicy::Overwrite);
batch.extend_with_cache(DB_COL_EXTRA, &mut *write_txs, update.transactions_addresses, CacheUpdatePolicy::Overwrite);
batch.extend_with_cache(db::COL_EXTRA, &mut *write_hashes, update.block_hashes, CacheUpdatePolicy::Overwrite);
batch.extend_with_cache(db::COL_EXTRA, &mut *write_txs, update.transactions_addresses, CacheUpdatePolicy::Overwrite);
}
}
@ -1078,10 +1077,9 @@ mod tests {
use devtools::*;
use blockchain::generator::{ChainGenerator, ChainIterator, BlockFinalizer};
use views::BlockView;
use client;
fn new_db(path: &str) -> Arc<Database> {
Arc::new(Database::open(&DatabaseConfig::with_columns(client::DB_NO_OF_COLUMNS), path).unwrap())
Arc::new(Database::open(&DatabaseConfig::with_columns(::db::NUM_COLUMNS), path).unwrap())
}
#[test]

View File

@ -143,20 +143,6 @@ pub struct Client {
const HISTORY: u64 = 1200;
// database columns
/// Column for State
pub const DB_COL_STATE: Option<u32> = Some(0);
/// Column for Block headers
pub const DB_COL_HEADERS: Option<u32> = Some(1);
/// Column for Block bodies
pub const DB_COL_BODIES: Option<u32> = Some(2);
/// Column for Extras
pub const DB_COL_EXTRA: Option<u32> = Some(3);
/// Column for Traces
pub const DB_COL_TRACE: Option<u32> = Some(4);
/// Number of columns in DB
pub const DB_NO_OF_COLUMNS: Option<u32> = Some(5);
/// Append a path element to the given path and return the string.
pub fn append_path<P>(path: P, item: &str) -> String where P: AsRef<Path> {
let mut p = path.as_ref().to_path_buf();
@ -175,7 +161,7 @@ impl Client {
) -> Result<Arc<Client>, ClientError> {
let path = path.to_path_buf();
let gb = spec.genesis_block();
let mut db_config = DatabaseConfig::with_columns(DB_NO_OF_COLUMNS);
let mut db_config = DatabaseConfig::with_columns(::db::NUM_COLUMNS);
db_config.cache_size = config.db_cache_size;
db_config.compaction = config.db_compaction.compaction_profile();
db_config.wal = config.db_wal;
@ -184,7 +170,7 @@ impl Client {
let chain = Arc::new(BlockChain::new(config.blockchain, &gb, db.clone()));
let tracedb = Arc::new(try!(TraceDB::new(config.tracing, db.clone(), chain.clone())));
let mut state_db = journaldb::new(db.clone(), config.pruning, DB_COL_STATE);
let mut state_db = journaldb::new(db.clone(), config.pruning, ::db::COL_STATE);
if state_db.is_empty() && try!(spec.ensure_db_good(state_db.as_hashdb_mut())) {
let batch = DBTransaction::new(&db);
try!(state_db.commit(&batch, 0, &spec.genesis_header().hash(), None));

View File

@ -14,7 +14,7 @@
// You should have received a copy of the GNU General Public License
// along with Parity. If not, see <http://www.gnu.org/licenses/>.
//! Extras db utils.
//! Database utilities and definitions.
use std::ops::Deref;
use std::hash::Hash;
@ -22,18 +22,38 @@ use std::collections::HashMap;
use util::{DBTransaction, Database, RwLock};
use util::rlp::{encode, Encodable, decode, Decodable};
// database columns
/// Column for State
pub const COL_STATE: Option<u32> = Some(0);
/// Column for Block headers
pub const COL_HEADERS: Option<u32> = Some(1);
/// Column for Block bodies
pub const COL_BODIES: Option<u32> = Some(2);
/// Column for Extras
pub const COL_EXTRA: Option<u32> = Some(3);
/// Column for Traces
pub const COL_TRACE: Option<u32> = Some(4);
/// Number of columns in DB
pub const NUM_COLUMNS: Option<u32> = Some(5);
/// Modes for updating caches.
#[derive(Clone, Copy)]
pub enum CacheUpdatePolicy {
/// Overwrite entries.
Overwrite,
/// Remove entries.
Remove,
}
/// A cache for arbitrary key-value pairs.
pub trait Cache<K, V> {
/// Insert an entry into the cache and get the old value.
fn insert(&mut self, k: K, v: V) -> Option<V>;
/// Remove an entry from the cache, getting the old value if it existed.
fn remove(&mut self, k: &K) -> Option<V>;
/// Query the cache for a key's associated value.
fn get(&self, k: &K) -> Option<&V>;
}
@ -53,6 +73,7 @@ impl<K, V> Cache<K, V> for HashMap<K, V> where K: Hash + Eq {
/// Should be used to get database key associated with given value.
pub trait Key<T> {
/// The db key associated with this value.
type Target: Deref<Target = [u8]>;
/// Returns db key.

View File

@ -122,11 +122,11 @@ pub mod migrations;
pub mod miner;
pub mod snapshot;
pub mod action_params;
pub mod db;
#[macro_use] pub mod evm;
mod cache_manager;
mod blooms;
mod db;
mod common;
mod basic_types;
mod env_info;

View File

@ -408,7 +408,7 @@ impl StateRebuilder {
/// Create a new state rebuilder to write into the given backing DB.
pub fn new(db: Arc<Database>, pruning: Algorithm) -> Self {
StateRebuilder {
db: journaldb::new(db.clone(), pruning, ::client::DB_COL_STATE),
db: journaldb::new(db.clone(), pruning, ::db::COL_STATE),
state_root: SHA3_NULL_RLP,
}
}

View File

@ -99,7 +99,7 @@ struct Restoration {
impl Restoration {
// make a new restoration, building databases in the given path.
fn new(manifest: &ManifestData, pruning: Algorithm, path: &Path, gb: &[u8]) -> Result<Self, Error> {
let cfg = DatabaseConfig::with_columns(::client::DB_NO_OF_COLUMNS);
let cfg = DatabaseConfig::with_columns(::db::NUM_COLUMNS);
let raw_db = Arc::new(try!(Database::open(&cfg, &*path.to_string_lossy())
.map_err(UtilError::SimpleString)));

View File

@ -32,7 +32,7 @@ fn chunk_and_restore(amount: u64) {
let mut canon_chain = ChainGenerator::default();
let mut finalizer = BlockFinalizer::default();
let genesis = canon_chain.generate(&mut finalizer).unwrap();
let db_cfg = DatabaseConfig::with_columns(::client::DB_NO_OF_COLUMNS);
let db_cfg = DatabaseConfig::with_columns(::db::NUM_COLUMNS);
let orig_path = RandomTempPath::create_dir();
let new_path = RandomTempPath::create_dir();

View File

@ -35,7 +35,7 @@ fn snap_and_restore() {
let mut producer = StateProducer::new();
let mut rng = rand::thread_rng();
let mut old_db = MemoryDB::new();
let db_cfg = DatabaseConfig::with_columns(::client::DB_NO_OF_COLUMNS);
let db_cfg = DatabaseConfig::with_columns(::db::NUM_COLUMNS);
for _ in 0..150 {
producer.tick(&mut rng, &mut old_db);
@ -76,7 +76,7 @@ fn snap_and_restore() {
new_db
};
let new_db = journaldb::new(db, Algorithm::Archive, ::client::DB_COL_STATE);
let new_db = journaldb::new(db, Algorithm::Archive, ::db::COL_STATE);
compare_dbs(&old_db, new_db.as_hashdb());
}

View File

@ -15,7 +15,7 @@
// along with Parity. If not, see <http://www.gnu.org/licenses/>.
use io::*;
use client::{self, BlockChainClient, Client, ClientConfig};
use client::{BlockChainClient, Client, ClientConfig};
use common::*;
use spec::*;
use block::{OpenBlock, Drain};
@ -250,7 +250,7 @@ pub fn get_test_client_with_blocks(blocks: Vec<Bytes>) -> GuardedTempResult<Arc<
fn new_db(path: &str) -> Arc<Database> {
Arc::new(
Database::open(&DatabaseConfig::with_columns(client::DB_NO_OF_COLUMNS), path)
Database::open(&DatabaseConfig::with_columns(::db::NUM_COLUMNS), path)
.expect("Opening database for tests should always work.")
)
}

View File

@ -23,10 +23,9 @@ use bloomchain::group::{BloomGroupDatabase, BloomGroupChain, GroupPosition, Bloo
use util::{H256, H264, Database, DBTransaction, RwLock, HeapSizeOf};
use header::BlockNumber;
use trace::{LocalizedTrace, Config, Switch, Filter, Database as TraceDatabase, ImportRequest, DatabaseExtras, Error};
use db::{Key, Writable, Readable, CacheUpdatePolicy};
use db::{self, Key, Writable, Readable, CacheUpdatePolicy};
use blooms;
use super::flat::{FlatTrace, FlatBlockTraces, FlatTransactionTraces};
use client::DB_COL_TRACE;
use cache_manager::CacheManager;
const TRACE_DB_VER: &'static [u8] = b"1.0";
@ -119,7 +118,7 @@ pub struct TraceDB<T> where T: DatabaseExtras {
impl<T> BloomGroupDatabase for TraceDB<T> where T: DatabaseExtras {
fn blooms_at(&self, position: &GroupPosition) -> Option<BloomGroup> {
let position = TraceGroupPosition::from(position.clone());
let result = self.tracesdb.read_with_cache(DB_COL_TRACE, &self.blooms, &position).map(Into::into);
let result = self.tracesdb.read_with_cache(db::COL_TRACE, &self.blooms, &position).map(Into::into);
self.note_used(CacheID::Bloom(position));
result
}
@ -129,7 +128,7 @@ impl<T> TraceDB<T> where T: DatabaseExtras {
/// Creates new instance of `TraceDB`.
pub fn new(config: Config, tracesdb: Arc<Database>, extras: Arc<T>) -> Result<Self, Error> {
// check if in previously tracing was enabled
let old_tracing = match tracesdb.get(DB_COL_TRACE, b"enabled").unwrap() {
let old_tracing = match tracesdb.get(db::COL_TRACE, b"enabled").unwrap() {
Some(ref value) if value as &[u8] == &[0x1] => Switch::On,
Some(ref value) if value as &[u8] == &[0x0] => Switch::Off,
Some(_) => { panic!("tracesdb is corrupted") },
@ -144,8 +143,8 @@ impl<T> TraceDB<T> where T: DatabaseExtras {
};
let batch = DBTransaction::new(&tracesdb);
batch.put(DB_COL_TRACE, b"enabled", &encoded_tracing);
batch.put(DB_COL_TRACE, b"version", TRACE_DB_VER);
batch.put(db::COL_TRACE, b"enabled", &encoded_tracing);
batch.put(db::COL_TRACE, b"version", TRACE_DB_VER);
tracesdb.write(batch).unwrap();
let db = TraceDB {
@ -197,7 +196,7 @@ impl<T> TraceDB<T> where T: DatabaseExtras {
/// Returns traces for block with hash.
fn traces(&self, block_hash: &H256) -> Option<FlatBlockTraces> {
let result = self.tracesdb.read_with_cache(DB_COL_TRACE, &self.traces, block_hash);
let result = self.tracesdb.read_with_cache(db::COL_TRACE, &self.traces, block_hash);
self.note_used(CacheID::Trace(block_hash.clone()));
result
}
@ -273,7 +272,7 @@ impl<T> TraceDatabase for TraceDB<T> where T: DatabaseExtras {
let mut traces = self.traces.write();
// it's important to use overwrite here,
// cause this value might be queried by hash later
batch.write_with_cache(DB_COL_TRACE, &mut *traces, request.block_hash, request.traces, CacheUpdatePolicy::Overwrite);
batch.write_with_cache(db::COL_TRACE, &mut *traces, request.block_hash, request.traces, CacheUpdatePolicy::Overwrite);
// note_used must be called after locking traces to avoid cache/traces deadlock on garbage collection
self.note_used(CacheID::Trace(request.block_hash.clone()));
}
@ -302,7 +301,7 @@ impl<T> TraceDatabase for TraceDB<T> where T: DatabaseExtras {
let blooms_keys: Vec<_> = blooms_to_insert.keys().cloned().collect();
let mut blooms = self.blooms.write();
batch.extend_with_cache(DB_COL_TRACE, &mut *blooms, blooms_to_insert, CacheUpdatePolicy::Remove);
batch.extend_with_cache(db::COL_TRACE, &mut *blooms, blooms_to_insert, CacheUpdatePolicy::Remove);
// note_used must be called after locking blooms to avoid cache/traces deadlock on garbage collection
for key in blooms_keys.into_iter() {
self.note_used(CacheID::Bloom(key));
@ -417,7 +416,6 @@ mod tests {
use trace::{Filter, LocalizedTrace, AddressesFilter};
use trace::trace::{Call, Action, Res};
use trace::flat::{FlatTrace, FlatBlockTraces, FlatTransactionTraces};
use client::DB_NO_OF_COLUMNS;
use types::executed::CallType;
struct NoopExtras;
@ -459,7 +457,7 @@ mod tests {
}
fn new_db(path: &str) -> Arc<Database> {
Arc::new(Database::open(&DatabaseConfig::with_columns(DB_NO_OF_COLUMNS), path).unwrap())
Arc::new(Database::open(&DatabaseConfig::with_columns(::db::NUM_COLUMNS), path).unwrap())
}
#[test]
@ -588,7 +586,7 @@ mod tests {
#[test]
fn test_import() {
let temp = RandomTempPath::new();
let db = Arc::new(Database::open(&DatabaseConfig::with_columns(DB_NO_OF_COLUMNS), temp.as_str()).unwrap());
let db = Arc::new(Database::open(&DatabaseConfig::with_columns(::db::NUM_COLUMNS), temp.as_str()).unwrap());
let mut config = Config::default();
config.enabled = Switch::On;
let block_0 = H256::from(0xa1);

View File

@ -23,7 +23,7 @@ use util::journaldb::Algorithm;
use util::migration::{Manager as MigrationManager, Config as MigrationConfig, Error as MigrationError, Migration};
use util::kvdb::{CompactionProfile, Database, DatabaseConfig};
use ethcore::migrations;
use ethcore::client;
use ethcore::db;
use ethcore::migrations::Extract;
/// Database is assumed to be at default version, when no version file is found.
@ -240,11 +240,11 @@ pub fn migrate(path: &Path, pruning: Algorithm, compaction_profile: CompactionPr
let db_path = consolidated_database_path(path);
// Remove the database dir (it shouldn't exist anyway, but it might when migration was interrupted)
let _ = fs::remove_dir_all(db_path.clone());
try!(consolidate_database(legacy::blocks_database_path(path), db_path.clone(), client::DB_COL_HEADERS, Extract::Header, &compaction_profile));
try!(consolidate_database(legacy::blocks_database_path(path), db_path.clone(), client::DB_COL_BODIES, Extract::Body, &compaction_profile));
try!(consolidate_database(legacy::extras_database_path(path), db_path.clone(), client::DB_COL_EXTRA, Extract::All, &compaction_profile));
try!(consolidate_database(legacy::state_database_path(path), db_path.clone(), client::DB_COL_STATE, Extract::All, &compaction_profile));
try!(consolidate_database(legacy::trace_database_path(path), db_path.clone(), client::DB_COL_TRACE, Extract::All, &compaction_profile));
try!(consolidate_database(legacy::blocks_database_path(path), db_path.clone(), db::COL_HEADERS, Extract::Header, &compaction_profile));
try!(consolidate_database(legacy::blocks_database_path(path), db_path.clone(), db::COL_BODIES, Extract::Body, &compaction_profile));
try!(consolidate_database(legacy::extras_database_path(path), db_path.clone(), db::COL_EXTRA, Extract::All, &compaction_profile));
try!(consolidate_database(legacy::state_database_path(path), db_path.clone(), db::COL_STATE, Extract::All, &compaction_profile));
try!(consolidate_database(legacy::trace_database_path(path), db_path.clone(), db::COL_TRACE, Extract::All, &compaction_profile));
let _ = fs::remove_dir_all(legacy::blocks_database_path(path));
let _ = fs::remove_dir_all(legacy::extras_database_path(path));
let _ = fs::remove_dir_all(legacy::state_database_path(path));