ClickHouse/tests/queries/0_stateless/00967_live_view_watch_http.py

50 lines
1.5 KiB
Python
Raw Normal View History

2020-10-02 16:54:07 +00:00
#!/usr/bin/env python3
2021-09-12 12:35:27 +00:00
# Tags: no-replicated-database, no-parallel, no-fasttest
import os
import sys
CURDIR = os.path.dirname(os.path.realpath(__file__))
sys.path.insert(0, os.path.join(CURDIR, "helpers"))
from client import client, prompt, end_of_block
from httpclient import client as http_client
log = None
# uncomment the line below for debugging
# log=sys.stdout
with client(name="client1>", log=log) as client1:
client1.expect(prompt)
client1.send("SET allow_experimental_live_view = 1")
client1.expect(prompt)
client1.send("DROP TABLE IF EXISTS test.lv")
client1.expect(prompt)
client1.send(" DROP TABLE IF EXISTS test.mt")
client1.expect(prompt)
client1.send("CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()")
client1.expect(prompt)
client1.send("CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt")
client1.expect(prompt)
Fix hang check for 00966_live_view_watch_events_http If INSERT was failed then WATCH EVENTS will hang: 2021.03.12 23:45:33.666223 [ 1901 ] {} <Trace> DynamicQueryHandler: Request URI: /?allow_experimental_live_view=1&query=WATCH%20test.lv%20EVENTS ... 2021.03.12 23:45:34.119465 [ 35492 ] {55f2c8ec-df9f-4e97-9acc-385ed5b5fde2} <Error> executeQuery: Code: 241, e.displayText() = DB::Exception: Memory limit (for user) exceeded: would use 153.99 MiB (attempt to allocate chunk of 4194729 bytes), maximum: 150.00 MiB (version 21.4.1.6239) (from [::1]:43910) (in query: INSERT INTO test.mt VALUES ), Stack trace (when copying this message, always include the lines below): ... Hung check: elapsed: 597.835824299 is_cancelled: 0 query: WATCH test.lv EVENTS thread_ids: [1901] gdb: Thread 133 (Thread 0x7f26ac95b700 (LWP 1901)): 0 0x00007f2793d447b1 in pthread_cond_timedwait@@GLIBC_2.3.2 () from /lib/x86_64-linux-gnu/libpthread.so.0 1 0x0000000008cbd5c5 in __tsan::call_pthread_cancel_with_cleanup(int (*)(void*, void*, void*), void*, void*, void*, void (*)(void*), void*) () 2 0x0000000008c4f134 in cond_wait(__tsan::ThreadState*, unsigned long, __tsan::ScopedInterceptor*, int (*)(void*, void*, void*), void*, void*, void*) () 3 0x0000000008c4f339 in pthread_cond_timedwait () 4 0x000000001831c8b6 in std::__1::__libcpp_condvar_timedwait (__cv=0x7b5801053520, __m=0x189, __ts=0x7f26ac9132b8) at ../contrib/libcxx/include/__threading_support:442 5 std::__1::condition_variable::__do_timed_wait (this=0x7b5801053520, lk=..., tp=...) at ../contrib/libcxx/src/condition_variable.cpp:74 6 0x00000000134fe7f2 in std::__1::condition_variable::wait_for<long long, std::__1::ratio<1l, 1000000l> > (this=this@entry=0x7b5801053520, __lk=..., __d=...) at ../contrib/libcxx/include/__mutex_base:462 7 0x00000000134fe2c9 in DB::LiveViewEventsBlockInputStream::tryReadImpl (this=this@entry=0x7b5c0070ff98, blocking=false) at ../src/Storages/LiveView/LiveViewEventsBlockInputStream.h:180 8 0x00000000134fd6e4 in DB::LiveViewEventsBlockInputStream::readImpl (this=0x189) at ../src/Storages/LiveView/LiveViewEventsBlockInputStream.h:115 9 0x0000000012285593 in DB::IBlockInputStream::read (this=0x7b5c0070ff98) at ../src/DataStreams/IBlockInputStream.cpp:58 10 0x00000000122afc7e in DB::copyDataImpl<std::__1::function<bool ()> const&, std::__1::function<void (DB::Block const&)> const&>(DB::IBlockInputStream&, DB::IBlockOutputStream&, std::__1::function<bool ()> const&, std::__1::function<void (DB::Block const&)> const&) (from=..., to=..., is_cancelled=..., progress=...) at ../src/DataStreams/copyData.cpp:26 11 0x00000000122af8e1 in DB::copyData(DB::IBlockInputStream&, DB::IBlockOutputStream&, std::__1::function<bool ()> const&, std::__1::function<void (DB::Block const&)> const&) (from=..., to=..., is_cancelled=..., progress=...) at ../src/DataStreams/copyData.cpp:73 12 0x0000000012cfc5e2 in DB::executeQuery(DB::ReadBuffer&, DB::WriteBuffer&, bool, DB::Context&, std::__1::function<void (std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&)>) (istr=..., ostr=..., allow_into_outfile=false, context=..., set_result_details=...) at ../src/Interpreters/executeQuery.cpp:1033 13 0x000000001357941d in DB::HTTPHandler::processQuery (this=this@entry=0x7b1c006f41b0, context=..., request=..., params=..., response=..., used_output=..., query_scope=...) at ../src/Server/HTTPHandler.cpp:772 14 0x000000001357c507 in DB::HTTPHandler::handleRequest (this=<optimized out>, request=..., response=...) at ../src/Server/HTTPHandler.cpp:910 15 0x00000000135f3081 in DB::HTTPServerConnection::run (this=<optimized out>) at ../src/Server/HTTP/HTTPServerConnection.cpp:48 16 0x00000000161f9e73 in Poco::Net::TCPServerConnection::start (this=0x7b8c01554000) at ../contrib/poco/Net/src/TCPServerConnection.cpp:43 17 0x00000000161fa59f in Poco::Net::TCPServerDispatcher::run (this=<optimized out>) at ../contrib/poco/Net/src/TCPServerDispatcher.cpp:113 18 0x0000000016361732 in Poco::PooledThread::run (this=<optimized out>) at ../contrib/poco/Foundation/src/ThreadPool.cpp:199 19 0x000000001635fcd0 in Poco::(anonymous namespace)::RunnableHolder::run (this=<optimized out>) at ../contrib/poco/Foundation/src/Thread.cpp:55 20 0x000000001635e4d8 in Poco::ThreadImpl::runnableEntry (pThread=0x7b5400082538) at ../contrib/poco/Foundation/src/Thread_POSIX.cpp:345 21 0x0000000008c4e36d in __tsan_thread_start_func () 22 0x00007f2793d3d609 in start_thread () from /lib/x86_64-linux-gnu/libpthread.so.0 23 0x00007f2793c64293 in clone () from /lib/x86_64-linux-gnu/libc.so.6 Cc: @vzakaznikov
2021-03-13 08:29:40 +00:00
try:
with http_client(
{
"method": "GET",
"url": "/?allow_experimental_live_view=1&query=WATCH%20test.lv",
},
name="client2>",
log=log,
) as client2:
client2.expect(".*0\t1\n")
client1.send("INSERT INTO test.mt VALUES (1),(2),(3)")
Fix hang check for 00966_live_view_watch_events_http If INSERT was failed then WATCH EVENTS will hang: 2021.03.12 23:45:33.666223 [ 1901 ] {} <Trace> DynamicQueryHandler: Request URI: /?allow_experimental_live_view=1&query=WATCH%20test.lv%20EVENTS ... 2021.03.12 23:45:34.119465 [ 35492 ] {55f2c8ec-df9f-4e97-9acc-385ed5b5fde2} <Error> executeQuery: Code: 241, e.displayText() = DB::Exception: Memory limit (for user) exceeded: would use 153.99 MiB (attempt to allocate chunk of 4194729 bytes), maximum: 150.00 MiB (version 21.4.1.6239) (from [::1]:43910) (in query: INSERT INTO test.mt VALUES ), Stack trace (when copying this message, always include the lines below): ... Hung check: elapsed: 597.835824299 is_cancelled: 0 query: WATCH test.lv EVENTS thread_ids: [1901] gdb: Thread 133 (Thread 0x7f26ac95b700 (LWP 1901)): 0 0x00007f2793d447b1 in pthread_cond_timedwait@@GLIBC_2.3.2 () from /lib/x86_64-linux-gnu/libpthread.so.0 1 0x0000000008cbd5c5 in __tsan::call_pthread_cancel_with_cleanup(int (*)(void*, void*, void*), void*, void*, void*, void (*)(void*), void*) () 2 0x0000000008c4f134 in cond_wait(__tsan::ThreadState*, unsigned long, __tsan::ScopedInterceptor*, int (*)(void*, void*, void*), void*, void*, void*) () 3 0x0000000008c4f339 in pthread_cond_timedwait () 4 0x000000001831c8b6 in std::__1::__libcpp_condvar_timedwait (__cv=0x7b5801053520, __m=0x189, __ts=0x7f26ac9132b8) at ../contrib/libcxx/include/__threading_support:442 5 std::__1::condition_variable::__do_timed_wait (this=0x7b5801053520, lk=..., tp=...) at ../contrib/libcxx/src/condition_variable.cpp:74 6 0x00000000134fe7f2 in std::__1::condition_variable::wait_for<long long, std::__1::ratio<1l, 1000000l> > (this=this@entry=0x7b5801053520, __lk=..., __d=...) at ../contrib/libcxx/include/__mutex_base:462 7 0x00000000134fe2c9 in DB::LiveViewEventsBlockInputStream::tryReadImpl (this=this@entry=0x7b5c0070ff98, blocking=false) at ../src/Storages/LiveView/LiveViewEventsBlockInputStream.h:180 8 0x00000000134fd6e4 in DB::LiveViewEventsBlockInputStream::readImpl (this=0x189) at ../src/Storages/LiveView/LiveViewEventsBlockInputStream.h:115 9 0x0000000012285593 in DB::IBlockInputStream::read (this=0x7b5c0070ff98) at ../src/DataStreams/IBlockInputStream.cpp:58 10 0x00000000122afc7e in DB::copyDataImpl<std::__1::function<bool ()> const&, std::__1::function<void (DB::Block const&)> const&>(DB::IBlockInputStream&, DB::IBlockOutputStream&, std::__1::function<bool ()> const&, std::__1::function<void (DB::Block const&)> const&) (from=..., to=..., is_cancelled=..., progress=...) at ../src/DataStreams/copyData.cpp:26 11 0x00000000122af8e1 in DB::copyData(DB::IBlockInputStream&, DB::IBlockOutputStream&, std::__1::function<bool ()> const&, std::__1::function<void (DB::Block const&)> const&) (from=..., to=..., is_cancelled=..., progress=...) at ../src/DataStreams/copyData.cpp:73 12 0x0000000012cfc5e2 in DB::executeQuery(DB::ReadBuffer&, DB::WriteBuffer&, bool, DB::Context&, std::__1::function<void (std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&)>) (istr=..., ostr=..., allow_into_outfile=false, context=..., set_result_details=...) at ../src/Interpreters/executeQuery.cpp:1033 13 0x000000001357941d in DB::HTTPHandler::processQuery (this=this@entry=0x7b1c006f41b0, context=..., request=..., params=..., response=..., used_output=..., query_scope=...) at ../src/Server/HTTPHandler.cpp:772 14 0x000000001357c507 in DB::HTTPHandler::handleRequest (this=<optimized out>, request=..., response=...) at ../src/Server/HTTPHandler.cpp:910 15 0x00000000135f3081 in DB::HTTPServerConnection::run (this=<optimized out>) at ../src/Server/HTTP/HTTPServerConnection.cpp:48 16 0x00000000161f9e73 in Poco::Net::TCPServerConnection::start (this=0x7b8c01554000) at ../contrib/poco/Net/src/TCPServerConnection.cpp:43 17 0x00000000161fa59f in Poco::Net::TCPServerDispatcher::run (this=<optimized out>) at ../contrib/poco/Net/src/TCPServerDispatcher.cpp:113 18 0x0000000016361732 in Poco::PooledThread::run (this=<optimized out>) at ../contrib/poco/Foundation/src/ThreadPool.cpp:199 19 0x000000001635fcd0 in Poco::(anonymous namespace)::RunnableHolder::run (this=<optimized out>) at ../contrib/poco/Foundation/src/Thread.cpp:55 20 0x000000001635e4d8 in Poco::ThreadImpl::runnableEntry (pThread=0x7b5400082538) at ../contrib/poco/Foundation/src/Thread_POSIX.cpp:345 21 0x0000000008c4e36d in __tsan_thread_start_func () 22 0x00007f2793d3d609 in start_thread () from /lib/x86_64-linux-gnu/libpthread.so.0 23 0x00007f2793c64293 in clone () from /lib/x86_64-linux-gnu/libc.so.6 Cc: @vzakaznikov
2021-03-13 08:29:40 +00:00
client1.expect(prompt)
client2.expect(".*6\t2\n")
Fix hang check for 00966_live_view_watch_events_http If INSERT was failed then WATCH EVENTS will hang: 2021.03.12 23:45:33.666223 [ 1901 ] {} <Trace> DynamicQueryHandler: Request URI: /?allow_experimental_live_view=1&query=WATCH%20test.lv%20EVENTS ... 2021.03.12 23:45:34.119465 [ 35492 ] {55f2c8ec-df9f-4e97-9acc-385ed5b5fde2} <Error> executeQuery: Code: 241, e.displayText() = DB::Exception: Memory limit (for user) exceeded: would use 153.99 MiB (attempt to allocate chunk of 4194729 bytes), maximum: 150.00 MiB (version 21.4.1.6239) (from [::1]:43910) (in query: INSERT INTO test.mt VALUES ), Stack trace (when copying this message, always include the lines below): ... Hung check: elapsed: 597.835824299 is_cancelled: 0 query: WATCH test.lv EVENTS thread_ids: [1901] gdb: Thread 133 (Thread 0x7f26ac95b700 (LWP 1901)): 0 0x00007f2793d447b1 in pthread_cond_timedwait@@GLIBC_2.3.2 () from /lib/x86_64-linux-gnu/libpthread.so.0 1 0x0000000008cbd5c5 in __tsan::call_pthread_cancel_with_cleanup(int (*)(void*, void*, void*), void*, void*, void*, void (*)(void*), void*) () 2 0x0000000008c4f134 in cond_wait(__tsan::ThreadState*, unsigned long, __tsan::ScopedInterceptor*, int (*)(void*, void*, void*), void*, void*, void*) () 3 0x0000000008c4f339 in pthread_cond_timedwait () 4 0x000000001831c8b6 in std::__1::__libcpp_condvar_timedwait (__cv=0x7b5801053520, __m=0x189, __ts=0x7f26ac9132b8) at ../contrib/libcxx/include/__threading_support:442 5 std::__1::condition_variable::__do_timed_wait (this=0x7b5801053520, lk=..., tp=...) at ../contrib/libcxx/src/condition_variable.cpp:74 6 0x00000000134fe7f2 in std::__1::condition_variable::wait_for<long long, std::__1::ratio<1l, 1000000l> > (this=this@entry=0x7b5801053520, __lk=..., __d=...) at ../contrib/libcxx/include/__mutex_base:462 7 0x00000000134fe2c9 in DB::LiveViewEventsBlockInputStream::tryReadImpl (this=this@entry=0x7b5c0070ff98, blocking=false) at ../src/Storages/LiveView/LiveViewEventsBlockInputStream.h:180 8 0x00000000134fd6e4 in DB::LiveViewEventsBlockInputStream::readImpl (this=0x189) at ../src/Storages/LiveView/LiveViewEventsBlockInputStream.h:115 9 0x0000000012285593 in DB::IBlockInputStream::read (this=0x7b5c0070ff98) at ../src/DataStreams/IBlockInputStream.cpp:58 10 0x00000000122afc7e in DB::copyDataImpl<std::__1::function<bool ()> const&, std::__1::function<void (DB::Block const&)> const&>(DB::IBlockInputStream&, DB::IBlockOutputStream&, std::__1::function<bool ()> const&, std::__1::function<void (DB::Block const&)> const&) (from=..., to=..., is_cancelled=..., progress=...) at ../src/DataStreams/copyData.cpp:26 11 0x00000000122af8e1 in DB::copyData(DB::IBlockInputStream&, DB::IBlockOutputStream&, std::__1::function<bool ()> const&, std::__1::function<void (DB::Block const&)> const&) (from=..., to=..., is_cancelled=..., progress=...) at ../src/DataStreams/copyData.cpp:73 12 0x0000000012cfc5e2 in DB::executeQuery(DB::ReadBuffer&, DB::WriteBuffer&, bool, DB::Context&, std::__1::function<void (std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&)>) (istr=..., ostr=..., allow_into_outfile=false, context=..., set_result_details=...) at ../src/Interpreters/executeQuery.cpp:1033 13 0x000000001357941d in DB::HTTPHandler::processQuery (this=this@entry=0x7b1c006f41b0, context=..., request=..., params=..., response=..., used_output=..., query_scope=...) at ../src/Server/HTTPHandler.cpp:772 14 0x000000001357c507 in DB::HTTPHandler::handleRequest (this=<optimized out>, request=..., response=...) at ../src/Server/HTTPHandler.cpp:910 15 0x00000000135f3081 in DB::HTTPServerConnection::run (this=<optimized out>) at ../src/Server/HTTP/HTTPServerConnection.cpp:48 16 0x00000000161f9e73 in Poco::Net::TCPServerConnection::start (this=0x7b8c01554000) at ../contrib/poco/Net/src/TCPServerConnection.cpp:43 17 0x00000000161fa59f in Poco::Net::TCPServerDispatcher::run (this=<optimized out>) at ../contrib/poco/Net/src/TCPServerDispatcher.cpp:113 18 0x0000000016361732 in Poco::PooledThread::run (this=<optimized out>) at ../contrib/poco/Foundation/src/ThreadPool.cpp:199 19 0x000000001635fcd0 in Poco::(anonymous namespace)::RunnableHolder::run (this=<optimized out>) at ../contrib/poco/Foundation/src/Thread.cpp:55 20 0x000000001635e4d8 in Poco::ThreadImpl::runnableEntry (pThread=0x7b5400082538) at ../contrib/poco/Foundation/src/Thread_POSIX.cpp:345 21 0x0000000008c4e36d in __tsan_thread_start_func () 22 0x00007f2793d3d609 in start_thread () from /lib/x86_64-linux-gnu/libpthread.so.0 23 0x00007f2793c64293 in clone () from /lib/x86_64-linux-gnu/libc.so.6 Cc: @vzakaznikov
2021-03-13 08:29:40 +00:00
finally:
client1.send("DROP TABLE test.lv")
Fix hang check for 00966_live_view_watch_events_http If INSERT was failed then WATCH EVENTS will hang: 2021.03.12 23:45:33.666223 [ 1901 ] {} <Trace> DynamicQueryHandler: Request URI: /?allow_experimental_live_view=1&query=WATCH%20test.lv%20EVENTS ... 2021.03.12 23:45:34.119465 [ 35492 ] {55f2c8ec-df9f-4e97-9acc-385ed5b5fde2} <Error> executeQuery: Code: 241, e.displayText() = DB::Exception: Memory limit (for user) exceeded: would use 153.99 MiB (attempt to allocate chunk of 4194729 bytes), maximum: 150.00 MiB (version 21.4.1.6239) (from [::1]:43910) (in query: INSERT INTO test.mt VALUES ), Stack trace (when copying this message, always include the lines below): ... Hung check: elapsed: 597.835824299 is_cancelled: 0 query: WATCH test.lv EVENTS thread_ids: [1901] gdb: Thread 133 (Thread 0x7f26ac95b700 (LWP 1901)): 0 0x00007f2793d447b1 in pthread_cond_timedwait@@GLIBC_2.3.2 () from /lib/x86_64-linux-gnu/libpthread.so.0 1 0x0000000008cbd5c5 in __tsan::call_pthread_cancel_with_cleanup(int (*)(void*, void*, void*), void*, void*, void*, void (*)(void*), void*) () 2 0x0000000008c4f134 in cond_wait(__tsan::ThreadState*, unsigned long, __tsan::ScopedInterceptor*, int (*)(void*, void*, void*), void*, void*, void*) () 3 0x0000000008c4f339 in pthread_cond_timedwait () 4 0x000000001831c8b6 in std::__1::__libcpp_condvar_timedwait (__cv=0x7b5801053520, __m=0x189, __ts=0x7f26ac9132b8) at ../contrib/libcxx/include/__threading_support:442 5 std::__1::condition_variable::__do_timed_wait (this=0x7b5801053520, lk=..., tp=...) at ../contrib/libcxx/src/condition_variable.cpp:74 6 0x00000000134fe7f2 in std::__1::condition_variable::wait_for<long long, std::__1::ratio<1l, 1000000l> > (this=this@entry=0x7b5801053520, __lk=..., __d=...) at ../contrib/libcxx/include/__mutex_base:462 7 0x00000000134fe2c9 in DB::LiveViewEventsBlockInputStream::tryReadImpl (this=this@entry=0x7b5c0070ff98, blocking=false) at ../src/Storages/LiveView/LiveViewEventsBlockInputStream.h:180 8 0x00000000134fd6e4 in DB::LiveViewEventsBlockInputStream::readImpl (this=0x189) at ../src/Storages/LiveView/LiveViewEventsBlockInputStream.h:115 9 0x0000000012285593 in DB::IBlockInputStream::read (this=0x7b5c0070ff98) at ../src/DataStreams/IBlockInputStream.cpp:58 10 0x00000000122afc7e in DB::copyDataImpl<std::__1::function<bool ()> const&, std::__1::function<void (DB::Block const&)> const&>(DB::IBlockInputStream&, DB::IBlockOutputStream&, std::__1::function<bool ()> const&, std::__1::function<void (DB::Block const&)> const&) (from=..., to=..., is_cancelled=..., progress=...) at ../src/DataStreams/copyData.cpp:26 11 0x00000000122af8e1 in DB::copyData(DB::IBlockInputStream&, DB::IBlockOutputStream&, std::__1::function<bool ()> const&, std::__1::function<void (DB::Block const&)> const&) (from=..., to=..., is_cancelled=..., progress=...) at ../src/DataStreams/copyData.cpp:73 12 0x0000000012cfc5e2 in DB::executeQuery(DB::ReadBuffer&, DB::WriteBuffer&, bool, DB::Context&, std::__1::function<void (std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&)>) (istr=..., ostr=..., allow_into_outfile=false, context=..., set_result_details=...) at ../src/Interpreters/executeQuery.cpp:1033 13 0x000000001357941d in DB::HTTPHandler::processQuery (this=this@entry=0x7b1c006f41b0, context=..., request=..., params=..., response=..., used_output=..., query_scope=...) at ../src/Server/HTTPHandler.cpp:772 14 0x000000001357c507 in DB::HTTPHandler::handleRequest (this=<optimized out>, request=..., response=...) at ../src/Server/HTTPHandler.cpp:910 15 0x00000000135f3081 in DB::HTTPServerConnection::run (this=<optimized out>) at ../src/Server/HTTP/HTTPServerConnection.cpp:48 16 0x00000000161f9e73 in Poco::Net::TCPServerConnection::start (this=0x7b8c01554000) at ../contrib/poco/Net/src/TCPServerConnection.cpp:43 17 0x00000000161fa59f in Poco::Net::TCPServerDispatcher::run (this=<optimized out>) at ../contrib/poco/Net/src/TCPServerDispatcher.cpp:113 18 0x0000000016361732 in Poco::PooledThread::run (this=<optimized out>) at ../contrib/poco/Foundation/src/ThreadPool.cpp:199 19 0x000000001635fcd0 in Poco::(anonymous namespace)::RunnableHolder::run (this=<optimized out>) at ../contrib/poco/Foundation/src/Thread.cpp:55 20 0x000000001635e4d8 in Poco::ThreadImpl::runnableEntry (pThread=0x7b5400082538) at ../contrib/poco/Foundation/src/Thread_POSIX.cpp:345 21 0x0000000008c4e36d in __tsan_thread_start_func () 22 0x00007f2793d3d609 in start_thread () from /lib/x86_64-linux-gnu/libpthread.so.0 23 0x00007f2793c64293 in clone () from /lib/x86_64-linux-gnu/libc.so.6 Cc: @vzakaznikov
2021-03-13 08:29:40 +00:00
client1.expect(prompt)
client1.send("DROP TABLE test.mt")
client1.expect(prompt)