* Started to add LIVE VIEW table tests

* Added uexpect.py module
* Fixed support for CREATE TEMPORARY LIVE VIEW
This commit is contained in:
Vitaliy Zakaznikov 2019-06-03 15:53:11 -04:00
parent f06f0e3947
commit fc21200bdd
26 changed files with 499 additions and 5 deletions

View File

@ -481,7 +481,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
}
/// Temporary tables are created out of databases.
if (create.temporary && !create.database.empty())
if (create.temporary && !create.database.empty() && !create.is_live_view)
throw Exception("Temporary tables cannot be inside a database. You should not specify a database for a temporary table.",
ErrorCodes::BAD_DATABASE_FOR_TEMPORARY_TABLE);
@ -539,7 +539,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
String data_path;
DatabasePtr database;
if (!create.temporary)
if (!create.temporary || create.is_live_view)
{
database = context.getDatabase(database_name);
data_path = database->getDataPath();
@ -582,7 +582,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
create.attach,
false);
if (create.temporary)
if (create.temporary && !create.is_live_view)
context.getSessionContext().addExternalTable(table_name, res, query_ptr);
else
database->createTable(context, table_name, res, query_ptr);

View File

@ -243,9 +243,9 @@ void StorageLiveView::noUsersThread()
bool drop_table = false;
{
Poco::FastMutex::ScopedLock lock(noUsersThreadMutex);
while (1)
{
Poco::FastMutex::ScopedLock lock(noUsersThreadMutex);
if (!noUsersThreadWakeUp && !noUsersThreadCondition.tryWait(noUsersThreadMutex, global_context.getSettingsRef().temporary_live_view_timeout.totalSeconds() * 1000))
{
noUsersThreadWakeUp = false;
@ -271,6 +271,7 @@ void StorageLiveView::noUsersThread()
auto drop_query = std::make_shared<ASTDropQuery>();
drop_query->database = database_name;
drop_query->table = table_name;
drop_query->kind = ASTDropQuery::Kind::Drop;
ASTPtr ast_drop_query = drop_query;
InterpreterDropQuery drop_interpreter(ast_drop_query, global_context);
drop_interpreter.execute();

View File

@ -258,7 +258,7 @@ def main(args):
# Keep same default values as in queries/shell_config.sh
os.environ.setdefault("CLICKHOUSE_BINARY", args.binary)
#os.environ.setdefault("CLICKHOUSE_CLIENT", args.client)
os.environ.setdefault("CLICKHOUSE_CLIENT", args.client)
os.environ.setdefault("CLICKHOUSE_CONFIG", args.configserver)
if args.configclient:
os.environ.setdefault("CLICKHOUSE_CONFIG_CLIENT", args.configclient)

View File

@ -0,0 +1 @@
lv

View File

@ -0,0 +1,9 @@
DROP TABLE IF EXISTS test.mt;
CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple();
CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt;
SHOW TABLES LIKE 'lv';
DROP TABLE test.lv;
DROP TABLE test.mt;

View File

@ -0,0 +1,4 @@
lv
1
2
3

View File

@ -0,0 +1,14 @@
DROP TABLE IF EXISTS test.lv;
DROP TABLE IF EXISTS test.mt;
CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple();
CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt;
SHOW TABLES LIKE 'lv';
INSERT INTO test.mt VALUES (1),(2),(3);
SELECT * FROM test.lv;
DROP TABLE test.lv;
DROP TABLE test.mt;

View File

@ -0,0 +1,4 @@
lv
1 1
2 1
3 1

View File

@ -0,0 +1,14 @@
DROP TABLE IF EXISTS test.lv;
DROP TABLE IF EXISTS test.mt;
CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple();
CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt;
SHOW TABLES LIKE 'lv';
INSERT INTO test.mt VALUES (1),(2),(3);
SELECT *,_version FROM test.lv;
DROP TABLE test.lv;
DROP TABLE test.mt;

View File

@ -0,0 +1,18 @@
DROP TABLE IF EXISTS test.lv;
DROP TABLE IF EXISTS test.mt;
CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple();
CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt;
SHOW TABLES LIKE 'lv';
INSERT INTO test.mt VALUES (1),(2),(3);
SELECT sum(a) FROM test.lv;
INSERT INTO test.mt VALUES (4),(5),(6);
SELECT sum(a) FROM test.lv;
DROP TABLE test.lv;
DROP TABLE test.mt;

View File

@ -0,0 +1,4 @@
lv
1 c9d39b11cce79112219a73aaa319b475
1 4cd0592103888d4682de9a32a23602e3
1 2186dbea325ee4c56b67e9b792e993a3

View File

@ -0,0 +1,20 @@
DROP TABLE IF EXISTS test.lv;
DROP TABLE IF EXISTS test.mt;
CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple();
CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt;
SHOW TABLES LIKE 'lv';
WATCH test.lv EVENTS LIMIT 0;
INSERT INTO test.mt VALUES (1),(2),(3);
WATCH test.lv EVENTS LIMIT 0;
INSERT INTO test.mt VALUES (4),(5),(6);
WATCH test.lv EVENTS LIMIT 0;
DROP TABLE test.lv;
DROP TABLE test.mt;

View File

@ -0,0 +1,4 @@
lv
0 1
6 1
21 1

View File

@ -0,0 +1,20 @@
DROP TABLE IF EXISTS test.lv;
DROP TABLE IF EXISTS test.mt;
CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple();
CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt;
SHOW TABLES LIKE 'lv';
WATCH test.lv LIMIT 0;
INSERT INTO test.mt VALUES (1),(2),(3);
WATCH test.lv LIMIT 0;
INSERT INTO test.mt VALUES (4),(5),(6);
WATCH test.lv LIMIT 0;
DROP TABLE test.lv;
DROP TABLE test.mt;

View File

@ -0,0 +1,46 @@
#!/usr/bin/env python
import imp
import os
import sys
import signal
CURDIR = os.path.dirname(os.path.realpath(__file__))
uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'uexpect.py'))
def client(name=''):
client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT'))
client.eol('\r')
# Note: uncomment this line for debugging
#client.logger(sys.stdout, prefix=name)
client.timeout(2)
return client
prompt = ':\) '
end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n'
client1 = client('client1>')
client2 = client('client2>')
client1.expect(prompt)
client2.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)
client1.send('WATCH test.lv')
client2.send('INSERT INTO test.mt VALUES (1),(2),(3)')
client1.expect(r'6.*2' + end_of_block)
client2.send('INSERT INTO test.mt VALUES (4),(5),(6)')
client1.expect(r'21.*3' + end_of_block)
# send Ctrl-C
os.kill(client1.process.pid,signal.SIGINT)
client1.expect(prompt)
client1.send('DROP TABLE test.lv')
client1.expect(prompt)
client1.send('DROP TABLE test.mt')
client1.expect(prompt)

View File

@ -0,0 +1,4 @@
temporary_live_view_timeout 5
lv
0
0

View File

@ -0,0 +1,14 @@
DROP TABLE IF EXISTS test.lv;
DROP TABLE IF EXISTS test.mt;
SELECT name, value from system.settings WHERE name = 'temporary_live_view_timeout';
CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple();
CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt;
SHOW TABLES LIKE 'lv';
SELECT sleep(3);
SELECT sleep(2);
SHOW TABLES LIKE 'lv';
DROP TABLE test.mt;

View File

@ -0,0 +1,4 @@
lv
0 1
6 1
21 1

View File

@ -0,0 +1,20 @@
DROP TABLE IF EXISTS test.lv;
DROP TABLE IF EXISTS test.mt;
CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple();
CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt;
SHOW TABLES LIKE 'lv';
WATCH test.lv LIMIT 0;
INSERT INTO test.mt VALUES (1),(2),(3);
WATCH test.lv LIMIT 0;
INSERT INTO test.mt VALUES (4),(5),(6);
WATCH test.lv LIMIT 0;
DROP TABLE test.lv;
DROP TABLE test.mt;

View File

@ -0,0 +1,46 @@
#!/usr/bin/env python
import imp
import os
import sys
import signal
CURDIR = os.path.dirname(os.path.realpath(__file__))
uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'uexpect.py'))
def client(name=''):
client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT'))
client.eol('\r')
# Note: uncomment this line for debugging
#client.logger(sys.stdout, prefix=name)
client.timeout(2)
return client
prompt = ':\) '
end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n'
client1 = client('client1>')
client2 = client('client2>')
client1.expect(prompt)
client2.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 TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt')
client1.expect(prompt)
client1.send('WATCH test.lv')
client2.send('INSERT INTO test.mt VALUES (1),(2),(3)')
client1.expect(r'6.*2' + end_of_block)
client2.send('INSERT INTO test.mt VALUES (4),(5),(6)')
client1.expect(r'21.*3' + end_of_block)
# send Ctrl-C
os.kill(client1.process.pid,signal.SIGINT)
client1.expect(prompt)
client1.send('DROP TABLE test.lv')
client1.expect(prompt)
client1.send('DROP TABLE test.mt')
client1.expect(prompt)

View File

@ -0,0 +1,57 @@
#!/usr/bin/env python
import imp
import os
import sys
import signal
CURDIR = os.path.dirname(os.path.realpath(__file__))
uexpect = imp.load_source('uexpect', os.path.join(CURDIR, 'uexpect.py'))
def client(name=''):
client = uexpect.spawn(os.environ.get('CLICKHOUSE_CLIENT'))
client.eol('\r')
# Note: uncomment this line for debugging
#client.logger(sys.stdout, prefix=name)
client.timeout(2)
return client
prompt = ':\) '
end_of_block = r'.*\xe2\x94\x82\r\n.*\xe2\x94\x98\r\n'
client1 = client('client1>')
client2 = client('client2>')
client1.expect(prompt)
client2.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 TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt')
client1.expect(prompt)
client1.send('SELECT 4')
client1.expect(prompt, timeout=4)
client1.send('WATCH test.lv')
client2.send('INSERT INTO test.mt VALUES (1),(2),(3)')
client2.expect(prompt)
client1.expect(r'6.*2' + end_of_block)
client2.send('SELECT sleep(3)')
client2.expect(prompt, timeout=4)
client2.send('INSERT INTO test.mt VALUES (4),(5),(6)')
client2.expect(prompt)
client1.expect(r'21.*3' + end_of_block)
# send Ctrl-C
os.kill(client1.process.pid,signal.SIGINT)
client1.expect(prompt)
client1.send('SELECT sleep(3)')
client1.expect(prompt, timeout=4)
client1.send('SELECT sleep(3)')
client1.expect(prompt, timeout=4)
client1.send('DROP TABLE test.lv')
client1.expect('Table test.lv doesn\'t exist')
client1.expect(prompt)
client1.send('DROP TABLE test.mt')
client1.expect(prompt)

View File

@ -0,0 +1,187 @@
# Copyright (c) 2019 Vitaliy Zakaznikov
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
# You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
import os
import pty
import time
import sys
import re
from threading import Thread
from subprocess import Popen
from Queue import Queue, Empty
class TimeoutError(Exception):
def __init__(self, timeout):
self.timeout = timeout
def __str__(self):
return 'Timeout %.3fs' % float(self.timeout)
class ExpectTimeoutError(Exception):
def __init__(self, pattern, timeout, buffer):
self.pattern = pattern
self.timeout = timeout
self.buffer = buffer
def __str__(self):
return ('Timeout %.3fs ' % float(self.timeout) +
'for %s ' % repr(self.pattern.pattern) +
'buffer ends with %s ' % repr(self.buffer[-80:]) +
'or \'%s\'' % ','.join(['%x' % ord(c) for c in self.buffer[-80:]]))
class IO(object):
class EOF(object):
pass
class Timeout(object):
pass
EOF = EOF
TIMEOUT = Timeout
class Logger(object):
def __init__(self, logger, prefix=''):
self._logger = logger
self._prefix = prefix
def write(self, data):
self._logger.write(('\n' + data).replace('\n','\n' + self._prefix))
def flush(self):
self._logger.flush()
def __init__(self, process, master, queue):
self.process = process
self.master = master
self.queue = queue
self.buffer = None
self.before = None
self.after = None
self.match = None
self.pattern = None
self._timeout = None
self._logger = None
self._eol = ''
def logger(self, logger=None, prefix=''):
if logger:
self._logger = self.Logger(logger, prefix=prefix)
return self._logger
def timeout(self, timeout=None):
if timeout:
self._timeout = timeout
return self._timeout
def eol(self, eol=None):
if eol:
self._eol = eol
return self._eol
def close(self):
if self._logger:
self._logger.write('\n')
self._logger.flush()
def send(self, data, eol=None):
if eol is None:
eol = self._eol
return self.write(data + eol)
def write(self, data):
return os.write(self.master, data)
def expect(self, pattern, timeout=None):
self.match = None
self.before = None
self.after = None
pattern = re.compile(pattern)
if timeout is None:
timeout = self._timeout
while timeout >= 0:
start_time = time.time()
try:
data = self.read(timeout=timeout, raise_exception=True)
except TimeoutError:
if self._logger:
self._logger.write(self.buffer + '\n')
self._logger.flush()
exception = ExpectTimeoutError(pattern, timeout, self.buffer)
self.buffer = None
raise exception
timeout -= (time.time() - start_time)
if data:
self.buffer = self.buffer + data if self.buffer else data
self.match = pattern.search(self.buffer, 0)
if self.match:
self.after = self.buffer[self.match.start():self.match.end()]
self.before = self.buffer[:self.match.start()]
self.buffer = self.buffer[self.match.end():]
break
if self._logger:
self._logger.write(self.before + self.after)
self._logger.flush()
return self.match
def read(self, timeout=0, raise_exception=False):
data = ''
try:
while timeout >= 0 :
start_time = time.time()
data += self.queue.get(timeout=timeout)
if data:
break
timeout -= (time.time() - start_time)
except Empty:
if data:
return data
if raise_exception:
raise TimeoutError(timeout)
pass
return data
def spawn(*command):
master, slave = pty.openpty()
process = Popen(command, preexec_fn=os.setsid, stdout=slave, stdin=slave, stderr=slave, bufsize=1)
os.close(slave)
queue = Queue()
thread = Thread(target=reader, args=(master, queue))
thread.daemon = True
thread.start()
return IO(process, master, queue)
def reader(out, queue):
while True:
data = os.read(out, 65536)
queue.put(data)
if __name__ == '__main__':
io = spawn('/bin/bash')
prompt = '\$ '
io.logger(sys.stdout)
io.timeout(2)
io.eol('\r')
io.expect(prompt)
io.send('clickhouse-client')
prompt = ':\) '
io.expect(prompt)
io.send('SELECT 1')
io.expect(prompt)
io.send('SHOW TABLES')
io.expect('test')
io.expect(prompt)
io.close()