1
0
mirror of https://github.com/mariadb-corporation/mariadb-columnstore-engine.git synced 2025-07-30 19:23:07 +03:00

MCOL-4368 Unified exceptions handling code in dbcon/joblist

This commit is contained in:
Roman Nozdrin
2020-10-19 16:49:52 +00:00
parent 92d861d980
commit cd48df99e5
20 changed files with 296 additions and 475 deletions

View File

@ -615,18 +615,12 @@ void CrossEngineStep::execute()
fOutputDL->insert(rgDataDelivered); fOutputDL->insert(rgDataDelivered);
fRowsRetrieved = mysql->getRowCount(); fRowsRetrieved = mysql->getRowCount();
} }
catch (IDBExcept& iex)
{
catchHandler(iex.what(), iex.errorCode(), fErrorInfo, fSessionId);
}
catch (const std::exception& ex)
{
catchHandler(ex.what(), ERR_CROSS_ENGINE_CONNECT, fErrorInfo, fSessionId);
}
catch (...) catch (...)
{ {
catchHandler("CrossEngineStep execute caught an unknown exception", handleException(std::current_exception(),
ERR_CROSS_ENGINE_CONNECT, fErrorInfo, fSessionId); logging::ERR_CROSS_ENGINE_CONNECT,
logging::ERR_ALWAYS_CRITICAL,
"CrossEngineStep::execute()");
} }
sts.msg_type = StepTeleStats::ST_SUMMARY; sts.msg_type = StepTeleStats::ST_SUMMARY;
@ -792,20 +786,12 @@ uint32_t CrossEngineStep::nextBand(messageqcpp::ByteStream& bs)
fEndOfResult = true; fEndOfResult = true;
} }
} }
catch (const std::exception& ex)
{
catchHandler(ex.what(), ERR_IN_DELIVERY, fErrorInfo, fSessionId);
while (more)
more = fOutputDL->next(fOutputIterator, &rgDataOut);
fEndOfResult = true;
}
catch (...) catch (...)
{ {
catchHandler("CrossEngineStep next band caught an unknown exception", handleException(std::current_exception(),
ERR_IN_DELIVERY, fErrorInfo, fSessionId); logging::ERR_IN_DELIVERY,
logging::ERR_ALWAYS_CRITICAL,
"CrossEngineStep::nextBand()");
while (more) while (more)
more = fOutputDL->next(fOutputIterator, &rgDataOut); more = fOutputDL->next(fOutputIterator, &rgDataOut);

View File

@ -25,34 +25,6 @@ using namespace rowgroup;
using namespace joiner; using namespace joiner;
using namespace logging; using namespace logging;
// a couple space-savers...
#define LOG(x) {\
logging::Logger logger(16); \
LoggingID id(16, fSessionId); \
logger.logMessage(LOG_TYPE_ERROR, x, id); \
}
#define CATCH_AND_LOG \
catch(IDBExcept &e) { \
if (!status()) { \
errorMessage(e.what()); \
status(e.errorCode()); \
LOG(string(e.what())); \
} \
abort(); \
} \
catch(exception &e) { \
if (!status()) { \
ostringstream os; \
os << "Disk join caught an error: " << e.what(); \
errorMessage(os.str().c_str()); \
LOG(os.str()); \
status(ERR_DBJ_UNKNOWN_ERROR); \
} \
abort(); \
}
namespace joblist namespace joblist
{ {
@ -231,10 +203,17 @@ void DiskJoinStep::smallReader()
abort(); abort();
} }
} }
} // try
catch (...)
{
handleException(std::current_exception(),
logging::ERR_EXEMGR_MALFUNCTION,
logging::ERR_ALWAYS_CRITICAL,
"DiskJoinStep::smallReader()");
status(logging::ERR_EXEMGR_MALFUNCTION);
abort();
} }
CATCH_AND_LOG;
while (more) while (more)
more = smallDL->next(0, &rgData); more = smallDL->next(0, &rgData);
} }
@ -272,8 +251,15 @@ void DiskJoinStep::largeReader()
if (!more) if (!more)
lastLargeIteration = true; lastLargeIteration = true;
} }
catch (...)
CATCH_AND_LOG; {
handleException(std::current_exception(),
logging::ERR_EXEMGR_MALFUNCTION,
logging::ERR_ALWAYS_CRITICAL,
"DiskJoinStep::largeReader()");
status(logging::ERR_EXEMGR_MALFUNCTION);
abort();
}
if (cancelled()) if (cancelled())
while (more) while (more)
@ -301,8 +287,15 @@ void DiskJoinStep::loadFcn()
} }
while (ret && !cancelled()); while (ret && !cancelled());
} }
catch (...)
CATCH_AND_LOG; {
handleException(std::current_exception(),
logging::ERR_EXEMGR_MALFUNCTION,
logging::ERR_ALWAYS_CRITICAL,
"DiskJoinStep::loadFcn()");
status(logging::ERR_EXEMGR_MALFUNCTION);
abort();
}
loadFIFO->endOfInput(); loadFIFO->endOfInput();
} }
@ -509,8 +502,15 @@ void DiskJoinStep::joinFcn()
} }
} }
} // the try stmt above; need to reformat. } // the try stmt above; need to reformat.
catch (...)
CATCH_AND_LOG; {
handleException(std::current_exception(),
logging::ERR_EXEMGR_MALFUNCTION,
logging::ERR_ALWAYS_CRITICAL,
"DiskJoinStep::joinFcn()");
status(logging::ERR_EXEMGR_MALFUNCTION);
abort();
}
out: out:
@ -559,8 +559,15 @@ void DiskJoinStep::mainRunner()
jobstepThreadPool.join(thrds); jobstepThreadPool.join(thrds);
} }
} }
catch (...)
CATCH_AND_LOG; {
handleException(std::current_exception(),
logging::ERR_EXEMGR_MALFUNCTION,
logging::ERR_ALWAYS_CRITICAL,
"DiskJoinStep::mainRunner()");
status(logging::ERR_EXEMGR_MALFUNCTION);
abort();
}
// make sure all inputs were drained & output closed // make sure all inputs were drained & output closed
if (cancelled()) if (cancelled())

View File

@ -58,7 +58,10 @@ Logger::Logger() : fLogId(5),
fImpl->msgMap(msgMap); fImpl->msgMap(msgMap);
} }
void catchHandler(const string& ex, int c, SErrorInfo& ei, unsigned sid, logging::LOG_TYPE level) void catchHandler(const string& ex,
int c, SErrorInfo& ei,
unsigned sid,
logging::LOG_TYPE level)
{ {
boost::mutex::scoped_lock lk(logMutex); boost::mutex::scoped_lock lk(logMutex);

View File

@ -1,4 +1,5 @@
/* Copyright (C) 2014 InfiniDB, Inc. /* Copyright (C) 2014 InfiniDB, Inc.
Copyright (c) 2016-2020 MariaDB
This program is free software; you can redistribute it and/or This program is free software; you can redistribute it and/or
modify it under the terms of the GNU General Public License modify it under the terms of the GNU General Public License
@ -209,6 +210,60 @@ bool JobStep::traceOn() const
return fTraceFlags & execplan::CalpontSelectExecutionPlan::TRACE_LOG; return fTraceFlags & execplan::CalpontSelectExecutionPlan::TRACE_LOG;
} }
//////////////////////////////////////////////////////////////////////
// DESCRIPTION:
// The m() rethrows a query runtime exception and handles it across
// all steps in a uniform
// way.
// PARAMETERS:
// e ptr to the exception raised
// errorCode error code to log
// critErrorCode is this a crit IDBExcept or not
// methodName method name to log
//////////////////////////////////////////////////////////////////////
void JobStep::handleException(std::exception_ptr e,
const int errorCode,
const unsigned infoErrorCode,
const std::string& methodName)
{
try
{
std::rethrow_exception(e);
}
catch (const IDBExcept& iex)
{
std::cerr << methodName << " caught a internal exception. "
<< std::endl;
catchHandler(methodName + " " + iex.what(), iex.errorCode(),
fErrorInfo, fSessionId, (iex.errorCode() == infoErrorCode
? LOG_TYPE_INFO
: LOG_TYPE_CRITICAL));
}
catch (boost::exception& e)
{
std::cerr << methodName << " caught a boost::exception. "
<< std::endl;
catchHandler(methodName + " caught " + boost::diagnostic_information(e),
errorCode, fErrorInfo, fSessionId);
}
catch (const std::exception& ex)
{
std::cerr << methodName << " caught an exception. " << std::endl;
catchHandler(methodName + " caught " + ex.what(), errorCode,
fErrorInfo, fSessionId);
}
catch (...)
{
std::ostringstream oss;
std::cerr << methodName << " caught an unknown exception."
<< std::endl;
catchHandler(methodName + " caught an unknown exception ",
errorCode, fErrorInfo, fSessionId);
}
}
} //namespace joblist } //namespace joblist
// vim:ts=4 sw=4: // vim:ts=4 sw=4:

View File

@ -430,6 +430,11 @@ public:
return fTimeZone; return fTimeZone;
} }
void handleException(std::exception_ptr e,
const int errorCode,
const unsigned infoErrorCode,
const std::string& methodName);
static threadpool::ThreadPool jobstepThreadPool; static threadpool::ThreadPool jobstepThreadPool;
protected: protected:

View File

@ -1,4 +1,5 @@
/* Copyright (C) 2014 InfiniDB, Inc. /* Copyright (C) 2014 InfiniDB, Inc.
Copyright (c) 2016-2020 MariaDB
This program is free software; you can redistribute it and/or This program is free software; you can redistribute it and/or
modify it under the terms of the GNU General Public License modify it under the terms of the GNU General Public License
@ -533,7 +534,7 @@ const string pDictionaryStep::toString() const
#ifdef FIFO_SINK #ifdef FIFO_SINK
if (fOid < 3000)) if (fOid < 3000)
oss << " (sink)"; oss << " (sink)";
#endif #endif

View File

@ -1,4 +1,5 @@
/* Copyright (C) 2014 InfiniDB, Inc. /* Copyright (C) 2014 InfiniDB, Inc.
Copyright (c) 2016-2020 MariaDB
This program is free software; you can redistribute it and/or This program is free software; you can redistribute it and/or
modify it under the terms of the GNU General Public License modify it under the terms of the GNU General Public License
@ -426,15 +427,12 @@ void pDictionaryScan::sendPrimitiveMessages()
} }
} // end of loop through LBID ranges to be requested from primproc } // end of loop through LBID ranges to be requested from primproc
}//try }//try
catch (const exception& e)
{
catchHandler(e.what(), ERR_DICTIONARY_SCAN, fErrorInfo, fSessionId);
sendError(ERR_DICTIONARY_SCAN);
}
catch (...) catch (...)
{ {
catchHandler("pDictionaryScan caught an unknown exception.", handleException(std::current_exception(),
ERR_DICTIONARY_SCAN, fErrorInfo, fSessionId); logging::ERR_DICTIONARY_SCAN,
logging::ERR_ALWAYS_CRITICAL,
"pDictionaryScan::sendPrimitiveMessages()");
sendError(ERR_DICTIONARY_SCAN); sendError(ERR_DICTIONARY_SCAN);
} }
@ -547,24 +545,14 @@ void pDictionaryScan::sendAPrimitiveMessage(
{ {
fDec->write(uniqueID, primMsg); fDec->write(uniqueID, primMsg);
} }
catch (const IDBExcept& e)
{
abort();
cerr << "pDictionaryScan::send() caught: " << e.what() << endl;
catchHandler(e.what(), e.errorCode(), fErrorInfo, fSessionId);
}
catch (const std::exception& e)
{
abort();
cerr << "pDictionaryScan::send() caught: " << e.what() << endl;
catchHandler(e.what(), ERR_DICTIONARY_SCAN, fErrorInfo, fSessionId);
}
catch (...) catch (...)
{ {
abort(); abort();
cerr << "pDictionaryScan::send() caught unknown exception" << endl; handleException(std::current_exception(),
catchHandler("pDictionaryScan::send() caught unknown exception", logging::ERR_DICTIONARY_SCAN,
ERR_DICTIONARY_SCAN, fErrorInfo, fSessionId); logging::ERR_ALWAYS_CRITICAL,
"pDictionaryScan::sendAPrimitiveMessage()");
sendError(ERR_DICTIONARY_SCAN);
} }
fMsgsToPm++; fMsgsToPm++;
@ -757,15 +745,12 @@ void pDictionaryScan::receivePrimitiveMessages()
catchHandler(ex.what(), ERR_DICTIONARY_SCAN, fErrorInfo, fSessionId); catchHandler(ex.what(), ERR_DICTIONARY_SCAN, fErrorInfo, fSessionId);
mutex.unlock(); mutex.unlock();
} }
catch (const exception& e)
{
catchHandler(e.what(), ERR_DICTIONARY_SCAN, fErrorInfo, fSessionId);
mutex.unlock();
}
catch (...) catch (...)
{ {
catchHandler("pDictionaryScan caught an unknown exception.", handleException(std::current_exception(),
ERR_DICTIONARY_SCAN, fErrorInfo, fSessionId); logging::ERR_DICTIONARY_SCAN,
logging::ERR_ALWAYS_CRITICAL,
"pDictionaryScan::receivePrimitiveMessages()");
mutex.unlock(); mutex.unlock();
} }
@ -927,24 +912,13 @@ void pDictionaryScan::serializeEqualityFilter()
{ {
fDec->write(uniqueID, msg); fDec->write(uniqueID, msg);
} }
catch (const IDBExcept& e)
{
abort();
cerr << "pDictionaryScan::serializeEqualityFilter() caught: " << e.what() << endl;
catchHandler(e.what(), e.errorCode(), fErrorInfo, fSessionId);
}
catch (const std::exception& e)
{
abort();
cerr << "pDictionaryScan::serializeEqualityFilter() caught: " << e.what() << endl;
catchHandler(e.what(), ERR_DICTIONARY_SCAN, fErrorInfo, fSessionId);
}
catch (...) catch (...)
{ {
abort(); abort();
cerr << "pDictionaryScan::serializeEqualityFilter() caught unknown exception" << endl; handleException(std::current_exception(),
catchHandler("pDictionaryScan::serializeEqualityFilter() caught unknown exception", logging::ERR_DICTIONARY_SCAN,
ERR_DICTIONARY_SCAN, fErrorInfo, fSessionId); logging::ERR_ALWAYS_CRITICAL,
"pDictionaryScan::serializeEqualityFilter()");
} }
empty.swap(equalityFilter); empty.swap(equalityFilter);
@ -965,23 +939,13 @@ void pDictionaryScan::destroyEqualityFilter()
{ {
fDec->write(uniqueID, msg); fDec->write(uniqueID, msg);
} }
catch (const IDBExcept& e)
{
abort();
catchHandler(e.what(), e.errorCode(), fErrorInfo, fSessionId);
}
catch (const std::exception& e)
{
abort();
cerr << "pDictionaryScan::destroyEqualityFilter() caught: " << e.what() << endl;
catchHandler(e.what(), ERR_DICTIONARY_SCAN, fErrorInfo, fSessionId);
}
catch (...) catch (...)
{ {
abort(); abort();
cerr << "pDictionaryScan::destroyEqualityFilter() caught unknown exception" << endl; handleException(std::current_exception(),
catchHandler("pDictionaryScan::destroyEqualityFilter() caught unknown exception", logging::ERR_DICTIONARY_SCAN,
ERR_DICTIONARY_SCAN, fErrorInfo, fSessionId); logging::ERR_ALWAYS_CRITICAL,
"pDictionaryScan::destroyEqualityFilter()");
} }
} }

View File

@ -1343,7 +1343,6 @@ public:
protected: protected:
void sendError(uint16_t status); void sendError(uint16_t status);
void processError(const std::string& ex, uint16_t err, const std::string& src);
private: private:
void formatMiniStats(); void formatMiniStats();

View File

@ -1,4 +1,5 @@
/* Copyright (C) 2014 InfiniDB, Inc. /* Copyright (C) 2014 InfiniDB, Inc.
Copyright (c) 2016-2020 MariaDB
This program is free software; you can redistribute it and/or This program is free software; you can redistribute it and/or
modify it under the terms of the GNU General Public License modify it under the terms of the GNU General Public License
@ -235,23 +236,14 @@ uint32_t SubAdapterStep::nextBand(messageqcpp::ByteStream& bs)
rowCount = fRowGroupDeliver.getRowCount(); rowCount = fRowGroupDeliver.getRowCount();
} }
} }
catch (const std::exception& ex)
{
catchHandler(ex.what(), ERR_IN_DELIVERY, fErrorInfo, fSessionId);
while (more)
more = fOutputDL->next(fOutputIterator, &rgDataOut);
fEndOfResult = true;
}
catch (...) catch (...)
{ {
catchHandler("SubAdapterStep next band caught an unknown exception", handleException(std::current_exception(),
ERR_IN_DELIVERY, fErrorInfo, fSessionId); logging::ERR_IN_DELIVERY,
logging::ERR_ALWAYS_CRITICAL,
"SubAdapterStep::nextBand()");
while (more) while (more)
more = fOutputDL->next(fOutputIterator, &rgDataOut); more = fOutputDL->next(fOutputIterator, &rgDataOut);
fEndOfResult = true; fEndOfResult = true;
} }
@ -441,14 +433,12 @@ void SubAdapterStep::execute()
more = fInputDL->next(fInputIterator, &rgDataIn); more = fInputDL->next(fInputIterator, &rgDataIn);
} }
} }
catch (const std::exception& ex)
{
catchHandler(ex.what(), ERR_EXEMGR_MALFUNCTION, fErrorInfo, fSessionId);
}
catch (...) catch (...)
{ {
catchHandler("SubAdapterStep execute caught an unknown exception", handleException(std::current_exception(),
ERR_EXEMGR_MALFUNCTION, fErrorInfo, fSessionId); logging::ERR_EXEMGR_MALFUNCTION,
logging::ERR_ALWAYS_CRITICAL,
"SubAdapterStep::execute()");
} }
if (cancelled()) if (cancelled())

View File

@ -1,4 +1,5 @@
/* Copyright (C) 2014 InfiniDB, Inc. /* Copyright (C) 2014 InfiniDB, Inc.
Copyright (C) 2019-2020 MariaDB Corporation
This program is free software; you can redistribute it and/or This program is free software; you can redistribute it and/or
modify it under the terms of the GNU General Public License modify it under the terms of the GNU General Public License
@ -1204,18 +1205,12 @@ void TupleBPS::run()
fProducerThreads.reserve(fMaxNumThreads); fProducerThreads.reserve(fMaxNumThreads);
startAggregationThread(); startAggregationThread();
} }
catch (const std::exception& e)
{
// log the exception
cerr << "tuple-bps::run() caught: " << e.what() << endl;
catchHandler(e.what(), ERR_TUPLE_BPS, fErrorInfo, fSessionId);
fOutputJobStepAssociation.outAt(0)->rowGroupDL()->endOfInput();
}
catch (...) catch (...)
{ {
cerr << "tuple-bps::run() caught unknown exception" << endl; handleException(std::current_exception(),
catchHandler("tuple-bps::run() caught unknown exception", logging::ERR_TUPLE_BPS,
ERR_TUPLE_BPS, fErrorInfo, fSessionId); logging::ERR_ALWAYS_CRITICAL,
"TupleBPS::run()");
fOutputJobStepAssociation.outAt(0)->rowGroupDL()->endOfInput(); fOutputJobStepAssociation.outAt(0)->rowGroupDL()->endOfInput();
} }
} }
@ -1254,17 +1249,12 @@ void TupleBPS::join()
{ {
fDec->write(uniqueID, bs); fDec->write(uniqueID, bs);
} }
catch (const std::exception& e)
{
// log the exception
cerr << "tuple-bps::join() write(bs) caught: " << e.what() << endl;
catchHandler(e.what(), ERR_TUPLE_BPS, fErrorInfo, fSessionId);
}
catch (...) catch (...)
{ {
cerr << "tuple-bps::join() write(bs) caught unknown exception" << endl; handleException(std::current_exception(),
catchHandler("tuple-bps::join() write(bs) caught unknown exception", logging::ERR_TUPLE_BPS,
ERR_TUPLE_BPS, fErrorInfo, fSessionId); logging::ERR_ALWAYS_CRITICAL,
"TupleBPS::join()");
} }
BPPIsAllocated = false; BPPIsAllocated = false;
@ -1848,20 +1838,15 @@ void TupleBPS::sendPrimitiveMessages()
interleaveJobs(&jobs); interleaveJobs(&jobs);
sendJobs(jobs); sendJobs(jobs);
} }
catch (const IDBExcept& e)
{
sendError(e.errorCode());
processError(e.what(), e.errorCode(), "TupleBPS::sendPrimitiveMessages()");
}
catch (const std::exception& ex)
{
sendError(ERR_TUPLE_BPS);
processError(ex.what(), ERR_TUPLE_BPS, "TupleBPS::sendPrimitiveMessages()");
}
catch (...) catch (...)
{ {
sendError(ERR_TUPLE_BPS); sendError(logging::ERR_TUPLE_BPS);
processError("unknown", ERR_TUPLE_BPS, "TupleBPS::sendPrimitiveMessages()"); handleException(std::current_exception(),
logging::ERR_TUPLE_BPS,
logging::ERR_ALWAYS_CRITICAL,
"st: " + std::to_string(fStepId) +
" TupleBPS::sendPrimitiveMessages()");
abort_nolock();
} }
abort: abort:
@ -2389,13 +2374,14 @@ void TupleBPS::receiveMultiPrimitiveMessages(uint32_t threadID)
} // done reading } // done reading
}//try }//try
catch (const std::exception& ex)
{
processError(ex.what(), ERR_TUPLE_BPS, "TupleBPS::receiveMultiPrimitiveMessages()");
}
catch (...) catch (...)
{ {
processError("unknown", ERR_TUPLE_BPS, "TupleBPS::receiveMultiPrimitiveMessages()"); handleException(std::current_exception(),
logging::ERR_TUPLE_BPS,
logging::ERR_ALWAYS_CRITICAL,
"st: " + std::to_string(fStepId) +
" TupleBPS::receiveMultiPrimitiveMessages()");
abort_nolock();
} }
out: out:
@ -2643,15 +2629,6 @@ out:
dlp->endOfInput(); dlp->endOfInput();
} }
void TupleBPS::processError(const string& ex, uint16_t err, const string& src)
{
ostringstream oss;
oss << "st: " << fStepId << " " << src << " caught an exception: " << ex << endl;
catchHandler(oss.str(), err, fErrorInfo, fSessionId);
abort_nolock();
cerr << oss.str();
}
const string TupleBPS::toString() const const string TupleBPS::toString() const
{ {
ostringstream oss; ostringstream oss;

View File

@ -1,5 +1,5 @@
/* Copyright (C) 2014 InfiniDB, Inc. /* Copyright (C) 2014 InfiniDB, Inc.
Copyright (C) 2019 MariaDB Corporation Copyright (C) 2019-2020 MariaDB Corporation
This program is free software; you can redistribute it and/or This program is free software; you can redistribute it and/or
modify it under the terms of the GNU General Public License modify it under the terms of the GNU General Public License
@ -449,7 +449,6 @@ void TupleAggregateStep::doThreadedSecondPhaseAggregate(uint32_t threadID)
scoped_array<RowBucketVec> rowBucketVecs(new RowBucketVec[fNumOfBuckets]); scoped_array<RowBucketVec> rowBucketVecs(new RowBucketVec[fNumOfBuckets]);
scoped_array<bool> bucketDone(new bool[fNumOfBuckets]); scoped_array<bool> bucketDone(new bool[fNumOfBuckets]);
uint32_t hashlen = fAggregator->aggMapKeyLength(); uint32_t hashlen = fAggregator->aggMapKeyLength();
bool caughtException = false;
try try
{ {
@ -561,26 +560,15 @@ void TupleAggregateStep::doThreadedSecondPhaseAggregate(uint32_t threadID)
} }
} // try } // try
catch (IDBExcept& iex)
{
catchHandler(iex.what(), iex.errorCode(), fErrorInfo, fSessionId,
(iex.errorCode() == ERR_AGGREGATION_TOO_BIG ? LOG_TYPE_INFO : LOG_TYPE_CRITICAL));
caughtException = true;
}
catch (const std::exception& ex)
{
catchHandler(ex.what(), tupleAggregateStepErr, fErrorInfo, fSessionId);
caughtException = true;
}
catch (...) catch (...)
{ {
catchHandler("doThreadedSecondPhaseAggregate() caught an unknown exception", handleException(std::current_exception(),
tupleAggregateStepErr, fErrorInfo, fSessionId); logging::tupleAggregateStepErr,
caughtException = true; logging::ERR_AGGREGATION_TOO_BIG,
"TupleAggregateStep::doThreadedSecondPhaseAggregate()");
fEndOfResult = true;
} }
if (caughtException)
fEndOfResult = true;
fDoneAggregate = true; fDoneAggregate = true;
@ -630,24 +618,12 @@ uint32_t TupleAggregateStep::nextBand_singleThread(messageqcpp::ByteStream& bs)
} }
} }
} // try } // try
catch (IDBExcept& iex)
{
catchHandler(iex.what(), iex.errorCode(), fErrorInfo, fSessionId,
(iex.errorCode() == ERR_AGGREGATION_TOO_BIG ? LOG_TYPE_INFO : LOG_TYPE_CRITICAL));
fEndOfResult = true;
}
catch (const std::exception& ex)
{
catchHandler(ex.what(), tupleAggregateStepErr, fErrorInfo, fSessionId);
fEndOfResult = true;
}
catch (...) catch (...)
{ {
catchHandler("TupleAggregateStep next band caught an unknown exception", handleException(std::current_exception(),
tupleAggregateStepErr, fErrorInfo, fSessionId); logging::tupleAggregateStepErr,
logging::ERR_AGGREGATION_TOO_BIG,
"TupleAggregateStep::doThreadedSecondPhaseAggregate()");
fEndOfResult = true; fEndOfResult = true;
} }
@ -5194,21 +5170,12 @@ void TupleAggregateStep::aggregateRowGroups()
} }
} }
} // try } // try
catch (IDBExcept& iex)
{
catchHandler(iex.what(), iex.errorCode(), fErrorInfo, fSessionId,
(iex.errorCode() == ERR_AGGREGATION_TOO_BIG ? LOG_TYPE_INFO : LOG_TYPE_CRITICAL));
fEndOfResult = true;
}
catch (const std::exception& ex)
{
catchHandler(ex.what(), tupleAggregateStepErr, fErrorInfo, fSessionId);
fEndOfResult = true;
}
catch (...) catch (...)
{ {
catchHandler("TupleAggregateStep::aggregateRowGroups() caught an unknown exception", handleException(std::current_exception(),
tupleAggregateStepErr, fErrorInfo, fSessionId); logging::tupleAggregateStepErr,
logging::ERR_AGGREGATION_TOO_BIG,
"TupleAggregateStep::aggregateRowGroups()");
fEndOfResult = true; fEndOfResult = true;
} }
} }
@ -5484,31 +5451,17 @@ void TupleAggregateStep::threadedAggregateRowGroups(uint32_t threadID)
} }
} }
} // try } // try
catch (IDBExcept& iex)
{
catchHandler(iex.what(), iex.errorCode(), fErrorInfo, fSessionId,
(iex.errorCode() == ERR_AGGREGATION_TOO_BIG ? LOG_TYPE_INFO : LOG_TYPE_CRITICAL));
caughtException = true;
}
catch (const std::exception& ex)
{
catchHandler(ex.what(), tupleAggregateStepErr, fErrorInfo, fSessionId);
caughtException = true;
}
catch (...) catch (...)
{ {
catchHandler("threadedAggregateRowGroups() caught an unknown exception", handleException(std::current_exception(),
tupleAggregateStepErr, fErrorInfo, fSessionId); logging::tupleAggregateStepErr,
caughtException = true; logging::ERR_AGGREGATION_TOO_BIG,
"TupleAggregateStep::threadedAggregateRowGroups()");
fEndOfResult = true;
fDoneAggregate = true;
} }
} }
if (caughtException)
{
fEndOfResult = true;
fDoneAggregate = true;
}
if (!locked) fMutex.lock(); if (!locked) fMutex.lock();
while (more) more = dlIn->next(fInputIter, &rgData); while (more) more = dlIn->next(fInputIter, &rgData);
@ -5558,19 +5511,12 @@ void TupleAggregateStep::doAggregate_singleThread()
} }
} }
} // try } // try
catch (IDBExcept& iex)
{
catchHandler(iex.what(), iex.errorCode(), fErrorInfo, fSessionId,
(iex.errorCode() == ERR_AGGREGATION_TOO_BIG ? LOG_TYPE_INFO : LOG_TYPE_CRITICAL));
}
catch (const std::exception& ex)
{
catchHandler(ex.what(), tupleAggregateStepErr, fErrorInfo, fSessionId);
}
catch (...) catch (...)
{ {
catchHandler("TupleAggregateStep next band caught an unknown exception", handleException(std::current_exception(),
tupleAggregateStepErr, fErrorInfo, fSessionId); logging::tupleAggregateStepErr,
logging::ERR_AGGREGATION_TOO_BIG,
"TupleAggregateStep::doAggregate_singleThread()");
} }
if (traceOn()) if (traceOn())
@ -5770,16 +5716,13 @@ uint64_t TupleAggregateStep::doThreadedAggregate(ByteStream& bs, RowGroupDL* dlp
fEndOfResult = true; fEndOfResult = true;
} }
} }
} } //try
catch (IDBExcept& iex) catch (...)
{ {
catchHandler(iex.what(), iex.errorCode(), fErrorInfo, fSessionId, handleException(std::current_exception(),
(iex.errorCode() == ERR_AGGREGATION_TOO_BIG ? LOG_TYPE_INFO : LOG_TYPE_CRITICAL)); logging::tupleAggregateStepErr,
fEndOfResult = true; logging::ERR_AGGREGATION_TOO_BIG,
} "TupleAggregateStep::doThreadedAggregate()");
catch (const std::exception& ex)
{
catchHandler(ex.what(), tupleAggregateStepErr, fErrorInfo, fSessionId);
fEndOfResult = true; fEndOfResult = true;
} }

View File

@ -328,20 +328,12 @@ uint32_t TupleAnnexStep::nextBand(messageqcpp::ByteStream& bs)
fEndOfResult = true; fEndOfResult = true;
} }
} }
catch (const std::exception& ex)
{
catchHandler(ex.what(), ERR_IN_DELIVERY, fErrorInfo, fSessionId);
while (more)
more = fOutputDL->next(fOutputIterator, &rgDataOut);
fEndOfResult = true;
}
catch (...) catch (...)
{ {
catchHandler("TupleAnnexStep next band caught an unknown exception", handleException(std::current_exception(),
ERR_IN_DELIVERY, fErrorInfo, fSessionId); logging::ERR_IN_DELIVERY,
logging::ERR_ALWAYS_CRITICAL,
"TupleAnnexStep::nextBand()");
while (more) while (more)
more = fOutputDL->next(fOutputIterator, &rgDataOut); more = fOutputDL->next(fOutputIterator, &rgDataOut);
@ -464,14 +456,12 @@ void TupleAnnexStep::executeNoOrderBy()
more = fInputDL->next(fInputIterator, &rgDataIn); more = fInputDL->next(fInputIterator, &rgDataIn);
} }
} }
catch (const std::exception& ex)
{
catchHandler(ex.what(), ERR_IN_PROCESS, fErrorInfo, fSessionId);
}
catch (...) catch (...)
{ {
catchHandler("TupleAnnexStep execute caught an unknown exception", handleException(std::current_exception(),
ERR_IN_PROCESS, fErrorInfo, fSessionId); logging::ERR_IN_PROCESS,
logging::ERR_ALWAYS_CRITICAL,
"TupleAnnexStep::executeNoOrderBy()");
} }
while (more) while (more)
@ -566,14 +556,12 @@ void TupleAnnexStep::executeNoOrderByWithDistinct()
fOutputDL->insert(rgDataOut); fOutputDL->insert(rgDataOut);
} }
} }
catch (const std::exception& ex)
{
catchHandler(ex.what(), ERR_IN_PROCESS, fErrorInfo, fSessionId);
}
catch (...) catch (...)
{ {
catchHandler("TupleAnnexStep execute caught an unknown exception", handleException(std::current_exception(),
ERR_IN_PROCESS, fErrorInfo, fSessionId); logging::ERR_IN_PROCESS,
logging::ERR_ALWAYS_CRITICAL,
"TupleAnnexStep::executeNoOrderByWithDistinct()");
} }
while (more) while (more)
@ -662,14 +650,12 @@ void TupleAnnexStep::executeWithOrderBy()
} }
} }
} }
catch (const std::exception& ex)
{
catchHandler(ex.what(), ERR_IN_PROCESS, fErrorInfo, fSessionId);
}
catch (...) catch (...)
{ {
catchHandler("TupleAnnexStep execute caught an unknown exception", handleException(std::current_exception(),
ERR_IN_PROCESS, fErrorInfo, fSessionId); logging::ERR_IN_PROCESS,
logging::ERR_ALWAYS_CRITICAL,
"TupleAnnexStep::executeWithOrderBy()");
} }
while (more) while (more)
@ -736,15 +722,12 @@ void TupleAnnexStep::finalizeParallelOrderByDistinct()
} }
} }
} }
catch (const std::exception& ex)
{
catchHandler(ex.what(),ERR_IN_PROCESS, fErrorInfo, fSessionId);
}
catch (...) catch (...)
{ {
catchHandler("TupleAnnexStep::finalizeParallelOrderByDistinct execute\ handleException(std::current_exception(),
caught an unknown exception 1", logging::ERR_IN_PROCESS,
ERR_IN_PROCESS, fErrorInfo, fSessionId); logging::ERR_ALWAYS_CRITICAL,
"TupleAnnexStep::finalizeParallelOrderByDistinct()");
} }
// OFFSET processing // OFFSET processing
@ -938,15 +921,12 @@ void TupleAnnexStep::finalizeParallelOrderBy()
} }
} }
} }
catch (const std::exception& ex)
{
catchHandler(ex.what(),ERR_IN_PROCESS, fErrorInfo, fSessionId);
}
catch (...) catch (...)
{ {
catchHandler("TupleAnnexStep::finalizeParallelOrderBy execute\ handleException(std::current_exception(),
caught an unknown exception 1", logging::ERR_IN_PROCESS,
ERR_IN_PROCESS, fErrorInfo, fSessionId); logging::ERR_ALWAYS_CRITICAL,
"TupleAnnexStep::finalizeParallelOrderBy()");
} }
// OFFSET processing // OFFSET processing
@ -1149,14 +1129,12 @@ void TupleAnnexStep::executeParallelOrderBy(uint64_t id)
if(more) dlOffset++; if(more) dlOffset++;
} }
} }
catch (const std::exception& ex)
{
catchHandler(ex.what(), ERR_IN_PROCESS, fErrorInfo, fSessionId);
}
catch (...) catch (...)
{ {
catchHandler("TupleAnnexStep::executeParallelOrderBy execute caught an unknown exception", handleException(std::current_exception(),
ERR_IN_PROCESS, fErrorInfo, fSessionId); logging::ERR_IN_PROCESS,
logging::ERR_ALWAYS_CRITICAL,
"TupleAnnexStep::executeParallelOrderBy()");
} }
// read out the input DL // read out the input DL

View File

@ -360,20 +360,12 @@ uint32_t TupleConstantStep::nextBand(messageqcpp::ByteStream& bs)
fEndOfResult = true; fEndOfResult = true;
} }
} }
catch (const std::exception& ex)
{
catchHandler(ex.what(), tupleConstantStepErr, fErrorInfo, fSessionId);
while (more)
more = fInputDL->next(fInputIterator, &rgDataIn);
fEndOfResult = true;
}
catch (...) catch (...)
{ {
catchHandler("TupleConstantStep next band caught an unknown exception", handleException(std::current_exception(),
tupleConstantStepErr, fErrorInfo, fSessionId); logging::tupleConstantStepErr,
logging::ERR_ALWAYS_CRITICAL,
"TupleConstantStep::nextBand()");
while (more) while (more)
more = fInputDL->next(fInputIterator, &rgDataIn); more = fInputDL->next(fInputIterator, &rgDataIn);
@ -445,17 +437,14 @@ void TupleConstantStep::execute()
} }
} }
} }
catch (const std::exception& ex)
{
catchHandler(ex.what(), tupleConstantStepErr, fErrorInfo, fSessionId);
}
catch (...) catch (...)
{ {
catchHandler("TupleConstantStep execute caught an unknown exception", handleException(std::current_exception(),
tupleConstantStepErr, fErrorInfo, fSessionId); logging::tupleConstantStepErr,
logging::ERR_ALWAYS_CRITICAL,
"TupleConstantStep::execute()");
} }
// if (!fEndOfResult)
while (more) while (more)
more = fInputDL->next(fInputIterator, &rgDataIn); more = fInputDL->next(fInputIterator, &rgDataIn);
@ -717,8 +706,10 @@ void TupleConstantOnlyStep::run()
} }
catch (...) catch (...)
{ {
catchHandler("TupleConstantOnlyStep run caught an unknown exception", handleException(std::current_exception(),
tupleConstantStepErr, fErrorInfo, fSessionId); logging::tupleConstantStepErr,
logging::ERR_ALWAYS_CRITICAL,
"TupleConstantOnlyStep::run()");
} }
if (traceOn()) if (traceOn())
@ -756,14 +747,12 @@ uint32_t TupleConstantOnlyStep::nextBand(messageqcpp::ByteStream& bs)
fRowGroupOut.serializeRGData(bs); fRowGroupOut.serializeRGData(bs);
rowCount = fRowGroupOut.getRowCount(); rowCount = fRowGroupOut.getRowCount();
} }
catch (const std::exception& ex)
{
catchHandler(ex.what(), tupleConstantStepErr, fErrorInfo, fSessionId);
}
catch (...) catch (...)
{ {
catchHandler("TupleConstantStep next band caught an unknown exception", handleException(std::current_exception(),
tupleConstantStepErr, fErrorInfo, fSessionId); logging::tupleConstantStepErr,
logging::ERR_ALWAYS_CRITICAL,
"TupleConstantOnlyStep::nextBand()");
} }
fEndOfResult = true; fEndOfResult = true;

View File

@ -1,4 +1,5 @@
/* Copyright (C) 2014 InfiniDB, Inc. /* Copyright (C) 2014 InfiniDB, Inc.
Copyright (c) 2016-2020 MariaDB
This program is free software; you can redistribute it and/or This program is free software; you can redistribute it and/or
modify it under the terms of the GNU General Public License modify it under the terms of the GNU General Public License
@ -447,23 +448,12 @@ next:
dlMutex.unlock(); dlMutex.unlock();
} }
} }
catch (boost::exception& e)
{
ostringstream oss;
oss << "TupleHashJoinStep::smallRunnerFcn failed due to " << boost::diagnostic_information(e);
fLogger->logMessage(logging::LOG_TYPE_ERROR, oss.str());
status(logging::ERR_EXEMGR_MALFUNCTION);
}
catch (std::exception& e)
{
ostringstream oss;
oss << "TupleHashJoinStep::smallRunnerFcn failed due to " << e.what();
fLogger->logMessage(logging::LOG_TYPE_ERROR, oss.str());
status(logging::ERR_EXEMGR_MALFUNCTION);
}
catch (...) catch (...)
{ {
fLogger->logMessage(logging::LOG_TYPE_ERROR, "TupleHashJoinStep::smallRunnerFcn failed due to an unknown reason (...)"); handleException(std::current_exception(),
logging::ERR_EXEMGR_MALFUNCTION,
logging::ERR_JOIN_TOO_BIG,
"TupleHashJoinStep::smallRunnerFcn()");
status(logging::ERR_EXEMGR_MALFUNCTION); status(logging::ERR_EXEMGR_MALFUNCTION);
} }
@ -755,28 +745,13 @@ void TupleHashJoinStep::hjRunner()
rgData[djsJoinerMap[i]].swap(empty); rgData[djsJoinerMap[i]].swap(empty);
} }
} }
catch (logging::IDBExcept& e) catch (...)
{ {
cout << e.what() << endl; handleException(std::current_exception(),
logging::ERR_EXEMGR_MALFUNCTION,
if (!status()) logging::ERR_JOIN_TOO_BIG,
{ "TupleHashJoinStep::hjRunner()");
errorMessage(logging::IDBErrorInfo::instance()->errorMsg(e.errorCode())); status(logging::ERR_EXEMGR_MALFUNCTION);
status(e.errorCode());
}
abort();
}
catch (std::exception& e)
{
cout << e.what() << endl;
if (!status())
{
errorMessage(logging::IDBErrorInfo::instance()->errorMsg(logging::ERR_DBJ_UNKNOWN_ERROR));
status(logging::ERR_DBJ_UNKNOWN_ERROR);
}
abort(); abort();
} }

View File

@ -227,21 +227,13 @@ uint32_t TupleHavingStep::nextBand(messageqcpp::ByteStream& bs)
fEndOfResult = true; fEndOfResult = true;
} }
} }
catch (const std::exception& ex)
{
catchHandler(ex.what(), tupleHavingStepErr, fErrorInfo, fSessionId);
while (more)
more = fInputDL->next(fInputIterator, &rgDataIn);
fEndOfResult = true;
}
catch (...) catch (...)
{ {
catchHandler("TupleHavingStep next band caught an unknown exception", handleException(std::current_exception(),
tupleHavingStepErr, fErrorInfo, fSessionId); logging::tupleHavingStepErr,
logging::ERR_ALWAYS_CRITICAL,
while (more) "TupleHavingStep::nextBand()");
while (more)
more = fInputDL->next(fInputIterator, &rgDataIn); more = fInputDL->next(fInputIterator, &rgDataIn);
fEndOfResult = true; fEndOfResult = true;
@ -310,14 +302,12 @@ void TupleHavingStep::execute()
} }
} }
} }
catch (const std::exception& ex)
{
catchHandler(ex.what(), tupleHavingStepErr, fErrorInfo, fSessionId);
}
catch (...) catch (...)
{ {
catchHandler("TupleHavingStep execute caught an unknown exception", handleException(std::current_exception(),
tupleHavingStepErr, fErrorInfo, fSessionId); logging::tupleHavingStepErr,
logging::ERR_ALWAYS_CRITICAL,
"TupleHavingStep::nextBand()");
} }
while (more) while (more)

View File

@ -276,13 +276,11 @@ void TupleUnion::readInput(uint32_t which)
} }
catch (...) catch (...)
{ {
if (status() == 0) handleException(std::current_exception(),
{ logging::unionStepErr,
errorMessage("Union step caught an unknown exception."); logging::ERR_UNION_TOO_BIG,
status(logging::unionStepErr); "TupleUnion::readInput()");
fLogger->logMessage(logging::LOG_TYPE_CRITICAL, "Union step caught an unknown exception."); status(logging::unionStepErr);
}
abort(); abort();
} }

View File

@ -1,4 +1,5 @@
/* Copyright (C) 2014 InfiniDB, Inc. /* Copyright (C) 2014 InfiniDB, Inc.
Copyright (c) 2016-2020 MariaDB
This program is free software; you can redistribute it and/or This program is free software; you can redistribute it and/or
modify it under the terms of the GNU General Public License modify it under the terms of the GNU General Public License
@ -228,31 +229,14 @@ uint32_t WindowFunctionStep::nextBand(messageqcpp::ByteStream& bs)
fEndOfResult = true; fEndOfResult = true;
} }
} }
catch (IDBExcept& iex)
{
handleException(iex.what(), iex.errorCode());
while (more)
more = fOutputDL->next(fOutputIterator, &rgDataOut);
fEndOfResult = true;
}
catch (const std::exception& ex)
{
handleException(ex.what(), ERR_IN_DELIVERY);
while (more)
more = fOutputDL->next(fOutputIterator, &rgDataOut);
fEndOfResult = true;
}
catch (...) catch (...)
{ {
handleException("WindowFunctionStep caught an unknown exception", ERR_IN_DELIVERY); handleException(std::current_exception(),
logging::ERR_IN_DELIVERY,
logging::ERR_WF_DATA_SET_TOO_BIG,
"WindowFunctionStep::nextBand()");
while (more) while (more)
more = fOutputDL->next(fOutputIterator, &rgDataOut); more = fOutputDL->next(fOutputIterator, &rgDataOut);
fEndOfResult = true; fEndOfResult = true;
} }
@ -961,17 +945,12 @@ void WindowFunctionStep::execute()
more = fInputDL->next(fInputIterator, &rgData); more = fInputDL->next(fInputIterator, &rgData);
} }
} // try } // try
catch (const IDBExcept& idb)
{
handleException(idb.what(), idb.errorCode());
}
catch (const std::exception& ex)
{
handleException(ex.what(), ERR_READ_INPUT_DATALIST);
}
catch (...) catch (...)
{ {
handleException("WindowFunctionStep caught an unknown exception", ERR_READ_INPUT_DATALIST); handleException(std::current_exception(),
logging::ERR_READ_INPUT_DATALIST,
logging::ERR_WF_DATA_SET_TOO_BIG,
"WindowFunctionStep::execute()");
} }
if (traceOn()) if (traceOn())
@ -1030,14 +1009,12 @@ void WindowFunctionStep::execute()
} }
} }
catch (const std::exception& ex)
{
handleException(ex.what(), ERR_EXECUTE_WINDOW_FUNCTION);
}
catch (...) catch (...)
{ {
handleException("WindowFunctionStep caught an unknown exception", handleException(std::current_exception(),
ERR_EXECUTE_WINDOW_FUNCTION); logging::ERR_EXECUTE_WINDOW_FUNCTION,
logging::ERR_WF_DATA_SET_TOO_BIG,
"WindowFunctionStep::execute()");
} }
fOutputDL->endOfInput(); fOutputDL->endOfInput();
@ -1084,21 +1061,15 @@ void WindowFunctionStep::doFunction()
(*fFunctions[i].get())(); (*fFunctions[i].get())();
} }
} }
catch (IDBExcept& iex)
{
handleException(iex.what(), iex.errorCode());
}
catch (const std::exception& ex)
{
handleException(ex.what(), ERR_EXECUTE_WINDOW_FUNCTION);
}
catch (...) catch (...)
{ {
handleException("doFunction caught an unknown exception", ERR_EXECUTE_WINDOW_FUNCTION); handleException(std::current_exception(),
logging::ERR_EXECUTE_WINDOW_FUNCTION,
logging::ERR_WF_DATA_SET_TOO_BIG,
"WindowFunctionStep::doFunction()");
} }
} }
void WindowFunctionStep::doPostProcessForSelect() void WindowFunctionStep::doPostProcessForSelect()
{ {
FuncExp* fe = funcexp::FuncExp::instance(); FuncExp* fe = funcexp::FuncExp::instance();
@ -1197,13 +1168,6 @@ void WindowFunctionStep::doPostProcessForDml()
} }
void WindowFunctionStep::handleException(string errStr, int errCode)
{
cerr << "Exception: " << errStr << endl;
catchHandler(errStr, errCode, fErrorInfo, fSessionId);
}
boost::shared_ptr<FrameBound> WindowFunctionStep::parseFrameBoundRows( boost::shared_ptr<FrameBound> WindowFunctionStep::parseFrameBoundRows(
const execplan::WF_Boundary& b, const execplan::WF_Boundary& b,
const map<uint64_t, uint64_t>& m, const map<uint64_t, uint64_t>& m,

View File

@ -1,4 +1,5 @@
/* Copyright (C) 2014 InfiniDB, Inc. /* Copyright (C) 2014 InfiniDB, Inc.
Copyright (c) 2016-2020 MariaDB
This program is free software; you can redistribute it and/or This program is free software; you can redistribute it and/or
modify it under the terms of the GNU General Public License modify it under the terms of the GNU General Public License
@ -108,8 +109,6 @@ public:
{ {
return fRows; return fRows;
} }
void handleException(std::string, int);
// for string table // for string table
rowgroup::Row::Pointer getPointer(RowPosition& pos) rowgroup::Row::Pointer getPointer(RowPosition& pos)
{ {

View File

@ -14,6 +14,9 @@
# The tokens should be separated by one tab character. The error message text may contain # The tokens should be separated by one tab character. The error message text may contain
# any character(s) except tab. The line must end with a single '\n'. # any character(s) except tab. The line must end with a single '\n'.
# The id to treat all messages as CRITICAL
444 ERR_ALWAYS_CRITICAL
# Non support errors 1000 ~ 2000. # Non support errors 1000 ~ 2000.
# The query will go through the optimizer again with some optimization turned off # The query will go through the optimizer again with some optimization turned off
1000 ERR_MISS_JOIN %1% not joined. 1000 ERR_MISS_JOIN %1% not joined.

View File

@ -212,17 +212,12 @@ void WindowFunction::operator()()
} }
} }
} }
catch (IDBExcept& iex)
{
fStep->handleException(iex.what(), iex.errorCode());
}
catch (const std::exception& ex)
{
fStep->handleException(ex.what(), logging::ERR_EXECUTE_WINDOW_FUNCTION);
}
catch (...) catch (...)
{ {
fStep->handleException("unknown exception", logging::ERR_EXECUTE_WINDOW_FUNCTION); fStep->handleException(std::current_exception(),
logging::ERR_EXECUTE_WINDOW_FUNCTION,
logging::ERR_WF_DATA_SET_TOO_BIG,
"WindowFunction::operator()");
} }
} }