Skip to content

Commit 3ad8409

Browse files
committed
Better exception handling in connect_chdb
1 parent b5c1b09 commit 3ad8409

File tree

1 file changed

+45
-15
lines changed

1 file changed

+45
-15
lines changed

programs/local/LocalServer.cpp

Lines changed: 45 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -1330,13 +1330,13 @@ chdb_conn ** connect_chdb(int argc, char ** argv)
13301330
std::thread(
13311331
[&]()
13321332
{
1333+
auto * queue = static_cast<query_queue *>(conn->queue);
13331334
try
13341335
{
13351336
DB::LocalServer * server = bgClickHouseLocal(argc, argv);
13361337
conn->server = server;
13371338
conn->connected = true;
13381339

1339-
// Store globally
13401340
global_conn_ptr = conn;
13411341
global_db_path = path;
13421342

@@ -1348,7 +1348,6 @@ chdb_conn ** connect_chdb(int argc, char ** argv)
13481348
}
13491349
init_cv.notify_one();
13501350

1351-
auto * queue = static_cast<query_queue *>(conn->queue);
13521351
while (true)
13531352
{
13541353
query_request req;
@@ -1358,8 +1357,16 @@ chdb_conn ** connect_chdb(int argc, char ** argv)
13581357

13591358
if (queue->shutdown)
13601359
{
1361-
server->cleanup();
1362-
delete server;
1360+
try
1361+
{
1362+
server->cleanup();
1363+
delete server;
1364+
}
1365+
catch (...)
1366+
{
1367+
// Log error but continue shutdown
1368+
LOG_ERROR(&Poco::Logger::get("LocalServer"), "Error during server cleanup");
1369+
}
13631370
queue->cleanup_done = true;
13641371
queue->query_cv.notify_all();
13651372
break;
@@ -1368,7 +1375,7 @@ chdb_conn ** connect_chdb(int argc, char ** argv)
13681375
req = queue->current_query;
13691376
}
13701377

1371-
auto result = std::make_unique<local_result_v2>();
1378+
local_result_v2 * result = new local_result_v2();
13721379
try
13731380
{
13741381
if (!server->parseQueryTextWithOutputFormat(req.query, req.format))
@@ -1406,21 +1413,45 @@ chdb_conn ** connect_chdb(int argc, char ** argv)
14061413

14071414
{
14081415
std::lock_guard<std::mutex> lock(queue->mutex);
1409-
queue->current_result = result.release();
1416+
queue->current_result = result;
14101417
queue->has_query = false;
14111418
}
14121419
queue->result_cv.notify_one();
14131420
}
14141421
}
1422+
catch (const DB::Exception & e)
1423+
{
1424+
// Log the error
1425+
LOG_ERROR(&Poco::Logger::get("LocalServer"), "Query thread terminated with error: {}", e.what());
1426+
1427+
// Signal thread termination
1428+
{
1429+
std::lock_guard<std::mutex> init_lock(init_mutex);
1430+
init_exception = std::current_exception();
1431+
init_done = true;
1432+
std::lock_guard<std::mutex> lock(queue->mutex);
1433+
queue->shutdown = true;
1434+
queue->cleanup_done = true;
1435+
}
1436+
init_cv.notify_one();
1437+
queue->query_cv.notify_all();
1438+
queue->result_cv.notify_all();
1439+
}
14151440
catch (...)
14161441
{
1417-
// Signal initialization failure
1442+
LOG_ERROR(&Poco::Logger::get("LocalServer"), "Query thread terminated with unknown error");
1443+
14181444
{
14191445
std::lock_guard<std::mutex> init_lock(init_mutex);
14201446
init_exception = std::current_exception();
14211447
init_done = true;
1448+
std::lock_guard<std::mutex> lock(queue->mutex);
1449+
queue->shutdown = true;
1450+
queue->cleanup_done = true;
14221451
}
14231452
init_cv.notify_one();
1453+
queue->query_cv.notify_all();
1454+
queue->result_cv.notify_all();
14241455
}
14251456
})
14261457
.detach();
@@ -1473,18 +1504,17 @@ void close_conn(chdb_conn ** conn)
14731504
}
14741505
}
14751506

1476-
// Mark as disconnected before deleting queue
1477-
(*conn)->connected = false;
1478-
14791507
delete queue;
14801508
(*conn)->queue = nullptr;
14811509
}
14821510

1483-
if (*conn == global_conn_ptr)
1484-
{
1485-
global_conn_ptr = nullptr;
1486-
global_db_path.clear();
1487-
}
1511+
// Mark as disconnected BEFORE deleting queue and nulling global pointer
1512+
(*conn)->connected = false;
1513+
}
1514+
// Clear global pointer under lock before queue deletion
1515+
if (*conn != global_conn_ptr)
1516+
{
1517+
LOG_ERROR(&Poco::Logger::get("LocalServer"), "Connection mismatch during close_conn");
14881518
}
14891519

14901520
delete *conn;

0 commit comments

Comments
 (0)