add tests

This commit is contained in:
Mikhail Artemenko 2022-01-14 20:58:56 +03:00
parent e24d2b4d34
commit 698364f7b0
16 changed files with 20035 additions and 160 deletions

View File

@ -610,10 +610,3 @@ add_subdirectory (utils)
include (cmake/print_include_directories.cmake)
include (cmake/sanitize_target_link_libraries.cmake)
add_executable (meili src/Storages/MeiliSearch/meili.cpp
src/Storages/MeiliSearch/MeiliSearchConnection.cpp
)
target_link_libraries (meili PRIVATE CURL::libcurl)
target_link_libraries (meili PRIVATE clickhouse_common_io)

View File

@ -0,0 +1,16 @@
version: '2.3'
services:
meili1:
image: getmeili/meilisearch
restart: always
command: ./meilisearch
ports:
- ${MEILI_EXTERNAL_PORT}:${MEILI_INTERNAL_PORT}
meili_secure:
image: getmeili/meilisearch
restart: always
command: ./meilisearch --master-key="password"
ports:
- ${MEILI_SECURE_EXTERNAL_PORT}:${MEILI_SECURE_INTERNAL_PORT}

View File

@ -87,7 +87,7 @@ if(USE_RDKAFKA)
add_headers_and_sources(dbms Storages/Kafka)
endif()
if(ENABLE_CURL)
if (ENABLE_CURL)
add_headers_and_sources(dbms Storages/MeiliSearch)
endif()
@ -337,7 +337,7 @@ if (USE_RDKAFKA)
endif()
endif()
if(ENABLE_CURL)
if (ENABLE_CURL)
dbms_target_include_directories(SYSTEM BEFORE PRIVATE ${CURL_INCLUDE_DIRS})
endif()

View File

@ -9,6 +9,9 @@ namespace DB
namespace
{
// This class is a stub for the meiliMatch function in the where section of the query,
// this function is used to pass parameters to the MeiliSearch storage engine
class FunctionMeiliMatch : public IFunction
{
public:
@ -39,15 +42,19 @@ public:
return 0;
}
bool isVariadic() const override {
return true;
bool isVariadic() const override
{
return true;
}
bool isDeterministic() const override { return false; }
bool isDeterministic() const override
{
return false;
}
bool isDeterministicInScopeOfQuery() const override
{
return false;
bool isDeterministicInScopeOfQuery() const override
{
return false;
}
DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override
@ -55,7 +62,7 @@ public:
return std::make_shared<DataTypeUInt8>();
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName&, const DataTypePtr&, size_t input_rows_count) const override
ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override
{
return ColumnUInt8::create(input_rows_count, 1u);
}

View File

@ -1,5 +1,5 @@
#include "MeiliSearchConnection.h"
#include <iostream>
#include <string_view>
#include <curl/curl.h>
#include <Common/Exception.h>
@ -14,29 +14,47 @@ MeiliSearchConnection::MeiliSearchConnection(const MeiliConfig & conf) : config{
{
}
MeiliSearchConnection::MeiliSearchConnection(MeiliConfig && conf) : config{std::move(conf)}
{
}
static size_t writeCallback(void * contents, size_t size, size_t nmemb, void * userp)
{
(static_cast<std::string *>(userp))->append(static_cast<char *>(contents), size * nmemb);
return size * nmemb;
}
String MeiliSearchConnection::searchQuery(const std::unordered_map<String, String> & query_params) const
CURLcode MeiliSearchConnection::execQuery(
std::string_view url,
std::string_view post_fields,
std::string& response_buffer) const
{
CURLcode ret_code;
CURL * hnd;
struct curl_slist * slist1;
CURL * handle;
struct curl_slist * headers_list;
slist1 = nullptr;
slist1 = curl_slist_append(slist1, "Content-Type: application/json");
slist1 = curl_slist_append(slist1, config.key.c_str());
headers_list = nullptr;
headers_list = curl_slist_append(headers_list, "Content-Type: application/json");
headers_list = curl_slist_append(headers_list, config.key.c_str());
handle = curl_easy_init();
curl_easy_setopt(handle, CURLOPT_URL, url.data());
curl_easy_setopt(handle, CURLOPT_POSTFIELDS, post_fields.data());
curl_easy_setopt(handle, CURLOPT_POSTFIELDSIZE_LARGE, post_fields.size());
curl_easy_setopt(handle, CURLOPT_HTTPHEADER, headers_list);
curl_easy_setopt(handle, CURLOPT_MAXREDIRS, 50L);
curl_easy_setopt(handle, CURLOPT_CUSTOMREQUEST, "POST");
curl_easy_setopt(handle, CURLOPT_WRITEFUNCTION, writeCallback);
curl_easy_setopt(handle, CURLOPT_WRITEDATA, &response_buffer);
ret_code = curl_easy_perform(handle);
curl_easy_cleanup(handle);
curl_slist_free_all(headers_list);
return ret_code;
}
String MeiliSearchConnection::searchQuery(const std::unordered_map<String, String> & query_params) const
{
std::string response_buffer;
std::string post_fields = "{";
for (const auto & q_attr : query_params)
post_fields += q_attr.first + ":" + q_attr.second + ",";
@ -44,25 +62,9 @@ String MeiliSearchConnection::searchQuery(const std::unordered_map<String, Strin
std::string url = config.connection_string + "search";
hnd = curl_easy_init();
curl_easy_setopt(hnd, CURLOPT_BUFFERSIZE, 102400L);
curl_easy_setopt(hnd, CURLOPT_URL, url.c_str());
curl_easy_setopt(hnd, CURLOPT_NOPROGRESS, 1L);
curl_easy_setopt(hnd, CURLOPT_POSTFIELDS, post_fields.c_str());
curl_easy_setopt(hnd, CURLOPT_POSTFIELDSIZE_LARGE, post_fields.size());
curl_easy_setopt(hnd, CURLOPT_HTTPHEADER, slist1);
curl_easy_setopt(hnd, CURLOPT_MAXREDIRS, 50L);
curl_easy_setopt(hnd, CURLOPT_HTTP_VERSION, CURL_HTTP_VERSION_2TLS);
curl_easy_setopt(hnd, CURLOPT_CUSTOMREQUEST, "POST");
curl_easy_setopt(hnd, CURLOPT_WRITEFUNCTION, writeCallback);
curl_easy_setopt(hnd, CURLOPT_WRITEDATA, &response_buffer);
CURLcode ret_code = execQuery(url, post_fields, response_buffer);
ret_code = curl_easy_perform(hnd);
curl_easy_cleanup(hnd);
curl_slist_free_all(slist1);
if (ret_code != 0)
if (ret_code != CURLE_OK)
throw Exception(ErrorCodes::NETWORK_ERROR, curl_easy_strerror(ret_code));
return response_buffer;
@ -70,36 +72,13 @@ String MeiliSearchConnection::searchQuery(const std::unordered_map<String, Strin
String MeiliSearchConnection::updateQuery(std::string_view data) const
{
CURLcode ret_code;
CURL * hnd;
struct curl_slist * slist1;
slist1 = nullptr;
slist1 = curl_slist_append(slist1, "Content-Type: application/json");
slist1 = curl_slist_append(slist1, config.key.c_str());
std::string response_buffer;
std::string url = config.connection_string + "documents";
hnd = curl_easy_init();
curl_easy_setopt(hnd, CURLOPT_BUFFERSIZE, 102400L);
curl_easy_setopt(hnd, CURLOPT_URL, url.c_str());
curl_easy_setopt(hnd, CURLOPT_NOPROGRESS, 1L);
curl_easy_setopt(hnd, CURLOPT_POSTFIELDS, data.data());
curl_easy_setopt(hnd, CURLOPT_POSTFIELDSIZE_LARGE, data.size());
curl_easy_setopt(hnd, CURLOPT_HTTPHEADER, slist1);
curl_easy_setopt(hnd, CURLOPT_MAXREDIRS, 50L);
curl_easy_setopt(hnd, CURLOPT_HTTP_VERSION, CURL_HTTP_VERSION_2TLS);
curl_easy_setopt(hnd, CURLOPT_CUSTOMREQUEST, "POST");
curl_easy_setopt(hnd, CURLOPT_WRITEFUNCTION, writeCallback);
curl_easy_setopt(hnd, CURLOPT_WRITEDATA, &response_buffer);
CURLcode ret_code = execQuery(url, data, response_buffer);
ret_code = curl_easy_perform(hnd);
curl_easy_cleanup(hnd);
curl_slist_free_all(slist1);
if (ret_code != 0)
if (ret_code != CURLE_OK)
throw Exception(ErrorCodes::NETWORK_ERROR, curl_easy_strerror(ret_code));
return response_buffer;

View File

@ -1,14 +1,10 @@
#pragma once
#include <cstdint>
#include <iostream>
#include <memory>
#include <string>
#include <string_view>
#include <unordered_map>
#include <utility>
#include <vector>
#include <base/types.h>
#include "curl/curl.h"
namespace DB
{
@ -21,7 +17,7 @@ struct MeiliSearchConfiguration
MeiliSearchConfiguration(const String & url_, const String & index_, const String & key_) : index{index_}
{
connection_string = url_ + "/indexes/" + index_ + "/";
key = "X-Meili-API-Key:" + key_;
key = "Authorization: Bearer " + key_;
}
};
@ -32,13 +28,17 @@ class MeiliSearchConnection
public:
explicit MeiliSearchConnection(const MeiliConfig & config);
explicit MeiliSearchConnection(MeiliConfig && config);
String searchQuery(const std::unordered_map<String, String> & query_params) const;
String updateQuery(std::string_view data) const;
private:
CURLcode execQuery(
std::string_view url,
std::string_view post_fields,
std::string& response_buffer) const;
MeiliConfig config;
};

View File

@ -14,11 +14,10 @@ namespace ErrorCodes
}
SinkMeiliSearch::SinkMeiliSearch(
const MeiliSearchConfiguration & config_, const Block & sample_block_, ContextPtr local_context_, UInt64 max_block_size_)
const MeiliSearchConfiguration & config_, const Block & sample_block_, ContextPtr local_context_)
: SinkToStorage(sample_block_)
, connection(config_)
, local_context{local_context_}
, max_block_size{max_block_size_}
, sample_block{sample_block_}
{
}
@ -27,7 +26,8 @@ void extractData(std::string_view& view) {
int ind = view.find("\"data\":") + 9;
view.remove_prefix(ind);
int bal = ind = 1;
while (bal > 0) {
while (bal > 0)
{
if (view[ind] == '[') ++bal;
else if (view[ind] == ']') --bal;
++ind;
@ -54,43 +54,10 @@ void SinkMeiliSearch::writeBlockData(const Block & block) const
throw Exception(ErrorCodes::MEILISEARCH_EXCEPTION, jres.getValue().toString());
}
Blocks SinkMeiliSearch::splitBlocks(const Block & block, const size_t & max_rows) const
{
/// Avoid Excessive copy when block is small enough
if (block.rows() <= max_rows)
return Blocks{std::move(block)};
const size_t split_block_size = ceil(block.rows() * 1.0 / max_rows);
Blocks split_blocks(split_block_size);
for (size_t idx = 0; idx < split_block_size; ++idx)
split_blocks[idx] = block.cloneEmpty();
const size_t columns = block.columns();
const size_t rows = block.rows();
size_t offsets = 0;
UInt64 limits = max_block_size;
for (size_t idx = 0; idx < split_block_size; ++idx)
{
/// For last batch, limits should be the remain size
if (idx == split_block_size - 1)
limits = rows - offsets;
for (size_t col_idx = 0; col_idx < columns; ++col_idx)
split_blocks[idx].getByPosition(col_idx).column = block.getByPosition(col_idx).column->cut(offsets, limits);
offsets += max_block_size;
}
return split_blocks;
}
void SinkMeiliSearch::consume(Chunk chunk)
{
auto block = getHeader().cloneWithColumns(chunk.detachColumns());
auto blocks = splitBlocks(block, max_block_size);
for (const auto & b : blocks)
writeBlockData(b);
writeBlockData(block);
}
}

View File

@ -7,11 +7,12 @@
namespace DB
{
class SinkMeiliSearch : public SinkToStorage
{
public:
SinkMeiliSearch(
const MeiliSearchConfiguration & config_, const Block & sample_block_, ContextPtr local_context_, UInt64 max_block_size_);
const MeiliSearchConfiguration & config_, const Block & sample_block_, ContextPtr local_context_);
String getName() const override { return "SinkMeiliSearch"; }
@ -19,12 +20,9 @@ public:
void writeBlockData(const Block & block) const;
Blocks splitBlocks(const Block & block, const size_t & max_rows) const;
private:
MeiliSearchConnection connection;
ContextPtr local_context;
const UInt64 max_block_size;
Block sample_block;
};

View File

@ -20,13 +20,12 @@ MeiliSearchSource::MeiliSearchSource(
const MeiliSearchConfiguration & config,
const Block & sample_block,
UInt64 max_block_size_,
UInt64 offset_,
std::unordered_map<String, String> query_params_)
: SourceWithProgress(sample_block.cloneEmpty())
, connection(config)
, max_block_size{max_block_size_}
, query_params{query_params_}
, offset{offset_}
, offset{0}
{
description.init(sample_block);
@ -43,24 +42,31 @@ MeiliSearchSource::MeiliSearchSource(
MeiliSearchSource::~MeiliSearchSource() = default;
void insertWithTypeId(MutableColumnPtr & column, JSON kv_pair, int type_id)
void insertWithTypeId(MutableColumnPtr & column, JSON kv_pair, ExternalResultDescription::ValueType type_id)
{
if (type_id == Field::Types::UInt64)
if (type_id == ExternalResultDescription::ValueType::vtUInt64 ||
type_id == ExternalResultDescription::ValueType::vtUInt32 ||
type_id == ExternalResultDescription::ValueType::vtUInt16 ||
type_id == ExternalResultDescription::ValueType::vtUInt8)
{
auto value = kv_pair.getValue().get<UInt64>();
column->insert(value);
}
else if (type_id == Field::Types::Int64)
else if (type_id == ExternalResultDescription::ValueType::vtInt64 ||
type_id == ExternalResultDescription::ValueType::vtInt32 ||
type_id == ExternalResultDescription::ValueType::vtInt16 ||
type_id == ExternalResultDescription::ValueType::vtInt8)
{
auto value = kv_pair.getValue().get<Int64>();
column->insert(value);
}
else if (type_id == Field::Types::String)
else if (type_id == ExternalResultDescription::ValueType::vtString)
{
auto value = kv_pair.getValue().get<String>();
column->insert(value);
}
else if (type_id == Field::Types::Float64)
else if (type_id == ExternalResultDescription::ValueType::vtFloat64 ||
type_id == ExternalResultDescription::ValueType::vtFloat32)
{
auto value = kv_pair.getValue().get<Float64>();
column->insert(value);
@ -72,11 +78,7 @@ Chunk MeiliSearchSource::generate()
if (all_read)
return {};
MutableColumns columns(description.sample_block.columns());
const size_t size = columns.size();
for (const auto i : collections::range(0, size))
columns[i] = description.sample_block.getByPosition(i).column->cloneEmpty();
MutableColumns columns = description.sample_block.cloneEmptyColumns();
query_params[doubleQuoteString("offset")] = std::to_string(offset);
auto response = connection.searchQuery(query_params);
@ -95,8 +97,9 @@ Chunk MeiliSearchSource::generate()
for (const auto kv_pair : json)
{
const auto & name = kv_pair.getName();
auto & col = columns[description.sample_block.getPositionByName(name)];
Field::Types::Which type_id = description.sample_block.getByName(name).type->getDefault().getType();
int pos = description.sample_block.getPositionByName(name);
auto & col = columns[pos];
ExternalResultDescription::ValueType type_id = description.types[pos].first;
insertWithTypeId(col, kv_pair, type_id);
}
}

View File

@ -15,7 +15,6 @@ public:
const MeiliSearchConfiguration & config,
const Block & sample_block,
UInt64 max_block_size_,
UInt64 offset_,
std::unordered_map<String, String> query_params_);
~MeiliSearchSource() override;

View File

@ -1,4 +1,3 @@
#include <iostream>
#include <Parsers/ASTSelectQuery.h>
#include <QueryPipeline/Pipe.h>
#include <Storages/IStorage.h>
@ -120,20 +119,15 @@ Pipe StorageMeiliSearch::read(
for (const auto & el : kv_pairs_params)
LOG_TRACE(log, "Parsed parameter: key = " + el.first + ", value = " + el.second);
Block sample_block;
for (const String & column_name : column_names)
{
auto column_data = metadata_snapshot->getColumns().getPhysical(column_name);
sample_block.insert({column_data.type, column_data.name});
}
auto sample_block = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID());
return Pipe(std::make_shared<MeiliSearchSource>(config, sample_block, max_block_size, 0, kv_pairs_params));
return Pipe(std::make_shared<MeiliSearchSource>(config, sample_block, max_block_size, kv_pairs_params));
}
SinkToStoragePtr StorageMeiliSearch::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context)
{
LOG_TRACE(log, "Trying update index: " + config.index);
return std::make_shared<SinkMeiliSearch>(config, metadata_snapshot->getSampleBlock(), local_context, 20000);
return std::make_shared<SinkMeiliSearch>(config, metadata_snapshot->getSampleBlock(), local_context);
}
MeiliSearchConfiguration getConfiguration(ASTs engine_args)

View File

@ -1,14 +0,0 @@
#include <iostream>
#include <string>
#include "MeiliSearchConnection.h"
#include <Common/Exception.h>
#include <base/JSON.h>
int main() {
String s = "{\"updateId\":4}";
JSON jres(s);
std::cout << jres.begin().getName();
return 0;
}

View File

@ -21,6 +21,7 @@ from cassandra.policies import RoundRobinPolicy
import cassandra.cluster
import psycopg2
import pymongo
import meilisearch
import pymysql
import requests
from confluent_kafka.avro.cached_schema_registry_client import \
@ -266,6 +267,7 @@ class ClickHouseCluster:
self.with_kerberized_hdfs = False
self.with_mongo = False
self.with_mongo_secure = False
self.with_meili = False
self.with_net_trics = False
self.with_redis = False
self.with_cassandra = False
@ -319,6 +321,12 @@ class ClickHouseCluster:
self.mongo_host = "mongo1"
self.mongo_port = get_free_port()
# available when with_meili == True
self.meili_host = "meili1"
self.meili_port = get_free_port()
self.meili_secure_host = "meili_secure"
self.meili_secure_port = get_free_port()
# available when with_cassandra == True
self.cassandra_host = "cassandra1"
self.cassandra_port = 9042
@ -729,6 +737,21 @@ class ClickHouseCluster:
self.base_mongo_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name,
'--file', p.join(docker_compose_yml_dir, 'docker_compose_mongo.yml')]
return self.base_mongo_cmd
def setup_meili_cmd(self, instance, env_variables, docker_compose_yml_dir):
self.with_meili = True
env_variables['MEILI_HOST'] = self.meili_host
env_variables['MEILI_EXTERNAL_PORT'] = str(self.meili_port)
env_variables['MEILI_INTERNAL_PORT'] = "7700"
env_variables['MEILI_SECURE_HOST'] = self.meili_secure_host
env_variables['MEILI_SECURE_EXTERNAL_PORT'] = str(self.meili_secure_port)
env_variables['MEILI_SECURE_INTERNAL_PORT'] = "7700"
self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_meili.yml')])
self.base_meili_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name,
'--file', p.join(docker_compose_yml_dir, 'docker_compose_meili.yml')]
return self.base_meili_cmd
def setup_minio_cmd(self, instance, env_variables, docker_compose_yml_dir):
self.with_minio = True
@ -772,7 +795,7 @@ class ClickHouseCluster:
with_mysql_client=False, with_mysql=False, with_mysql8=False, with_mysql_cluster=False,
with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False, clickhouse_path_dir=None,
with_odbc_drivers=False, with_postgres=False, with_postgres_cluster=False, with_hdfs=False,
with_kerberized_hdfs=False, with_mongo=False, with_mongo_secure=False, with_nginx=False,
with_kerberized_hdfs=False, with_mongo=False, with_mongo_secure=False, with_meili=False, with_nginx=False,
with_redis=False, with_minio=False, with_cassandra=False, with_jdbc_bridge=False,
hostname=None, env_variables=None, image="clickhouse/integration-test", tag=None,
stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, external_dirs=None, tmpfs=None,
@ -827,6 +850,7 @@ class ClickHouseCluster:
with_nginx=with_nginx,
with_kerberized_hdfs=with_kerberized_hdfs,
with_mongo=with_mongo or with_mongo_secure,
with_meili=with_meili,
with_redis=with_redis,
with_minio=with_minio,
with_cassandra=with_cassandra,
@ -922,6 +946,10 @@ class ClickHouseCluster:
else:
cmds.append(self.setup_mongo_cmd(instance, env_variables, docker_compose_yml_dir))
if with_meili and not self.with_meili:
cmds.append(self.setup_meili_cmd(instance, env_variables, docker_compose_yml_dir))
if self.with_net_trics:
for cmd in cmds:
cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_net.yml')])
@ -1341,6 +1369,26 @@ class ClickHouseCluster:
except Exception as ex:
logging.debug("Can't connect to Mongo " + str(ex))
time.sleep(1)
def wait_meili_to_start(self, timeout=30):
connection_str = 'http://{host}:{port}'.format(
host='localhost', port=self.meili_port)
client = meilisearch.Client(connection_str)
connection_str_secure = 'http://{host}:{port}'.format(
host='localhost', port=self.meili_secure_port)
client_secure = meilisearch.Client(connection_str_secure, "password")
start = time.time()
while time.time() - start < timeout:
try:
client.get_all_stats()
client_secure.get_all_stats()
logging.debug(f"Connected to MeiliSearch dbs: {client.get_all_stats()}\n{client_secure.get_all_stats()}")
return
except Exception as ex:
logging.debug("Can't connect to MeiliSearch " + str(ex))
time.sleep(1)
def wait_minio_to_start(self, timeout=180, secure=False):
self.minio_ip = self.get_instance_ip(self.minio_host)
@ -1602,6 +1650,13 @@ class ClickHouseCluster:
self.up_called = True
self.wait_mongo_to_start(30, secure=self.with_mongo_secure)
if self.with_meili and self.base_meili_cmd:
logging.debug('Setup MeiliSearch')
run_and_check(self.base_meili_cmd + common_opts)
self.up_called = True
self.wait_meili_to_start()
if self.with_redis and self.base_redis_cmd:
logging.debug('Setup Redis')
subprocess_check_call(self.base_redis_cmd + common_opts)
@ -1843,7 +1898,7 @@ class ClickHouseInstance:
self, cluster, base_path, name, base_config_dir, custom_main_configs, custom_user_configs,
custom_dictionaries,
macros, with_zookeeper, zookeeper_config_path, with_mysql_client, with_mysql, with_mysql8, with_mysql_cluster, with_kafka, with_kerberized_kafka,
with_rabbitmq, with_nginx, with_kerberized_hdfs, with_mongo, with_redis, with_minio, with_jdbc_bridge,
with_rabbitmq, with_nginx, with_kerberized_hdfs, with_mongo, with_meili, with_redis, with_minio, with_jdbc_bridge,
with_cassandra, server_bin_path, odbc_bridge_bin_path, library_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, with_postgres, with_postgres_cluster,
clickhouse_start_command=CLICKHOUSE_START_COMMAND,
main_config_name="config.xml", users_config_name="users.xml", copy_common_configs=True,
@ -1885,6 +1940,7 @@ class ClickHouseInstance:
self.with_nginx = with_nginx
self.with_kerberized_hdfs = with_kerberized_hdfs
self.with_mongo = with_mongo
self.with_meili = with_meili
self.with_redis = with_redis
self.with_minio = with_minio
self.with_cassandra = with_cassandra

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,328 @@
import json
import os
from time import sleep
import meilisearch
import pytest
from helpers.client import QueryRuntimeException
from helpers.cluster import ClickHouseCluster
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
@pytest.fixture(scope="module")
def started_cluster(request):
try:
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('meili',
with_meili=True)
cluster.start()
yield cluster
finally:
cluster.shutdown()
def get_meili_client(started_cluster):
connection_str = 'http://localhost:{}'.format(started_cluster.meili_port)
return meilisearch.Client(connection_str)
def get_meili_secure_client(started_cluster):
connection_str = 'http://localhost:{}'.format(started_cluster.meili_secure_port)
return meilisearch.Client(connection_str, "password")
def push_data(client, table, documents):
ans = table.add_documents(documents)
client.wait_for_task(ans['uid'])
def push_movies(client):
print(SCRIPT_DIR + '/movies.json')
json_file = open(SCRIPT_DIR + '/movies.json')
movies = json.load(json_file)
ans = client.index('movies').add_documents(movies)
client.wait_for_task(ans['uid'], 100000)
@pytest.mark.parametrize('started_cluster', [False], indirect=['started_cluster'])
def test_simple_select(started_cluster):
client = get_meili_client(started_cluster)
table = client.index("new_table")
data = []
for i in range(0, 100):
data.append({'id': i, 'data': hex(i * i)})
push_data(client, table, data)
node = started_cluster.instances['meili']
node.query(
"CREATE TABLE simple_meili_table(id UInt64, data String) ENGINE = MeiliSearch('meili1:7700', 'new_table', '')")
assert node.query("SELECT COUNT() FROM simple_meili_table") == '100\n'
assert node.query("SELECT sum(id) FROM simple_meili_table") == str(sum(range(0, 100))) + '\n'
assert node.query("SELECT data FROM simple_meili_table WHERE id = 42") == hex(42 * 42) + '\n'
node.query("DROP TABLE simple_meili_table")
table.delete()
@pytest.mark.parametrize('started_cluster', [False], indirect=['started_cluster'])
def test_insert(started_cluster):
client = get_meili_client(started_cluster)
new_table = client.index("new_table")
big_table = client.index("big_table")
node = started_cluster.instances['meili']
node.query("CREATE TABLE new_table(id UInt64, data String) ENGINE = MeiliSearch('meili1:7700', 'new_table', '')")
node.query("INSERT INTO new_table (id, data) VALUES (1, '1') (2, '2') (3, '3') (4, '4') (5, '5') (6, '6') (7, '7')")
sleep(1)
assert len(new_table.get_documents()) == 7
node.query("CREATE TABLE big_table(id UInt64, data String) ENGINE = MeiliSearch('meili1:7700', 'big_table', '')")
values = ""
for i in range(1, 40001):
values += "(" + str(i) + ", " + "\'" + str(i) + "\'" + ") "
node.query("INSERT INTO big_table (id, data) VALUES " + values)
sleep(5)
ans = big_table.update_sortable_attributes(['id'])
client.wait_for_task(ans['uid'])
docs = big_table.search("", {"limit":50000, 'sort': ['id:asc']})["hits"]
assert len(docs) == 40000
for i in range(1, 40001):
assert docs[i - 1] == {"id": i, "data": str(i)}
node.query("DROP TABLE new_table")
node.query("DROP TABLE big_table")
new_table.delete()
big_table.delete()
@pytest.mark.parametrize('started_cluster', [False], indirect=['started_cluster'])
def test_meilimatch(started_cluster):
client = get_meili_client(started_cluster)
table = client.index("movies")
table.update_sortable_attributes(['release_date'])
table.update_filterable_attributes(['release_date'])
push_movies(client)
node = started_cluster.instances['meili']
node.query("CREATE TABLE movies_table(id String, title String, release_date Int64) ENGINE = MeiliSearch('meili1:7700', 'movies', '')")
assert node.query("SELECT COUNT() FROM movies_table") == '19546\n'
assert node.query("SELECT COUNT() FROM movies_table WHERE meiliMatch('\"q\"=\"abaca\"')") == '13\n'
assert node.query("SELECT sum(release_date) FROM movies_table WHERE meiliMatch('\"q\"=\"abaca\"')") == '12887532000\n'
real_json = table.search("abaca", {"attributesToRetrieve":["id", "title", "release_date"], "limit":20000})["hits"]
click_ans = "[" + ", ".join(node.query("SELECT * FROM movies_table WHERE \
meiliMatch('\"q\"=\"abaca\"') \
format JSONEachRow settings output_format_json_quote_64bit_integers=0").split("\n")[:-1]) + "]"
click_json = json.loads(click_ans)
assert real_json == click_json
real_json = table.search("abaca", {"attributesToRetrieve":["id", "title", "release_date"], "limit":20000, 'sort': ['release_date:asc']})["hits"]
click_ans = "[" + ", ".join(node.query("SELECT * FROM movies_table WHERE \
meiliMatch('\"q\"=\"abaca\"', '\"sort\"=[\"release_date:asc\"]') \
format JSONEachRow settings output_format_json_quote_64bit_integers=0").split("\n")[:-1]) + "]"
click_json = json.loads(click_ans)
assert real_json == click_json
real_json = table.search("abaca", {"attributesToRetrieve":["id", "title", "release_date"], "limit":20000, 'sort': ['release_date:desc'], 'filter': 'release_date < 700000000'})["hits"]
click_ans = "[" + ", ".join(node.query("SELECT * FROM movies_table WHERE \
meiliMatch('\"q\"=\"abaca\"', '\"sort\"=[\"release_date:asc\"]', '\"filter\"=\"release_date < 700000000\"') \
format JSONEachRow settings output_format_json_quote_64bit_integers=0").split("\n")[:-1]) + "]"
click_json = json.loads(click_ans)
assert real_json == click_json
node.query("DROP TABLE movies_table")
table.delete()
@pytest.mark.parametrize('started_cluster', [False], indirect=['started_cluster'])
def test_incorrect_data_type(started_cluster):
client = get_meili_client(started_cluster)
table = client.index("new_table")
data = []
for i in range(0, 100):
data.append({'id': i, 'data': hex(i * i), 'aaaa': 'Hello'})
push_data(client, table, data)
node = started_cluster.instances['meili']
node.query("CREATE TABLE strange_meili_table(id UInt64, data String, bbbb String) ENGINE = MeiliSearch('meili1:7700', 'new_table', '')")
with pytest.raises(QueryRuntimeException):
node.query("SELECT bbbb FROM strange_meili_table")
node.query("DROP TABLE strange_meili_table")
table.delete()
@pytest.mark.parametrize('started_cluster', [False], indirect=['started_cluster'])
def test_simple_select_secure(started_cluster):
client = get_meili_secure_client(started_cluster)
table = client.index("new_table")
data = []
for i in range(0, 100):
data.append({'id': i, 'data': hex(i * i)})
push_data(client, table, data)
node = started_cluster.instances['meili']
node.query(
"CREATE TABLE simple_meili_table(id UInt64, data String) ENGINE = MeiliSearch('meili_secure:7700', 'new_table', 'password')")
node.query(
"CREATE TABLE wrong_meili_table(id UInt64, data String) ENGINE = MeiliSearch('meili_secure:7700', 'new_table', 'wrong_password')")
assert node.query("SELECT COUNT() FROM simple_meili_table") == '100\n'
assert node.query("SELECT sum(id) FROM simple_meili_table") == str(sum(range(0, 100))) + '\n'
assert node.query("SELECT data FROM simple_meili_table WHERE id = 42") == hex(42 * 42) + '\n'
with pytest.raises(QueryRuntimeException):
node.query("SELECT COUNT() FROM wrong_meili_table")
with pytest.raises(QueryRuntimeException):
node.query("SELECT sum(id) FROM wrong_meili_table")
with pytest.raises(QueryRuntimeException):
node.query("SELECT data FROM wrong_meili_table WHERE id = 42")
node.query("DROP TABLE simple_meili_table")
node.query("DROP TABLE wrong_meili_table")
table.delete()
@pytest.mark.parametrize('started_cluster', [False], indirect=['started_cluster'])
def test_meilimatch_secure(started_cluster):
client = get_meili_secure_client(started_cluster)
table = client.index("movies")
table.update_sortable_attributes(['release_date'])
table.update_filterable_attributes(['release_date'])
push_movies(client)
node = started_cluster.instances['meili']
node.query("CREATE TABLE movies_table(id String, title String, release_date Int64) ENGINE = MeiliSearch('meili_secure:7700', 'movies', 'password')")
assert node.query("SELECT COUNT() FROM movies_table") == '19546\n'
assert node.query("SELECT COUNT() FROM movies_table WHERE meiliMatch('\"q\"=\"abaca\"')") == '13\n'
assert node.query("SELECT sum(release_date) FROM movies_table WHERE meiliMatch('\"q\"=\"abaca\"')") == '12887532000\n'
real_json = table.search("abaca", {"attributesToRetrieve":["id", "title", "release_date"], "limit":20000})["hits"]
click_ans = "[" + ", ".join(node.query("SELECT * FROM movies_table WHERE \
meiliMatch('\"q\"=\"abaca\"') \
format JSONEachRow settings output_format_json_quote_64bit_integers=0").split("\n")[:-1]) + "]"
click_json = json.loads(click_ans)
assert real_json == click_json
real_json = table.search("abaca", {"attributesToRetrieve":["id", "title", "release_date"], "limit":20000, 'sort': ['release_date:asc']})["hits"]
click_ans = "[" + ", ".join(node.query("SELECT * FROM movies_table WHERE \
meiliMatch('\"q\"=\"abaca\"', '\"sort\"=[\"release_date:asc\"]') \
format JSONEachRow settings output_format_json_quote_64bit_integers=0").split("\n")[:-1]) + "]"
click_json = json.loads(click_ans)
assert real_json == click_json
real_json = table.search("abaca", {"attributesToRetrieve":["id", "title", "release_date"], "limit":20000, 'sort': ['release_date:desc'], 'filter': 'release_date < 700000000'})["hits"]
click_ans = "[" + ", ".join(node.query("SELECT * FROM movies_table WHERE \
meiliMatch('\"q\"=\"abaca\"', '\"sort\"=[\"release_date:asc\"]', '\"filter\"=\"release_date < 700000000\"') \
format JSONEachRow settings output_format_json_quote_64bit_integers=0").split("\n")[:-1]) + "]"
click_json = json.loads(click_ans)
assert real_json == click_json
node.query("DROP TABLE movies_table")
table.delete()
@pytest.mark.parametrize('started_cluster', [False], indirect=['started_cluster'])
def test_incorrect_data_type_secure(started_cluster):
client = get_meili_secure_client(started_cluster)
table = client.index("new_table")
data = []
for i in range(0, 100):
data.append({'id': i, 'data': hex(i * i), 'aaaa': 'Hello'})
push_data(client, table, data)
node = started_cluster.instances['meili']
node.query("CREATE TABLE strange_meili_table(id UInt64, data String, bbbb String) ENGINE = MeiliSearch('meili_secure:7700', 'new_table', 'password')")
with pytest.raises(QueryRuntimeException):
node.query("SELECT bbbb FROM strange_meili_table")
node.query("DROP TABLE strange_meili_table")
table.delete()
@pytest.mark.parametrize('started_cluster', [False], indirect=['started_cluster'])
def test_insert_secure(started_cluster):
client = get_meili_secure_client(started_cluster)
new_table = client.index("new_table")
big_table = client.index("big_table")
node = started_cluster.instances['meili']
node.query("CREATE TABLE new_table(id UInt64, data String) ENGINE = MeiliSearch('meili_secure:7700', 'new_table', 'password')")
node.query("INSERT INTO new_table (id, data) VALUES (1, '1') (2, '2') (3, '3') (4, '4') (5, '5') (6, '6') (7, '7')")
sleep(1)
assert len(new_table.get_documents()) == 7
node.query("CREATE TABLE big_table(id UInt64, data String) ENGINE = MeiliSearch('meili_secure:7700', 'big_table', 'password')")
values = ""
for i in range(1, 40001):
values += "(" + str(i) + ", " + "\'" + str(i) + "\'" + ") "
node.query("INSERT INTO big_table (id, data) VALUES " + values)
sleep(5)
ans = big_table.update_sortable_attributes(['id'])
client.wait_for_task(ans['uid'])
docs = big_table.search("", {"limit":50000, 'sort': ['id:asc']})["hits"]
assert len(docs) == 40000
for i in range(1, 40001):
assert docs[i - 1] == {"id": i, "data": str(i)}
node.query("DROP TABLE new_table")
node.query("DROP TABLE big_table")
new_table.delete()
big_table.delete()
@pytest.mark.parametrize('started_cluster', [False], indirect=['started_cluster'])
def test_security_levels(started_cluster):
client = get_meili_secure_client(started_cluster)
search_key = client.get_keys()['results'][0]['key']
admin_key = client.get_keys()['results'][1]['key']
values = ""
for i in range(1, 101):
values += "(" + str(i) + ", " + "\'" + str(i) + "\'" + ") "
node = started_cluster.instances['meili']
node.query(f"CREATE TABLE read_table(id UInt64, data String) ENGINE = MeiliSearch('meili_secure:7700', 'new_table', '{search_key}')")
node.query(f"CREATE TABLE write_table(id UInt64, data String) ENGINE = MeiliSearch('meili_secure:7700', 'new_table', '{admin_key}')")
with pytest.raises(QueryRuntimeException):
node.query("INSERT INTO read_table (id, data) VALUES " + values)
node.query("INSERT INTO write_table (id, data) VALUES " + values)
sleep(1)
ans1 = "[" + ", ".join(node.query("SELECT * FROM read_table \
format JSONEachRow settings output_format_json_quote_64bit_integers=0").split("\n")[:-1]) + "]"
ans2 = "[" + ", ".join(node.query("SELECT * FROM write_table \
format JSONEachRow settings output_format_json_quote_64bit_integers=0").split("\n")[:-1]) + "]"
assert ans1 == ans2
docs = json.loads(ans1)
assert len(docs) == 100
node.query("DROP TABLE read_table")
node.query("DROP TABLE write_table")
client.index("new_table").delete()