Commit 9305f720 authored by Chris Toshok's avatar Chris Toshok

build tracebacks incrementally while c++ unwinding

Add a tb_next to the BoxedTraceback object, which during normal unwinding is how we add lines to a traceback.
For getTraceback() we take advantage fo the fact that each BoxedTraceback can have multiple lines in it, and
only create one.  Also add BoxedTraceback::Here which is kinda like PyTraceback_Here, except that since we
don't have access to the ExcInfo inside it, we have to pass a pointer to the BoxedTraceback*.

three wrinkles in traceback generation while unwinding are:

   1) if the previous frame was an osr frame (so we should skip the current frame)
   2) if the traceback object was passed to raise3, or raise0 was used (treated as a re-raise by cpython)
   3) the interpreter map

1 and 2 are fixed by keeping a per-thread interpreter state variable, "unwind_why" ("why" is probably
a bad name to use in this context, but cpython's logic uses that word as well), to record if we're in
either of those cases.  Both cases have the same effect (skip the next traceback line, and reset state
to NORMAL.)

The interpreter map problem comes about because of the way c++ unwinding works.  since ASTInterpreter::execute
uses RAII for RegisterHelper, unwinding through the frame causes us to jump to a cleanup block and then resume
unwinding (with the IP still in ASTInterpreter::execute).  Since the IP is there, we assume the frame is valid
and we can look up $rbp in the interpreter map.  Given that we just cleaned up the RegisterHelper (and removed
the $rbp mapping), we crash.

The fix here is to keep the same RegisterHelper in ASTInterpreter::execute, but the ip checked for by the
unwinder (and the $rbp in the mapping) correspond to ASTInterpreter::executeInner.

Lastly, AST_Interpreter::invoke catches exceptions, so we won't make it to the top-most ASTInterpreter::execute,
and so won't add the final line.  We make use of BoxedTraceback::Here here as well to add the final line.
parent 46d6bba3
......@@ -110,6 +110,15 @@ typedef struct _ts {
} PyThreadState;
#endif
// somewhat similar to CPython's WHY_* enum
// UNWIND_WHY_NORMAL : == WHY_EXCEPTION. we call it "NORMAL" since we often unwind due to things other than exceptions (getGlobals, getLocals, etc)
// UNWIND_WHY_RERAISE: same as NORMAL, except we are supposed to skip the first frame.
// UNWIND_WHY_OSR : The previous frame was an osr replacement for the next one, so we should skip it
enum {
UNWIND_WHY_NORMAL = 0,
UNWIND_WHY_RERAISE,
UNWIND_WHY_OSR
};
typedef struct _ts {
int recursion_depth;
......@@ -119,6 +128,9 @@ typedef struct _ts {
PyObject *dict; /* Stores per-thread state */
// one of the UNWIND_WHY_* above
int unwind_why;
// Pyston note: additions in here need to be mirrored in ThreadStateInternal::accept
} PyThreadState;
......
......@@ -41,6 +41,7 @@
#include "runtime/long.h"
#include "runtime/objmodel.h"
#include "runtime/set.h"
#include "runtime/traceback.h"
#include "runtime/types.h"
#ifndef NDEBUG
......@@ -55,29 +56,6 @@ namespace {
static BoxedClass* astinterpreter_cls;
class ASTInterpreter;
// Map from stack frame pointers for frames corresponding to ASTInterpreter::execute() to the ASTInterpreter handling
// them. Used to look up information about that frame. This is used for getting tracebacks, for CPython introspection
// (sys._getframe & co), and for GC scanning.
static std::unordered_map<void*, ASTInterpreter*> s_interpreterMap;
static_assert(THREADING_USE_GIL, "have to make the interpreter map thread safe!");
class RegisterHelper {
private:
void* frame_addr;
ASTInterpreter* interpreter;
public:
RegisterHelper(ASTInterpreter* interpreter, void* frame_addr);
~RegisterHelper();
static void deregister(void* frame_addr) {
assert(s_interpreterMap.count(frame_addr));
s_interpreterMap.erase(frame_addr);
}
};
union Value {
bool b;
int64_t n;
......@@ -93,6 +71,20 @@ union Value {
}
};
class ASTInterpreter;
class RegisterHelper {
private:
void* frame_addr;
ASTInterpreter* interpreter;
public:
RegisterHelper();
~RegisterHelper();
void doRegister(void* frame_addr, ASTInterpreter* interpreter);
static void deregister(void* frame_addr);
};
class ASTInterpreter : public Box {
public:
typedef ContiguousMap<InternedString, Box*> SymMap;
......@@ -102,11 +94,13 @@ public:
void initArguments(int nargs, BoxedClosure* closure, BoxedGenerator* generator, Box* arg1, Box* arg2, Box* arg3,
Box** args);
static Value execute(ASTInterpreter& interpreter, CFGBlock* start_block = NULL, AST_stmt* start_at = NULL);
// This must not be inlined, because we rely on being able to detect when we're inside of it (by checking whether
// %rip is inside its instruction range) during a stack-trace in order to produce tracebacks inside interpreted
// code.
__attribute__((__no_inline__)) static Value
execute(ASTInterpreter& interpreter, CFGBlock* start_block = NULL, AST_stmt* start_at = NULL);
executeInner(ASTInterpreter& interpreter, CFGBlock* start_block, AST_stmt* start_at, RegisterHelper* reg);
private:
Box* createFunction(AST* node, AST_arguments* args, const std::vector<AST_stmt*>& body);
......@@ -318,22 +312,40 @@ void ASTInterpreter::initArguments(int nargs, BoxedClosure* _closure, BoxedGener
}
}
RegisterHelper::RegisterHelper(ASTInterpreter* interpreter, void* frame_addr)
: frame_addr(frame_addr), interpreter(interpreter) {
interpreter->frame_addr = frame_addr;
s_interpreterMap[frame_addr] = interpreter;
}
// Map from stack frame pointers for frames corresponding to ASTInterpreter::execute() to the ASTInterpreter handling
// them. Used to look up information about that frame. This is used for getting tracebacks, for CPython introspection
// (sys._getframe & co), and for GC scanning.
static std::unordered_map<void*, ASTInterpreter*> s_interpreterMap;
static_assert(THREADING_USE_GIL, "have to make the interpreter map thread safe!");
RegisterHelper::RegisterHelper() : frame_addr(NULL), interpreter(NULL) { }
RegisterHelper::~RegisterHelper() {
assert(interpreter);
assert(interpreter->frame_addr == frame_addr);
interpreter->frame_addr = nullptr;
deregister(frame_addr);
}
Value ASTInterpreter::execute(ASTInterpreter& interpreter, CFGBlock* start_block, AST_stmt* start_at) {
STAT_TIMER(t0, "us_timer_astinterpreter_execute");
void RegisterHelper::doRegister(void* frame_addr, ASTInterpreter* interpreter) {
assert(!this->interpreter);
assert(!this->frame_addr);
this->frame_addr = frame_addr;
this->interpreter = interpreter;
interpreter->frame_addr = frame_addr;
s_interpreterMap[frame_addr] = interpreter;
}
void RegisterHelper::deregister(void* frame_addr) {
assert(frame_addr);
assert(s_interpreterMap.count(frame_addr));
s_interpreterMap.erase(frame_addr);
}
Value ASTInterpreter::executeInner(ASTInterpreter& interpreter, CFGBlock* start_block, AST_stmt* start_at,
RegisterHelper* reg) {
void* frame_addr = __builtin_frame_address(0);
RegisterHelper frame_registerer(&interpreter, frame_addr);
reg->doRegister(frame_addr, &interpreter);
Value v;
......@@ -373,6 +385,14 @@ Value ASTInterpreter::execute(ASTInterpreter& interpreter, CFGBlock* start_block
return v;
}
Value ASTInterpreter::execute(ASTInterpreter& interpreter, CFGBlock* start_block, AST_stmt* start_at) {
STAT_TIMER(t0, "us_timer_astinterpreter_execute");
RegisterHelper frame_registerer;
return executeInner(interpreter, start_block, start_at, &frame_registerer);
}
Value ASTInterpreter::doBinOp(Box* left, Box* right, int op, BinExpType exp_type) {
switch (exp_type) {
case BinExpType::AugBinOp:
......@@ -610,6 +630,21 @@ Value ASTInterpreter::visit_invoke(AST_Invoke* node) {
v = visit_stmt(node->stmt);
next_block = node->normal_dest;
} catch (ExcInfo e) {
if (cur_thread_state.unwind_why == UNWIND_WHY_NORMAL) {
// when generating the traceback incrementally we only
// include an interpreter frame if we unwind through
// ASTInterpreter::execute_inner. this will keep a toplevel
// invoke from showing up, since we catch the exception
// here.
auto source = getCF()->clfunc->source.get();
BoxedTraceback::Here(LineInfo(node->lineno, node->col_offset, source->fn, source->getName()),
reinterpret_cast<BoxedTraceback**>(&e.traceback));
}
cur_thread_state.unwind_why = UNWIND_WHY_NORMAL;
next_block = node->exc_dest;
last_exception = e;
}
......@@ -1269,7 +1304,7 @@ Value ASTInterpreter::visit_attribute(AST_Attribute* node) {
}
}
const void* interpreter_instr_addr = (void*)&ASTInterpreter::execute;
const void* interpreter_instr_addr = (void*)&ASTInterpreter::executeInner;
Box* astInterpretFunction(CompiledFunction* cf, int nargs, Box* closure, Box* generator, Box* globals, Box* arg1,
Box* arg2, Box* arg3, Box** args) {
......
......@@ -277,6 +277,10 @@ struct PythonFrameId {
uint64_t ip;
uint64_t bp;
PythonFrameId() {}
PythonFrameId(FrameType type, uint64_t ip, uint64_t bp) : type(type), ip(ip), bp(bp) {}
bool operator==(const PythonFrameId& rhs) const { return (this->type == rhs.type) && (this->ip == rhs.ip); }
};
......@@ -289,13 +293,16 @@ public:
intptr_t regs[16];
uint16_t regs_valid;
PythonFrameIteratorImpl(const PythonFrameIteratorImpl&) = delete;
void operator=(const PythonFrameIteratorImpl&) = delete;
PythonFrameIteratorImpl(const PythonFrameIteratorImpl&&) = delete;
void operator=(const PythonFrameIteratorImpl&&) = delete;
// PythonFrameIteratorImpl(const PythonFrameIteratorImpl&) = delete;
// void operator=(const PythonFrameIteratorImpl&) = delete;
// PythonFrameIteratorImpl(const PythonFrameIteratorImpl&&) = delete;
// void operator=(const PythonFrameIteratorImpl&&) = delete;
PythonFrameIteratorImpl() : regs_valid(0) {}
PythonFrameIteratorImpl(PythonFrameId::FrameType type, uint64_t ip, uint64_t bp, CompiledFunction* cf)
: id(PythonFrameId(type, ip, bp)), cf(cf), regs_valid(0) {}
CompiledFunction* getCF() const {
assert(cf);
return cf;
......@@ -411,45 +418,35 @@ static unw_word_t getFunctionEnd(unw_word_t ip) {
return pip.end_ip;
}
// While I'm not a huge fan of the callback-passing style, libunwind cursors are only valid for
// the stack frame that they were created in, so we need to use this approach (as opposed to
// C++11 range loops, for example).
// Return true from the handler to stop iteration at that frame.
void unwindPythonStack(std::function<bool(std::unique_ptr<PythonFrameIteratorImpl>)> func) {
static bool inASTInterpreterExecuteInner(unw_word_t ip) {
static unw_word_t interpreter_instr_end = getFunctionEnd((unw_word_t)interpreter_instr_addr);
static unw_word_t generator_entry_end = getFunctionEnd((unw_word_t)generatorEntry);
unw_context_t ctx;
unw_cursor_t cursor;
unw_getcontext(&ctx);
unw_init_local(&cursor, &ctx);
// If the previous (lower, ie newer) frame was a frame that got OSR'd into,
// we skip the previous frame, its OSR parent.
bool was_osr = false;
while (true) {
int r = unw_step(&cursor);
return ((unw_word_t)interpreter_instr_addr <= ip && ip < interpreter_instr_end);
}
assert(r >= 0);
if (r == 0)
break;
static bool inGeneratorEntry(unw_word_t ip) {
static unw_word_t generator_entry_end = getFunctionEnd((unw_word_t)generatorEntry);
return ((unw_word_t)generatorEntry <= ip && ip < generator_entry_end);
}
unw_word_t ip;
unw_get_reg(&cursor, UNW_REG_IP, &ip);
unw_word_t bp;
unw_get_reg(&cursor, UNW_TDEP_BP, &bp);
static inline unw_word_t get_cursor_reg(unw_cursor_t* cursor, int reg) {
unw_word_t v;
unw_get_reg(cursor, reg, &v);
return v;
}
static inline unw_word_t get_cursor_ip(unw_cursor_t* cursor) {
return get_cursor_reg(cursor, UNW_REG_IP) - 1;
}
static inline unw_word_t get_cursor_bp(unw_cursor_t* cursor) {
return get_cursor_reg(cursor, UNW_TDEP_BP);
}
template <typename FrameFunc>
bool unwindProcessFrame(unw_word_t ip, unw_word_t bp, unw_cursor_t* cursor, FrameFunc func) {
CompiledFunction* cf = getCFForAddress(ip);
if (cf) {
std::unique_ptr<PythonFrameIteratorImpl> info(new PythonFrameIteratorImpl());
info->id.type = PythonFrameId::COMPILED;
info->id.ip = ip;
info->id.bp = bp;
info->cf = cf;
PythonFrameIteratorImpl info(PythonFrameId::COMPILED, ip, bp, cf);
if (!was_osr) {
// Try getting all the callee-save registers, and save the ones we were able to get.
// Some of them may be inaccessible, I think because they weren't defined by that
// stack frame, which can show up as a -UNW_EBADREG return code.
......@@ -457,41 +454,68 @@ void unwindPythonStack(std::function<bool(std::unique_ptr<PythonFrameIteratorImp
if (!assembler::Register::fromDwarf(i).isCalleeSave())
continue;
unw_word_t r;
int code = unw_get_reg(&cursor, i, &r);
int code = unw_get_reg(cursor, i, &r);
ASSERT(code == 0 || code == -UNW_EBADREG, "%d %d", code, i);
if (code == 0) {
info->regs[i] = r;
info->regs_valid |= (1 << i);
info.regs[i] = r;
info.regs_valid |= (1 << i);
}
}
bool stop = func(std::move(info));
if (cur_thread_state.unwind_why == UNWIND_WHY_NORMAL) {
bool stop = func(&info);
if (stop)
break;
return true;
}
was_osr = (bool)cf->entry_descriptor;
continue;
cur_thread_state.unwind_why = (bool)cf->entry_descriptor ? UNWIND_WHY_OSR : UNWIND_WHY_NORMAL;
return false;
}
if ((unw_word_t)interpreter_instr_addr <= ip && ip < interpreter_instr_end) {
std::unique_ptr<PythonFrameIteratorImpl> info(new PythonFrameIteratorImpl());
info->id.type = PythonFrameId::INTERPRETED;
info->id.ip = ip;
info->id.bp = bp;
cf = info->cf = getCFForInterpretedFrame((void*)bp);
if (inASTInterpreterExecuteInner(ip)) {
cf = getCFForInterpretedFrame((void*)bp);
assert(cf);
if (!was_osr) {
bool stop = func(std::move(info));
PythonFrameIteratorImpl info(PythonFrameId::INTERPRETED, ip, bp, cf);
if (cur_thread_state.unwind_why == UNWIND_WHY_NORMAL) {
bool stop = func(&info);
if (stop)
break;
return true;
}
was_osr = (bool)cf->entry_descriptor;
continue;
cur_thread_state.unwind_why = (bool)cf->entry_descriptor ? UNWIND_WHY_OSR : UNWIND_WHY_NORMAL;
return false;
}
if ((unw_word_t)generatorEntry <= ip && ip < generator_entry_end) {
return false;
}
// While I'm not a huge fan of the callback-passing style, libunwind cursors are only valid for
// the stack frame that they were created in, so we need to use this approach (as opposed to
// C++11 range loops, for example).
// Return true from the handler to stop iteration at that frame.
template <typename Func> void unwindPythonStack(Func func) {
cur_thread_state.unwind_why = UNWIND_WHY_NORMAL; // ensure we won't be skipping any python frames at the start
unw_context_t ctx;
unw_cursor_t cursor;
unw_getcontext(&ctx);
unw_init_local(&cursor, &ctx);
while (true) {
int r = unw_step(&cursor);
assert(r >= 0);
if (r == 0)
break;
unw_word_t ip = get_cursor_ip(&cursor);
unw_word_t bp = get_cursor_bp(&cursor);
if (unwindProcessFrame(ip, bp, &cursor, func))
break;
if (inGeneratorEntry(ip)) {
// for generators continue unwinding in the context in which the generator got called
Context* remote_ctx = getReturnContextForGeneratorFrame((void*)bp);
// setup unw_context_t struct from the infos we have, seems like this is enough to make unwinding work.
......@@ -514,21 +538,33 @@ void unwindPythonStack(std::function<bool(std::unique_ptr<PythonFrameIteratorImp
static std::unique_ptr<PythonFrameIteratorImpl> getTopPythonFrame() {
STAT_TIMER(t0, "us_timer_getTopPythonFrame");
std::unique_ptr<PythonFrameIteratorImpl> rtn(nullptr);
unwindPythonStack([&](std::unique_ptr<PythonFrameIteratorImpl> iter) {
rtn = std::move(iter);
unwindPythonStack([&](PythonFrameIteratorImpl* iter) {
rtn = std::unique_ptr<PythonFrameIteratorImpl>(new PythonFrameIteratorImpl(*iter));
return true;
});
return rtn;
}
static const LineInfo* lineInfoForFrame(PythonFrameIteratorImpl& frame_it) {
AST_stmt* current_stmt = frame_it.getCurrentStatement();
auto* cf = frame_it.getCF();
static const LineInfo lineInfoForFrame(PythonFrameIteratorImpl* frame_it) {
AST_stmt* current_stmt = frame_it->getCurrentStatement();
auto* cf = frame_it->getCF();
assert(cf);
auto source = cf->clfunc->source.get();
return new LineInfo(current_stmt->lineno, current_stmt->col_offset, source->fn, source->getName());
return LineInfo(current_stmt->lineno, current_stmt->col_offset, source->fn, source->getName());
}
void maybeTracebackHere(void* unw_cursor, BoxedTraceback** tb) {
unw_cursor_t* cursor = (unw_cursor_t*)unw_cursor;
unw_word_t ip = get_cursor_ip(cursor);
unw_word_t bp = get_cursor_bp(cursor);
unwindProcessFrame(ip, bp, cursor, [&](PythonFrameIteratorImpl* frame_iter) {
BoxedTraceback::Here(lineInfoForFrame(frame_iter), tb);
return false;
});
}
// To produce a traceback, we:
......@@ -584,11 +620,9 @@ BoxedTraceback* getTraceback() {
Timer _t("getTraceback", 1000);
std::vector<const LineInfo*> entries;
unwindPythonStack([&](std::unique_ptr<PythonFrameIteratorImpl> frame_iter) {
const LineInfo* line_info = lineInfoForFrame(*frame_iter.get());
if (line_info)
entries.push_back(line_info);
BoxedTraceback::LinesVector entries;
unwindPythonStack([&](PythonFrameIteratorImpl* frame_iter) {
entries.push_back(lineInfoForFrame(frame_iter));
return false;
});
......@@ -605,7 +639,7 @@ ExcInfo* getFrameExcInfo() {
ExcInfo* copy_from_exc = NULL;
ExcInfo* cur_exc = NULL;
unwindPythonStack([&](std::unique_ptr<PythonFrameIteratorImpl> frame_iter) {
unwindPythonStack([&](PythonFrameIteratorImpl* frame_iter) {
FrameInfo* frame_info = frame_iter->getFrameInfo();
copy_from_exc = &frame_info->exc;
......@@ -665,9 +699,9 @@ BoxedModule* getCurrentModule() {
PythonFrameIterator getPythonFrame(int depth) {
std::unique_ptr<PythonFrameIteratorImpl> rtn(nullptr);
unwindPythonStack([&](std::unique_ptr<PythonFrameIteratorImpl> frame_iter) {
unwindPythonStack([&](PythonFrameIteratorImpl* frame_iter) {
if (depth == 0) {
rtn = std::move(frame_iter);
rtn = std::unique_ptr<PythonFrameIteratorImpl>(new PythonFrameIteratorImpl(*frame_iter));
return true;
}
depth--;
......@@ -697,7 +731,7 @@ PythonFrameIterator::PythonFrameIterator(std::unique_ptr<PythonFrameIteratorImpl
FrameStackState getFrameStackState() {
FrameStackState rtn(NULL, NULL);
bool found = false;
unwindPythonStack([&](std::unique_ptr<PythonFrameIteratorImpl> frame_iter) {
unwindPythonStack([&](PythonFrameIteratorImpl* frame_iter) {
BoxedDict* d;
BoxedClosure* closure;
CompiledFunction* cf;
......@@ -957,9 +991,9 @@ FrameInfo* PythonFrameIterator::getFrameInfo() {
PythonFrameIterator PythonFrameIterator::getCurrentVersion() {
std::unique_ptr<PythonFrameIteratorImpl> rtn(nullptr);
auto& impl = this->impl;
unwindPythonStack([&](std::unique_ptr<PythonFrameIteratorImpl> frame_iter) {
unwindPythonStack([&](PythonFrameIteratorImpl* frame_iter) {
if (frame_iter->pointsToTheSameAs(*impl.get())) {
rtn = std::move(frame_iter);
rtn = std::unique_ptr<PythonFrameIteratorImpl>(new PythonFrameIteratorImpl(*frame_iter));
return true;
}
return false;
......@@ -975,9 +1009,9 @@ PythonFrameIterator PythonFrameIterator::back() {
std::unique_ptr<PythonFrameIteratorImpl> rtn(nullptr);
auto& impl = this->impl;
bool found = false;
unwindPythonStack([&](std::unique_ptr<PythonFrameIteratorImpl> frame_iter) {
unwindPythonStack([&](PythonFrameIteratorImpl* frame_iter) {
if (found) {
rtn = std::move(frame_iter);
rtn = std::unique_ptr<PythonFrameIteratorImpl>(new PythonFrameIteratorImpl(*frame_iter));
return true;
}
......
......@@ -36,6 +36,8 @@ CompiledFunction* getCFForAddress(uint64_t addr);
BoxedTraceback* getTraceback();
void maybeTracebackHere(void* unw_cursor, BoxedTraceback** tb);
struct ExecutionPoint {
CompiledFunction* cf;
AST_stmt* current_stmt;
......
......@@ -38,7 +38,7 @@ namespace threading {
extern "C" {
__thread PyThreadState cur_thread_state
= { 0, NULL, NULL, NULL, NULL }; // not sure if we need to explicitly request zero-initialization
= { 0, NULL, NULL, NULL, NULL, UNWIND_WHY_NORMAL }; // not sure if we need to explicitly request zero-initialization
}
PthreadFastMutex threading_lock;
......
......@@ -666,7 +666,7 @@ void raiseSyntaxErrorHelper(llvm::StringRef file, llvm::StringRef func, AST* nod
struct LineInfo {
public:
const int line, column;
int line, column;
std::string file, func;
LineInfo(int line, int column, llvm::StringRef file, llvm::StringRef func)
......
......@@ -20,6 +20,7 @@
#include "llvm/Support/LEB128.h" // for {U,S}LEB128 decoding
#include "asm_writing/assembler.h" // assembler
#include "codegen/ast_interpreter.h" // interpreter_instr_addr
#include "codegen/unwinding.h" // getCFForAddress
#include "core/ast.h"
......@@ -27,6 +28,7 @@
#include "core/types.h" // for ExcInfo
#include "core/util.h" // Timer
#include "runtime/generator.h" // generatorEntry
#include "runtime/traceback.h" // BoxedTraceback::addLine
#define UNW_LOCAL_ONLY
#include <libunwind.h>
......@@ -101,15 +103,6 @@ thread_local ExcData exception_ferry;
static_assert(offsetof(ExcData, exc) == 0, "wrong offset");
// Timer that auto-logs.
struct LogTimer {
StatCounter& counter;
Timer timer;
LogTimer(const char* desc, StatCounter& ctr, long min_usec = -1) : counter(ctr), timer(desc, min_usec) {}
~LogTimer() { counter.log(timer.end()); }
};
static StatCounter us_unwind_loop("us_unwind_loop");
static StatCounter us_unwind_resume_catch("us_unwind_resume_catch");
static StatCounter us_unwind_cleanup("us_unwind_cleanup");
......@@ -508,16 +501,8 @@ static inline int64_t determine_action(const lsda_info_t* info, const call_site_
RELEASE_ASSERT(0, "action chain exhausted and no cleanup indicated");
}
static inline int step(unw_cursor_t* cp) {
LogTimer t("unw_step", us_unwind_step, 5);
return unw_step(cp);
}
// The stack-unwinding loop.
// TODO: integrate incremental traceback generation into this function
static inline void unwind_loop(const ExcData* exc_data) {
Timer t("unwind_loop", 50);
static inline void unwind_loop(ExcData* exc_data) {
// NB. https://monoinfinito.wordpress.com/series/exception-handling-in-c/ is a very useful resource
// as are http://www.airs.com/blog/archives/460 and http://www.airs.com/blog/archives/464
unw_cursor_t cursor;
......@@ -530,11 +515,10 @@ static inline void unwind_loop(const ExcData* exc_data) {
unw_getcontext(&uc);
unw_init_local(&cursor, &uc);
while (step(&cursor) > 0) {
while (unw_step(&cursor) > 0) {
unw_proc_info_t pip;
{
// NB. unw_get_proc_info is slow; a significant chunk of all time spent unwinding is spent here.
LogTimer t_procinfo("get_proc_info", us_unwind_get_proc_info, 10);
check(unw_get_proc_info(&cursor, &pip));
}
assert((pip.lsda == 0) == (pip.handler == 0));
......@@ -544,6 +528,8 @@ static inline void unwind_loop(const ExcData* exc_data) {
print_frame(&cursor, &pip);
}
maybeTracebackHere(&cursor, reinterpret_cast<BoxedTraceback**>(&exc_data->exc.traceback));
// Skip frames without handlers
if (pip.handler == 0) {
continue;
......@@ -560,8 +546,6 @@ static inline void unwind_loop(const ExcData* exc_data) {
call_site_entry_t entry;
{
LogTimer t_call_site("find_call_site_entry", us_unwind_find_call_site_entry, 10);
// 2. Find our current IP in the call site table.
unw_word_t ip;
unw_get_reg(&cursor, UNW_REG_IP, &ip);
......@@ -592,17 +576,15 @@ static inline void unwind_loop(const ExcData* exc_data) {
}
int64_t switch_value = determine_action(&info, &entry);
us_unwind_loop.log(t.end());
resume(&cursor, entry.landing_pad, switch_value, exc_data);
}
us_unwind_loop.log(t.end());
// Hit end of stack! return & let unwindException determine what to do.
}
// The unwinder entry-point.
static void unwind(const ExcData* exc) {
static void unwind(ExcData* exc) {
exc->check();
unwind_loop(exc);
// unwind_loop returned, couldn't find any handler. ruh-roh.
......@@ -637,7 +619,7 @@ extern "C" void _Unwind_Resume(struct _Unwind_Exception* _exc) {
if (VERBOSITY("cxx_unwind") >= 4)
printf("***** _Unwind_Resume() *****\n");
// we give `_exc' type `struct _Unwind_Exception*' because unwind.h demands it; it's not actually accurate
const pyston::ExcData* data = (const pyston::ExcData*)_exc;
pyston::ExcData* data = (pyston::ExcData*)_exc;
pyston::unwind(data);
}
......@@ -708,7 +690,7 @@ extern "C" void __cxa_throw(void* exc_obj, std::type_info* tinfo, void (*dtor)(v
if (VERBOSITY("cxx_unwind") >= 4)
printf("***** __cxa_throw() *****\n");
const pyston::ExcData* exc_data = (const pyston::ExcData*)exc_obj;
pyston::ExcData* exc_data = (pyston::ExcData*)exc_obj;
exc_data->check();
pyston::unwind(exc_data);
}
......
......@@ -76,7 +76,7 @@ void raiseRaw(const ExcInfo& e) {
}
void raiseExc(Box* exc_obj) {
raiseRaw(ExcInfo(exc_obj->cls, exc_obj, getTraceback()));
raiseRaw(ExcInfo(exc_obj->cls, exc_obj, new BoxedTraceback()));
}
// Have a special helper function for syntax errors, since we want to include the location
......@@ -84,10 +84,9 @@ void raiseExc(Box* exc_obj) {
void raiseSyntaxError(const char* msg, int lineno, int col_offset, llvm::StringRef file, llvm::StringRef func) {
Box* exc = runtimeCall(SyntaxError, ArgPassSpec(1), boxString(msg), NULL, NULL, NULL, NULL);
auto tb = getTraceback();
std::vector<const LineInfo*> entries = tb->lines;
entries.push_back(new LineInfo(lineno, col_offset, file, func));
raiseRaw(ExcInfo(exc->cls, exc, new BoxedTraceback(std::move(entries))));
auto tb = new BoxedTraceback();
tb->addLine(LineInfo(lineno, col_offset, file, func));
raiseRaw(ExcInfo(exc->cls, exc, tb));
}
void raiseSyntaxErrorHelper(llvm::StringRef file, llvm::StringRef func, AST* node_at, const char* msg, ...) {
......@@ -205,6 +204,7 @@ extern "C" void raise0() {
if (exc_info->type == None)
raiseExcHelper(TypeError, "exceptions must be old-style classes or derived from BaseException, not NoneType");
cur_thread_state.unwind_why = UNWIND_WHY_RERAISE;
raiseRaw(*exc_info);
}
......@@ -229,8 +229,12 @@ ExcInfo excInfoForRaise(Box* type, Box* value, Box* tb) {
assert(type && value && tb); // use None for default behavior, not nullptr
// TODO switch this to PyErr_Normalize
if (tb == None)
tb = getTraceback();
if (tb == None) {
tb = NULL;
} else if (tb != NULL && !PyTraceBack_Check(tb)) {
raiseExcHelper(TypeError, "raise: arg 3 must be a traceback or None");
}
/* Next, repeatedly, replace a tuple exception with its first item */
while (PyTuple_Check(type) && PyTuple_Size(type) > 0) {
......@@ -242,6 +246,7 @@ ExcInfo excInfoForRaise(Box* type, Box* value, Box* tb) {
if (PyExceptionClass_Check(type)) {
PyErr_NormalizeException(&type, &value, &tb);
if (!PyExceptionInstance_Check(value)) {
raiseExcHelper(TypeError, "calling %s() should have returned an instance of "
"BaseException, not '%s'",
......@@ -268,11 +273,18 @@ ExcInfo excInfoForRaise(Box* type, Box* value, Box* tb) {
assert(PyExceptionClass_Check(type));
if (tb == NULL) {
tb = new BoxedTraceback();
}
return ExcInfo(type, value, tb);
}
extern "C" void raise3(Box* arg0, Box* arg1, Box* arg2) {
raiseRaw(excInfoForRaise(arg0, arg1, arg2));
bool reraise = arg2 != NULL && arg2 != None;
auto exc_info = excInfoForRaise(arg0, arg1, arg2);
cur_thread_state.unwind_why = reraise ? UNWIND_WHY_RERAISE : UNWIND_WHY_NORMAL;
raiseRaw(exc_info);
}
void raiseExcHelper(BoxedClass* cls, Box* arg) {
......
......@@ -24,6 +24,7 @@
#include "core/stats.h"
#include "core/types.h"
#include "gc/collector.h"
#include "runtime/list.h"
#include "runtime/objmodel.h"
#include "runtime/types.h"
#include "runtime/util.h"
......@@ -40,6 +41,8 @@ void BoxedTraceback::gcHandler(GCVisitor* v, Box* b) {
if (self->py_lines)
v->visit(self->py_lines);
if (self->tb_next)
v->visit(self->tb_next);
boxGCHandler(v, b);
}
......@@ -53,16 +56,17 @@ void printTraceback(Box* b) {
fprintf(stderr, "Traceback (most recent call last):\n");
for (; tb && tb != None; tb = static_cast<BoxedTraceback*>(tb->tb_next)) {
for (auto line : tb->lines) {
fprintf(stderr, " File \"%s\", line %d, in %s:\n", line->file.c_str(), line->line, line->func.c_str());
fprintf(stderr, " File \"%s\", line %d, in %s:\n", line.file.c_str(), line.line, line.func.c_str());
if (line->line < 0)
if (line.line < 0)
continue;
FILE* f = fopen(line->file.c_str(), "r");
FILE* f = fopen(line.file.c_str(), "r");
if (f) {
assert(line->line < 10000000 && "Refusing to try to seek that many lines forward");
for (int i = 1; i < line->line; i++) {
assert(line.line < 10000000 && "Refusing to try to seek that many lines forward");
for (int i = 1; i < line.line; i++) {
char* buf = NULL;
size_t size;
size_t r = getline(&buf, &size, f);
......@@ -88,6 +92,11 @@ void printTraceback(Box* b) {
fclose(f);
}
}
}
}
void BoxedTraceback::addLine(const LineInfo line) {
lines.insert(lines.begin(), line);
}
Box* BoxedTraceback::getLines(Box* b) {
......@@ -97,17 +106,24 @@ Box* BoxedTraceback::getLines(Box* b) {
if (!tb->py_lines) {
BoxedList* lines = new BoxedList();
lines->ensure(tb->lines.size());
for (auto line : tb->lines) {
auto l = BoxedTuple::create({ boxString(line->file), boxString(line->func), boxInt(line->line) });
for (BoxedTraceback* wtb = tb; wtb && wtb != None; wtb = static_cast<BoxedTraceback*>(wtb->tb_next)) {
lines->ensure(wtb->lines.size());
for (auto& line : wtb->lines) {
auto l = BoxedTuple::create({ boxString(line.file), boxString(line.func), boxInt(line.line) });
listAppendInternal(lines, l);
}
}
tb->py_lines = lines;
}
return tb->py_lines;
}
void BoxedTraceback::Here(LineInfo lineInfo, BoxedTraceback** tb) {
*tb = new BoxedTraceback(*tb);
(*tb)->addLine(lineInfo);
}
void setupTraceback() {
traceback_cls = BoxedHeapClass::create(type_cls, object_cls, BoxedTraceback::gcHandler, 0, 0,
sizeof(BoxedTraceback), false, "traceback");
......
......@@ -27,19 +27,25 @@ class GCVisitor;
extern "C" BoxedClass* traceback_cls;
class BoxedTraceback : public Box {
public:
std::vector<const LineInfo*> lines;
typedef llvm::SmallVector<LineInfo, 1> LinesVector;
Box* tb_next;
LinesVector lines;
Box* py_lines;
BoxedTraceback(std::vector<const LineInfo*> lines) : lines(std::move(lines)), py_lines(NULL) {}
BoxedTraceback() : py_lines(NULL) {}
BoxedTraceback(LinesVector&& lines) : tb_next(None), lines(std::move(lines)), py_lines(NULL) {}
BoxedTraceback(BoxedTraceback* tb_next) : tb_next(tb_next), py_lines(NULL) {}
BoxedTraceback() : tb_next(None), py_lines(NULL) {}
DEFAULT_CLASS(traceback_cls);
void addLine(const LineInfo* line);
void addLine(const LineInfo line);
static Box* getLines(Box* b);
static void gcHandler(gc::GCVisitor* v, Box* b);
// somewhat equivalent to PyTraceBack_Here
static void Here(LineInfo lineInfo, BoxedTraceback** tb);
};
void printTraceback(Box* b);
......
import traceback
import sys
def f():
a, b, c = sys.exc_info()
raise a, b, c
et0, ev0, tb0 = None, None, None
try:
1/0
except:
pass
for i in xrange(10):
try:
f()
except:
et0, ev0, tb0 = sys.exc_info()
print "******** 0", ''.join(traceback.format_exception(et0, ev0, tb0))
et1, ev1, tb1 = None, None, None
et2, ev2, tb2 = None, None, None
def f1():
raise
def f2():
f1()
def f21():
raise Exception()
def f3():
try:
f21()
except:
global et1, tv1, tb1
et1, tv1, tb1 = sys.exc_info()
f2()
try:
f3()
except:
et2, tv2, tb2 = sys.exc_info()
print "******** 1", ''.join(traceback.format_exception(et1, ev1, tb1))
print "******** 2", ''.join(traceback.format_exception(et2, ev2, tb2))
print et1 is et2
print ev1 is ev2
print tb1 is tb2
# expected: fail
# - we don't stop tracebacks at the catching except handler. this is hard do the way it gets added to
# (ie a bare "raise" statement will add more traceback entries to the traceback it raises)
import sys
import traceback
......
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment