calloc -> stasis_calloc

This commit is contained in:
Rusty Sears 2012-11-30 18:29:28 -08:00
parent 4d51fe2212
commit f72498969d
36 changed files with 66 additions and 65 deletions

View file

@ -10,7 +10,7 @@
#include <stdio.h> #include <stdio.h>
int main(int argc, char * argv[]) { int main(int argc, char * argv[]) {
char * foo = calloc(1024*1024*1024,1); char * foo = stasis_calloc(1024*1024*1024,char);
struct timeval start, stop; struct timeval start, stop;
gettimeofday(&start, 0); gettimeofday(&start, 0);
for(long i = 0; i < (1024*1024*1024/sizeof(long)); i++) { for(long i = 0; i < (1024*1024*1024/sizeof(long)); i++) {

View file

@ -57,14 +57,14 @@ int main(int argc, char * argv[]) {
long start_off = atoll(argv[5]); long start_off = atoll(argv[5]);
long random_mode = atoi(argv[6]); long random_mode = atoi(argv[6]);
double** sum_x = calloc(steps, sizeof(double*)); double** sum_x = stasis_calloc(steps, double*);
double** sum_x2 = calloc(steps, sizeof(double*)); double** sum_x2 = stasis_calloc(steps, double*);
long** sample_count = calloc(steps, sizeof(long*)); long** sample_count = stasis_calloc(steps, long*);
for(int s = 0; s < steps; s++) { for(int s = 0; s < steps; s++) {
sum_x[s] = calloc(steps, sizeof(double)); sum_x[s] = stasis_calloc(steps, double);
sum_x2[s] = calloc(steps, sizeof(double)); sum_x2[s] = stasis_calloc(steps, double);
sample_count[s] = calloc(steps, sizeof(long)); sample_count[s] = stasis_calloc(steps, long);
} }
long stride = length / steps; long stride = length / steps;

View file

@ -138,7 +138,7 @@ int main(int argc, char ** argv) {
} else { } else {
if(log_mode) { if(log_mode) {
lsn_t prevLSN = -1; lsn_t prevLSN = -1;
byte * arg = calloc(PAGE_SIZE, 1); byte * arg = stasis_calloc(PAGE_SIZE, byte);
stasis_log_t * l = stasis_log(); stasis_log_t * l = stasis_log();
for(long i = 0; i < page_count; i++) { for(long i = 0; i < page_count; i++) {

View file

@ -38,7 +38,7 @@ void * func(void * argp) {
buf = (void*)(((intptr_t)buf) & ~((intptr_t)arg->write_size-1)); buf = (void*)(((intptr_t)buf) & ~((intptr_t)arg->write_size-1));
#endif #endif
memset(buf, 0, arg->write_size); memset(buf, 0, arg->write_size);
// void * buf = calloc(arg->write_size, 1); // void * buf = stasis_calloc(arg->write_size, byte);
pthread_mutex_lock(&arg->mutex); pthread_mutex_lock(&arg->mutex);
uint64_t offset = 0; uint64_t offset = 0;
if(many_handles) { if(many_handles) {

View file

@ -132,8 +132,8 @@ DfaSet * cHtInit(int cht_type,
dfaSet = dfa_malloc(DFA_MACHINE_COUNT, ns); dfaSet = dfa_malloc(DFA_MACHINE_COUNT, ns);
twoPC_state = calloc(1, sizeof(TwoPCAppState)); twoPC_state = stasis_calloc(1, TwoPCAppState);
chtApp_state = calloc(1, sizeof(CHTAppState)); chtApp_state = stasis_calloc(1, CHTAppState);
if(cht_type == CHT_CLIENT) { if(cht_type == CHT_CLIENT) {
error = dfa_reinitialize(dfaSet, ns->localhost, client_transitions_2pc, client_transition_count_2pc, states_2pc, state_count_2pc); error = dfa_reinitialize(dfaSet, ns->localhost, client_transitions_2pc, client_transition_count_2pc, states_2pc, state_count_2pc);

View file

@ -159,7 +159,7 @@ static int stasis_getattr(const char *path, struct stat *stbuf)
static stasis_dir_entry* malloc_nod(int*sz) { static stasis_dir_entry* malloc_nod(int*sz) {
*sz = sizeof(stasis_dir_entry); *sz = sizeof(stasis_dir_entry);
return calloc(1, sizeof(stasis_dir_entry)); return stasis_calloc(1, stasis_dir_entry);
} }
@ -265,7 +265,7 @@ static int stasis_write(const char *path, const char *buf, const size_t sz, cons
*blob_rid = Talloc(xid, sz+off); *blob_rid = Talloc(xid, sz+off);
char* realbuf; char* realbuf;
if(off) { if(off) {
realbuf = calloc(sz+off,sizeof(char)); realbuf = stasis_calloc(sz+off,char);
memcpy(realbuf+off,buf,sz); memcpy(realbuf+off,buf,sz);
} else { } else {
realbuf = (char*)buf; realbuf = (char*)buf;
@ -284,7 +284,7 @@ static int stasis_write(const char *path, const char *buf, const size_t sz, cons
Tset(xid, *blob_rid, tmp); Tset(xid, *blob_rid, tmp);
free(tmp); free(tmp);
} else { } else {
byte * tmp = calloc(sz+off,sizeof(char)); byte * tmp = stasis_calloc(sz+off,char);
Tread(xid, *blob_rid, tmp); Tread(xid, *blob_rid, tmp);
memcpy(tmp+off,buf,sz); memcpy(tmp+off,buf,sz);
Tdealloc(xid, *blob_rid); Tdealloc(xid, *blob_rid);

View file

@ -680,7 +680,7 @@ lladdIterator_t* ReferentialAlgebra_Join(int xid,
j->have_outer = 0; j->have_outer = 0;
j->pred = pred; j->pred = pred;
j->inner_tpls = calloc(1, sizeof(tuple_t*)); j->inner_tpls = stasis_calloc(1, tuple_t*);
int i = 0; int i = 0;
while(Titerator_next(xid, inner_it)) { while(Titerator_next(xid, inner_it)) {
byte * in_val; byte * in_val;

View file

@ -605,7 +605,7 @@ DfaSet * dfa_malloc_old(int count, short port,
char *** broadcast_lists, char *** broadcast_lists,
int broadcast_lists_count, int broadcast_lists_count,
int * broadcast_list_host_count) { int * broadcast_list_host_count) {
DfaSet * dfaSet = calloc(1, sizeof(DfaSet)); DfaSet * dfaSet = stasis_calloc(1, DfaSet);
dfa_initialize_new(dfaSet, port, count); dfa_initialize_new(dfaSet, port, count);
dfaSet->networkSetup.broadcast_lists = broadcast_lists; dfaSet->networkSetup.broadcast_lists = broadcast_lists;
@ -616,7 +616,7 @@ DfaSet * dfa_malloc_old(int count, short port,
} }
DfaSet * dfa_malloc(int count, NetworkSetup * ns) { DfaSet * dfa_malloc(int count, NetworkSetup * ns) {
DfaSet * dfaSet = calloc(1, sizeof(DfaSet)); DfaSet * dfaSet = stasis_calloc(1, DfaSet);
dfa_initialize_new(dfaSet, ns->localport, count); dfa_initialize_new(dfaSet, ns->localport, count);
memcpy(&dfaSet->networkSetup, ns, sizeof(NetworkSetup)); memcpy(&dfaSet->networkSetup, ns, sizeof(NetworkSetup));

View file

@ -77,7 +77,7 @@ NetworkSetup * readNetworkConfig(char * name, int hostnumber) {
} else { } else {
DEBUG("I am subordinate # %d\n", hostnumber); DEBUG("I am subordinate # %d\n", hostnumber);
} }
NetworkSetup * ret = calloc(1, sizeof(NetworkSetup)); NetworkSetup * ret = stasis_calloc(1, NetworkSetup);
ret->localport = hostnumber == COORDINATOR ret->localport = hostnumber == COORDINATOR
? parse_port(cfg_getstr(cfg, "coordinator")) ? parse_port(cfg_getstr(cfg, "coordinator"))

View file

@ -46,7 +46,7 @@ terms specified in this license.
smash_t * init_Smash(int size) { smash_t * init_Smash(int size) {
int xid = Tbegin(); int xid = Tbegin();
recordid store = Talloc(xid, sizeof(smash_t)); recordid store = Talloc(xid, sizeof(smash_t));
smash_t * ret = calloc(1, sizeof(smash_t)); smash_t * ret = stasis_calloc(1, smash_t);
ret->size = size; ret->size = size;
ret->contents = 0; ret->contents = 0;

View file

@ -34,7 +34,7 @@ static Page * paLoadPage(stasis_buffer_manager_t *bm, stasis_buffer_manager_hand
pa->pageMap[pageid]->inCache = 1; pa->pageMap[pageid]->inCache = 1;
pa->pageMap[pageid]->rwlatch = initlock(); pa->pageMap[pageid]->rwlatch = initlock();
pa->pageMap[pageid]->loadlatch = initlock(); pa->pageMap[pageid]->loadlatch = initlock();
pa->pageMap[pageid]->memAddr= calloc(PAGE_SIZE, sizeof(byte)); pa->pageMap[pageid]->memAddr= stasis_calloc(PAGE_SIZE, byte);
} else{ } else{
if(type != UNKNOWN_TYPE_PAGE) { assert(type == pa->pageMap[pageid]->pageType); } if(type != UNKNOWN_TYPE_PAGE) { assert(type == pa->pageMap[pageid]->pageType); }
} }

View file

@ -297,7 +297,7 @@ recordid TlsmCreate(int xid, int comparator,
&& tmp.slot == COMPARATOR && tmp.slot == COMPARATOR
&& tmp.size == sizeof(lsmTreeNodeRecord) + keySize); && tmp.size == sizeof(lsmTreeNodeRecord) + keySize);
byte *dummy = calloc(1,keySize); byte *dummy = stasis_calloc(keySize, byte);
writeNodeRecord(xid, p, DEPTH, dummy, keySize, 0); writeNodeRecord(xid, p, DEPTH, dummy, keySize, 0);
writeNodeRecord(xid, p, COMPARATOR, dummy, keySize, comparator); writeNodeRecord(xid, p, COMPARATOR, dummy, keySize, comparator);
@ -343,7 +343,7 @@ static recordid buildPathToLeaf(int xid, recordid root, Page *root_p,
} else { } else {
// set leaf // set leaf
byte *dummy = calloc(1, key_len); byte *dummy = stasis_calloc(key_len, byte);
// backward link. // backward link.
writeNodeRecord(xid,child_p,PREV_LEAF,dummy,key_len,lastLeaf); writeNodeRecord(xid,child_p,PREV_LEAF,dummy,key_len,lastLeaf);
@ -575,7 +575,7 @@ recordid TlsmAppendPage(int xid, recordid tree,
stasis_record_write_done(xid,p,pFirstSlot,(byte*)nr); stasis_record_write_done(xid,p,pFirstSlot,(byte*)nr);
stasis_page_lsn_write(xid, p, 0); // XXX need real LSN? stasis_page_lsn_write(xid, p, 0); // XXX need real LSN?
byte *dummy = calloc(1,keySize); byte *dummy = stasis_calloc(keySize, byte);
if(!depth) { if(!depth) {
s->lastLeaf = lc->id; s->lastLeaf = lc->id;
writeNodeRecord(xid,lc,PREV_LEAF,dummy,keySize,-1); writeNodeRecord(xid,lc,PREV_LEAF,dummy,keySize,-1);

View file

@ -153,7 +153,7 @@ static range ** rangeTrackerToArray(rangeTracker * rt) {
range_count++; range_count++;
} }
range ** ret = calloc(range_count + 1, sizeof(range *)); range ** ret = stasis_calloc(range_count + 1, range *);
int next_range = 0; int next_range = 0;
in_range = 0; in_range = 0;

View file

@ -252,7 +252,7 @@ static void stasis_log_file_pool_prealloc_file(stasis_log_file_pool_state * fp)
posix_fallocate(fd, 0, fp->target_chunk_size + bufsz); posix_fallocate(fd, 0, fp->target_chunk_size + bufsz);
#endif #endif
printf("Writing zeros to empty log file...\n"); printf("Writing zeros to empty log file...\n");
byte * buffer = calloc(bufsz, sizeof(byte)); byte * buffer = stasis_calloc(bufsz, byte);
for(off_t i = 0; i <= fp->target_chunk_size; i += bufsz) { for(off_t i = 0; i <= fp->target_chunk_size; i += bufsz) {
int ret = pwrite(fd, buffer, bufsz, i); int ret = pwrite(fd, buffer, bufsz, i);
if(ret != bufsz) { if(ret != bufsz) {

View file

@ -856,7 +856,7 @@ stasis_log_t* stasis_log_safe_writes_open(const char * filename,
memcpy(log,&proto, sizeof(proto)); memcpy(log,&proto, sizeof(proto));
log->impl = sw; log->impl = sw;
sw->buffer = calloc(stasis_log_file_write_buffer_size, sizeof(char)); sw->buffer = stasis_calloc(stasis_log_file_write_buffer_size, char);
if(!sw->buffer) { return 0; /*LLADD_NO_MEM;*/ } if(!sw->buffer) { return 0; /*LLADD_NO_MEM;*/ }

View file

@ -29,7 +29,7 @@ void BtreeInit() {
// todo: register iterator // todo: register iterator
// register comparators // register comparators
btree_comparators = calloc(MAX_COMPARATOR, sizeof(stasis_comparator_t)); btree_comparators = stasis_calloc(MAX_COMPARATOR, stasis_comparator_t);
btree_comparators[BYTE_ARRAY_COMPARATOR] = stasis_btree_byte_array_comparator; btree_comparators[BYTE_ARRAY_COMPARATOR] = stasis_btree_byte_array_comparator;
} }
@ -113,7 +113,7 @@ static slotid_t find_in_page(int xid, Page * p, byte * key, size_t keySize, int
static slotid_t stasis_btree_helper(int xid, stasis_op_btree_page_header h, byte* key, size_t keySize, static slotid_t stasis_btree_helper(int xid, stasis_op_btree_page_header h, byte* key, size_t keySize,
int * found, pageid_t ** path, stasis_comparator_t cmp, void * cmp_arg) { int * found, pageid_t ** path, stasis_comparator_t cmp, void * cmp_arg) {
pageid_t next = h.root; pageid_t next = h.root;
*path = calloc(h.height, sizeof(pageid_t)); *path = stasis_calloc(h.height, pageid_t);
for(int i = 0; i < h.height-1; i++) { for(int i = 0; i < h.height-1; i++) {
Page *p = loadPage(xid, next); Page *p = loadPage(xid, next);
(*path)[i] = next; (*path)[i] = next;

View file

@ -64,7 +64,7 @@ void stasis_blob_write(int xid, Page * p, recordid rid, const byte* dat) {
TpageSetRange(xid,rec.offset+chunk,0,((const byte*)dat)+(chunk*USABLE_SIZE_OF_PAGE),USABLE_SIZE_OF_PAGE); TpageSetRange(xid,rec.offset+chunk,0,((const byte*)dat)+(chunk*USABLE_SIZE_OF_PAGE),USABLE_SIZE_OF_PAGE);
} }
// Painful; allocate buffer for zero padding. TODO: Remove zero padding? // Painful; allocate buffer for zero padding. TODO: Remove zero padding?
byte * buf = calloc(1,USABLE_SIZE_OF_PAGE); byte * buf = stasis_calloc(USABLE_SIZE_OF_PAGE, byte);
memcpy(buf, ((const byte*)dat)+(chunk*USABLE_SIZE_OF_PAGE), rid.size % USABLE_SIZE_OF_PAGE); memcpy(buf, ((const byte*)dat)+(chunk*USABLE_SIZE_OF_PAGE), rid.size % USABLE_SIZE_OF_PAGE);
TpageSetRange(xid,rec.offset+chunk,0,buf,USABLE_SIZE_OF_PAGE); TpageSetRange(xid,rec.offset+chunk,0,buf,USABLE_SIZE_OF_PAGE);
free(buf); free(buf);

View file

@ -89,7 +89,7 @@ recordid ThashCreate(int xid, int keySize, int valueSize) {
} else { } else {
#ifdef ARRAY_LIST_OLD_ALLOC #ifdef ARRAY_LIST_OLD_ALLOC
byte * entry = calloc(1, lhh.buckets.size); byte * entry = stasis_calloc(lhh.buckets.size, byte);
for(i = 0; i < HASH_INIT_ARRAY_LIST_COUNT; i++) { for(i = 0; i < HASH_INIT_ARRAY_LIST_COUNT; i++) {
bucket.slot = i; bucket.slot = i;
begin_action_ret(free, entry, NULLRID) { begin_action_ret(free, entry, NULLRID) {
@ -327,7 +327,7 @@ static void ThashSplitBucket(int xid, recordid hashHeader, lladd_hash_header * l
Tset(xid, new_bucket_rid, &new_bucket_list); Tset(xid, new_bucket_rid, &new_bucket_list);
} else { } else {
#ifdef ARRAY_LIST_OLD_ALLOC #ifdef ARRAY_LIST_OLD_ALLOC
byte * entry = calloc(1, lhh->buckets.size); byte * entry = stasis_calloc(lhh->buckets.size, byte);
Tset(xid, new_bucket_rid, entry); Tset(xid, new_bucket_rid, entry);
free(entry); free(entry);
#endif #endif
@ -378,7 +378,7 @@ static void ThashSplitBucket(int xid, recordid hashHeader, lladd_hash_header * l
} }
lladd_hash_iterator * ThashIterator(int xid, recordid hashHeader, int keySize, int valueSize) { lladd_hash_iterator * ThashIterator(int xid, recordid hashHeader, int keySize, int valueSize) {
hashHeader.size = sizeof(lladd_hash_header); hashHeader.size = sizeof(lladd_hash_header);
lladd_hash_iterator * it = calloc(1,sizeof(lladd_hash_iterator)); lladd_hash_iterator * it = stasis_calloc(1,lladd_hash_iterator);
it->hashHeader = hashHeader; it->hashHeader = hashHeader;
lladd_hash_header lhh; lladd_hash_header lhh;
Tread(xid, hashHeader, &lhh); Tread(xid, hashHeader, &lhh);

View file

@ -329,7 +329,7 @@ recordid TlinkedListCreate(int xid, int keySize, int valueSize) {
recordid ret; recordid ret;
ret = Talloc(xid, sizeof(stasis_linkedList_entry) + keySize + valueSize); ret = Talloc(xid, sizeof(stasis_linkedList_entry) + keySize + valueSize);
byte * cleared = calloc(sizeof(stasis_linkedList_entry) + keySize + valueSize, sizeof(byte)); byte * cleared = stasis_calloc(sizeof(stasis_linkedList_entry) + keySize + valueSize, byte);
Tset(xid, ret, cleared); Tset(xid, ret, cleared);
free(cleared); free(cleared);

View file

@ -43,7 +43,7 @@ int TsetReorderable(int xid, stasis_log_reordering_handle_t * h,
} else { } else {
rid.size = stasis_record_type_to_size(rid.size); rid.size = stasis_record_type_to_size(rid.size);
intptr_t sz = 2 * (sizeof(pageoff_t) + rid.size); intptr_t sz = 2 * (sizeof(pageoff_t) + rid.size);
byte * buf = calloc(sz, 1); byte * buf = stasis_calloc(sz, byte);
pageoff_t * a = (pageoff_t*)buf; pageoff_t * a = (pageoff_t*)buf;
// XXX hack! // XXX hack!
byte * writeBuf = stasis_record_write_begin(xid, p, rid); byte * writeBuf = stasis_record_write_begin(xid, p, rid);
@ -74,7 +74,7 @@ int TsetReorderableWriteBack(int xid, stasis_log_reordering_handle_t * h,
pageid_t page, pageoff_t off, pageoff_t len, pageid_t page, pageoff_t off, pageoff_t len,
const void * dat, const void * olddat) { const void * dat, const void * olddat) {
intptr_t sz = 2 * (sizeof(pageoff_t) + len); intptr_t sz = 2 * (sizeof(pageoff_t) + len);
byte * buf = calloc(sz,1); byte * buf = stasis_calloc(sz,byte);
pageoff_t * a = (pageoff_t*)buf; pageoff_t * a = (pageoff_t*)buf;
a[0] = off; a[0] = off;
a[1] = len; a[1] = len;

View file

@ -116,9 +116,9 @@ static void rehash(int xid, recordid hashRid, pageid_t next_split, pageid_t i, u
recordid ba = hashRid; ba.slot = next_split; recordid ba = hashRid; ba.slot = next_split;
recordid bb = hashRid; bb.slot = next_split + stasis_util_two_to_the(i-1); recordid bb = hashRid; bb.slot = next_split + stasis_util_two_to_the(i-1);
hashEntry * D_contents = calloc(1,sizeof(hashEntry) + keySize + valSize); hashEntry * D_contents = stasis_calloc_trailing_array(hashEntry, keySize + valSize);
hashEntry * A_contents = calloc(1,sizeof(hashEntry) + keySize + valSize); hashEntry * A_contents = stasis_calloc_trailing_array(hashEntry, keySize + valSize);
hashEntry * B_contents = calloc(1,sizeof(hashEntry) + keySize + valSize); hashEntry * B_contents = stasis_calloc_trailing_array(hashEntry, keySize + valSize);
Tread(xid, ba, A_contents); Tread(xid, ba, A_contents);
Tread(xid, bb, D_contents); Tread(xid, bb, D_contents);
@ -346,8 +346,8 @@ recordid TnaiveHashCreate(int xid, int keySize, int valSize) {
assert(rid.size == sizeof(hashEntry) + keySize + valSize); assert(rid.size == sizeof(hashEntry) + keySize + valSize);
TarrayListExtend(xid, rid, 4096+2); TarrayListExtend(xid, rid, 4096+2);
recordid * headerRidA = calloc (1, sizeof(recordid) + keySize + valSize); recordid * headerRidA = stasis_calloc_trailing_array(recordid, keySize + valSize);
recordid * headerRidB = calloc (1, sizeof(recordid) + keySize + valSize); recordid * headerRidB = stasis_calloc_trailing_array(recordid, keySize + valSize);
assert(headerRidB); assert(headerRidB);
@ -407,11 +407,11 @@ void TnaiveHashInsert(int xid, recordid hashRid,
int bucket = int bucket =
2 + stasis_linear_hash(key, keySize, headerHashBits, headerNextSplit - 2); 2 + stasis_linear_hash(key, keySize, headerHashBits, headerNextSplit - 2);
hashEntry * e = calloc(1,sizeof(hashEntry) + keySize + valSize); hashEntry * e = stasis_calloc_trailing_array(hashEntry, keySize + valSize);
memcpy(e+1, key, keySize); memcpy(e+1, key, keySize);
memcpy(((byte*)(e+1)) + keySize, val, valSize); memcpy(((byte*)(e+1)) + keySize, val, valSize);
hashEntry * bucket_contents = calloc(1,sizeof(hashEntry) + keySize + valSize); hashEntry * bucket_contents = stasis_calloc_trailing_array(hashEntry, keySize + valSize);
hashRid.slot = bucket; hashRid.slot = bucket;
Tread(xid, hashRid, bucket_contents); Tread(xid, hashRid, bucket_contents);

View file

@ -84,7 +84,7 @@ stasis_bloom_filter_t * stasis_bloom_filter_create(uint64_t(*func_a)(const char*
ret->num_expected_items = num_expected_items; ret->num_expected_items = num_expected_items;
ret->desired_false_positive_rate = false_positive_rate; ret->desired_false_positive_rate = false_positive_rate;
ret->num_buckets = stasis_bloom_filter_calc_num_buckets(ret->num_expected_items, ret->desired_false_positive_rate); ret->num_buckets = stasis_bloom_filter_calc_num_buckets(ret->num_expected_items, ret->desired_false_positive_rate);
ret->buckets = calloc((ret->num_buckets / 8) + ((ret->num_buckets % 8 == 0) ? 0 : 1), 1); ret->buckets = stasis_calloc((ret->num_buckets / 8) + ((ret->num_buckets % 8 == 0) ? 0 : 1), uint8_t);
ret->num_functions = stasis_bloom_filter_calc_num_functions(ret->num_expected_items, ret->num_buckets); ret->num_functions = stasis_bloom_filter_calc_num_functions(ret->num_expected_items, ret->num_buckets);
ret->result_scratch_space = stasis_malloc(ret->num_functions, uint64_t); ret->result_scratch_space = stasis_malloc(ret->num_functions, uint64_t);
ret->actual_number_of_items = 0; ret->actual_number_of_items = 0;

View file

@ -425,7 +425,7 @@ hashtable_t * hashtable_init(pageid_t size) {
hashtable_t *ht = stasis_alloc(hashtable_t); hashtable_t *ht = stasis_alloc(hashtable_t);
ht->maxbucketid = (newsize) - 1; ht->maxbucketid = (newsize) - 1;
ht->buckets = calloc(ht->maxbucketid+1, sizeof(bucket_t)); ht->buckets = stasis_calloc(ht->maxbucketid+1, bucket_t);
for(int i = 0; i <= ht->maxbucketid; i++) { for(int i = 0; i <= ht->maxbucketid; i++) {
ht->buckets[i].key = -1; ht->buckets[i].key = -1;
} }

View file

@ -221,7 +221,7 @@ static void extendHashTable(struct LH_ENTRY(table) * table) {
struct LH_ENTRY(table) * LH_ENTRY(create)(int initialSize) { struct LH_ENTRY(table) * LH_ENTRY(create)(int initialSize) {
struct LH_ENTRY(table) * ret = stasis_alloc(struct LH_ENTRY(table)); struct LH_ENTRY(table) * ret = stasis_alloc(struct LH_ENTRY(table));
ret->bucketList = calloc(initialSize, sizeof(struct LH_ENTRY(pair_t))); ret->bucketList = stasis_calloc(initialSize, struct LH_ENTRY(pair_t));
HASH_ENTRY(_get_size_params)(initialSize, HASH_ENTRY(_get_size_params)(initialSize,
&(ret->bucketListBits), &(ret->bucketListBits),
&(ret->bucketListNextExtension)); &(ret->bucketListNextExtension));

View file

@ -143,8 +143,8 @@ static inline hazard_ptr_rec_t * hazard_ensure_tls(hazard_t * h) {
hazard_ptr_rec_t * rec = pthread_getspecific(h->hp); hazard_ptr_rec_t * rec = pthread_getspecific(h->hp);
if(rec == NULL) { if(rec == NULL) {
rec = stasis_alloc(hazard_ptr_rec_t); rec = stasis_alloc(hazard_ptr_rec_t);
rec->hp = calloc(h->num_slots, sizeof(hazard_ptr)); rec->hp = stasis_calloc(h->num_slots, hazard_ptr);
rec->rlist = calloc(h->num_r_slots, sizeof(hazard_ptr)); rec->rlist = stasis_calloc(h->num_r_slots, void*);
rec->rlist_len = 0; rec->rlist_len = 0;
rec->h = h; rec->h = h;
pthread_setspecific(h->hp, rec); pthread_setspecific(h->hp, rec);

View file

@ -14,6 +14,7 @@
#define stasis_malloc(cnt, typ) ((typ*)malloc((cnt)*sizeof(typ))) #define stasis_malloc(cnt, typ) ((typ*)malloc((cnt)*sizeof(typ)))
#define stasis_malloc_trailing_array(typ, array_sz) ((typ*)malloc(sizeof(typ)+(array_sz))) #define stasis_malloc_trailing_array(typ, array_sz) ((typ*)malloc(sizeof(typ)+(array_sz)))
#define stasis_calloc(cnt, typ) ((typ*)calloc((cnt),sizeof(typ))) #define stasis_calloc(cnt, typ) ((typ*)calloc((cnt),sizeof(typ)))
#define stasis_calloc_trailing_array(typ, array_sz) ((typ*)calloc(sizeof(typ)+(array_sz),1))
#define stasis_realloc(ptr, cnt, typ) ((typ*)realloc(ptr, (cnt)*sizeof(typ))) #define stasis_realloc(ptr, cnt, typ) ((typ*)realloc(ptr, (cnt)*sizeof(typ)))
#define stasis_free(ptr) free(ptr) #define stasis_free(ptr) free(ptr)

View file

@ -75,7 +75,7 @@ callback_fcn recv_new, recv_ack;
*/ */
int main (int argc, char** argv) { int main (int argc, char** argv) {
DfaSet * dfaSet = calloc(1, sizeof(DfaSet)); DfaSet * dfaSet = stasis_calloc(1, DfaSet);
/* callback_fcn* callbacks[MAX_MESSAGE_COUNT]; */ /* callback_fcn* callbacks[MAX_MESSAGE_COUNT]; */
Transition * transitions = stasis_malloc(3, Transition); Transition * transitions = stasis_malloc(3, Transition);
@ -85,9 +85,9 @@ int main (int argc, char** argv) {
StateMachine * initial_sm1; StateMachine * initial_sm1;
int transition_count; int transition_count;
/* dfaSet->monoTree.buffer = calloc(DFA_MACHINE_COUNT, sizeof(StateMachine)); /* dfaSet->monoTree.buffer = stasis_calloc(DFA_MACHINE_COUNT, StateMachine);
dfa_initialize_new (dfaSet, 10001, DFA_MACHINE_COUNT); */ dfa_initialize_new (dfaSet, 10001, DFA_MACHINE_COUNT); */
/* dfaSet->monoTree.buffer = calloc(100, sizeof(StateMachine)); */ /* dfaSet->monoTree.buffer = stasis_calloc(100, StateMachine); */
dfa_initialize_new (dfaSet, 10001, 100); dfa_initialize_new (dfaSet, 10001, 100);
/* initial_sm1 = allocMachine(&(dfaSet->monoTree)); */ /* initial_sm1 = allocMachine(&(dfaSet->monoTree)); */

View file

@ -68,7 +68,7 @@ START_TEST (pingpong_check) {
//int main (int argc, char** argv) { //int main (int argc, char** argv) {
DfaSet * dfaSet = calloc(1, sizeof(DfaSet)); DfaSet * dfaSet = stasis_calloc(1, DfaSet);
/* callback_fcn* callbacks[MAX_MESSAGE_COUNT]; */ /* callback_fcn* callbacks[MAX_MESSAGE_COUNT]; */
Transition * transitions = stasis_malloc(4, Transition); Transition * transitions = stasis_malloc(4, Transition);
@ -78,7 +78,7 @@ START_TEST (pingpong_check) {
StateMachine * initial_sm2; StateMachine * initial_sm2;
int i; int i;
/*dfaSet->monoTree.buffer = calloc(DFA_MACHINE_COUNT, sizeof(StateMachine)); */ /*dfaSet->monoTree.buffer = stasis_calloc(DFA_MACHINE_COUNT, StateMachine); */
dfa_initialize_new (dfaSet, 10000, DFA_MACHINE_COUNT); dfa_initialize_new (dfaSet, 10000, DFA_MACHINE_COUNT);
for(i = 0; i < DFA_MACHINE_COUNT/4; i++) { for(i = 0; i < DFA_MACHINE_COUNT/4; i++) {
/* StateMachine initial_sm1_stack; /* StateMachine initial_sm1_stack;

View file

@ -57,7 +57,7 @@ int main () {
int i = 0; int i = 0;
int rb_size = 1 + (int) (5000.0*rand()/(RAND_MAX+1.0)); int rb_size = 1 + (int) (5000.0*rand()/(RAND_MAX+1.0));
int tx_count = 1 + (int)(30000.0*rand()/(RAND_MAX+1.0)); int tx_count = 1 + (int)(30000.0*rand()/(RAND_MAX+1.0));
unsigned char * bitmap = calloc(tx_count, sizeof(unsigned char)); unsigned char * bitmap = stasis_calloc(tx_count, unsigned char);
unsigned int new_seed = (int) ((1.0* INT_MAX*rand())/(RAND_MAX+1.0)); unsigned int new_seed = (int) ((1.0* INT_MAX*rand())/(RAND_MAX+1.0));
state_machine_id remaining_xact = 0; state_machine_id remaining_xact = 0;
state_machine_id last_xact = 0; state_machine_id last_xact = 0;

View file

@ -280,7 +280,7 @@ void * blindRandomWorker(void * v) {
// int idx = *(int*)v; /// Don't need index; want pinned pages to overlap! // int idx = *(int*)v; /// Don't need index; want pinned pages to overlap!
pageid_t * pageids = stasis_malloc(PINNED_PAGE_COUNT, pageid_t); pageid_t * pageids = stasis_malloc(PINNED_PAGE_COUNT, pageid_t);
Page ** pages = calloc(PINNED_PAGE_COUNT, sizeof(Page*)); Page ** pages = stasis_calloc(PINNED_PAGE_COUNT, Page*);
for(int i = 0; i < PINNED_PAGE_COUNT; i++) { for(int i = 0; i < PINNED_PAGE_COUNT; i++) {
pageids[i] = -1; pageids[i] = -1;

View file

@ -140,9 +140,9 @@ START_TEST(lhtableRandomized) {
int64_t nextVal = 1; int64_t nextVal = 1;
int64_t eventCount = 0; int64_t eventCount = 0;
int* setNextAlloc = calloc(numSets, sizeof(int)); int* setNextAlloc = stasis_calloc(numSets, int);
int* setNextDel = calloc(numSets, sizeof(int)); int* setNextDel = stasis_calloc(numSets, int);
int* setNextRead = calloc(numSets, sizeof(int)); int* setNextRead = stasis_calloc(numSets, int);
for(int i =0; i < numSets; i++) { for(int i =0; i < numSets; i++) {
setLength[i] = stasis_util_random64(MAXSETLEN); setLength[i] = stasis_util_random64(MAXSETLEN);

View file

@ -182,7 +182,7 @@ START_TEST(multiplexTest) {
lsn_t i; lsn_t i;
array = (lsn_t*)calloc(NUM_INSERTS, sizeof(lsn_t)); array = stasis_calloc(NUM_INSERTS, lsn_t);
for(i = 0; i < NUM_INSERTS; i++) { for(i = 0; i < NUM_INSERTS; i++) {

View file

@ -170,7 +170,7 @@ START_TEST(pagedListCheck) {
assert(!memcmp(bb, &b, sizeof(recordid))); assert(!memcmp(bb, &b, sizeof(recordid)));
} }
byte * seen = calloc(NUM_ENTRIES, sizeof(byte)); byte * seen = stasis_calloc(NUM_ENTRIES, byte);
lladd_pagedList_iterator * it = TpagedListIterator(xid, list); lladd_pagedList_iterator * it = TpagedListIterator(xid, list);

View file

@ -286,7 +286,7 @@ START_TEST (rangeTracker_randomTest) {
gettimeofday(&time,0); gettimeofday(&time,0);
int * explicit_pins = calloc(RANGE_SIZE, sizeof(int)); int * explicit_pins = stasis_calloc(RANGE_SIZE, int);
long seed = time.tv_usec + time.tv_sec * 1000000; //1170807889195512; //time.tv_usec + time.tv_sec * 1000000; //1170729550013502; //time.tv_usec + time.tv_sec * 1000000;// 1170727703805787; // 1170810757441165; 1170811024737237; 1171329519584370; long seed = time.tv_usec + time.tv_sec * 1000000; //1170807889195512; //time.tv_usec + time.tv_sec * 1000000; //1170729550013502; //time.tv_usec + time.tv_sec * 1000000;// 1170727703805787; // 1170810757441165; 1170811024737237; 1171329519584370;
@ -295,7 +295,7 @@ START_TEST (rangeTracker_randomTest) {
range ** r_arry; range ** r_arry;
range * ranges = stasis_malloc(RANGE_COUNT, range); range * ranges = stasis_malloc(RANGE_COUNT, range);
int * pins = calloc(RANGE_COUNT, sizeof(int)); int * pins = stasis_calloc(RANGE_COUNT, int);
rangeTracker * rt = rangeTrackerInit(QUANTIZATION); rangeTracker * rt = rangeTrackerInit(QUANTIZATION);
for(long i = 0; i < RANGE_COUNT; i++) { for(long i = 0; i < RANGE_COUNT; i++) {
randomRange(&(ranges[i])); randomRange(&(ranges[i]));

View file

@ -34,8 +34,8 @@ void randomSetup() {
cachedCount = 0; cachedCount = 0;
t = calloc(OBJECT_COUNT, sizeof(tracker)); t = stasis_calloc(OBJECT_COUNT, tracker);
pages = calloc(OBJECT_COUNT, sizeof(Page)); pages = stasis_calloc(OBJECT_COUNT, Page);
for(int i = 0; i < OBJECT_COUNT; i++) { for(int i = 0; i < OBJECT_COUNT; i++) {
pages[i].id = i; pages[i].id = i;
pages[i].pinCount = 1; pages[i].pinCount = 1;

View file

@ -68,8 +68,8 @@ runChecker() {
* possibly committed (insert table) and definitely committed (commit table). * possibly committed (insert table) and definitely committed (commit table).
*/ */
tableLength = baseKey + (numThreads * opsPerThread); tableLength = baseKey + (numThreads * opsPerThread);
insertTable = (int*) calloc(tableLength, sizeof(int)); insertTable = stasis_calloc(tableLength, int);
commitTable = (int*) calloc(tableLength, sizeof(int)); commitTable = stasis_calloc(tableLength, int);
/* Read all the entries from the insert log (commit log) and insert into the /* Read all the entries from the insert log (commit log) and insert into the
* insert table (commit table). * insert table (commit table).