You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mynewt.apache.org by cc...@apache.org on 2017/05/13 01:40:47 UTC

[1/3] incubator-mynewt-core git commit: MYNEWT-745 Sim - deadlock involving system calls

Repository: incubator-mynewt-core
Updated Branches:
  refs/heads/master b26a78d8f -> dd4baa28e


MYNEWT-745 Sim - deadlock involving system calls

This commit splits sim into two separate implementations:
    * "signals"
    * "no-signals"

The user chooses which implementation to use via the
MCU_NATIVE_USE_SIGNALS syscfg setting (defined in hw/mcu/native).  The
two implementations are described below:

signals:
    More correctness; less stability.  The OS tick timer will
    cause a high-priority task to preempt a low-priority task.
    This causes stability issues because a task can be preempted
    while it is in the middle of a system call, potentially
    causing deadlock or memory corruption.

no-signals:
    Less correctness; more stability.  The OS tick timer only
    runs while the idle task is active.  Therefore, a sleeping
    high-priority task will not preempt a low-priority task due
    to a timing event (e.g., delay or callout expired).
    However, this version of sim does not suffer from the
    stability issues that affect the "signals" implementation.


Project: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/commit/cc1acfe8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/tree/cc1acfe8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/diff/cc1acfe8

Branch: refs/heads/master
Commit: cc1acfe8ddc8d88c1637b7bd7374c35fc0ace90f
Parents: cb23f34
Author: Christopher Collins <cc...@apache.org>
Authored: Tue May 9 17:40:29 2017 -0700
Committer: Christopher Collins <cc...@apache.org>
Committed: Tue May 9 17:54:13 2017 -0700

----------------------------------------------------------------------
 hw/mcu/native/syscfg.yml                   |  21 ++
 kernel/os/src/arch/sim/os_arch_sim.c       | 461 ------------------------
 kernel/os/src/arch/sim/os_arch_sim_gen.c   | 248 +++++++++++++
 kernel/os/src/arch/sim/os_arch_sim_nosig.c | 241 +++++++++++++
 kernel/os/src/arch/sim/os_arch_sim_priv.h  |  29 ++
 kernel/os/src/arch/sim/os_arch_sim_sig.c   | 266 ++++++++++++++
 6 files changed, 805 insertions(+), 461 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/cc1acfe8/hw/mcu/native/syscfg.yml
----------------------------------------------------------------------
diff --git a/hw/mcu/native/syscfg.yml b/hw/mcu/native/syscfg.yml
index d0c40be..230450f 100644
--- a/hw/mcu/native/syscfg.yml
+++ b/hw/mcu/native/syscfg.yml
@@ -24,3 +24,24 @@ syscfg.defs:
             Specifies the required alignment for internal flash writes.
             Used internally by the newt tool.
         value: 1
+
+    MCU_NATIVE_USE_SIGNALS:
+        description: >
+            Whether to use POSIX signals to implement context switches.  Valid
+            values are as follows:
+                1: More correctness; less stability.  The OS tick timer will
+                   cause a high-priority task to preempt a low-priority task.
+                   This causes stability issues because a task can be preempted
+                   while it is in the middle of a system call, potentially
+                   causing deadlock or memory corruption.
+
+                0: Less correctness; more stability.  The OS tick timer only
+                   runs while the idle task is active.  Therefore, a sleeping
+                   high-priority task will not preempt a low-priority task due
+                   to a timing event (e.g., delay or callout expired).
+                   However, this version of sim does not suffer from the
+                   stability issues that affect the "signals" implementation.
+
+            Unit tests should use 1.  Long-running sim processes should use 0.
+
+        value: 1

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/cc1acfe8/kernel/os/src/arch/sim/os_arch_sim.c
----------------------------------------------------------------------
diff --git a/kernel/os/src/arch/sim/os_arch_sim.c b/kernel/os/src/arch/sim/os_arch_sim.c
deleted file mode 100644
index bc9608c..0000000
--- a/kernel/os/src/arch/sim/os_arch_sim.c
+++ /dev/null
@@ -1,461 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *  http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-#include "os/os.h"
-#include "os_priv.h"
-
-#include <hal/hal_bsp.h>
-
-#ifdef __APPLE__
-#define _XOPEN_SOURCE
-#endif
-
-#include <string.h>
-#include <stdio.h>
-#include <unistd.h>
-#include <setjmp.h>
-#include <signal.h>
-#include <sys/time.h>
-#include <assert.h>
-
-struct stack_frame {
-    int sf_mainsp;              /* stack on which main() is executing */
-    sigjmp_buf sf_jb;
-    struct os_task *sf_task;
-};
-
-/*
- * Assert that 'sf_mainsp' and 'sf_jb' are at the specific offsets where
- * os_arch_frame_init() expects them to be.
- */
-CTASSERT(offsetof(struct stack_frame, sf_mainsp) == 0);
-CTASSERT(offsetof(struct stack_frame, sf_jb) == 4);
-
-extern void os_arch_frame_init(struct stack_frame *sf);
-
-#define sim_setjmp(__jb) sigsetjmp(__jb, 0)
-#define sim_longjmp(__jb, __ret) siglongjmp(__jb, __ret)
-
-#define OS_USEC_PER_TICK    (1000000 / OS_TICKS_PER_SEC)
-
-static pid_t mypid;
-static sigset_t allsigs, nosigs;
-static void timer_handler(int sig);
-
-static bool suspended;      /* process is blocked in sigsuspend() */
-static sigset_t suspsigs;   /* signals delivered in sigsuspend() */
-
-/*
- * Called from 'os_arch_frame_init()' when setjmp returns indirectly via
- * longjmp. The return value of setjmp is passed to this function as 'rc'.
- */
-void
-os_arch_task_start(struct stack_frame *sf, int rc)
-{
-    struct os_task *task;
-
-    /*
-     * Interrupts are disabled when a task starts executing. This happens in
-     * two different ways:
-     * - via os_arch_os_start() for the first task.
-     * - via os_sched() for all other tasks.
-     *
-     * Enable interrupts before starting the task.
-     */
-    OS_EXIT_CRITICAL(0);
-
-    task = sf->sf_task;
-    task->t_func(task->t_arg);
-
-    /* This should never return */
-    assert(0);
-}
-
-os_stack_t *
-os_arch_task_stack_init(struct os_task *t, os_stack_t *stack_top, int size)
-{
-    struct stack_frame *sf;
-
-    sf = (struct stack_frame *) ((uint8_t *) stack_top - sizeof(*sf));
-    sf->sf_task = t;
-
-    os_arch_frame_init(sf);
-
-    return ((os_stack_t *)sf);
-}
-
-void
-os_arch_ctx_sw(struct os_task *next_t)
-{
-    /*
-     * gdb will stop execution of the program on most signals (e.g. SIGUSR1)
-     * whereas it passes SIGURG to the process without any special settings.
-     */
-    kill(mypid, SIGURG);
-}
-
-static void
-ctxsw_handler(int sig)
-{
-    struct os_task *t, *next_t;
-    struct stack_frame *sf;
-    int rc;
-
-    OS_ASSERT_CRITICAL();
-
-    /*
-     * Just record that this handler was called when the process was blocked.
-     * The handler will be called after sigsuspend() returns in the correct
-     * order.
-     */
-    if (suspended) {
-        sigaddset(&suspsigs, sig);
-        return;
-    }
-
-    t = os_sched_get_current_task();
-    next_t = os_sched_next_task();
-    if (t == next_t) {
-        /*
-         * Context switch not needed - just return.
-         */
-        return;
-    }
-
-    if (t) {
-        sf = (struct stack_frame *) t->t_stackptr;
-
-        rc = sim_setjmp(sf->sf_jb);
-        if (rc != 0) {
-            OS_ASSERT_CRITICAL();
-            return;
-        }
-    }
-
-    os_sched_ctx_sw_hook(next_t);
-
-    os_sched_set_current_task(next_t);
-
-    sf = (struct stack_frame *) next_t->t_stackptr;
-    sim_longjmp(sf->sf_jb, 1);
-}
-
-/*
- * Disable signals and enter a critical section.
- *
- * Returns 1 if signals were already blocked and 0 otherwise.
- */
-os_sr_t
-os_arch_save_sr(void)
-{
-    int error;
-    sigset_t omask;
-
-    error = sigprocmask(SIG_BLOCK, &allsigs, &omask);
-    assert(error == 0);
-
-    /*
-     * If any one of the signals in 'allsigs' is present in 'omask' then
-     * we are already inside a critical section.
-     */
-    return (sigismember(&omask, SIGALRM));
-}
-
-void
-os_arch_restore_sr(os_sr_t osr)
-{
-    int error;
-
-    OS_ASSERT_CRITICAL();
-    assert(osr == 0 || osr == 1);
-
-    if (osr == 1) {
-        /* Exiting a nested critical section */
-        return;
-    }
-
-    error = sigprocmask(SIG_UNBLOCK, &allsigs, NULL);
-    assert(error == 0);
-}
-
-int
-os_arch_in_critical(void)
-{
-    int error;
-    sigset_t omask;
-
-    error = sigprocmask(SIG_SETMASK, NULL, &omask);
-    assert(error == 0);
-
-    /*
-     * If any one of the signals in 'allsigs' is present in 'omask' then
-     * we are already inside a critical section.
-     */
-    return (sigismember(&omask, SIGALRM));
-}
-
-static struct {
-    int num;
-    void (*handler)(int sig);
-} signals[] = {
-    { SIGALRM, timer_handler },
-    { SIGURG, ctxsw_handler },
-};
-
-#define NUMSIGS     (sizeof(signals)/sizeof(signals[0]))
-
-void
-os_tick_idle(os_time_t ticks)
-{
-    int i, rc, sig;
-    struct itimerval it;
-    void (*handler)(int sig);
-
-    OS_ASSERT_CRITICAL();
-
-    if (ticks > 0) {
-        /*
-         * Enter tickless regime and set the timer to fire after 'ticks'
-         * worth of time has elapsed.
-         */
-        it.it_value.tv_sec = ticks / OS_TICKS_PER_SEC;
-        it.it_value.tv_usec = (ticks % OS_TICKS_PER_SEC) * OS_USEC_PER_TICK;
-        it.it_interval.tv_sec = 0;
-        it.it_interval.tv_usec = OS_USEC_PER_TICK;
-        rc = setitimer(ITIMER_REAL, &it, NULL);
-        assert(rc == 0);
-    }
-
-    suspended = true;
-    sigemptyset(&suspsigs);
-    sigsuspend(&nosigs);        /* Wait for a signal to wake us up */
-    suspended = false;
-
-    /*
-     * Call handlers for signals delivered to the process during sigsuspend().
-     * The SIGALRM handler is called before any other handlers to ensure that
-     * OS time is always correct.
-     */
-    if (sigismember(&suspsigs, SIGALRM)) {
-        timer_handler(SIGALRM);
-    }
-    for (i = 0; i < NUMSIGS; i++) {
-        sig = signals[i].num;
-        handler = signals[i].handler;
-        if (sig != SIGALRM && sigismember(&suspsigs, sig)) {
-            handler(sig);
-        }
-    }
-
-    if (ticks > 0) {
-        /*
-         * Enable the periodic timer interrupt.
-         */
-        it.it_value.tv_sec = 0;
-        it.it_value.tv_usec = OS_USEC_PER_TICK;
-        it.it_interval.tv_sec = 0;
-        it.it_interval.tv_usec = OS_USEC_PER_TICK;
-        rc = setitimer(ITIMER_REAL, &it, NULL);
-        assert(rc == 0);
-    }
-}
-
-static void
-signals_init(void)
-{
-    int i, error;
-    struct sigaction sa;
-
-    sigemptyset(&nosigs);
-    sigemptyset(&allsigs);
-    for (i = 0; i < NUMSIGS; i++) {
-        sigaddset(&allsigs, signals[i].num);
-    }
-
-    for (i = 0; i < NUMSIGS; i++) {
-        memset(&sa, 0, sizeof sa);
-        sa.sa_handler = signals[i].handler;
-        sa.sa_mask = allsigs;
-        sa.sa_flags = SA_RESTART;
-        error = sigaction(signals[i].num, &sa, NULL);
-        assert(error == 0);
-    }
-
-    /*
-     * We use SIGALRM as a proxy for 'allsigs' to check if we are inside
-     * a critical section (for e.g. see os_arch_in_critical()). Make sure
-     * that SIGALRM is indeed present in 'allsigs'.
-     */
-    assert(sigismember(&allsigs, SIGALRM));
-}
-
-static void
-signals_cleanup(void)
-{
-    int i, error;
-    struct sigaction sa;
-
-    for (i = 0; i < NUMSIGS; i++) {
-        memset(&sa, 0, sizeof sa);
-        sa.sa_handler = SIG_DFL;
-        error = sigaction(signals[i].num, &sa, NULL);
-        assert(error == 0);
-    }
-}
-
-static void
-timer_handler(int sig)
-{
-    struct timeval time_now, time_diff;
-    int ticks;
-
-    static struct timeval time_last;
-    static int time_inited;
-
-    OS_ASSERT_CRITICAL();
-
-    /*
-     * Just record that this handler was called when the process was blocked.
-     * The handler will be called after sigsuspend() returns in the proper
-     * order.
-     */
-    if (suspended) {
-        sigaddset(&suspsigs, sig);
-        return;
-    }
-
-    if (!time_inited) {
-        gettimeofday(&time_last, NULL);
-        time_inited = 1;
-    }
-
-    gettimeofday(&time_now, NULL);
-    if (timercmp(&time_now, &time_last, <)) {
-        /*
-         * System time going backwards.
-         */
-        time_last = time_now;
-    } else {
-        timersub(&time_now, &time_last, &time_diff);
-
-        ticks = time_diff.tv_sec * OS_TICKS_PER_SEC;
-        ticks += time_diff.tv_usec / OS_USEC_PER_TICK;
-
-        /*
-         * Update 'time_last' but account for the remainder usecs that did not
-         * contribute towards whole 'ticks'.
-         */
-        time_diff.tv_sec = 0;
-        time_diff.tv_usec %= OS_USEC_PER_TICK;
-        timersub(&time_now, &time_diff, &time_last);
-
-        os_time_advance(ticks);
-    }
-}
-
-static void
-start_timer(void)
-{
-    struct itimerval it;
-    int rc;
-
-    memset(&it, 0, sizeof(it));
-    it.it_value.tv_sec = 0;
-    it.it_value.tv_usec = OS_USEC_PER_TICK;
-    it.it_interval.tv_sec = 0;
-    it.it_interval.tv_usec = OS_USEC_PER_TICK;
-
-    rc = setitimer(ITIMER_REAL, &it, NULL);
-    assert(rc == 0);
-}
-
-static void
-stop_timer(void)
-{
-    struct itimerval it;
-    int rc;
-
-    memset(&it, 0, sizeof(it));
-
-    rc = setitimer(ITIMER_REAL, &it, NULL);
-    assert(rc == 0);
-}
-
-os_error_t
-os_arch_os_init(void)
-{
-    mypid = getpid();
-    g_current_task = NULL;
-
-    STAILQ_INIT(&g_os_task_list);
-    TAILQ_INIT(&g_os_run_list);
-    TAILQ_INIT(&g_os_sleep_list);
-
-    /*
-     * Setup all interrupt handlers.
-     *
-     * This must be done early because task initialization uses critical
-     * sections which function correctly only when 'allsigs' is initialized.
-     */
-    signals_init();
-
-    os_init_idle_task();
-
-    return OS_OK;
-}
-
-os_error_t
-os_arch_os_start(void)
-{
-    struct stack_frame *sf;
-    struct os_task *t;
-    os_sr_t sr;
-
-    /*
-     * Disable interrupts before enabling any interrupt sources. Pending
-     * interrupts will be recognized when the first task starts executing.
-     */
-    OS_ENTER_CRITICAL(sr);
-    assert(sr == 0);
-
-    /* Enable the interrupt sources */
-    start_timer();
-
-    t = os_sched_next_task();
-    os_sched_set_current_task(t);
-
-    g_os_started = 1;
-
-    sf = (struct stack_frame *) t->t_stackptr;
-    sim_longjmp(sf->sf_jb, 1);
-
-    return 0;
-}
-
-/**
- * Stops the tick timer and clears the "started" flag.  This function is only
- * implemented for sim.
- */
-void
-os_arch_os_stop(void)
-{
-    stop_timer();
-    signals_cleanup();
-    g_os_started = 0;
-}

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/cc1acfe8/kernel/os/src/arch/sim/os_arch_sim_gen.c
----------------------------------------------------------------------
diff --git a/kernel/os/src/arch/sim/os_arch_sim_gen.c b/kernel/os/src/arch/sim/os_arch_sim_gen.c
new file mode 100644
index 0000000..66202e1
--- /dev/null
+++ b/kernel/os/src/arch/sim/os_arch_sim_gen.c
@@ -0,0 +1,248 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * This file contains code that is shared by both sim implementations (signals
+ * and no-signals).
+ */
+
+#include "os/os.h"
+#include "os_priv.h"
+
+#include <hal/hal_bsp.h>
+
+#ifdef __APPLE__
+#define _XOPEN_SOURCE
+#endif
+
+#include <string.h>
+#include <stdio.h>
+#include <unistd.h>
+#include <setjmp.h>
+#include <signal.h>
+#include <sys/time.h>
+#include <assert.h>
+#include "os_arch_sim_priv.h"
+
+#define sim_setjmp(__jb) sigsetjmp(__jb, 0)
+#define sim_longjmp(__jb, __ret) siglongjmp(__jb, __ret)
+
+pid_t os_arch_sim_pid;
+
+extern void os_arch_frame_init(struct stack_frame *sf);
+
+void
+os_arch_sim_ctx_sw(void)
+{
+    struct os_task *t, *next_t;
+    struct stack_frame *sf;
+    int rc;
+
+    OS_ASSERT_CRITICAL();
+
+    t = os_sched_get_current_task();
+    next_t = os_sched_next_task();
+    if (t == next_t) {
+        /*
+         * Context switch not needed - just return.
+         */
+        return;
+    }
+
+    if (t) {
+        sf = (struct stack_frame *) t->t_stackptr;
+
+        rc = sim_setjmp(sf->sf_jb);
+        if (rc != 0) {
+            OS_ASSERT_CRITICAL();
+            return;
+        }
+    }
+
+    os_sched_ctx_sw_hook(next_t);
+
+    os_sched_set_current_task(next_t);
+
+    sf = (struct stack_frame *) next_t->t_stackptr;
+    sim_longjmp(sf->sf_jb, 1);
+}
+
+void
+os_arch_sim_tick(void)
+{
+    struct timeval time_now, time_diff;
+    int ticks;
+
+    static struct timeval time_last;
+    static int time_inited;
+
+    OS_ASSERT_CRITICAL();
+
+    if (!time_inited) {
+        gettimeofday(&time_last, NULL);
+        time_inited = 1;
+    }
+
+    gettimeofday(&time_now, NULL);
+    if (timercmp(&time_now, &time_last, <)) {
+        /*
+         * System time going backwards.
+         */
+        time_last = time_now;
+    } else {
+        timersub(&time_now, &time_last, &time_diff);
+
+        ticks = time_diff.tv_sec * OS_TICKS_PER_SEC;
+        ticks += time_diff.tv_usec / OS_USEC_PER_TICK;
+
+        /*
+         * Update 'time_last' but account for the remainder usecs that did not
+         * contribute towards whole 'ticks'.
+         */
+        time_diff.tv_sec = 0;
+        time_diff.tv_usec %= OS_USEC_PER_TICK;
+        timersub(&time_now, &time_diff, &time_last);
+
+        os_time_advance(ticks);
+    }
+}
+
+static void
+os_arch_sim_start_timer(void)
+{
+    struct itimerval it;
+    int rc;
+
+    memset(&it, 0, sizeof(it));
+    it.it_value.tv_sec = 0;
+    it.it_value.tv_usec = OS_USEC_PER_TICK;
+    it.it_interval.tv_sec = 0;
+    it.it_interval.tv_usec = OS_USEC_PER_TICK;
+
+    rc = setitimer(ITIMER_REAL, &it, NULL);
+    assert(rc == 0);
+}
+
+static void
+os_arch_sim_stop_timer(void)
+{
+    struct itimerval it;
+    int rc;
+
+    memset(&it, 0, sizeof(it));
+
+    rc = setitimer(ITIMER_REAL, &it, NULL);
+    assert(rc == 0);
+}
+
+/*
+ * Called from 'os_arch_frame_init()' when setjmp returns indirectly via
+ * longjmp. The return value of setjmp is passed to this function as 'rc'.
+ */
+void
+os_arch_task_start(struct stack_frame *sf, int rc)
+{
+    struct os_task *task;
+
+    /*
+     * Interrupts are disabled when a task starts executing. This happens in
+     * two different ways:
+     * - via os_arch_os_start() for the first task.
+     * - via os_sched() for all other tasks.
+     *
+     * Enable interrupts before starting the task.
+     */
+    OS_EXIT_CRITICAL(0);
+
+    task = sf->sf_task;
+    task->t_func(task->t_arg);
+
+    /* This should never return */
+    assert(0);
+}
+
+os_stack_t *
+os_arch_task_stack_init(struct os_task *t, os_stack_t *stack_top, int size)
+{
+    struct stack_frame *sf;
+
+    sf = (struct stack_frame *) ((uint8_t *) stack_top - sizeof(*sf));
+    sf->sf_task = t;
+
+    os_arch_frame_init(sf);
+
+    return ((os_stack_t *)sf);
+}
+
+os_error_t
+os_arch_os_start(void)
+{
+    struct stack_frame *sf;
+    struct os_task *t;
+    os_sr_t sr;
+
+    /*
+     * Disable interrupts before enabling any interrupt sources. Pending
+     * interrupts will be recognized when the first task starts executing.
+     */
+    OS_ENTER_CRITICAL(sr);
+    assert(sr == 0);
+
+    /* Enable the interrupt sources */
+    os_arch_sim_start_timer();
+
+    t = os_sched_next_task();
+    os_sched_set_current_task(t);
+
+    g_os_started = 1;
+
+    sf = (struct stack_frame *) t->t_stackptr;
+    sim_longjmp(sf->sf_jb, 1);
+
+    return 0;
+}
+
+/**
+ * Stops the tick timer and clears the "started" flag.  This function is only
+ * implemented for sim.
+ */
+void
+os_arch_os_stop(void)
+{
+    os_arch_sim_stop_timer();
+    os_arch_sim_signals_cleanup();
+    g_os_started = 0;
+}
+
+os_error_t
+os_arch_os_init(void)
+{
+    os_arch_sim_pid = getpid();
+    g_current_task = NULL;
+
+    STAILQ_INIT(&g_os_task_list);
+    TAILQ_INIT(&g_os_run_list);
+    TAILQ_INIT(&g_os_sleep_list);
+
+    os_arch_sim_signals_init();
+
+    os_init_idle_task();
+
+    return OS_OK;
+}

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/cc1acfe8/kernel/os/src/arch/sim/os_arch_sim_nosig.c
----------------------------------------------------------------------
diff --git a/kernel/os/src/arch/sim/os_arch_sim_nosig.c b/kernel/os/src/arch/sim/os_arch_sim_nosig.c
new file mode 100644
index 0000000..9f23e67
--- /dev/null
+++ b/kernel/os/src/arch/sim/os_arch_sim_nosig.c
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * This file implements the "no-signals" version of sim.  This implementation
+ * does not use signals to perform context switches.  This is the less correct
+ * version of sim: the OS tick timer only runs while the idle task is active.
+ * Therefore, a sleeping high-priority task will not preempt a low-priority
+ * task due to a timing event (e.g., delay or callout expired).  However, this
+ * version of sim does not suffer from the stability issues that affect the
+ * "signals" implementation.
+ *
+ * To use this version of sim, disable the MCU_NATIVE_USE_SIGNALS syscfg
+ * setting.  
+ */
+
+#include "syscfg/syscfg.h"
+
+#if !MYNEWT_VAL(MCU_NATIVE_USE_SIGNALS)
+
+#include "os/os.h"
+#include "os_priv.h"
+
+#include <hal/hal_bsp.h>
+
+#ifdef __APPLE__
+#define _XOPEN_SOURCE
+#endif
+
+#include <string.h>
+#include <stdio.h>
+#include <unistd.h>
+#include <setjmp.h>
+#include <signal.h>
+#include <sys/time.h>
+#include <assert.h>
+#include "os_arch_sim_priv.h"
+
+static sigset_t nosigs;
+static sigset_t suspsigs;   /* signals delivered in sigsuspend() */
+
+static int ctx_sw_pending;
+static int interrupts_enabled = 1;
+
+void
+os_arch_ctx_sw(struct os_task *next_t)
+{
+    if (interrupts_enabled) {
+        /* Perform the context switch immediately. */
+        os_arch_sim_ctx_sw();
+    } else {
+        /* Remember that we want to perform a context switch.  Perform it when
+         * interrupts are re-enabled.
+         */
+        ctx_sw_pending = 1;
+    }
+}
+
+/*
+ * Enter a critical section.
+ *
+ * Returns 1 if interrupts were already disabled; 0 otherwise.
+ */
+os_sr_t
+os_arch_save_sr(void)
+{
+    if (!interrupts_enabled) {
+        return 1;
+    }
+
+    interrupts_enabled = 0;
+    return 0;
+}
+
+void
+os_arch_restore_sr(os_sr_t osr)
+{
+    OS_ASSERT_CRITICAL();
+    assert(osr == 0 || osr == 1);
+
+    if (osr == 1) {
+        /* Exiting a nested critical section */
+        return;
+    }
+
+    if (ctx_sw_pending) {
+        /* A context switch was requested while interrupts were disabled.
+         * Perform it now that interrupts are enabled again.
+         */
+        ctx_sw_pending = 0;
+        os_arch_sim_ctx_sw();
+    }
+    interrupts_enabled = 1;
+}
+
+int
+os_arch_in_critical(void)
+{
+    return !interrupts_enabled;
+}
+
+/**
+ * Unblocks the SIGALRM signal that is delivered by the OS tick timer.
+ */
+static void
+unblock_timer(void)
+{
+    sigset_t sigs;
+    int rc;
+
+    sigemptyset(&sigs);
+    sigaddset(&sigs, SIGALRM);
+
+    rc = sigprocmask(SIG_UNBLOCK, &sigs, NULL);
+    assert(rc == 0);
+}
+
+/**
+ * Blocks the SIGALRM signal that is delivered by the OS tick timer.
+ */
+static void
+block_timer(void)
+{
+    sigset_t sigs;
+    int rc;
+
+    sigemptyset(&sigs);
+    sigaddset(&sigs, SIGALRM);
+
+    rc = sigprocmask(SIG_BLOCK, &sigs, NULL);
+    assert(rc == 0);
+}
+
+static void
+sig_handler_alrm(int sig)
+{
+    /* Wake the idle task. */
+    sigaddset(&suspsigs, sig);
+}
+
+void
+os_tick_idle(os_time_t ticks)
+{
+    int rc;
+    struct itimerval it;
+
+    OS_ASSERT_CRITICAL();
+
+    if (ticks > 0) {
+        /*
+         * Enter tickless regime and set the timer to fire after 'ticks'
+         * worth of time has elapsed.
+         */
+        it.it_value.tv_sec = ticks / OS_TICKS_PER_SEC;
+        it.it_value.tv_usec = (ticks % OS_TICKS_PER_SEC) * OS_USEC_PER_TICK;
+        it.it_interval.tv_sec = 0;
+        it.it_interval.tv_usec = OS_USEC_PER_TICK;
+        rc = setitimer(ITIMER_REAL, &it, NULL);
+        assert(rc == 0);
+    }
+
+    unblock_timer();
+
+    sigemptyset(&suspsigs);
+    sigsuspend(&nosigs);        /* Wait for a signal to wake us up */
+
+    block_timer();
+
+    /*
+     * Call handlers for signals delivered to the process during sigsuspend().
+     * The SIGALRM handler is called before any other handlers to ensure that
+     * OS time is always correct.
+     */
+    if (sigismember(&suspsigs, SIGALRM)) {
+        os_arch_sim_tick();
+    }
+
+    if (ticks > 0) {
+        /*
+         * Enable the periodic timer interrupt.
+         */
+        it.it_value.tv_sec = 0;
+        it.it_value.tv_usec = OS_USEC_PER_TICK;
+        it.it_interval.tv_sec = 0;
+        it.it_interval.tv_usec = OS_USEC_PER_TICK;
+        rc = setitimer(ITIMER_REAL, &it, NULL);
+        assert(rc == 0);
+    }
+}
+
+void
+os_arch_sim_signals_init(void)
+{
+    sigset_t sigset_alrm;
+    struct sigaction sa;
+    int error;
+
+    block_timer();
+
+    sigemptyset(&nosigs);
+
+    sigemptyset(&sigset_alrm);
+    sigaddset(&sigset_alrm, SIGALRM);
+
+    memset(&sa, 0, sizeof sa);
+    sa.sa_handler = sig_handler_alrm;
+    sa.sa_mask = sigset_alrm;
+    sa.sa_flags = SA_RESTART;
+    error = sigaction(SIGALRM, &sa, NULL);
+    assert(error == 0);
+}
+
+void
+os_arch_sim_signals_cleanup(void)
+{
+    int error;
+    struct sigaction sa;
+
+    memset(&sa, 0, sizeof sa);
+    sa.sa_handler = SIG_DFL;
+    error = sigaction(SIGALRM, &sa, NULL);
+    assert(error == 0);
+}
+
+#endif /* !MYNEWT_VAL(MCU_NATIVE_USE_SIGNALS) */

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/cc1acfe8/kernel/os/src/arch/sim/os_arch_sim_priv.h
----------------------------------------------------------------------
diff --git a/kernel/os/src/arch/sim/os_arch_sim_priv.h b/kernel/os/src/arch/sim/os_arch_sim_priv.h
new file mode 100644
index 0000000..6a19cc5
--- /dev/null
+++ b/kernel/os/src/arch/sim/os_arch_sim_priv.h
@@ -0,0 +1,29 @@
+#ifndef H_OS_ARCH_SIM_PRIV_
+#define H_OS_ARCH_SIM_PRIV_
+
+#include <setjmp.h>
+#include <os/os.h>
+
+struct stack_frame {
+    int sf_mainsp;              /* stack on which main() is executing */
+    sigjmp_buf sf_jb;
+    struct os_task *sf_task;
+};
+
+/*
+ * Assert that 'sf_mainsp' and 'sf_jb' are at the specific offsets where
+ * os_arch_frame_init() expects them to be.
+ */
+CTASSERT(offsetof(struct stack_frame, sf_mainsp) == 0);
+CTASSERT(offsetof(struct stack_frame, sf_jb) == 4);
+
+#define OS_USEC_PER_TICK    (1000000 / OS_TICKS_PER_SEC)
+
+void os_arch_sim_ctx_sw(void);
+void os_arch_sim_tick(void);
+void os_arch_sim_signals_init(void);
+void os_arch_sim_signals_cleanup(void);
+
+extern pid_t os_arch_sim_pid;
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/cc1acfe8/kernel/os/src/arch/sim/os_arch_sim_sig.c
----------------------------------------------------------------------
diff --git a/kernel/os/src/arch/sim/os_arch_sim_sig.c b/kernel/os/src/arch/sim/os_arch_sim_sig.c
new file mode 100644
index 0000000..d1df204
--- /dev/null
+++ b/kernel/os/src/arch/sim/os_arch_sim_sig.c
@@ -0,0 +1,266 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * This file implements the "signals" version of sim.  This implementation uses
+ * signals to perform context switches.  This is the more correct version of
+ * sim: the OS tick timer will cause a high-priority task to preempt a
+ * low-priority task.  Unfortunately, there are stability issues because a task
+ * can be preempted while it is in the middle of a system call, potentially
+ * causing deadlock or memory corruption.
+ *
+ * To use this version of sim, enable the MCU_NATIVE_USE_SIGNALS syscfg
+ * setting.  
+ */
+
+#include "syscfg/syscfg.h"
+
+#if MYNEWT_VAL(MCU_NATIVE_USE_SIGNALS)
+
+#include "os/os.h"
+#include "os_priv.h"
+#include "os_arch_sim_priv.h"
+
+#include <hal/hal_bsp.h>
+
+#ifdef __APPLE__
+#define _XOPEN_SOURCE
+#endif
+
+#include <string.h>
+#include <stdio.h>
+#include <unistd.h>
+#include <setjmp.h>
+#include <signal.h>
+#include <sys/time.h>
+#include <assert.h>
+
+static bool suspended;      /* process is blocked in sigsuspend() */
+static sigset_t suspsigs;   /* signals delivered in sigsuspend() */
+static sigset_t allsigs;
+static sigset_t nosigs;
+
+void
+os_arch_ctx_sw(struct os_task *next_t)
+{
+    /*
+     * gdb will stop execution of the program on most signals (e.g. SIGUSR1)
+     * whereas it passes SIGURG to the process without any special settings.
+     */
+    kill(os_arch_sim_pid, SIGURG);
+}
+
+static void
+ctxsw_handler(int sig)
+{
+    OS_ASSERT_CRITICAL();
+
+    /*
+     * Just record that this handler was called when the process was blocked.
+     * The handler will be called after sigsuspend() returns in the correct
+     * order.
+     */
+    if (suspended) {
+        sigaddset(&suspsigs, sig);
+    } else {
+        os_arch_sim_ctx_sw();
+    }
+}
+
+/*
+ * Disable signals and enter a critical section.
+ *
+ * Returns 1 if signals were already blocked and 0 otherwise.
+ */
+os_sr_t
+os_arch_save_sr(void)
+{
+    int error;
+    sigset_t omask;
+
+    error = sigprocmask(SIG_BLOCK, &allsigs, &omask);
+    assert(error == 0);
+
+    /*
+     * If any one of the signals in 'allsigs' is present in 'omask' then
+     * we are already inside a critical section.
+     */
+    return (sigismember(&omask, SIGALRM));
+}
+
+void
+os_arch_restore_sr(os_sr_t osr)
+{
+    int error;
+
+    OS_ASSERT_CRITICAL();
+    assert(osr == 0 || osr == 1);
+
+    if (osr == 1) {
+        /* Exiting a nested critical section */
+        return;
+    }
+
+    error = sigprocmask(SIG_UNBLOCK, &allsigs, NULL);
+    assert(error == 0);
+}
+
+int
+os_arch_in_critical(void)
+{
+    int error;
+    sigset_t omask;
+
+    error = sigprocmask(SIG_SETMASK, NULL, &omask);
+    assert(error == 0);
+
+    /*
+     * If any one of the signals in 'allsigs' is present in 'omask' then
+     * we are already inside a critical section.
+     */
+    return (sigismember(&omask, SIGALRM));
+}
+
+static void
+timer_handler(int sig)
+{
+    OS_ASSERT_CRITICAL();
+
+    /*
+     * Just record that this handler was called when the process was blocked.
+     * The handler will be called after sigsuspend() returns in the proper
+     * order.
+     */
+    if (suspended) {
+        sigaddset(&suspsigs, sig);
+    } else {
+        os_arch_sim_tick();
+    }
+}
+
+static struct {
+    int num;
+    void (*handler)(int sig);
+} signals[] = {
+    { SIGALRM, timer_handler },
+    { SIGURG, ctxsw_handler },
+};
+
+#define NUMSIGS     (sizeof(signals)/sizeof(signals[0]))
+
+void
+os_tick_idle(os_time_t ticks)
+{
+    int i, rc, sig;
+    struct itimerval it;
+    void (*handler)(int sig);
+
+    OS_ASSERT_CRITICAL();
+
+    if (ticks > 0) {
+        /*
+         * Enter tickless regime and set the timer to fire after 'ticks'
+         * worth of time has elapsed.
+         */
+        it.it_value.tv_sec = ticks / OS_TICKS_PER_SEC;
+        it.it_value.tv_usec = (ticks % OS_TICKS_PER_SEC) * OS_USEC_PER_TICK;
+        it.it_interval.tv_sec = 0;
+        it.it_interval.tv_usec = OS_USEC_PER_TICK;
+        rc = setitimer(ITIMER_REAL, &it, NULL);
+        assert(rc == 0);
+    }
+
+    suspended = true;
+    sigemptyset(&suspsigs);
+    sigsuspend(&nosigs);        /* Wait for a signal to wake us up */
+    suspended = false;
+
+    /*
+     * Call handlers for signals delivered to the process during sigsuspend().
+     * The SIGALRM handler is called before any other handlers to ensure that
+     * OS time is always correct.
+     */
+    if (sigismember(&suspsigs, SIGALRM)) {
+        os_arch_sim_tick();
+    }
+    for (i = 0; i < NUMSIGS; i++) {
+        sig = signals[i].num;
+        handler = signals[i].handler;
+        if (sig != SIGALRM && sigismember(&suspsigs, sig)) {
+            handler(sig);
+        }
+    }
+
+    if (ticks > 0) {
+        /*
+         * Enable the periodic timer interrupt.
+         */
+        it.it_value.tv_sec = 0;
+        it.it_value.tv_usec = OS_USEC_PER_TICK;
+        it.it_interval.tv_sec = 0;
+        it.it_interval.tv_usec = OS_USEC_PER_TICK;
+        rc = setitimer(ITIMER_REAL, &it, NULL);
+        assert(rc == 0);
+    }
+}
+
+void
+os_arch_sim_signals_init(void)
+{
+    int i, error;
+    struct sigaction sa;
+
+    sigemptyset(&nosigs);
+    sigemptyset(&allsigs);
+    for (i = 0; i < NUMSIGS; i++) {
+        sigaddset(&allsigs, signals[i].num);
+    }
+
+    for (i = 0; i < NUMSIGS; i++) {
+        memset(&sa, 0, sizeof sa);
+        sa.sa_handler = signals[i].handler;
+        sa.sa_mask = allsigs;
+        sa.sa_flags = SA_RESTART;
+        error = sigaction(signals[i].num, &sa, NULL);
+        assert(error == 0);
+    }
+
+    /*
+     * We use SIGALRM as a proxy for 'allsigs' to check if we are inside
+     * a critical section (for e.g. see os_arch_in_critical()). Make sure
+     * that SIGALRM is indeed present in 'allsigs'.
+     */
+    assert(sigismember(&allsigs, SIGALRM));
+}
+
+void
+os_arch_sim_signals_cleanup(void)
+{
+    int i, error;
+    struct sigaction sa;
+
+    for (i = 0; i < NUMSIGS; i++) {
+        memset(&sa, 0, sizeof sa);
+        sa.sa_handler = SIG_DFL;
+        error = sigaction(signals[i].num, &sa, NULL);
+        assert(error == 0);
+    }
+}
+
+#endif /* MYNEWT_VAL(MCU_NATIVE_USE_SIGNALS) */


[3/3] incubator-mynewt-core git commit: This closes #269.

Posted by cc...@apache.org.
This closes #269.

Merge remote-tracking branch 'ccollins476ad/sim-no-signals'

* ccollins476ad/sim-no-signals:
  MYNEWT-745 Make native sockets nonblocking.
  MYNEWT-745 Sim - deadlock involving system calls


Project: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/commit/dd4baa28
Tree: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/tree/dd4baa28
Diff: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/diff/dd4baa28

Branch: refs/heads/master
Commit: dd4baa28ea899c0bfd444551ccba78309f3847df
Parents: b26a78d 71811d4
Author: Christopher Collins <cc...@apache.org>
Authored: Fri May 12 18:39:11 2017 -0700
Committer: Christopher Collins <cc...@apache.org>
Committed: Fri May 12 18:39:11 2017 -0700

----------------------------------------------------------------------
 hw/mcu/native/syscfg.yml                   |  21 ++
 kernel/os/src/arch/sim/os_arch_sim.c       | 461 ------------------------
 kernel/os/src/arch/sim/os_arch_sim_gen.c   | 248 +++++++++++++
 kernel/os/src/arch/sim/os_arch_sim_nosig.c | 241 +++++++++++++
 kernel/os/src/arch/sim/os_arch_sim_priv.h  |  29 ++
 kernel/os/src/arch/sim/os_arch_sim_sig.c   | 266 ++++++++++++++
 net/ip/native_sockets/src/native_sock.c    |   6 +
 7 files changed, 811 insertions(+), 461 deletions(-)
----------------------------------------------------------------------



[2/3] incubator-mynewt-core git commit: MYNEWT-745 Make native sockets nonblocking.

Posted by cc...@apache.org.
MYNEWT-745 Make native sockets nonblocking.

This prevents deadlock in the "no-signals" sim implementation.


Project: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/commit/71811d43
Tree: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/tree/71811d43
Diff: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/diff/71811d43

Branch: refs/heads/master
Commit: 71811d438c1c4722c09419646c4e6f198da1d1b2
Parents: cc1acfe
Author: Christopher Collins <cc...@apache.org>
Authored: Tue May 9 18:05:53 2017 -0700
Committer: Christopher Collins <cc...@apache.org>
Committed: Tue May 9 18:05:53 2017 -0700

----------------------------------------------------------------------
 net/ip/native_sockets/src/native_sock.c | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/71811d43/net/ip/native_sockets/src/native_sock.c
----------------------------------------------------------------------
diff --git a/net/ip/native_sockets/src/native_sock.c b/net/ip/native_sockets/src/native_sock.c
index 9e2d3d6..e0ecd79 100644
--- a/net/ip/native_sockets/src/native_sock.c
+++ b/net/ip/native_sockets/src/native_sock.c
@@ -250,6 +250,7 @@ native_sock_create(struct mn_socket **sp, uint8_t domain,
     struct native_sock_state *nss = &native_sock_state;
     struct native_sock *ns;
     int idx;
+    int rc;
 
     switch (domain) {
     case MN_PF_INET:
@@ -283,6 +284,11 @@ native_sock_create(struct mn_socket **sp, uint8_t domain,
     }
     os_sem_init(&ns->ns_sem, 0);
     idx = socket(domain, type, proto);
+
+    /* Make the socket nonblocking. */
+    rc = fcntl(idx, F_SETFL, fcntl(idx, F_GETFL, 0) | O_NONBLOCK);
+    assert(rc == 0);
+
     ns->ns_fd = idx;
     ns->ns_pf = domain;
     ns->ns_type = type;