aboutsummaryrefslogtreecommitdiffstats
path: root/kernel/auditsc.c
diff options
context:
space:
mode:
Diffstat (limited to 'kernel/auditsc.c')
-rw-r--r--kernel/auditsc.c739
1 files changed, 290 insertions, 449 deletions
diff --git a/kernel/auditsc.c b/kernel/auditsc.c
index 4819f3711973..8cbddff6c283 100644
--- a/kernel/auditsc.c
+++ b/kernel/auditsc.c
@@ -124,43 +124,6 @@ struct audit_aux_data {
/* Number of target pids per aux struct. */
#define AUDIT_AUX_PIDS 16
-struct audit_aux_data_mq_open {
- struct audit_aux_data d;
- int oflag;
- mode_t mode;
- struct mq_attr attr;
-};
-
-struct audit_aux_data_mq_sendrecv {
- struct audit_aux_data d;
- mqd_t mqdes;
- size_t msg_len;
- unsigned int msg_prio;
- struct timespec abs_timeout;
-};
-
-struct audit_aux_data_mq_notify {
- struct audit_aux_data d;
- mqd_t mqdes;
- struct sigevent notification;
-};
-
-struct audit_aux_data_mq_getsetattr {
- struct audit_aux_data d;
- mqd_t mqdes;
- struct mq_attr mqstat;
-};
-
-struct audit_aux_data_ipcctl {
- struct audit_aux_data d;
- struct ipc_perm p;
- unsigned long qbytes;
- uid_t uid;
- gid_t gid;
- mode_t mode;
- u32 osid;
-};
-
struct audit_aux_data_execve {
struct audit_aux_data d;
int argc;
@@ -168,23 +131,6 @@ struct audit_aux_data_execve {
struct mm_struct *mm;
};
-struct audit_aux_data_socketcall {
- struct audit_aux_data d;
- int nargs;
- unsigned long args[0];
-};
-
-struct audit_aux_data_sockaddr {
- struct audit_aux_data d;
- int len;
- char a[0];
-};
-
-struct audit_aux_data_fd_pair {
- struct audit_aux_data d;
- int fd[2];
-};
-
struct audit_aux_data_pids {
struct audit_aux_data d;
pid_t target_pid[AUDIT_AUX_PIDS];
@@ -219,14 +165,14 @@ struct audit_tree_refs {
struct audit_context {
int dummy; /* must be the first element */
int in_syscall; /* 1 if task is in a syscall */
- enum audit_state state;
+ enum audit_state state, current_state;
unsigned int serial; /* serial number for record */
struct timespec ctime; /* time of syscall entry */
int major; /* syscall number */
unsigned long argv[4]; /* syscall arguments */
int return_valid; /* return code is valid */
long return_code;/* syscall return code */
- int auditable; /* 1 if record should be written */
+ u64 prio;
int name_count;
struct audit_names names[AUDIT_NAMES];
char * filterkey; /* key for rule that triggered record */
@@ -234,7 +180,8 @@ struct audit_context {
struct audit_context *previous; /* For nested syscalls */
struct audit_aux_data *aux;
struct audit_aux_data *aux_pids;
-
+ struct sockaddr_storage *sockaddr;
+ size_t sockaddr_len;
/* Save things to print about task_struct */
pid_t pid, ppid;
uid_t uid, euid, suid, fsuid;
@@ -252,6 +199,49 @@ struct audit_context {
struct audit_tree_refs *trees, *first_trees;
int tree_count;
+ int type;
+ union {
+ struct {
+ int nargs;
+ long args[6];
+ } socketcall;
+ struct {
+ uid_t uid;
+ gid_t gid;
+ mode_t mode;
+ u32 osid;
+ int has_perm;
+ uid_t perm_uid;
+ gid_t perm_gid;
+ mode_t perm_mode;
+ unsigned long qbytes;
+ } ipc;
+ struct {
+ mqd_t mqdes;
+ struct mq_attr mqstat;
+ } mq_getsetattr;
+ struct {
+ mqd_t mqdes;
+ int sigev_signo;
+ } mq_notify;
+ struct {
+ mqd_t mqdes;
+ size_t msg_len;
+ unsigned int msg_prio;
+ struct timespec abs_timeout;
+ } mq_sendrecv;
+ struct {
+ int oflag;
+ mode_t mode;
+ struct mq_attr attr;
+ } mq_open;
+ struct {
+ pid_t pid;
+ struct audit_cap_data cap;
+ } capset;
+ };
+ int fds[2];
+
#if AUDIT_DEBUG
int put_count;
int ino_count;
@@ -608,19 +598,12 @@ static int audit_filter_rules(struct task_struct *tsk,
}
}
/* Find ipc objects that match */
- if (ctx) {
- struct audit_aux_data *aux;
- for (aux = ctx->aux; aux;
- aux = aux->next) {
- if (aux->type == AUDIT_IPC) {
- struct audit_aux_data_ipcctl *axi = (void *)aux;
- if (security_audit_rule_match(axi->osid, f->type, f->op, f->lsm_rule, ctx)) {
- ++result;
- break;
- }
- }
- }
- }
+ if (!ctx || ctx->type != AUDIT_IPC)
+ break;
+ if (security_audit_rule_match(ctx->ipc.osid,
+ f->type, f->op,
+ f->lsm_rule, ctx))
+ ++result;
}
break;
case AUDIT_ARG0:
@@ -647,8 +630,16 @@ static int audit_filter_rules(struct task_struct *tsk,
return 0;
}
}
- if (rule->filterkey && ctx)
- ctx->filterkey = kstrdup(rule->filterkey, GFP_ATOMIC);
+
+ if (ctx) {
+ if (rule->prio <= ctx->prio)
+ return 0;
+ if (rule->filterkey) {
+ kfree(ctx->filterkey);
+ ctx->filterkey = kstrdup(rule->filterkey, GFP_ATOMIC);
+ }
+ ctx->prio = rule->prio;
+ }
switch (rule->action) {
case AUDIT_NEVER: *state = AUDIT_DISABLED; break;
case AUDIT_ALWAYS: *state = AUDIT_RECORD_CONTEXT; break;
@@ -661,7 +652,7 @@ static int audit_filter_rules(struct task_struct *tsk,
* completely disabled for this task. Since we only have the task
* structure at this point, we can only check uid and gid.
*/
-static enum audit_state audit_filter_task(struct task_struct *tsk)
+static enum audit_state audit_filter_task(struct task_struct *tsk, char **key)
{
struct audit_entry *e;
enum audit_state state;
@@ -669,6 +660,8 @@ static enum audit_state audit_filter_task(struct task_struct *tsk)
rcu_read_lock();
list_for_each_entry_rcu(e, &audit_filter_list[AUDIT_FILTER_TASK], list) {
if (audit_filter_rules(tsk, &e->rule, NULL, NULL, &state)) {
+ if (state == AUDIT_RECORD_CONTEXT)
+ *key = kstrdup(e->rule.filterkey, GFP_ATOMIC);
rcu_read_unlock();
return state;
}
@@ -702,6 +695,7 @@ static enum audit_state audit_filter_syscall(struct task_struct *tsk,
audit_filter_rules(tsk, &e->rule, ctx, NULL,
&state)) {
rcu_read_unlock();
+ ctx->current_state = state;
return state;
}
}
@@ -715,15 +709,14 @@ static enum audit_state audit_filter_syscall(struct task_struct *tsk,
* buckets applicable to the inode numbers in audit_names[].
* Regarding audit_state, same rules apply as for audit_filter_syscall().
*/
-enum audit_state audit_filter_inodes(struct task_struct *tsk,
- struct audit_context *ctx)
+void audit_filter_inodes(struct task_struct *tsk, struct audit_context *ctx)
{
int i;
struct audit_entry *e;
enum audit_state state;
if (audit_pid && tsk->tgid == audit_pid)
- return AUDIT_DISABLED;
+ return;
rcu_read_lock();
for (i = 0; i < ctx->name_count; i++) {
@@ -740,17 +733,20 @@ enum audit_state audit_filter_inodes(struct task_struct *tsk,
if ((e->rule.mask[word] & bit) == bit &&
audit_filter_rules(tsk, &e->rule, ctx, n, &state)) {
rcu_read_unlock();
- return state;
+ ctx->current_state = state;
+ return;
}
}
}
rcu_read_unlock();
- return AUDIT_BUILD_CONTEXT;
}
-void audit_set_auditable(struct audit_context *ctx)
+static void audit_set_auditable(struct audit_context *ctx)
{
- ctx->auditable = 1;
+ if (!ctx->prio) {
+ ctx->prio = 1;
+ ctx->current_state = AUDIT_RECORD_CONTEXT;
+ }
}
static inline struct audit_context *audit_get_context(struct task_struct *tsk,
@@ -781,23 +777,11 @@ static inline struct audit_context *audit_get_context(struct task_struct *tsk,
else
context->return_code = return_code;
- if (context->in_syscall && !context->dummy && !context->auditable) {
- enum audit_state state;
-
- state = audit_filter_syscall(tsk, context, &audit_filter_list[AUDIT_FILTER_EXIT]);
- if (state == AUDIT_RECORD_CONTEXT) {
- context->auditable = 1;
- goto get_context;
- }
-
- state = audit_filter_inodes(tsk, context);
- if (state == AUDIT_RECORD_CONTEXT)
- context->auditable = 1;
-
+ if (context->in_syscall && !context->dummy) {
+ audit_filter_syscall(tsk, context, &audit_filter_list[AUDIT_FILTER_EXIT]);
+ audit_filter_inodes(tsk, context);
}
-get_context:
-
tsk->audit_context = NULL;
return context;
}
@@ -807,8 +791,7 @@ static inline void audit_free_names(struct audit_context *context)
int i;
#if AUDIT_DEBUG == 2
- if (context->auditable
- ||context->put_count + context->ino_count != context->name_count) {
+ if (context->put_count + context->ino_count != context->name_count) {
printk(KERN_ERR "%s:%d(:%d): major=%d in_syscall=%d"
" name_count=%d put_count=%d"
" ino_count=%d [NOT freeing]\n",
@@ -859,6 +842,7 @@ static inline void audit_zero_context(struct audit_context *context,
{
memset(context, 0, sizeof(*context));
context->state = state;
+ context->prio = state == AUDIT_RECORD_CONTEXT ? ~0ULL : 0;
}
static inline struct audit_context *audit_alloc_context(enum audit_state state)
@@ -884,18 +868,21 @@ int audit_alloc(struct task_struct *tsk)
{
struct audit_context *context;
enum audit_state state;
+ char *key = NULL;
if (likely(!audit_ever_enabled))
return 0; /* Return if not auditing. */
- state = audit_filter_task(tsk);
+ state = audit_filter_task(tsk, &key);
if (likely(state == AUDIT_DISABLED))
return 0;
if (!(context = audit_alloc_context(state))) {
+ kfree(key);
audit_log_lost("out of memory in audit_alloc");
return -ENOMEM;
}
+ context->filterkey = key;
tsk->audit_context = context;
set_tsk_thread_flag(tsk, TIF_SYSCALL_AUDIT);
@@ -921,6 +908,7 @@ static inline void audit_free_context(struct audit_context *context)
free_tree_refs(context);
audit_free_aux(context);
kfree(context->filterkey);
+ kfree(context->sockaddr);
kfree(context);
context = previous;
} while (context);
@@ -1230,6 +1218,97 @@ static void audit_log_fcaps(struct audit_buffer *ab, struct audit_names *name)
audit_log_format(ab, " cap_fe=%d cap_fver=%x", name->fcap.fE, name->fcap_ver);
}
+static void show_special(struct audit_context *context, int *call_panic)
+{
+ struct audit_buffer *ab;
+ int i;
+
+ ab = audit_log_start(context, GFP_KERNEL, context->type);
+ if (!ab)
+ return;
+
+ switch (context->type) {
+ case AUDIT_SOCKETCALL: {
+ int nargs = context->socketcall.nargs;
+ audit_log_format(ab, "nargs=%d", nargs);
+ for (i = 0; i < nargs; i++)
+ audit_log_format(ab, " a%d=%lx", i,
+ context->socketcall.args[i]);
+ break; }
+ case AUDIT_IPC: {
+ u32 osid = context->ipc.osid;
+
+ audit_log_format(ab, "ouid=%u ogid=%u mode=%#o",
+ context->ipc.uid, context->ipc.gid, context->ipc.mode);
+ if (osid) {
+ char *ctx = NULL;
+ u32 len;
+ if (security_secid_to_secctx(osid, &ctx, &len)) {
+ audit_log_format(ab, " osid=%u", osid);
+ *call_panic = 1;
+ } else {
+ audit_log_format(ab, " obj=%s", ctx);
+ security_release_secctx(ctx, len);
+ }
+ }
+ if (context->ipc.has_perm) {
+ audit_log_end(ab);
+ ab = audit_log_start(context, GFP_KERNEL,
+ AUDIT_IPC_SET_PERM);
+ audit_log_format(ab,
+ "qbytes=%lx ouid=%u ogid=%u mode=%#o",
+ context->ipc.qbytes,
+ context->ipc.perm_uid,
+ context->ipc.perm_gid,
+ context->ipc.perm_mode);
+ if (!ab)
+ return;
+ }
+ break; }
+ case AUDIT_MQ_OPEN: {
+ audit_log_format(ab,
+ "oflag=0x%x mode=%#o mq_flags=0x%lx mq_maxmsg=%ld "
+ "mq_msgsize=%ld mq_curmsgs=%ld",
+ context->mq_open.oflag, context->mq_open.mode,
+ context->mq_open.attr.mq_flags,
+ context->mq_open.attr.mq_maxmsg,
+ context->mq_open.attr.mq_msgsize,
+ context->mq_open.attr.mq_curmsgs);
+ break; }
+ case AUDIT_MQ_SENDRECV: {
+ audit_log_format(ab,
+ "mqdes=%d msg_len=%zd msg_prio=%u "
+ "abs_timeout_sec=%ld abs_timeout_nsec=%ld",
+ context->mq_sendrecv.mqdes,
+ context->mq_sendrecv.msg_len,
+ context->mq_sendrecv.msg_prio,
+ context->mq_sendrecv.abs_timeout.tv_sec,
+ context->mq_sendrecv.abs_timeout.tv_nsec);
+ break; }
+ case AUDIT_MQ_NOTIFY: {
+ audit_log_format(ab, "mqdes=%d sigev_signo=%d",
+ context->mq_notify.mqdes,
+ context->mq_notify.sigev_signo);
+ break; }
+ case AUDIT_MQ_GETSETATTR: {
+ struct mq_attr *attr = &context->mq_getsetattr.mqstat;
+ audit_log_format(ab,
+ "mqdes=%d mq_flags=0x%lx mq_maxmsg=%ld mq_msgsize=%ld "
+ "mq_curmsgs=%ld ",
+ context->mq_getsetattr.mqdes,
+ attr->mq_flags, attr->mq_maxmsg,
+ attr->mq_msgsize, attr->mq_curmsgs);
+ break; }
+ case AUDIT_CAPSET: {
+ audit_log_format(ab, "pid=%d", context->capset.pid);
+ audit_log_cap(ab, "cap_pi", &context->capset.cap.inheritable);
+ audit_log_cap(ab, "cap_pp", &context->capset.cap.permitted);
+ audit_log_cap(ab, "cap_pe", &context->capset.cap.effective);
+ break; }
+ }
+ audit_log_end(ab);
+}
+
static void audit_log_exit(struct audit_context *context, struct task_struct *tsk)
{
const struct cred *cred;
@@ -1307,94 +1386,12 @@ static void audit_log_exit(struct audit_context *context, struct task_struct *ts
continue; /* audit_panic has been called */
switch (aux->type) {
- case AUDIT_MQ_OPEN: {
- struct audit_aux_data_mq_open *axi = (void *)aux;
- audit_log_format(ab,
- "oflag=0x%x mode=%#o mq_flags=0x%lx mq_maxmsg=%ld "
- "mq_msgsize=%ld mq_curmsgs=%ld",
- axi->oflag, axi->mode, axi->attr.mq_flags,
- axi->attr.mq_maxmsg, axi->attr.mq_msgsize,
- axi->attr.mq_curmsgs);
- break; }
-
- case AUDIT_MQ_SENDRECV: {
- struct audit_aux_data_mq_sendrecv *axi = (void *)aux;
- audit_log_format(ab,
- "mqdes=%d msg_len=%zd msg_prio=%u "
- "abs_timeout_sec=%ld abs_timeout_nsec=%ld",
- axi->mqdes, axi->msg_len, axi->msg_prio,
- axi->abs_timeout.tv_sec, axi->abs_timeout.tv_nsec);
- break; }
-
- case AUDIT_MQ_NOTIFY: {
- struct audit_aux_data_mq_notify *axi = (void *)aux;
- audit_log_format(ab,
- "mqdes=%d sigev_signo=%d",
- axi->mqdes,
- axi->notification.sigev_signo);
- break; }
-
- case AUDIT_MQ_GETSETATTR: {
- struct audit_aux_data_mq_getsetattr *axi = (void *)aux;
- audit_log_format(ab,
- "mqdes=%d mq_flags=0x%lx mq_maxmsg=%ld mq_msgsize=%ld "
- "mq_curmsgs=%ld ",
- axi->mqdes,
- axi->mqstat.mq_flags, axi->mqstat.mq_maxmsg,
- axi->mqstat.mq_msgsize, axi->mqstat.mq_curmsgs);
- break; }
-
- case AUDIT_IPC: {
- struct audit_aux_data_ipcctl *axi = (void *)aux;
- audit_log_format(ab,
- "ouid=%u ogid=%u mode=%#o",
- axi->uid, axi->gid, axi->mode);
- if (axi->osid != 0) {
- char *ctx = NULL;
- u32 len;
- if (security_secid_to_secctx(
- axi->osid, &ctx, &len)) {
- audit_log_format(ab, " osid=%u",
- axi->osid);
- call_panic = 1;
- } else {
- audit_log_format(ab, " obj=%s", ctx);
- security_release_secctx(ctx, len);
- }
- }
- break; }
-
- case AUDIT_IPC_SET_PERM: {
- struct audit_aux_data_ipcctl *axi = (void *)aux;
- audit_log_format(ab,
- "qbytes=%lx ouid=%u ogid=%u mode=%#o",
- axi->qbytes, axi->uid, axi->gid, axi->mode);
- break; }
case AUDIT_EXECVE: {
struct audit_aux_data_execve *axi = (void *)aux;
audit_log_execve_info(context, &ab, axi);
break; }
- case AUDIT_SOCKETCALL: {
- struct audit_aux_data_socketcall *axs = (void *)aux;
- audit_log_format(ab, "nargs=%d", axs->nargs);
- for (i=0; i<axs->nargs; i++)
- audit_log_format(ab, " a%d=%lx", i, axs->args[i]);
- break; }
-
- case AUDIT_SOCKADDR: {
- struct audit_aux_data_sockaddr *axs = (void *)aux;
-
- audit_log_format(ab, "saddr=");
- audit_log_n_hex(ab, axs->a, axs->len);
- break; }
-
- case AUDIT_FD_PAIR: {
- struct audit_aux_data_fd_pair *axs = (void *)aux;
- audit_log_format(ab, "fd0=%d fd1=%d", axs->fd[0], axs->fd[1]);
- break; }
-
case AUDIT_BPRM_FCAPS: {
struct audit_aux_data_bprm_fcaps *axs = (void *)aux;
audit_log_format(ab, "fver=%x", axs->fcap_ver);
@@ -1409,18 +1406,32 @@ static void audit_log_exit(struct audit_context *context, struct task_struct *ts
audit_log_cap(ab, "new_pe", &axs->new_pcap.effective);
break; }
- case AUDIT_CAPSET: {
- struct audit_aux_data_capset *axs = (void *)aux;
- audit_log_format(ab, "pid=%d", axs->pid);
- audit_log_cap(ab, "cap_pi", &axs->cap.inheritable);
- audit_log_cap(ab, "cap_pp", &axs->cap.permitted);
- audit_log_cap(ab, "cap_pe", &axs->cap.effective);
- break; }
-
}
audit_log_end(ab);
}
+ if (context->type)
+ show_special(context, &call_panic);
+
+ if (context->fds[0] >= 0) {
+ ab = audit_log_start(context, GFP_KERNEL, AUDIT_FD_PAIR);
+ if (ab) {
+ audit_log_format(ab, "fd0=%d fd1=%d",
+ context->fds[0], context->fds[1]);
+ audit_log_end(ab);
+ }
+ }
+
+ if (context->sockaddr_len) {
+ ab = audit_log_start(context, GFP_KERNEL, AUDIT_SOCKADDR);
+ if (ab) {
+ audit_log_format(ab, "saddr=");
+ audit_log_n_hex(ab, (void *)context->sockaddr,
+ context->sockaddr_len);
+ audit_log_end(ab);
+ }
+ }
+
for (aux = context->aux_pids; aux; aux = aux->next) {
struct audit_aux_data_pids *axs = (void *)aux;
@@ -1536,7 +1547,7 @@ void audit_free(struct task_struct *tsk)
* We use GFP_ATOMIC here because we might be doing this
* in the context of the idle thread */
/* that can happen only if we are called from do_exit() */
- if (context->in_syscall && context->auditable)
+ if (context->in_syscall && context->current_state == AUDIT_RECORD_CONTEXT)
audit_log_exit(context, tsk);
audit_free_context(context);
@@ -1620,15 +1631,17 @@ void audit_syscall_entry(int arch, int major,
state = context->state;
context->dummy = !audit_n_rules;
- if (!context->dummy && (state == AUDIT_SETUP_CONTEXT || state == AUDIT_BUILD_CONTEXT))
+ if (!context->dummy && state == AUDIT_BUILD_CONTEXT) {
+ context->prio = 0;
state = audit_filter_syscall(tsk, context, &audit_filter_list[AUDIT_FILTER_ENTRY]);
+ }
if (likely(state == AUDIT_DISABLED))
return;
context->serial = 0;
context->ctime = CURRENT_TIME;
context->in_syscall = 1;
- context->auditable = !!(state == AUDIT_RECORD_CONTEXT);
+ context->current_state = state;
context->ppid = 0;
}
@@ -1636,17 +1649,20 @@ void audit_finish_fork(struct task_struct *child)
{
struct audit_context *ctx = current->audit_context;
struct audit_context *p = child->audit_context;
- if (!p || !ctx || !ctx->auditable)
+ if (!p || !ctx)
+ return;
+ if (!ctx->in_syscall || ctx->current_state != AUDIT_RECORD_CONTEXT)
return;
p->arch = ctx->arch;
p->major = ctx->major;
memcpy(p->argv, ctx->argv, sizeof(ctx->argv));
p->ctime = ctx->ctime;
p->dummy = ctx->dummy;
- p->auditable = ctx->auditable;
p->in_syscall = ctx->in_syscall;
p->filterkey = kstrdup(ctx->filterkey, GFP_KERNEL);
p->ppid = current->pid;
+ p->prio = ctx->prio;
+ p->current_state = ctx->current_state;
}
/**
@@ -1670,11 +1686,11 @@ void audit_syscall_exit(int valid, long return_code)
if (likely(!context))
return;
- if (context->in_syscall && context->auditable)
+ if (context->in_syscall && context->current_state == AUDIT_RECORD_CONTEXT)
audit_log_exit(context, tsk);
context->in_syscall = 0;
- context->auditable = 0;
+ context->prio = context->state == AUDIT_RECORD_CONTEXT ? ~0ULL : 0;
if (context->previous) {
struct audit_context *new_context = context->previous;
@@ -1689,8 +1705,13 @@ void audit_syscall_exit(int valid, long return_code)
context->aux_pids = NULL;
context->target_pid = 0;
context->target_sid = 0;
- kfree(context->filterkey);
- context->filterkey = NULL;
+ context->sockaddr_len = 0;
+ context->type = 0;
+ context->fds[0] = -1;
+ if (context->state != AUDIT_RECORD_CONTEXT) {
+ kfree(context->filterkey);
+ context->filterkey = NULL;
+ }
tsk->audit_context = context;
}
}
@@ -2081,7 +2102,10 @@ int auditsc_get_stamp(struct audit_context *ctx,
t->tv_sec = ctx->ctime.tv_sec;
t->tv_nsec = ctx->ctime.tv_nsec;
*serial = ctx->serial;
- ctx->auditable = 1;
+ if (!ctx->prio) {
+ ctx->prio = 1;
+ ctx->current_state = AUDIT_RECORD_CONTEXT;
+ }
return 1;
}
@@ -2127,132 +2151,46 @@ int audit_set_loginuid(struct task_struct *task, uid_t loginuid)
* @mode: mode bits
* @u_attr: queue attributes
*
- * Returns 0 for success or NULL context or < 0 on error.
*/
-int __audit_mq_open(int oflag, mode_t mode, struct mq_attr __user *u_attr)
+void __audit_mq_open(int oflag, mode_t mode, struct mq_attr *attr)
{
- struct audit_aux_data_mq_open *ax;
struct audit_context *context = current->audit_context;
- if (!audit_enabled)
- return 0;
-
- if (likely(!context))
- return 0;
-
- ax = kmalloc(sizeof(*ax), GFP_ATOMIC);
- if (!ax)
- return -ENOMEM;
-
- if (u_attr != NULL) {
- if (copy_from_user(&ax->attr, u_attr, sizeof(ax->attr))) {
- kfree(ax);
- return -EFAULT;
- }
- } else
- memset(&ax->attr, 0, sizeof(ax->attr));
+ if (attr)
+ memcpy(&context->mq_open.attr, attr, sizeof(struct mq_attr));
+ else
+ memset(&context->mq_open.attr, 0, sizeof(struct mq_attr));
- ax->oflag = oflag;
- ax->mode = mode;
+ context->mq_open.oflag = oflag;
+ context->mq_open.mode = mode;
- ax->d.type = AUDIT_MQ_OPEN;
- ax->d.next = context->aux;
- context->aux = (void *)ax;
- return 0;
+ context->type = AUDIT_MQ_OPEN;
}
/**
- * __audit_mq_timedsend - record audit data for a POSIX MQ timed send
+ * __audit_mq_sendrecv - record audit data for a POSIX MQ timed send/receive
* @mqdes: MQ descriptor
* @msg_len: Message length
* @msg_prio: Message priority
- * @u_abs_timeout: Message timeout in absolute time
+ * @abs_timeout: Message timeout in absolute time
*
- * Returns 0 for success or NULL context or < 0 on error.
*/
-int __audit_mq_timedsend(mqd_t mqdes, size_t msg_len, unsigned int msg_prio,
- const struct timespec __user *u_abs_timeout)
+void __audit_mq_sendrecv(mqd_t mqdes, size_t msg_len, unsigned int msg_prio,
+ const struct timespec *abs_timeout)
{
- struct audit_aux_data_mq_sendrecv *ax;
struct audit_context *context = current->audit_context;
+ struct timespec *p = &context->mq_sendrecv.abs_timeout;
- if (!audit_enabled)
- return 0;
-
- if (likely(!context))
- return 0;
-
- ax = kmalloc(sizeof(*ax), GFP_ATOMIC);
- if (!ax)
- return -ENOMEM;
-
- if (u_abs_timeout != NULL) {
- if (copy_from_user(&ax->abs_timeout, u_abs_timeout, sizeof(ax->abs_timeout))) {
- kfree(ax);
- return -EFAULT;
- }
- } else
- memset(&ax->abs_timeout, 0, sizeof(ax->abs_timeout));
-
- ax->mqdes = mqdes;
- ax->msg_len = msg_len;
- ax->msg_prio = msg_prio;
-
- ax->d.type = AUDIT_MQ_SENDRECV;
- ax->d.next = context->aux;
- context->aux = (void *)ax;
- return 0;
-}
-
-/**
- * __audit_mq_timedreceive - record audit data for a POSIX MQ timed receive
- * @mqdes: MQ descriptor
- * @msg_len: Message length
- * @u_msg_prio: Message priority
- * @u_abs_timeout: Message timeout in absolute time
- *
- * Returns 0 for success or NULL context or < 0 on error.
- */
-int __audit_mq_timedreceive(mqd_t mqdes, size_t msg_len,
- unsigned int __user *u_msg_prio,
- const struct timespec __user *u_abs_timeout)
-{
- struct audit_aux_data_mq_sendrecv *ax;
- struct audit_context *context = current->audit_context;
-
- if (!audit_enabled)
- return 0;
-
- if (likely(!context))
- return 0;
-
- ax = kmalloc(sizeof(*ax), GFP_ATOMIC);
- if (!ax)
- return -ENOMEM;
-
- if (u_msg_prio != NULL) {
- if (get_user(ax->msg_prio, u_msg_prio)) {
- kfree(ax);
- return -EFAULT;
- }
- } else
- ax->msg_prio = 0;
-
- if (u_abs_timeout != NULL) {
- if (copy_from_user(&ax->abs_timeout, u_abs_timeout, sizeof(ax->abs_timeout))) {
- kfree(ax);
- return -EFAULT;
- }
- } else
- memset(&ax->abs_timeout, 0, sizeof(ax->abs_timeout));
+ if (abs_timeout)
+ memcpy(p, abs_timeout, sizeof(struct timespec));
+ else
+ memset(p, 0, sizeof(struct timespec));
- ax->mqdes = mqdes;
- ax->msg_len = msg_len;
+ context->mq_sendrecv.mqdes = mqdes;
+ context->mq_sendrecv.msg_len = msg_len;
+ context->mq_sendrecv.msg_prio = msg_prio;
- ax->d.type = AUDIT_MQ_SENDRECV;
- ax->d.next = context->aux;
- context->aux = (void *)ax;
- return 0;
+ context->type = AUDIT_MQ_SENDRECV;
}
/**
@@ -2260,38 +2198,19 @@ int __audit_mq_timedreceive(mqd_t mqdes, size_t msg_len,
* @mqdes: MQ descriptor
* @u_notification: Notification event
*
- * Returns 0 for success or NULL context or < 0 on error.
*/
-int __audit_mq_notify(mqd_t mqdes, const struct sigevent __user *u_notification)
+void __audit_mq_notify(mqd_t mqdes, const struct sigevent *notification)
{
- struct audit_aux_data_mq_notify *ax;
struct audit_context *context = current->audit_context;
- if (!audit_enabled)
- return 0;
-
- if (likely(!context))
- return 0;
-
- ax = kmalloc(sizeof(*ax), GFP_ATOMIC);
- if (!ax)
- return -ENOMEM;
-
- if (u_notification != NULL) {
- if (copy_from_user(&ax->notification, u_notification, sizeof(ax->notification))) {
- kfree(ax);
- return -EFAULT;
- }
- } else
- memset(&ax->notification, 0, sizeof(ax->notification));
-
- ax->mqdes = mqdes;
+ if (notification)
+ context->mq_notify.sigev_signo = notification->sigev_signo;
+ else
+ context->mq_notify.sigev_signo = 0;
- ax->d.type = AUDIT_MQ_NOTIFY;
- ax->d.next = context->aux;
- context->aux = (void *)ax;
- return 0;
+ context->mq_notify.mqdes = mqdes;
+ context->type = AUDIT_MQ_NOTIFY;
}
/**
@@ -2299,55 +2218,29 @@ int __audit_mq_notify(mqd_t mqdes, const struct sigevent __user *u_notification)
* @mqdes: MQ descriptor
* @mqstat: MQ flags
*
- * Returns 0 for success or NULL context or < 0 on error.
*/
-int __audit_mq_getsetattr(mqd_t mqdes, struct mq_attr *mqstat)
+void __audit_mq_getsetattr(mqd_t mqdes, struct mq_attr *mqstat)
{
- struct audit_aux_data_mq_getsetattr *ax;
struct audit_context *context = current->audit_context;
-
- if (!audit_enabled)
- return 0;
-
- if (likely(!context))
- return 0;
-
- ax = kmalloc(sizeof(*ax), GFP_ATOMIC);
- if (!ax)
- return -ENOMEM;
-
- ax->mqdes = mqdes;
- ax->mqstat = *mqstat;
-
- ax->d.type = AUDIT_MQ_GETSETATTR;
- ax->d.next = context->aux;
- context->aux = (void *)ax;
- return 0;
+ context->mq_getsetattr.mqdes = mqdes;
+ context->mq_getsetattr.mqstat = *mqstat;
+ context->type = AUDIT_MQ_GETSETATTR;
}
/**
* audit_ipc_obj - record audit data for ipc object
* @ipcp: ipc permissions
*
- * Returns 0 for success or NULL context or < 0 on error.
*/
-int __audit_ipc_obj(struct kern_ipc_perm *ipcp)
+void __audit_ipc_obj(struct kern_ipc_perm *ipcp)
{
- struct audit_aux_data_ipcctl *ax;
struct audit_context *context = current->audit_context;
-
- ax = kmalloc(sizeof(*ax), GFP_ATOMIC);
- if (!ax)
- return -ENOMEM;
-
- ax->uid = ipcp->uid;
- ax->gid = ipcp->gid;
- ax->mode = ipcp->mode;
- security_ipc_getsecid(ipcp, &ax->osid);
- ax->d.type = AUDIT_IPC;
- ax->d.next = context->aux;
- context->aux = (void *)ax;
- return 0;
+ context->ipc.uid = ipcp->uid;
+ context->ipc.gid = ipcp->gid;
+ context->ipc.mode = ipcp->mode;
+ context->ipc.has_perm = 0;
+ security_ipc_getsecid(ipcp, &context->ipc.osid);
+ context->type = AUDIT_IPC;
}
/**
@@ -2357,26 +2250,17 @@ int __audit_ipc_obj(struct kern_ipc_perm *ipcp)
* @gid: msgq group id
* @mode: msgq mode (permissions)
*
- * Returns 0 for success or NULL context or < 0 on error.
+ * Called only after audit_ipc_obj().
*/
-int __audit_ipc_set_perm(unsigned long qbytes, uid_t uid, gid_t gid, mode_t mode)
+void __audit_ipc_set_perm(unsigned long qbytes, uid_t uid, gid_t gid, mode_t mode)
{
- struct audit_aux_data_ipcctl *ax;
struct audit_context *context = current->audit_context;
- ax = kmalloc(sizeof(*ax), GFP_ATOMIC);
- if (!ax)
- return -ENOMEM;
-
- ax->qbytes = qbytes;
- ax->uid = uid;
- ax->gid = gid;
- ax->mode = mode;
-
- ax->d.type = AUDIT_IPC_SET_PERM;
- ax->d.next = context->aux;
- context->aux = (void *)ax;
- return 0;
+ context->ipc.qbytes = qbytes;
+ context->ipc.perm_uid = uid;
+ context->ipc.perm_gid = gid;
+ context->ipc.perm_mode = mode;
+ context->ipc.has_perm = 1;
}
int audit_bprm(struct linux_binprm *bprm)
@@ -2406,27 +2290,17 @@ int audit_bprm(struct linux_binprm *bprm)
* @nargs: number of args
* @args: args array
*
- * Returns 0 for success or NULL context or < 0 on error.
*/
-int audit_socketcall(int nargs, unsigned long *args)
+void audit_socketcall(int nargs, unsigned long *args)
{
- struct audit_aux_data_socketcall *ax;
struct audit_context *context = current->audit_context;
if (likely(!context || context->dummy))
- return 0;
-
- ax = kmalloc(sizeof(*ax) + nargs * sizeof(unsigned long), GFP_KERNEL);
- if (!ax)
- return -ENOMEM;
-
- ax->nargs = nargs;
- memcpy(ax->args, args, nargs * sizeof(unsigned long));
+ return;
- ax->d.type = AUDIT_SOCKETCALL;
- ax->d.next = context->aux;
- context->aux = (void *)ax;
- return 0;
+ context->type = AUDIT_SOCKETCALL;
+ context->socketcall.nargs = nargs;
+ memcpy(context->socketcall.args, args, nargs * sizeof(unsigned long));
}
/**
@@ -2434,29 +2308,12 @@ int audit_socketcall(int nargs, unsigned long *args)
* @fd1: the first file descriptor
* @fd2: the second file descriptor
*
- * Returns 0 for success or NULL context or < 0 on error.
*/
-int __audit_fd_pair(int fd1, int fd2)
+void __audit_fd_pair(int fd1, int fd2)
{
struct audit_context *context = current->audit_context;
- struct audit_aux_data_fd_pair *ax;
-
- if (likely(!context)) {
- return 0;
- }
-
- ax = kmalloc(sizeof(*ax), GFP_KERNEL);
- if (!ax) {
- return -ENOMEM;
- }
-
- ax->fd[0] = fd1;
- ax->fd[1] = fd2;
-
- ax->d.type = AUDIT_FD_PAIR;
- ax->d.next = context->aux;
- context->aux = (void *)ax;
- return 0;
+ context->fds[0] = fd1;
+ context->fds[1] = fd2;
}
/**
@@ -2468,22 +2325,20 @@ int __audit_fd_pair(int fd1, int fd2)
*/
int audit_sockaddr(int len, void *a)
{
- struct audit_aux_data_sockaddr *ax;
struct audit_context *context = current->audit_context;
if (likely(!context || context->dummy))
return 0;
- ax = kmalloc(sizeof(*ax) + len, GFP_KERNEL);
- if (!ax)
- return -ENOMEM;
-
- ax->len = len;
- memcpy(ax->a, a, len);
+ if (!context->sockaddr) {
+ void *p = kmalloc(sizeof(struct sockaddr_storage), GFP_KERNEL);
+ if (!p)
+ return -ENOMEM;
+ context->sockaddr = p;
+ }
- ax->d.type = AUDIT_SOCKADDR;
- ax->d.next = context->aux;
- context->aux = (void *)ax;
+ context->sockaddr_len = len;
+ memcpy(context->sockaddr, a, len);
return 0;
}
@@ -2617,29 +2472,15 @@ int __audit_log_bprm_fcaps(struct linux_binprm *bprm,
* Record the aguments userspace sent to sys_capset for later printing by the
* audit system if applicable
*/
-int __audit_log_capset(pid_t pid,
+void __audit_log_capset(pid_t pid,
const struct cred *new, const struct cred *old)
{
- struct audit_aux_data_capset *ax;
struct audit_context *context = current->audit_context;
-
- if (likely(!audit_enabled || !context || context->dummy))
- return 0;
-
- ax = kmalloc(sizeof(*ax), GFP_KERNEL);
- if (!ax)
- return -ENOMEM;
-
- ax->d.type = AUDIT_CAPSET;
- ax->d.next = context->aux;
- context->aux = (void *)ax;
-
- ax->pid = pid;
- ax->cap.effective = new->cap_effective;
- ax->cap.inheritable = new->cap_effective;
- ax->cap.permitted = new->cap_permitted;
-
- return 0;
+ context->capset.pid = pid;
+ context->capset.cap.effective = new->cap_effective;
+ context->capset.cap.inheritable = new->cap_effective;
+ context->capset.cap.permitted = new->cap_permitted;
+ context->type = AUDIT_CAPSET;
}
/**