memory: add uv_replace_allocator

With uv_replace_allocator, it's possible to override the default
memory allocator's malloc and free calls with functions of the user's
choosing. This allows libuv to interoperate with projects requiring a
custom memory allocator.

Internally, all calls to malloc and free have been replaced with
uv__malloc and uv__free, respectively. The uv__malloc and uv__free
functions call malloc and free unless they have been overridden by a
previous call to uv_replace_allocator.

As part of this change, the special aligned memory allocations
performed in src/win/fs-event.c have been replaced with standard
allocations. The 4-byte alignment being requested in this file was
unnecessary, since standard allocators already guarantee at least an
8-byte alignment.

PR-URL: https://github.com/libuv/libuv/pull/231
Reviewed-By: Ben Noordhuis <info@bnoordhuis.nl>
Reviewed-By: Saúl Ibarra Corretgé <saghul@gmail.com>
This commit is contained in:
Brett Vickers 2015-02-26 14:53:07 -08:00 committed by Saúl Ibarra Corretgé
parent 5645b2d69f
commit c272f1f1bc
37 changed files with 347 additions and 300 deletions

View File

@ -15,6 +15,16 @@ Data types
Buffer data type.
.. c:type:: uv_malloc_func
Function pointer type for the malloc override used by
:c:func:`uv_replace_allocator`.
.. c:type:: uv_free_func
Function pointer type for the free override used by
:c:func:`uv_replace_allocator`.
.. c:type:: uv_file
Cross platform representation of a file handle.
@ -26,7 +36,7 @@ Data types
.. c:type:: uv_os_fd_t
Abstract representation of a file descriptor. On Unix systems this is a
`typedef` of `int` and on Windows fa `HANDLE`.
`typedef` of `int` and on Windows a `HANDLE`.
.. c:type:: uv_rusage_t
@ -115,6 +125,16 @@ API
Returns the libuv version number as a string. For non-release versions
"-pre" is appended, so the version number could be "1.2.3-pre".
.. c:function:: int uv_replace_allocator(uv_malloc_func malloc_func, uv_free_func free_func)
.. versionadded:: 1.5.0
Override the use of the standard library's malloc and free functions for
memory allocation. If used, this function must be called before any
other libuv function is called. On success, it returns 0. If called more
than once, the replacement request is ignored and the function returns
``UV_EINVAL``.
.. c:function:: uv_buf_t uv_buf_init(char* base, unsigned int len)
Constructor for :c:type:`uv_buf_t`.

View File

@ -43,16 +43,6 @@ typedef struct pollfd {
# define LOCALE_INVARIANT 0x007f
#endif
#ifndef _malloca
# if defined(_DEBUG)
# define _malloca(size) malloc(size)
# define _freea(ptr) free(ptr)
# else
# define _malloca(size) alloca(size)
# define _freea(ptr)
# endif
#endif
#include <mswsock.h>
#include <ws2tcpip.h>
#include <windows.h>

View File

@ -244,6 +244,12 @@ typedef enum {
UV_EXTERN unsigned int uv_version(void);
UV_EXTERN const char* uv_version_string(void);
typedef void* (*uv_malloc_func)(size_t size);
typedef void (*uv_free_func)(void* ptr);
UV_EXTERN int uv_replace_allocator(uv_malloc_func malloc_func,
uv_free_func free_func);
UV_EXTERN uv_loop_t* uv_default_loop(void);
UV_EXTERN int uv_loop_init(uv_loop_t* loop);
UV_EXTERN int uv_loop_close(uv_loop_t* loop);

View File

@ -96,7 +96,7 @@ int uv_fs_poll_start(uv_fs_poll_t* handle,
return 0;
error:
free(ctx);
uv__free(ctx);
return err;
}
@ -219,7 +219,7 @@ out:
static void timer_close_cb(uv_handle_t* handle) {
free(container_of(handle, struct poll_ctx, timer_handle));
uv__free(container_of(handle, struct poll_ctx, timer_handle));
}

View File

@ -122,7 +122,7 @@ UV_DESTRUCTOR(static void cleanup(void)) {
abort();
if (threads != default_threads)
free(threads);
uv__free(threads);
uv_mutex_destroy(&mutex);
uv_cond_destroy(&cond);
@ -149,7 +149,7 @@ static void init_once(void) {
threads = default_threads;
if (nthreads > ARRAY_SIZE(default_threads)) {
threads = malloc(nthreads * sizeof(threads[0]));
threads = uv__malloc(nthreads * sizeof(threads[0]));
if (threads == NULL) {
nthreads = ARRAY_SIZE(default_threads);
threads = default_threads;

View File

@ -550,7 +550,7 @@ static int uv__is_ahafs_mounted(void){
const char *dev = "/aha";
char *obj, *stub;
p = malloc(siz);
p = uv__malloc(siz);
if (p == NULL)
return -errno;
@ -561,8 +561,8 @@ static int uv__is_ahafs_mounted(void){
if (rv == 0) {
/* buffer was not large enough, reallocate to correct size */
siz = *(int*)p;
free(p);
p = malloc(siz);
uv__free(p);
p = uv__malloc(siz);
if (p == NULL)
return -errno;
rv = mntctl(MCTL_QUERY, siz, (char*)p);
@ -576,7 +576,7 @@ static int uv__is_ahafs_mounted(void){
stub = vmt2dataptr(vmt, VMT_STUB); /* mount point */
if (EQ(obj, dev) || EQ(uv__rawname(obj), dev) || EQ(stub, dev)) {
free(p); /* Found a match */
uv__free(p); /* Found a match */
return 0;
}
vmt = (struct vmount *) ((char *) vmt + vmt->vmt_length);
@ -949,11 +949,11 @@ int uv_fs_event_stop(uv_fs_event_t* handle) {
uv__handle_stop(handle);
if (uv__path_is_a_directory(handle->path) == 0) {
free(handle->dir_filename);
uv__free(handle->dir_filename);
handle->dir_filename = NULL;
}
free(handle->path);
uv__free(handle->path);
handle->path = NULL;
uv__close(handle->event_watcher.fd);
handle->event_watcher.fd = -1;
@ -1052,7 +1052,7 @@ int uv_cpu_info(uv_cpu_info_t** cpu_infos, int* count) {
return -ENOSYS;
}
ps_cpus = (perfstat_cpu_t*) malloc(ncpus * sizeof(perfstat_cpu_t));
ps_cpus = (perfstat_cpu_t*) uv__malloc(ncpus * sizeof(perfstat_cpu_t));
if (!ps_cpus) {
return -ENOMEM;
}
@ -1060,13 +1060,13 @@ int uv_cpu_info(uv_cpu_info_t** cpu_infos, int* count) {
strcpy(cpu_id.name, FIRST_CPU);
result = perfstat_cpu(&cpu_id, ps_cpus, sizeof(perfstat_cpu_t), ncpus);
if (result == -1) {
free(ps_cpus);
uv__free(ps_cpus);
return -ENOSYS;
}
*cpu_infos = (uv_cpu_info_t*) malloc(ncpus * sizeof(uv_cpu_info_t));
*cpu_infos = (uv_cpu_info_t*) uv__malloc(ncpus * sizeof(uv_cpu_info_t));
if (!*cpu_infos) {
free(ps_cpus);
uv__free(ps_cpus);
return -ENOMEM;
}
@ -1085,7 +1085,7 @@ int uv_cpu_info(uv_cpu_info_t** cpu_infos, int* count) {
idx++;
}
free(ps_cpus);
uv__free(ps_cpus);
return 0;
}
@ -1094,10 +1094,10 @@ void uv_free_cpu_info(uv_cpu_info_t* cpu_infos, int count) {
int i;
for (i = 0; i < count; ++i) {
free(cpu_infos[i].model);
uv__free(cpu_infos[i].model);
}
free(cpu_infos);
uv__free(cpu_infos);
}
@ -1119,7 +1119,7 @@ int uv_interface_addresses(uv_interface_address_t** addresses,
return -ENOSYS;
}
ifc.ifc_req = (struct ifreq*)malloc(size);
ifc.ifc_req = (struct ifreq*)uv__malloc(size);
ifc.ifc_len = size;
if (ioctl(sockfd, SIOCGIFCONF, &ifc) == -1) {
uv__close(sockfd);
@ -1153,7 +1153,7 @@ int uv_interface_addresses(uv_interface_address_t** addresses,
/* Alloc the return interface structs */
*addresses = (uv_interface_address_t*)
malloc(*count * sizeof(uv_interface_address_t));
uv__malloc(*count * sizeof(uv_interface_address_t));
if (!(*addresses)) {
uv__close(sockfd);
return -ENOMEM;
@ -1208,10 +1208,10 @@ void uv_free_interface_addresses(uv_interface_address_t* addresses,
int i;
for (i = 0; i < count; ++i) {
free(addresses[i].name);
uv__free(addresses[i].name);
}
free(addresses);
uv__free(addresses);
}
void uv__platform_invalidate_fd(uv_loop_t* loop, int fd) {

View File

@ -136,8 +136,8 @@ static struct nlmsghdr *getNetlinkResponse(int p_socket, int *p_size, int *p_don
{
int l_read;
free(l_buffer);
l_buffer = malloc(l_size);
uv__free(l_buffer);
l_buffer = uv__malloc(l_size);
if (l_buffer == NULL)
{
return NULL;
@ -147,7 +147,7 @@ static struct nlmsghdr *getNetlinkResponse(int p_socket, int *p_size, int *p_don
*p_size = l_read;
if(l_read == -2)
{
free(l_buffer);
uv__free(l_buffer);
return NULL;
}
if(l_read >= 0)
@ -169,7 +169,7 @@ static struct nlmsghdr *getNetlinkResponse(int p_socket, int *p_size, int *p_don
if(l_hdr->nlmsg_type == NLMSG_ERROR)
{
free(l_buffer);
uv__free(l_buffer);
return NULL;
}
}
@ -182,7 +182,7 @@ static struct nlmsghdr *getNetlinkResponse(int p_socket, int *p_size, int *p_don
static NetlinkList *newListItem(struct nlmsghdr *p_data, unsigned int p_size)
{
NetlinkList *l_item = malloc(sizeof(NetlinkList));
NetlinkList *l_item = uv__malloc(sizeof(NetlinkList));
if (l_item == NULL)
{
return NULL;
@ -201,8 +201,8 @@ static void freeResultList(NetlinkList *p_list)
{
l_cur = p_list;
p_list = p_list->m_next;
free(l_cur->m_data);
free(l_cur);
uv__free(l_cur->m_data);
uv__free(l_cur);
}
}
@ -348,7 +348,7 @@ static int interpretLink(struct nlmsghdr *p_hdr, struct ifaddrs **p_resultList)
}
}
l_entry = malloc(sizeof(struct ifaddrs) + sizeof(int) + l_nameSize + l_addrSize + l_dataSize);
l_entry = uv__malloc(sizeof(struct ifaddrs) + sizeof(int) + l_nameSize + l_addrSize + l_dataSize);
if (l_entry == NULL)
{
return -1;
@ -477,7 +477,7 @@ static int interpretAddr(struct nlmsghdr *p_hdr, struct ifaddrs **p_resultList,
}
}
l_entry = malloc(sizeof(struct ifaddrs) + l_nameSize + l_addrSize);
l_entry = uv__malloc(sizeof(struct ifaddrs) + l_nameSize + l_addrSize);
if (l_entry == NULL)
{
return -1;
@ -697,6 +697,6 @@ void freeifaddrs(struct ifaddrs *ifa)
{
l_cur = ifa;
ifa = ifa->ifa_next;
free(l_cur);
uv__free(l_cur);
}
}

View File

@ -198,7 +198,7 @@ int uv_cpu_info(uv_cpu_info_t** cpu_infos, int* count) {
return -EINVAL; /* FIXME(bnoordhuis) Translate error. */
}
*cpu_infos = malloc(numcpus * sizeof(**cpu_infos));
*cpu_infos = uv__malloc(numcpus * sizeof(**cpu_infos));
if (!(*cpu_infos))
return -ENOMEM; /* FIXME(bnoordhuis) Deallocate info? */
@ -226,10 +226,10 @@ void uv_free_cpu_info(uv_cpu_info_t* cpu_infos, int count) {
int i;
for (i = 0; i < count; i++) {
free(cpu_infos[i].model);
uv__free(cpu_infos[i].model);
}
free(cpu_infos);
uv__free(cpu_infos);
}
@ -255,7 +255,7 @@ int uv_interface_addresses(uv_interface_address_t** addresses, int* count) {
(*count)++;
}
*addresses = malloc(*count * sizeof(**addresses));
*addresses = uv__malloc(*count * sizeof(**addresses));
if (!(*addresses))
return -ENOMEM;
@ -324,8 +324,8 @@ void uv_free_interface_addresses(uv_interface_address_t* addresses,
int i;
for (i = 0; i < count; i++) {
free(addresses[i].name);
uv__free(addresses[i].name);
}
free(addresses);
uv__free(addresses);
}

View File

@ -40,7 +40,7 @@ int uv_dlopen(const char* filename, uv_lib_t* lib) {
void uv_dlclose(uv_lib_t* lib) {
if (lib->errmsg) {
free(lib->errmsg);
uv__free(lib->errmsg);
lib->errmsg = NULL;
}
@ -68,7 +68,7 @@ static int uv__dlerror(uv_lib_t* lib) {
const char* errmsg;
if (lib->errmsg)
free(lib->errmsg);
uv__free(lib->errmsg);
errmsg = dlerror();

View File

@ -159,7 +159,7 @@ char** uv_setup_args(int argc, char** argv) {
int uv_set_process_title(const char* title) {
int oid[4];
if (process_title) free(process_title);
if (process_title) uv__free(process_title);
process_title = strdup(title);
oid[0] = CTL_KERN;
@ -271,7 +271,7 @@ int uv_cpu_info(uv_cpu_info_t** cpu_infos, int* count) {
if (sysctlbyname("hw.ncpu", &numcpus, &size, NULL, 0))
return -errno;
*cpu_infos = malloc(numcpus * sizeof(**cpu_infos));
*cpu_infos = uv__malloc(numcpus * sizeof(**cpu_infos));
if (!(*cpu_infos))
return -ENOMEM;
@ -279,7 +279,7 @@ int uv_cpu_info(uv_cpu_info_t** cpu_infos, int* count) {
size = sizeof(cpuspeed);
if (sysctlbyname("hw.clockrate", &cpuspeed, &size, NULL, 0)) {
SAVE_ERRNO(free(*cpu_infos));
SAVE_ERRNO(uv__free(*cpu_infos));
return -errno;
}
@ -288,21 +288,21 @@ int uv_cpu_info(uv_cpu_info_t** cpu_infos, int* count) {
*/
size = sizeof(maxcpus);
if (sysctlbyname(maxcpus_key, &maxcpus, &size, NULL, 0)) {
SAVE_ERRNO(free(*cpu_infos));
SAVE_ERRNO(uv__free(*cpu_infos));
return -errno;
}
size = maxcpus * CPUSTATES * sizeof(long);
cp_times = malloc(size);
cp_times = uv__malloc(size);
if (cp_times == NULL) {
free(*cpu_infos);
uv__free(*cpu_infos);
return -ENOMEM;
}
if (sysctlbyname(cptimes_key, cp_times, &size, NULL, 0)) {
SAVE_ERRNO(free(cp_times));
SAVE_ERRNO(free(*cpu_infos));
SAVE_ERRNO(uv__free(cp_times));
SAVE_ERRNO(uv__free(*cpu_infos));
return -errno;
}
@ -321,7 +321,7 @@ int uv_cpu_info(uv_cpu_info_t** cpu_infos, int* count) {
cur+=CPUSTATES;
}
free(cp_times);
uv__free(cp_times);
return 0;
}
@ -330,10 +330,10 @@ void uv_free_cpu_info(uv_cpu_info_t* cpu_infos, int count) {
int i;
for (i = 0; i < count; i++) {
free(cpu_infos[i].model);
uv__free(cpu_infos[i].model);
}
free(cpu_infos);
uv__free(cpu_infos);
}
@ -359,7 +359,7 @@ int uv_interface_addresses(uv_interface_address_t** addresses, int* count) {
(*count)++;
}
*addresses = malloc(*count * sizeof(**addresses));
*addresses = uv__malloc(*count * sizeof(**addresses));
if (!(*addresses))
return -ENOMEM;
@ -428,8 +428,8 @@ void uv_free_interface_addresses(uv_interface_address_t* addresses,
int i;
for (i = 0; i < count; i++) {
free(addresses[i].name);
uv__free(addresses[i].name);
}
free(addresses);
uv__free(addresses);
}

View File

@ -85,7 +85,7 @@
size_t new_path_len; \
path_len = strlen((path)) + 1; \
new_path_len = strlen((new_path)) + 1; \
(req)->path = malloc(path_len + new_path_len); \
(req)->path = uv__malloc(path_len + new_path_len); \
if ((req)->path == NULL) \
return -ENOMEM; \
(req)->new_path = (req)->path + path_len; \
@ -272,7 +272,7 @@ static ssize_t uv__fs_read(uv_fs_t* req) {
done:
if (req->bufs != req->bufsml)
free(req->bufs);
uv__free(req->bufs);
return result;
}
@ -312,8 +312,8 @@ out:
int i;
for (i = 0; i < n; i++)
free(dents[i]);
free(dents);
uv__free(dents[i]);
uv__free(dents);
}
errno = saved_errno;
@ -337,7 +337,7 @@ static ssize_t uv__fs_readlink(uv_fs_t* req) {
#endif
}
buf = malloc(len + 1);
buf = uv__malloc(len + 1);
if (buf == NULL) {
errno = ENOMEM;
@ -347,7 +347,7 @@ static ssize_t uv__fs_readlink(uv_fs_t* req) {
len = readlink(req->path, buf, len);
if (len == -1) {
free(buf);
uv__free(buf);
return -1;
}
@ -633,7 +633,7 @@ done:
#endif
if (req->bufs != req->bufsml)
free(req->bufs);
uv__free(req->bufs);
return r;
}
@ -1025,7 +1025,7 @@ int uv_fs_read(uv_loop_t* loop, uv_fs_t* req,
req->nbufs = nbufs;
req->bufs = req->bufsml;
if (nbufs > ARRAY_SIZE(req->bufsml))
req->bufs = malloc(nbufs * sizeof(*bufs));
req->bufs = uv__malloc(nbufs * sizeof(*bufs));
if (req->bufs == NULL)
return -ENOMEM;
@ -1147,7 +1147,7 @@ int uv_fs_write(uv_loop_t* loop,
req->nbufs = nbufs;
req->bufs = req->bufsml;
if (nbufs > ARRAY_SIZE(req->bufsml))
req->bufs = malloc(nbufs * sizeof(*bufs));
req->bufs = uv__malloc(nbufs * sizeof(*bufs));
if (req->bufs == NULL)
return -ENOMEM;
@ -1160,7 +1160,7 @@ int uv_fs_write(uv_loop_t* loop,
void uv_fs_req_cleanup(uv_fs_t* req) {
free((void*) req->path);
uv__free((void*)req->path);
req->path = NULL;
req->new_path = NULL;
@ -1168,6 +1168,6 @@ void uv_fs_req_cleanup(uv_fs_t* req) {
uv__fs_scandir_cleanup(req);
if (req->ptr != &req->statbuf)
free(req->ptr);
uv__free(req->ptr);
req->ptr = NULL;
}

View File

@ -169,7 +169,7 @@ static void (*pFSEventStreamStop)(FSEventStreamRef);
if (!uv__is_closing((handle)) && uv__is_active((handle))) \
block \
/* Free allocated data */ \
free(event); \
uv__free(event); \
} \
if (err != 0 && !uv__is_closing((handle)) && uv__is_active((handle))) \
(handle)->cb((handle), NULL, 0, err); \
@ -280,7 +280,7 @@ static void uv__fsevents_event_cb(ConstFSEventStreamRef streamRef,
len = 0;
#endif /* MAC_OS_X_VERSION_10_7 */
event = malloc(sizeof(*event) + len);
event = uv__malloc(sizeof(*event) + len);
if (event == NULL)
break;
@ -425,7 +425,7 @@ static void uv__fsevents_reschedule(uv_fs_event_t* handle) {
uv_mutex_lock(&state->fsevent_mutex);
path_count = state->fsevent_handle_count;
if (path_count != 0) {
paths = malloc(sizeof(*paths) * path_count);
paths = uv__malloc(sizeof(*paths) * path_count);
if (paths == NULL) {
uv_mutex_unlock(&state->fsevent_mutex);
goto final;
@ -465,7 +465,7 @@ final:
if (cf_paths == NULL) {
while (i != 0)
pCFRelease(paths[--i]);
free(paths);
uv__free(paths);
} else {
/* CFArray takes ownership of both strings and original C-array */
pCFRelease(cf_paths);
@ -662,7 +662,7 @@ fail_sem_init:
uv_mutex_destroy(&loop->cf_mutex);
fail_mutex_init:
free(state);
uv__free(state);
return err;
}
@ -688,7 +688,7 @@ void uv__fsevents_loop_delete(uv_loop_t* loop) {
q = QUEUE_HEAD(&loop->cf_signals);
s = QUEUE_DATA(q, uv__cf_loop_signal_t, member);
QUEUE_REMOVE(q);
free(s);
uv__free(s);
}
/* Destroy state */
@ -696,7 +696,7 @@ void uv__fsevents_loop_delete(uv_loop_t* loop) {
uv_sem_destroy(&state->fsevent_sem);
uv_mutex_destroy(&state->fsevent_mutex);
pCFRelease(state->signal_source);
free(state);
uv__free(state);
loop->cf_state = NULL;
}
@ -756,7 +756,7 @@ static void uv__cf_loop_cb(void* arg) {
uv__fsevents_reschedule(s->handle);
QUEUE_REMOVE(item);
free(s);
uv__free(s);
}
}
@ -766,7 +766,7 @@ int uv__cf_loop_signal(uv_loop_t* loop, uv_fs_event_t* handle) {
uv__cf_loop_signal_t* item;
uv__cf_loop_state_t* state;
item = malloc(sizeof(*item));
item = uv__malloc(sizeof(*item));
if (item == NULL)
return -ENOMEM;
@ -808,7 +808,7 @@ int uv__fsevents_init(uv_fs_event_t* handle) {
* Events will occur in other thread.
* Initialize callback for getting them back into event loop's thread
*/
handle->cf_cb = malloc(sizeof(*handle->cf_cb));
handle->cf_cb = uv__malloc(sizeof(*handle->cf_cb));
if (handle->cf_cb == NULL) {
err = -ENOMEM;
goto fail_cf_cb_malloc;
@ -843,11 +843,11 @@ fail_loop_signal:
uv_mutex_destroy(&handle->cf_mutex);
fail_cf_mutex_init:
free(handle->cf_cb);
uv__free(handle->cf_cb);
handle->cf_cb = NULL;
fail_cf_cb_malloc:
free(handle->realpath);
uv__free(handle->realpath);
handle->realpath = NULL;
handle->realpath_len = 0;
@ -880,7 +880,7 @@ int uv__fsevents_close(uv_fs_event_t* handle) {
/* Wait for deinitialization */
uv_sem_wait(&state->fsevent_sem);
uv_close((uv_handle_t*) handle->cf_cb, (uv_close_cb) free);
uv_close((uv_handle_t*) handle->cf_cb, (uv_close_cb) uv__free);
handle->cf_cb = NULL;
/* Free data in queue */
@ -889,7 +889,7 @@ int uv__fsevents_close(uv_fs_event_t* handle) {
});
uv_mutex_destroy(&handle->cf_mutex);
free(handle->realpath);
uv__free(handle->realpath);
handle->realpath = NULL;
handle->realpath_len = 0;

View File

@ -112,11 +112,11 @@ static void uv__getaddrinfo_done(struct uv__work* w, int status) {
/* See initialization in uv_getaddrinfo(). */
if (req->hints)
free(req->hints);
uv__free(req->hints);
else if (req->service)
free(req->service);
uv__free(req->service);
else if (req->hostname)
free(req->hostname);
uv__free(req->hostname);
else
assert(0);
@ -152,7 +152,7 @@ int uv_getaddrinfo(uv_loop_t* loop,
hostname_len = hostname ? strlen(hostname) + 1 : 0;
service_len = service ? strlen(service) + 1 : 0;
hints_len = hints ? sizeof(*hints) : 0;
buf = malloc(hostname_len + service_len + hints_len);
buf = uv__malloc(hostname_len + service_len + hints_len);
if (buf == NULL)
return -ENOMEM;

View File

@ -403,7 +403,7 @@ int uv_fs_event_stop(uv_fs_event_t* handle) {
uv__io_close(handle->loop, &handle->event_watcher);
}
free(handle->path);
uv__free(handle->path);
handle->path = NULL;
uv__close(handle->event_watcher.fd);

View File

@ -755,10 +755,10 @@ void uv_free_cpu_info(uv_cpu_info_t* cpu_infos, int count) {
int i;
for (i = 0; i < count; i++) {
free(cpu_infos[i].model);
uv__free(cpu_infos[i].model);
}
free(cpu_infos);
uv__free(cpu_infos);
}
@ -792,7 +792,7 @@ int uv_interface_addresses(uv_interface_address_t** addresses,
if (*count == 0)
return 0;
*addresses = malloc(*count * sizeof(**addresses));
*addresses = uv__malloc(*count * sizeof(**addresses));
if (!(*addresses))
return -ENOMEM;
@ -862,10 +862,10 @@ void uv_free_interface_addresses(uv_interface_address_t* addresses,
int i;
for (i = 0; i < count; i++) {
free(addresses[i].name);
uv__free(addresses[i].name);
}
free(addresses);
uv__free(addresses);
}

View File

@ -207,7 +207,7 @@ int uv_fs_event_start(uv_fs_event_t* handle,
if (w)
goto no_insert;
w = malloc(sizeof(*w) + strlen(path) + 1);
w = uv__malloc(sizeof(*w) + strlen(path) + 1);
if (w == NULL)
return -ENOMEM;
@ -245,7 +245,7 @@ int uv_fs_event_stop(uv_fs_event_t* handle) {
/* No watchers left for this path. Clean up. */
RB_REMOVE(watcher_root, CAST(&handle->loop->inotify_watchers), w);
uv__inotify_rm_watch(handle->loop->inotify_fd, w->wd);
free(w);
uv__free(w);
}
return 0;

View File

@ -117,7 +117,7 @@ void uv__loop_close(uv_loop_t* loop) {
assert(loop->nfds == 0);
#endif
free(loop->watchers);
uv__free(loop->watchers);
loop->watchers = NULL;
loop->nwatchers = 0;
}

View File

@ -137,7 +137,7 @@ char** uv_setup_args(int argc, char** argv) {
int uv_set_process_title(const char* title) {
if (process_title) free(process_title);
if (process_title) uv__free(process_title);
process_title = strdup(title);
setproctitle("%s", title);
@ -234,17 +234,17 @@ int uv_cpu_info(uv_cpu_info_t** cpu_infos, int* count) {
cpuspeed = 0;
size = numcpus * CPUSTATES * sizeof(*cp_times);
cp_times = malloc(size);
cp_times = uv__malloc(size);
if (cp_times == NULL)
return -ENOMEM;
if (sysctlbyname("kern.cp_time", cp_times, &size, NULL, 0))
return -errno;
*cpu_infos = malloc(numcpus * sizeof(**cpu_infos));
*cpu_infos = uv__malloc(numcpus * sizeof(**cpu_infos));
if (!(*cpu_infos)) {
free(cp_times);
free(*cpu_infos);
uv__free(cp_times);
uv__free(*cpu_infos);
return -ENOMEM;
}
@ -259,7 +259,7 @@ int uv_cpu_info(uv_cpu_info_t** cpu_infos, int* count) {
cpu_info->speed = (int)(cpuspeed/(uint64_t) 1e6);
cur += CPUSTATES;
}
free(cp_times);
uv__free(cp_times);
return 0;
}
@ -268,10 +268,10 @@ void uv_free_cpu_info(uv_cpu_info_t* cpu_infos, int count) {
int i;
for (i = 0; i < count; i++) {
free(cpu_infos[i].model);
uv__free(cpu_infos[i].model);
}
free(cpu_infos);
uv__free(cpu_infos);
}
@ -296,7 +296,7 @@ int uv_interface_addresses(uv_interface_address_t** addresses, int* count) {
(*count)++;
}
*addresses = malloc(*count * sizeof(**addresses));
*addresses = uv__malloc(*count * sizeof(**addresses));
if (!(*addresses))
return -ENOMEM;
@ -361,8 +361,8 @@ void uv_free_interface_addresses(uv_interface_address_t* addresses, int count) {
int i;
for (i = 0; i < count; i++) {
free(addresses[i].name);
uv__free(addresses[i].name);
}
free(addresses);
uv__free(addresses);
}

View File

@ -124,7 +124,7 @@ int uv_exepath(char* buffer, size_t* size) {
err = 0;
out:
free(argsbuf);
uv__free(argsbuf);
return err;
}
@ -161,7 +161,7 @@ char** uv_setup_args(int argc, char** argv) {
int uv_set_process_title(const char* title) {
if (process_title) free(process_title);
if (process_title) uv__free(process_title);
process_title = strdup(title);
setproctitle(title);
return 0;
@ -238,7 +238,7 @@ int uv_cpu_info(uv_cpu_info_t** cpu_infos, int* count) {
if (sysctl(which, 2, &numcpus, &size, NULL, 0))
return -errno;
*cpu_infos = malloc(numcpus * sizeof(**cpu_infos));
*cpu_infos = uv__malloc(numcpus * sizeof(**cpu_infos));
if (!(*cpu_infos))
return -ENOMEM;
@ -247,7 +247,7 @@ int uv_cpu_info(uv_cpu_info_t** cpu_infos, int* count) {
which[1] = HW_CPUSPEED;
size = sizeof(cpuspeed);
if (sysctl(which, 2, &cpuspeed, &size, NULL, 0)) {
SAVE_ERRNO(free(*cpu_infos));
SAVE_ERRNO(uv__free(*cpu_infos));
return -errno;
}
@ -258,7 +258,7 @@ int uv_cpu_info(uv_cpu_info_t** cpu_infos, int* count) {
which[2] = i;
size = sizeof(info);
if (sysctl(which, 3, &info, &size, NULL, 0)) {
SAVE_ERRNO(free(*cpu_infos));
SAVE_ERRNO(uv__free(*cpu_infos));
return -errno;
}
@ -282,10 +282,10 @@ void uv_free_cpu_info(uv_cpu_info_t* cpu_infos, int count) {
int i;
for (i = 0; i < count; i++) {
free(cpu_infos[i].model);
uv__free(cpu_infos[i].model);
}
free(cpu_infos);
uv__free(cpu_infos);
}
@ -311,7 +311,7 @@ int uv_interface_addresses(uv_interface_address_t** addresses,
(*count)++;
}
*addresses = malloc(*count * sizeof(**addresses));
*addresses = uv__malloc(*count * sizeof(**addresses));
if (!(*addresses))
return -ENOMEM;
@ -377,8 +377,8 @@ void uv_free_interface_addresses(uv_interface_address_t* addresses,
int i;
for (i = 0; i < count; i++) {
free(addresses[i].name);
uv__free(addresses[i].name);
}
free(addresses);
uv__free(addresses);
}

View File

@ -88,7 +88,7 @@ err_bind:
uv__close(sockfd);
err_socket:
free((void*)pipe_fname);
uv__free((void*)pipe_fname);
return err;
}
@ -116,7 +116,7 @@ void uv__pipe_close(uv_pipe_t* handle) {
* another thread or process.
*/
unlink(handle->pipe_fname);
free((void*)handle->pipe_fname);
uv__free((void*)handle->pipe_fname);
handle->pipe_fname = NULL;
}

View File

@ -383,7 +383,7 @@ int uv_spawn(uv_loop_t* loop,
stdio_count = 3;
err = -ENOMEM;
pipes = malloc(stdio_count * sizeof(*pipes));
pipes = uv__malloc(stdio_count * sizeof(*pipes));
if (pipes == NULL)
goto error;
@ -482,7 +482,7 @@ int uv_spawn(uv_loop_t* loop,
process->pid = pid;
process->exit_cb = options->exit_cb;
free(pipes);
uv__free(pipes);
return exec_errorno;
error:
@ -496,7 +496,7 @@ error:
if (pipes[i][1] != -1)
close(pipes[i][1]);
}
free(pipes);
uv__free(pipes);
}
return err;

View File

@ -55,7 +55,7 @@ char** uv_setup_args(int argc, char** argv) {
/* Add space for the argv pointers. */
size += (argc + 1) * sizeof(char*);
new_argv = malloc(size);
new_argv = uv__malloc(size);
if (new_argv == NULL)
return argv;
args_mem = new_argv;
@ -97,6 +97,6 @@ int uv_get_process_title(char* buffer, size_t size) {
UV_DESTRUCTOR(static void free_args_mem(void)) {
free(args_mem); /* Keep valgrind happy. */
uv__free(args_mem); /* Keep valgrind happy. */
args_mem = NULL;
}

View File

@ -244,7 +244,7 @@ static void uv__stream_osx_cb_close(uv_handle_t* async) {
uv__stream_select_t* s;
s = container_of(async, uv__stream_select_t, async);
free(s);
uv__free(s);
}
@ -304,7 +304,7 @@ int uv__stream_try_select(uv_stream_t* stream, int* fd) {
sread_sz = (max_fd + NBBY) / NBBY;
swrite_sz = sread_sz;
s = malloc(sizeof(*s) + sread_sz + swrite_sz);
s = uv__malloc(sizeof(*s) + sread_sz + swrite_sz);
if (s == NULL) {
err = -ENOMEM;
goto failed_malloc;
@ -363,7 +363,7 @@ failed_close_sem_init:
return err;
failed_async_init:
free(s);
uv__free(s);
failed_malloc:
uv__close(fds[0]);
@ -600,7 +600,7 @@ done:
/* All read, free */
assert(queued_fds->offset > 0);
if (--queued_fds->offset == 0) {
free(queued_fds);
uv__free(queued_fds);
server->queued_fds = NULL;
} else {
/* Shift rest */
@ -698,7 +698,7 @@ static void uv__write_req_finish(uv_write_t* req) {
*/
if (req->error == 0) {
if (req->bufs != req->bufsml)
free(req->bufs);
uv__free(req->bufs);
req->bufs = NULL;
}
@ -909,7 +909,7 @@ static void uv__write_callbacks(uv_stream_t* stream) {
if (req->bufs != NULL) {
stream->write_queue_size -= uv__write_req_size(req);
if (req->bufs != req->bufsml)
free(req->bufs);
uv__free(req->bufs);
req->bufs = NULL;
}
@ -974,8 +974,8 @@ static int uv__stream_queue_fd(uv_stream_t* stream, int fd) {
queued_fds = stream->queued_fds;
if (queued_fds == NULL) {
queue_size = 8;
queued_fds = malloc((queue_size - 1) * sizeof(*queued_fds->fds) +
sizeof(*queued_fds));
queued_fds = uv__malloc((queue_size - 1) * sizeof(*queued_fds->fds) +
sizeof(*queued_fds));
if (queued_fds == NULL)
return -ENOMEM;
queued_fds->size = queue_size;
@ -1352,7 +1352,7 @@ int uv_write2(uv_write_t* req,
req->bufs = req->bufsml;
if (nbufs > ARRAY_SIZE(req->bufsml))
req->bufs = malloc(nbufs * sizeof(bufs[0]));
req->bufs = uv__malloc(nbufs * sizeof(bufs[0]));
if (req->bufs == NULL)
return -ENOMEM;
@ -1440,7 +1440,7 @@ int uv_try_write(uv_stream_t* stream,
QUEUE_REMOVE(&req.queue);
uv__req_unregister(stream->loop, &req);
if (req.bufs != req.bufsml)
free(req.bufs);
uv__free(req.bufs);
req.bufs = NULL;
/* Do not poll for writable, if we wasn't before calling this */
@ -1577,7 +1577,7 @@ void uv__stream_close(uv_stream_t* handle) {
queued_fds = handle->queued_fds;
for (i = 0; i < queued_fds->offset; i++)
uv__close(queued_fds->fds[i]);
free(handle->queued_fds);
uv__free(handle->queued_fds);
handle->queued_fds = NULL;
}

View File

@ -463,7 +463,7 @@ int uv_fs_event_stop(uv_fs_event_t* handle) {
}
handle->fd = PORT_DELETED;
free(handle->path);
uv__free(handle->path);
handle->path = NULL;
handle->fo.fo_name = NULL;
uv__handle_stop(handle);
@ -582,7 +582,7 @@ int uv_cpu_info(uv_cpu_info_t** cpu_infos, int* count) {
lookup_instance++;
}
*cpu_infos = malloc(lookup_instance * sizeof(**cpu_infos));
*cpu_infos = uv__malloc(lookup_instance * sizeof(**cpu_infos));
if (!(*cpu_infos)) {
kstat_close(kc);
return -ENOMEM;
@ -659,10 +659,10 @@ void uv_free_cpu_info(uv_cpu_info_t* cpu_infos, int count) {
int i;
for (i = 0; i < count; i++) {
free(cpu_infos[i].model);
uv__free(cpu_infos[i].model);
}
free(cpu_infos);
uv__free(cpu_infos);
}
@ -692,7 +692,7 @@ int uv_interface_addresses(uv_interface_address_t** addresses, int* count) {
(*count)++;
}
*addresses = malloc(*count * sizeof(**addresses));
*addresses = uv__malloc(*count * sizeof(**addresses));
if (!(*addresses))
return -ENOMEM;
@ -756,8 +756,8 @@ void uv_free_interface_addresses(uv_interface_address_t* addresses,
int i;
for (i = 0; i < count; i++) {
free(addresses[i].name);
uv__free(addresses[i].name);
}
free(addresses);
uv__free(addresses);
}

View File

@ -45,7 +45,7 @@ static void* uv__thread_start(void *arg)
ctx_p = arg;
ctx = *ctx_p;
free(ctx_p);
uv__free(ctx_p);
ctx.entry(ctx.arg);
return 0;
@ -56,7 +56,7 @@ int uv_thread_create(uv_thread_t *tid, void (*entry)(void *arg), void *arg) {
struct thread_ctx* ctx;
int err;
ctx = malloc(sizeof(*ctx));
ctx = uv__malloc(sizeof(*ctx));
if (ctx == NULL)
return UV_ENOMEM;
@ -66,7 +66,7 @@ int uv_thread_create(uv_thread_t *tid, void (*entry)(void *arg), void *arg) {
err = pthread_create(tid, NULL, uv__thread_start, ctx);
if (err)
free(ctx);
uv__free(ctx);
return -err;
}

View File

@ -100,7 +100,7 @@ static void uv__udp_run_completed(uv_udp_t* handle) {
handle->send_queue_count--;
if (req->bufs != req->bufsml)
free(req->bufs);
uv__free(req->bufs);
req->bufs = NULL;
if (req->send_cb == NULL)
@ -399,7 +399,7 @@ int uv__udp_send(uv_udp_send_t* req,
req->bufs = req->bufsml;
if (nbufs > ARRAY_SIZE(req->bufsml))
req->bufs = malloc(nbufs * sizeof(bufs[0]));
req->bufs = uv__malloc(nbufs * sizeof(bufs[0]));
if (req->bufs == NULL)
return -ENOMEM;

View File

@ -33,6 +33,34 @@
# include <net/if.h> /* if_nametoindex */
#endif
static uv_malloc_func replaced_malloc;
static uv_free_func replaced_free;
void* uv__malloc(size_t size) {
if (replaced_malloc)
return (*replaced_malloc)(size);
return malloc(size);
}
void uv__free(void* ptr) {
if (replaced_free)
(*replaced_free)(ptr);
else
free(ptr);
}
int uv_replace_allocator(uv_malloc_func malloc_func, uv_free_func free_func) {
if (replaced_malloc || replaced_free)
return UV_EINVAL;
replaced_malloc = malloc_func;
replaced_free = free_func;
return 0;
}
#define XX(uc, lc) case UV_##uc: return sizeof(uv_##lc##_t);
size_t uv_handle_size(uv_handle_type type) {
@ -400,7 +428,7 @@ void uv__fs_scandir_cleanup(uv_fs_t* req) {
if (*nbufs > 0 && *nbufs != (unsigned int) req->result)
(*nbufs)--;
for (; *nbufs < (unsigned int) req->result; (*nbufs)++)
free(dents[*nbufs]);
uv__free(dents[*nbufs]);
}
@ -414,11 +442,11 @@ int uv_fs_scandir_next(uv_fs_t* req, uv_dirent_t* ent) {
/* Free previous entity */
if (*nbufs > 0)
free(dents[*nbufs - 1]);
uv__free(dents[*nbufs - 1]);
/* End was already reached */
if (*nbufs == (unsigned int) req->result) {
free(dents);
uv__free(dents);
req->ptr = NULL;
return UV_EOF;
}
@ -492,12 +520,12 @@ uv_loop_t* uv_default_loop(void) {
uv_loop_t* uv_loop_new(void) {
uv_loop_t* loop;
loop = malloc(sizeof(*loop));
loop = uv__malloc(sizeof(*loop));
if (loop == NULL)
return NULL;
if (uv_loop_init(loop)) {
free(loop);
uv__free(loop);
return NULL;
}
@ -539,5 +567,5 @@ void uv_loop_delete(uv_loop_t* loop) {
err = uv_loop_close(loop);
assert(err == 0);
if (loop != default_loop)
free(loop);
uv__free(loop);
}

View File

@ -63,6 +63,10 @@ enum {
# define UV__HANDLE_CLOSING 0x01
#endif
void* uv__malloc(size_t size);
void uv__free(void* ptr);
int uv__loop_configure(uv_loop_t* loop, uv_loop_option option, va_list ap);
void uv__loop_close(uv_loop_t* loop);

View File

@ -73,13 +73,13 @@ static int uv_split_path(const WCHAR* filename, WCHAR** dir,
if (i == 0) {
if (dir) {
*dir = (WCHAR*)malloc((MAX_PATH + 1) * sizeof(WCHAR));
*dir = (WCHAR*)uv__malloc((MAX_PATH + 1) * sizeof(WCHAR));
if (!*dir) {
uv_fatal_error(ERROR_OUTOFMEMORY, "malloc");
uv_fatal_error(ERROR_OUTOFMEMORY, "uv__malloc");
}
if (!GetCurrentDirectoryW(MAX_PATH, *dir)) {
free(*dir);
uv__free(*dir);
*dir = NULL;
return -1;
}
@ -88,17 +88,17 @@ static int uv_split_path(const WCHAR* filename, WCHAR** dir,
*file = wcsdup(filename);
} else {
if (dir) {
*dir = (WCHAR*)malloc((i + 1) * sizeof(WCHAR));
*dir = (WCHAR*)uv__malloc((i + 1) * sizeof(WCHAR));
if (!*dir) {
uv_fatal_error(ERROR_OUTOFMEMORY, "malloc");
uv_fatal_error(ERROR_OUTOFMEMORY, "uv__malloc");
}
wcsncpy(*dir, filename, i);
(*dir)[i] = L'\0';
}
*file = (WCHAR*)malloc((len - i) * sizeof(WCHAR));
*file = (WCHAR*)uv__malloc((len - i) * sizeof(WCHAR));
if (!*file) {
uv_fatal_error(ERROR_OUTOFMEMORY, "malloc");
uv_fatal_error(ERROR_OUTOFMEMORY, "uv__malloc");
}
wcsncpy(*file, filename + i + 1, len - i - 1);
(*file)[len - i - 1] = L'\0';
@ -140,16 +140,16 @@ int uv_fs_event_start(uv_fs_event_t* handle,
handle->cb = cb;
handle->path = strdup(path);
if (!handle->path) {
uv_fatal_error(ERROR_OUTOFMEMORY, "malloc");
uv_fatal_error(ERROR_OUTOFMEMORY, "uv__malloc");
}
uv__handle_start(handle);
/* Convert name to UTF16. */
name_size = uv_utf8_to_utf16(path, NULL, 0) * sizeof(WCHAR);
pathw = (WCHAR*)malloc(name_size);
pathw = (WCHAR*)uv__malloc(name_size);
if (!pathw) {
uv_fatal_error(ERROR_OUTOFMEMORY, "malloc");
uv_fatal_error(ERROR_OUTOFMEMORY, "uv__malloc");
}
if (!uv_utf8_to_utf16(path, pathw,
@ -193,7 +193,7 @@ int uv_fs_event_start(uv_fs_event_t* handle,
}
dir_to_watch = dir;
free(pathw);
uv__free(pathw);
pathw = NULL;
}
@ -208,7 +208,7 @@ int uv_fs_event_start(uv_fs_event_t* handle,
NULL);
if (dir) {
free(dir);
uv__free(dir);
dir = NULL;
}
@ -226,11 +226,10 @@ int uv_fs_event_start(uv_fs_event_t* handle,
}
if (!handle->buffer) {
handle->buffer = (char*)_aligned_malloc(uv_directory_watcher_buffer_size,
sizeof(DWORD));
handle->buffer = (char*)uv__malloc(uv_directory_watcher_buffer_size);
}
if (!handle->buffer) {
uv_fatal_error(ERROR_OUTOFMEMORY, "malloc");
uv_fatal_error(ERROR_OUTOFMEMORY, "uv__malloc");
}
memset(&(handle->req.u.io.overlapped), 0,
@ -260,21 +259,21 @@ int uv_fs_event_start(uv_fs_event_t* handle,
error:
if (handle->path) {
free(handle->path);
uv__free(handle->path);
handle->path = NULL;
}
if (handle->filew) {
free(handle->filew);
uv__free(handle->filew);
handle->filew = NULL;
}
if (handle->short_filew) {
free(handle->short_filew);
uv__free(handle->short_filew);
handle->short_filew = NULL;
}
free(pathw);
uv__free(pathw);
if (handle->dir_handle != INVALID_HANDLE_VALUE) {
CloseHandle(handle->dir_handle);
@ -282,7 +281,7 @@ error:
}
if (handle->buffer) {
_aligned_free(handle->buffer);
uv__free(handle->buffer);
handle->buffer = NULL;
}
@ -302,22 +301,22 @@ int uv_fs_event_stop(uv_fs_event_t* handle) {
uv__handle_stop(handle);
if (handle->filew) {
free(handle->filew);
uv__free(handle->filew);
handle->filew = NULL;
}
if (handle->short_filew) {
free(handle->short_filew);
uv__free(handle->short_filew);
handle->short_filew = NULL;
}
if (handle->path) {
free(handle->path);
uv__free(handle->path);
handle->path = NULL;
}
if (handle->dirw) {
free(handle->dirw);
uv__free(handle->dirw);
handle->dirw = NULL;
}
@ -379,9 +378,9 @@ void uv_process_fs_event_req(uv_loop_t* loop, uv_req_t* req,
size = wcslen(handle->dirw) +
file_info->FileNameLength / sizeof(WCHAR) + 2;
filenamew = (WCHAR*)malloc(size * sizeof(WCHAR));
filenamew = (WCHAR*)uv__malloc(size * sizeof(WCHAR));
if (!filenamew) {
uv_fatal_error(ERROR_OUTOFMEMORY, "malloc");
uv_fatal_error(ERROR_OUTOFMEMORY, "uv__malloc");
}
_snwprintf(filenamew, size, L"%s\\%.*s", handle->dirw,
@ -394,26 +393,26 @@ void uv_process_fs_event_req(uv_loop_t* loop, uv_req_t* req,
size = GetLongPathNameW(filenamew, NULL, 0);
if (size) {
long_filenamew = (WCHAR*)malloc(size * sizeof(WCHAR));
long_filenamew = (WCHAR*)uv__malloc(size * sizeof(WCHAR));
if (!long_filenamew) {
uv_fatal_error(ERROR_OUTOFMEMORY, "malloc");
uv_fatal_error(ERROR_OUTOFMEMORY, "uv__malloc");
}
size = GetLongPathNameW(filenamew, long_filenamew, size);
if (size) {
long_filenamew[size] = '\0';
} else {
free(long_filenamew);
uv__free(long_filenamew);
long_filenamew = NULL;
}
}
free(filenamew);
uv__free(filenamew);
if (long_filenamew) {
/* Get the file name out of the long path. */
result = uv_split_path(long_filenamew, NULL, &filenamew);
free(long_filenamew);
uv__free(long_filenamew);
if (result == 0) {
long_filenamew = filenamew;
@ -448,9 +447,9 @@ void uv_process_fs_event_req(uv_loop_t* loop, uv_req_t* req,
NULL,
0);
if (size) {
filename = (char*)malloc(size + 1);
filename = (char*)uv__malloc(size + 1);
if (!filename) {
uv_fatal_error(ERROR_OUTOFMEMORY, "malloc");
uv_fatal_error(ERROR_OUTOFMEMORY, "uv__malloc");
}
size = uv_utf16_to_utf8(filenamew,
@ -460,7 +459,7 @@ void uv_process_fs_event_req(uv_loop_t* loop, uv_req_t* req,
if (size) {
filename[size] = '\0';
} else {
free(filename);
uv__free(filename);
filename = NULL;
}
}
@ -479,9 +478,9 @@ void uv_process_fs_event_req(uv_loop_t* loop, uv_req_t* req,
break;
}
free(filename);
uv__free(filename);
filename = NULL;
free(long_filenamew);
uv__free(long_filenamew);
long_filenamew = NULL;
}
@ -520,7 +519,7 @@ void uv_fs_event_endgame(uv_loop_t* loop, uv_fs_event_t* handle) {
assert(!(handle->flags & UV_HANDLE_CLOSED));
if (handle->buffer) {
_aligned_free(handle->buffer);
uv__free(handle->buffer);
handle->buffer = NULL;
}

View File

@ -167,7 +167,7 @@ INLINE static int fs__capture_path(uv_loop_t* loop, uv_fs_t* req,
return 0;
}
buf = (char*) malloc(buf_sz);
buf = (char*) uv__malloc(buf_sz);
if (buf == NULL) {
return ERROR_OUTOFMEMORY;
}
@ -353,7 +353,7 @@ INLINE static int fs__readlink_handle(HANDLE handle, char** target_ptr,
/* If requested, allocate memory and convert to UTF8. */
if (target_ptr != NULL) {
int r;
target = (char*) malloc(target_len + 1);
target = (char*) uv__malloc(target_len + 1);
if (target == NULL) {
SetLastError(ERROR_OUTOFMEMORY);
return -1;
@ -891,7 +891,7 @@ void fs__scandir(uv_fs_t* req) {
* includes room for the first character of the filename, but `utf8_len`
* doesn't count the NULL terminator at this point.
*/
dirent = malloc(sizeof *dirent + utf8_len);
dirent = uv__malloc(sizeof *dirent + utf8_len);
if (dirent == NULL)
goto out_of_memory_error;
@ -980,9 +980,9 @@ cleanup:
if (dir_handle != INVALID_HANDLE_VALUE)
CloseHandle(dir_handle);
while (dirents_used > 0)
free(dirents[--dirents_used]);
uv__free(dirents[--dirents_used]);
if (dirents != NULL)
free(dirents);
uv__free(dirents);
}
@ -1263,9 +1263,9 @@ static void fs__sendfile(uv_fs_t* req) {
size_t buf_size = length < max_buf_size ? length : max_buf_size;
int n, result = 0;
int64_t result_offset = 0;
char* buf = (char*) malloc(buf_size);
char* buf = (char*) uv__malloc(buf_size);
if (!buf) {
uv_fatal_error(ERROR_OUTOFMEMORY, "malloc");
uv_fatal_error(ERROR_OUTOFMEMORY, "uv__malloc");
}
if (offset != -1) {
@ -1296,7 +1296,7 @@ static void fs__sendfile(uv_fs_t* req) {
}
}
free(buf);
uv__free(buf);
SET_REQ_RESULT(req, result);
}
@ -1479,9 +1479,9 @@ static void fs__create_junction(uv_fs_t* req, const WCHAR* path,
2 * (target_len + 2) * sizeof(WCHAR);
/* Allocate the buffer */
buffer = (REPARSE_DATA_BUFFER*)malloc(needed_buf_size);
buffer = (REPARSE_DATA_BUFFER*)uv__malloc(needed_buf_size);
if (!buffer) {
uv_fatal_error(ERROR_OUTOFMEMORY, "malloc");
uv_fatal_error(ERROR_OUTOFMEMORY, "uv__malloc");
}
/* Grab a pointer to the part of the buffer where filenames go */
@ -1595,13 +1595,13 @@ static void fs__create_junction(uv_fs_t* req, const WCHAR* path,
/* Clean up */
CloseHandle(handle);
free(buffer);
uv__free(buffer);
SET_REQ_RESULT(req, 0);
return;
error:
free(buffer);
uv__free(buffer);
if (handle != INVALID_HANDLE_VALUE) {
CloseHandle(handle);
@ -1739,10 +1739,10 @@ void uv_fs_req_cleanup(uv_fs_t* req) {
return;
if (req->flags & UV_FS_FREE_PATHS)
free(req->file.pathw);
uv__free(req->file.pathw);
if (req->flags & UV_FS_FREE_PTR)
free(req->ptr);
uv__free(req->ptr);
req->path = NULL;
req->file.pathw = NULL;
@ -1805,7 +1805,7 @@ int uv_fs_read(uv_loop_t* loop,
req->fs.info.nbufs = nbufs;
req->fs.info.bufs = req->fs.info.bufsml;
if (nbufs > ARRAY_SIZE(req->fs.info.bufsml))
req->fs.info.bufs = malloc(nbufs * sizeof(*bufs));
req->fs.info.bufs = uv__malloc(nbufs * sizeof(*bufs));
if (req->fs.info.bufs == NULL)
return UV_ENOMEM;
@ -1838,7 +1838,7 @@ int uv_fs_write(uv_loop_t* loop,
req->fs.info.nbufs = nbufs;
req->fs.info.bufs = req->fs.info.bufsml;
if (nbufs > ARRAY_SIZE(req->fs.info.bufsml))
req->fs.info.bufs = malloc(nbufs * sizeof(*bufs));
req->fs.info.bufs = uv__malloc(nbufs * sizeof(*bufs));
if (req->fs.info.bufs == NULL)
return UV_ENOMEM;

View File

@ -111,7 +111,7 @@ static void uv__getaddrinfo_done(struct uv__work* w, int status) {
/* release input parameter memory */
if (req->alloc != NULL) {
free(req->alloc);
uv__free(req->alloc);
req->alloc = NULL;
}
@ -140,7 +140,7 @@ static void uv__getaddrinfo_done(struct uv__work* w, int status) {
}
/* allocate memory for addrinfo results */
alloc_ptr = (char*)malloc(addrinfo_len);
alloc_ptr = (char*)uv__malloc(addrinfo_len);
/* do conversions */
if (alloc_ptr != NULL) {
@ -221,7 +221,7 @@ void uv_freeaddrinfo(struct addrinfo* ai) {
/* release copied result memory */
if (alloc_ptr != NULL) {
free(alloc_ptr);
uv__free(alloc_ptr);
}
}
@ -286,7 +286,7 @@ int uv_getaddrinfo(uv_loop_t* loop,
}
/* allocate memory for inputs, and partition it as needed */
alloc_ptr = (char*)malloc(nodesize + servicesize + hintssize);
alloc_ptr = (char*)uv__malloc(nodesize + servicesize + hintssize);
if (!alloc_ptr) {
err = WSAENOBUFS;
goto error;
@ -356,7 +356,7 @@ int uv_getaddrinfo(uv_loop_t* loop,
error:
if (req != NULL && req->alloc != NULL) {
free(req->alloc);
uv__free(req->alloc);
}
return uv_translate_sys_error(err);
}

View File

@ -423,7 +423,7 @@ void uv_pipe_endgame(uv_loop_t* loop, uv_pipe_t* handle) {
&item->socket_info_ex.socket_info,
0,
WSA_FLAG_OVERLAPPED);
free(item);
uv__free(item);
if (socket != INVALID_SOCKET)
closesocket(socket);
@ -444,7 +444,7 @@ void uv_pipe_endgame(uv_loop_t* loop, uv_pipe_t* handle) {
if (handle->flags & UV_HANDLE_PIPESERVER) {
assert(handle->pipe.serv.accept_reqs);
free(handle->pipe.serv.accept_reqs);
uv__free(handle->pipe.serv.accept_reqs);
handle->pipe.serv.accept_reqs = NULL;
}
@ -478,9 +478,9 @@ int uv_pipe_bind(uv_pipe_t* handle, const char* name) {
}
handle->pipe.serv.accept_reqs = (uv_pipe_accept_t*)
malloc(sizeof(uv_pipe_accept_t) * handle->pipe.serv.pending_instances);
uv__malloc(sizeof(uv_pipe_accept_t) * handle->pipe.serv.pending_instances);
if (!handle->pipe.serv.accept_reqs) {
uv_fatal_error(ERROR_OUTOFMEMORY, "malloc");
uv_fatal_error(ERROR_OUTOFMEMORY, "uv__malloc");
}
for (i = 0; i < handle->pipe.serv.pending_instances; i++) {
@ -494,9 +494,9 @@ int uv_pipe_bind(uv_pipe_t* handle, const char* name) {
/* Convert name to UTF16. */
nameSize = uv_utf8_to_utf16(name, NULL, 0) * sizeof(WCHAR);
handle->name = (WCHAR*)malloc(nameSize);
handle->name = (WCHAR*)uv__malloc(nameSize);
if (!handle->name) {
uv_fatal_error(ERROR_OUTOFMEMORY, "malloc");
uv_fatal_error(ERROR_OUTOFMEMORY, "uv__malloc");
}
if (!uv_utf8_to_utf16(name, handle->name, nameSize / sizeof(WCHAR))) {
@ -540,7 +540,7 @@ int uv_pipe_bind(uv_pipe_t* handle, const char* name) {
error:
if (handle->name) {
free(handle->name);
uv__free(handle->name);
handle->name = NULL;
}
@ -607,9 +607,9 @@ void uv_pipe_connect(uv_connect_t* req, uv_pipe_t* handle,
/* Convert name to UTF16. */
nameSize = uv_utf8_to_utf16(name, NULL, 0) * sizeof(WCHAR);
handle->name = (WCHAR*)malloc(nameSize);
handle->name = (WCHAR*)uv__malloc(nameSize);
if (!handle->name) {
uv_fatal_error(ERROR_OUTOFMEMORY, "malloc");
uv_fatal_error(ERROR_OUTOFMEMORY, "uv__malloc");
}
if (!uv_utf8_to_utf16(name, handle->name, nameSize / sizeof(WCHAR))) {
@ -656,7 +656,7 @@ void uv_pipe_connect(uv_connect_t* req, uv_pipe_t* handle,
error:
if (handle->name) {
free(handle->name);
uv__free(handle->name);
handle->name = NULL;
}
@ -717,7 +717,7 @@ void uv_pipe_cleanup(uv_loop_t* loop, uv_pipe_t* handle) {
uv__pipe_stop_read(handle);
if (handle->name) {
free(handle->name);
uv__free(handle->name);
handle->name = NULL;
}
@ -845,7 +845,7 @@ int uv_pipe_accept(uv_pipe_t* server, uv_stream_t* client) {
if (err != 0)
return err;
free(item);
uv__free(item);
} else {
pipe_client = (uv_pipe_t*)client;
@ -1261,9 +1261,9 @@ static int uv_pipe_write_impl(uv_loop_t* loop,
if (handle->pipe.conn.ipc_header_write_req.type != UV_WRITE) {
ipc_header_req = (uv_write_t*)&handle->pipe.conn.ipc_header_write_req;
} else {
ipc_header_req = (uv_write_t*)malloc(sizeof(uv_write_t));
ipc_header_req = (uv_write_t*)uv__malloc(sizeof(uv_write_t));
if (!ipc_header_req) {
uv_fatal_error(ERROR_OUTOFMEMORY, "malloc");
uv_fatal_error(ERROR_OUTOFMEMORY, "uv__malloc");
}
}
@ -1498,9 +1498,9 @@ void uv__pipe_insert_pending_socket(uv_pipe_t* handle,
int tcp_connection) {
uv__ipc_queue_item_t* item;
item = (uv__ipc_queue_item_t*) malloc(sizeof(*item));
item = (uv__ipc_queue_item_t*) uv__malloc(sizeof(*item));
if (item == NULL)
uv_fatal_error(ERROR_OUTOFMEMORY, "malloc");
uv_fatal_error(ERROR_OUTOFMEMORY, "uv__malloc");
memcpy(&item->socket_info_ex, info, sizeof(item->socket_info_ex));
item->tcp_connection = tcp_connection;
@ -1667,7 +1667,7 @@ void uv_process_pipe_write_req(uv_loop_t* loop, uv_pipe_t* handle,
if (req == &handle->pipe.conn.ipc_header_write_req) {
req->type = UV_UNKNOWN_REQ;
} else {
free(req);
uv__free(req);
}
} else {
if (req->cb) {
@ -1788,7 +1788,7 @@ static void eof_timer_init(uv_pipe_t* pipe) {
assert(pipe->pipe.conn.eof_timer == NULL);
assert(pipe->flags & UV_HANDLE_CONNECTION);
pipe->pipe.conn.eof_timer = (uv_timer_t*) malloc(sizeof *pipe->pipe.conn.eof_timer);
pipe->pipe.conn.eof_timer = (uv_timer_t*) uv__malloc(sizeof *pipe->pipe.conn.eof_timer);
r = uv_timer_init(pipe->loop, pipe->pipe.conn.eof_timer);
assert(r == 0); /* timers can't fail */
@ -1863,7 +1863,7 @@ static void eof_timer_destroy(uv_pipe_t* pipe) {
static void eof_timer_close_cb(uv_handle_t* handle) {
assert(handle->type == UV_TIMER);
free(handle);
uv__free(handle);
}
@ -1941,7 +1941,7 @@ static int uv__pipe_getname(const uv_pipe_t* handle, char* buffer, size_t* size)
FileNameInformation);
if (nt_status == STATUS_BUFFER_OVERFLOW) {
name_size = sizeof(*name_info) + tmp_name_info.FileNameLength;
name_info = malloc(name_size);
name_info = uv__malloc(name_size);
if (!name_info) {
*size = 0;
err = UV_ENOMEM;
@ -2020,7 +2020,7 @@ static int uv__pipe_getname(const uv_pipe_t* handle, char* buffer, size_t* size)
goto cleanup;
error:
free(name_info);
uv__free(name_info);
cleanup:
uv__pipe_unpause_read((uv_pipe_t*)handle); /* cast away const warning */

View File

@ -279,7 +279,7 @@ int uv__stdio_create(uv_loop_t* loop,
}
/* Allocate the child stdio buffer */
buffer = (BYTE*) malloc(CHILD_STDIO_SIZE(count));
buffer = (BYTE*) uv__malloc(CHILD_STDIO_SIZE(count));
if (buffer == NULL) {
return ERROR_OUTOFMEMORY;
}
@ -459,7 +459,7 @@ void uv__stdio_destroy(BYTE* buffer) {
}
}
free(buffer);
uv__free(buffer);
}

View File

@ -120,7 +120,7 @@ static int uv_utf8_to_utf16_alloc(const char* s, WCHAR** ws_ptr) {
return GetLastError();
}
ws = (WCHAR*) malloc(ws_len * sizeof(WCHAR));
ws = (WCHAR*) uv__malloc(ws_len * sizeof(WCHAR));
if (ws == NULL) {
return ERROR_OUTOFMEMORY;
}
@ -197,7 +197,7 @@ static WCHAR* search_path_join_test(const WCHAR* dir,
}
/* Allocate buffer for output */
result = result_pos = (WCHAR*)malloc(sizeof(WCHAR) *
result = result_pos = (WCHAR*)uv__malloc(sizeof(WCHAR) *
(cwd_len + 1 + dir_len + 1 + name_len + 1 + ext_len + 1));
/* Copy cwd */
@ -246,7 +246,7 @@ static WCHAR* search_path_join_test(const WCHAR* dir,
return result;
}
free(result);
uv__free(result);
return NULL;
}
@ -555,14 +555,14 @@ int make_program_args(char** args, int verbatim_arguments, WCHAR** dst_ptr) {
dst_len = dst_len * 2 + arg_count * 2;
/* Allocate buffer for the final command line. */
dst = (WCHAR*) malloc(dst_len * sizeof(WCHAR));
dst = (WCHAR*) uv__malloc(dst_len * sizeof(WCHAR));
if (dst == NULL) {
err = ERROR_OUTOFMEMORY;
goto error;
}
/* Allocate temporary working buffer. */
temp_buffer = (WCHAR*) malloc(temp_buffer_len * sizeof(WCHAR));
temp_buffer = (WCHAR*) uv__malloc(temp_buffer_len * sizeof(WCHAR));
if (temp_buffer == NULL) {
err = ERROR_OUTOFMEMORY;
goto error;
@ -596,14 +596,14 @@ int make_program_args(char** args, int verbatim_arguments, WCHAR** dst_ptr) {
*pos++ = *(arg + 1) ? L' ' : L'\0';
}
free(temp_buffer);
uv__free(temp_buffer);
*dst_ptr = dst;
return 0;
error:
free(dst);
free(temp_buffer);
uv__free(dst);
uv__free(temp_buffer);
return err;
}
@ -707,7 +707,7 @@ int make_program_env(char* env_block[], WCHAR** dst_ptr) {
}
/* second pass: copy to UTF-16 environment block */
dst_copy = _malloca(env_len * sizeof(WCHAR));
dst_copy = (WCHAR*)uv__malloc(env_len * sizeof(WCHAR));
if (!dst_copy) {
return ERROR_OUTOFMEMORY;
}
@ -725,7 +725,7 @@ int make_program_env(char* env_block[], WCHAR** dst_ptr) {
(int) (env_len - (ptr - dst_copy)));
if (len <= 0) {
DWORD err = GetLastError();
_freea(dst_copy);
uv__free(dst_copy);
return err;
}
*ptr_copy++ = ptr;
@ -765,9 +765,9 @@ int make_program_env(char* env_block[], WCHAR** dst_ptr) {
}
/* final pass: copy, in sort order, and inserting required variables */
dst = malloc((1+env_len) * sizeof(WCHAR));
dst = uv__malloc((1+env_len) * sizeof(WCHAR));
if (!dst) {
_freea(dst_copy);
uv__free(dst_copy);
return ERROR_OUTOFMEMORY;
}
@ -812,7 +812,7 @@ int make_program_env(char* env_block[], WCHAR** dst_ptr) {
assert(env_len == (ptr - dst));
*ptr = L'\0';
_freea(dst_copy);
uv__free(dst_copy);
*dst_ptr = dst;
return 0;
}
@ -988,7 +988,7 @@ int uv_spawn(uv_loop_t* loop,
goto done;
}
cwd = (WCHAR*) malloc(cwd_len * sizeof(WCHAR));
cwd = (WCHAR*) uv__malloc(cwd_len * sizeof(WCHAR));
if (cwd == NULL) {
err = ERROR_OUTOFMEMORY;
goto done;
@ -1012,7 +1012,7 @@ int uv_spawn(uv_loop_t* loop,
goto done;
}
alloc_path = (WCHAR*) malloc(path_len * sizeof(WCHAR));
alloc_path = (WCHAR*) uv__malloc(path_len * sizeof(WCHAR));
if (alloc_path == NULL) {
err = ERROR_OUTOFMEMORY;
goto done;
@ -1146,12 +1146,12 @@ int uv_spawn(uv_loop_t* loop,
/* Cleanup, whether we succeeded or failed. */
done:
free(application);
free(application_path);
free(arguments);
free(cwd);
free(env);
free(alloc_path);
uv__free(application);
uv__free(application_path);
uv__free(arguments);
uv__free(cwd);
uv__free(env);
uv__free(alloc_path);
if (process->child_stdio_buffer != NULL) {
/* Clean up child stdio handles. */

View File

@ -215,7 +215,7 @@ void uv_tcp_endgame(uv_loop_t* loop, uv_tcp_t* handle) {
}
}
free(handle->tcp.serv.accept_reqs);
uv__free(handle->tcp.serv.accept_reqs);
handle->tcp.serv.accept_reqs = NULL;
}
@ -564,9 +564,9 @@ int uv_tcp_listen(uv_tcp_t* handle, int backlog, uv_connection_cb cb) {
if(!handle->tcp.serv.accept_reqs) {
handle->tcp.serv.accept_reqs = (uv_tcp_accept_t*)
malloc(uv_simultaneous_server_accepts * sizeof(uv_tcp_accept_t));
uv__malloc(uv_simultaneous_server_accepts * sizeof(uv_tcp_accept_t));
if (!handle->tcp.serv.accept_reqs) {
uv_fatal_error(ERROR_OUTOFMEMORY, "malloc");
uv_fatal_error(ERROR_OUTOFMEMORY, "uv__malloc");
}
for (i = 0; i < simultaneous_accepts; i++) {

View File

@ -134,7 +134,7 @@ static UINT __stdcall uv__thread_start(void* arg) {
ctx_p = arg;
ctx = *ctx_p;
free(ctx_p);
uv__free(ctx_p);
uv_once(&uv__current_thread_init_guard, uv__init_current_thread_key);
uv_key_set(&uv__current_thread_key, (void*) ctx.self);
@ -150,7 +150,7 @@ int uv_thread_create(uv_thread_t *tid, void (*entry)(void *arg), void *arg) {
int err;
HANDLE thread;
ctx = malloc(sizeof(*ctx));
ctx = uv__malloc(sizeof(*ctx));
if (ctx == NULL)
return UV_ENOMEM;
@ -167,7 +167,7 @@ int uv_thread_create(uv_thread_t *tid, void (*entry)(void *arg), void *arg) {
NULL);
if (thread == NULL) {
err = errno;
free(ctx);
uv__free(ctx);
} else {
err = 0;
*tid = thread;

View File

@ -123,7 +123,7 @@ int uv_exepath(char* buffer, size_t* size_ptr) {
utf16_buffer_len = (int) *size_ptr;
}
utf16_buffer = (WCHAR*) malloc(sizeof(WCHAR) * utf16_buffer_len);
utf16_buffer = (WCHAR*) uv__malloc(sizeof(WCHAR) * utf16_buffer_len);
if (!utf16_buffer) {
return UV_ENOMEM;
}
@ -152,7 +152,7 @@ int uv_exepath(char* buffer, size_t* size_ptr) {
goto error;
}
free(utf16_buffer);
uv__free(utf16_buffer);
/* utf8_len *does* include the terminating null at this point, but the */
/* returned size shouldn't. */
@ -160,7 +160,7 @@ int uv_exepath(char* buffer, size_t* size_ptr) {
return 0;
error:
free(utf16_buffer);
uv__free(utf16_buffer);
return uv_translate_sys_error(err);
}
@ -379,9 +379,9 @@ int uv_set_process_title(const char* title) {
}
/* Convert to wide-char string */
title_w = (WCHAR*)malloc(sizeof(WCHAR) * length);
title_w = (WCHAR*)uv__malloc(sizeof(WCHAR) * length);
if (!title_w) {
uv_fatal_error(ERROR_OUTOFMEMORY, "malloc");
uv_fatal_error(ERROR_OUTOFMEMORY, "uv__malloc");
}
length = uv_utf8_to_utf16(title, title_w, length);
@ -401,14 +401,14 @@ int uv_set_process_title(const char* title) {
}
EnterCriticalSection(&process_title_lock);
free(process_title);
uv__free(process_title);
process_title = strdup(title);
LeaveCriticalSection(&process_title_lock);
err = 0;
done:
free(title_w);
uv__free(title_w);
return uv_translate_sys_error(err);
}
@ -428,14 +428,14 @@ static int uv__get_process_title() {
}
assert(!process_title);
process_title = (char*)malloc(length);
process_title = (char*)uv__malloc(length);
if (!process_title) {
uv_fatal_error(ERROR_OUTOFMEMORY, "malloc");
uv_fatal_error(ERROR_OUTOFMEMORY, "uv__malloc");
}
/* Do utf16 -> utf8 conversion here */
if (!uv_utf16_to_utf8(title_w, -1, process_title, length)) {
free(process_title);
uv__free(process_title);
return -1;
}
@ -535,7 +535,7 @@ int uv_uptime(double* uptime) {
return uv_translate_sys_error(result);
}
free(malloced_buffer);
uv__free(malloced_buffer);
buffer_size *= 2;
/* Don't let the buffer grow infinitely. */
@ -543,7 +543,7 @@ int uv_uptime(double* uptime) {
goto internalError;
}
buffer = malloced_buffer = (BYTE*) malloc(buffer_size);
buffer = malloced_buffer = (BYTE*) uv__malloc(buffer_size);
if (malloced_buffer == NULL) {
*uptime = 0;
return UV_ENOMEM;
@ -585,7 +585,7 @@ int uv_uptime(double* uptime) {
uint64_t value = *((uint64_t*) address);
*uptime = (double) (object_type->PerfTime.QuadPart - value) /
(double) object_type->PerfFreq.QuadPart;
free(malloced_buffer);
uv__free(malloced_buffer);
return 0;
}
}
@ -595,12 +595,12 @@ int uv_uptime(double* uptime) {
}
/* If we get here, the uptime value was not found. */
free(malloced_buffer);
uv__free(malloced_buffer);
*uptime = 0;
return UV_ENOSYS;
internalError:
free(malloced_buffer);
uv__free(malloced_buffer);
*uptime = 0;
return UV_EIO;
}
@ -633,7 +633,7 @@ int uv_cpu_info(uv_cpu_info_t** cpu_infos_ptr, int* cpu_count_ptr) {
}
sppi_size = cpu_count * sizeof(*sppi);
sppi = malloc(sppi_size);
sppi = uv__malloc(sppi_size);
if (sppi == NULL) {
err = ERROR_OUTOFMEMORY;
goto error;
@ -726,7 +726,7 @@ int uv_cpu_info(uv_cpu_info_t** cpu_infos_ptr, int* cpu_count_ptr) {
assert(len > 0);
/* Allocate 1 extra byte for the null terminator. */
cpu_info->model = malloc(len + 1);
cpu_info->model = uv__malloc(len + 1);
if (cpu_info->model == NULL) {
err = ERROR_OUTOFMEMORY;
goto error;
@ -748,7 +748,7 @@ int uv_cpu_info(uv_cpu_info_t** cpu_infos_ptr, int* cpu_count_ptr) {
cpu_info->model[len] = '\0';
}
free(sppi);
uv__free(sppi);
*cpu_count_ptr = cpu_count;
*cpu_infos_ptr = cpu_infos;
@ -758,10 +758,10 @@ int uv_cpu_info(uv_cpu_info_t** cpu_infos_ptr, int* cpu_count_ptr) {
error:
/* This is safe because the cpu_infos array is zeroed on allocation. */
for (i = 0; i < cpu_count; i++)
free(cpu_infos[i].model);
uv__free(cpu_infos[i].model);
free(cpu_infos);
free(sppi);
uv__free(cpu_infos);
uv__free(sppi);
return uv_translate_sys_error(err);
}
@ -771,10 +771,10 @@ void uv_free_cpu_info(uv_cpu_info_t* cpu_infos, int count) {
int i;
for (i = 0; i < count; i++) {
free(cpu_infos[i].model);
uv__free(cpu_infos[i].model);
}
free(cpu_infos);
uv__free(cpu_infos);
}
@ -893,13 +893,13 @@ int uv_interface_addresses(uv_interface_address_t** addresses_ptr,
if (r == ERROR_SUCCESS)
break;
free(win_address_buf);
uv__free(win_address_buf);
switch (r) {
case ERROR_BUFFER_OVERFLOW:
/* This happens when win_address_buf is NULL or too small to hold */
/* all adapters. */
win_address_buf = malloc(win_address_buf_size);
win_address_buf = uv__malloc(win_address_buf_size);
if (win_address_buf == NULL)
return UV_ENOMEM;
@ -907,7 +907,7 @@ int uv_interface_addresses(uv_interface_address_t** addresses_ptr,
case ERROR_NO_DATA: {
/* No adapters were found. */
uv_address_buf = malloc(1);
uv_address_buf = uv__malloc(1);
if (uv_address_buf == NULL)
return UV_ENOMEM;
@ -967,7 +967,7 @@ int uv_interface_addresses(uv_interface_address_t** addresses_ptr,
NULL,
FALSE);
if (name_size <= 0) {
free(win_address_buf);
uv__free(win_address_buf);
return uv_translate_sys_error(GetLastError());
}
uv_address_buf_size += name_size;
@ -984,9 +984,9 @@ int uv_interface_addresses(uv_interface_address_t** addresses_ptr,
}
/* Allocate space to store interface data plus adapter names. */
uv_address_buf = malloc(uv_address_buf_size);
uv_address_buf = uv__malloc(uv_address_buf_size);
if (uv_address_buf == NULL) {
free(win_address_buf);
uv__free(win_address_buf);
return UV_ENOMEM;
}
@ -1020,8 +1020,8 @@ int uv_interface_addresses(uv_interface_address_t** addresses_ptr,
NULL,
FALSE);
if (name_size <= 0) {
free(win_address_buf);
free(uv_address_buf);
uv__free(win_address_buf);
uv__free(uv_address_buf);
return uv_translate_sys_error(GetLastError());
}
@ -1105,7 +1105,7 @@ int uv_interface_addresses(uv_interface_address_t** addresses_ptr,
name_buf += name_size;
}
free(win_address_buf);
uv__free(win_address_buf);
*addresses_ptr = uv_address_buf;
*count_ptr = count;
@ -1116,7 +1116,7 @@ int uv_interface_addresses(uv_interface_address_t** addresses_ptr,
void uv_free_interface_addresses(uv_interface_address_t* addresses,
int count) {
free(addresses);
uv__free(addresses);
}