mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Replace unit test to functional test
This commit is contained in:
parent
57b480e65d
commit
da8a938069
@ -415,6 +415,6 @@ if (ENABLE_TESTS AND USE_GTEST)
|
||||
-Wno-gnu-zero-variadic-macro-arguments
|
||||
)
|
||||
|
||||
target_link_libraries(unit_tests_dbms PRIVATE ${GTEST_BOTH_LIBRARIES} clickhouse_functions clickhouse_aggregate_functions clickhouse_table_functions clickhouse_parsers dbms clickhouse_common_zookeeper string_utils)
|
||||
target_link_libraries(unit_tests_dbms PRIVATE ${GTEST_BOTH_LIBRARIES} clickhouse_functions clickhouse_aggregate_functions clickhouse_parsers dbms clickhouse_common_zookeeper string_utils)
|
||||
add_check(unit_tests_dbms)
|
||||
endif ()
|
||||
|
@ -1,92 +0,0 @@
|
||||
#include <Parsers/obfuscateQueries.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/WriteBufferFromFileDescriptor.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/registerFunctions.h>
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/registerAggregateFunctions.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <TableFunctions/registerTableFunctions.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/registerStorages.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
#include <Common/tests/gtest_global_context.h>
|
||||
|
||||
|
||||
using namespace DB;
|
||||
|
||||
|
||||
TEST(ObfuscateQueries, Test1)
|
||||
{
|
||||
WordMap obfuscated_words_map;
|
||||
WordSet used_nouns;
|
||||
SipHash hash_func;
|
||||
|
||||
std::string salt = "Hello, world";
|
||||
hash_func.update(salt);
|
||||
|
||||
SharedContextHolder shared_context;
|
||||
const ContextHolder & context_holder = getContext();
|
||||
|
||||
registerFunctions();
|
||||
registerAggregateFunctions();
|
||||
registerTableFunctions();
|
||||
registerStorages();
|
||||
|
||||
std::unordered_set<std::string> additional_names;
|
||||
|
||||
auto all_known_storage_names = StorageFactory::instance().getAllRegisteredNames();
|
||||
auto all_known_data_type_names = DataTypeFactory::instance().getAllRegisteredNames();
|
||||
|
||||
additional_names.insert(all_known_storage_names.begin(), all_known_storage_names.end());
|
||||
additional_names.insert(all_known_data_type_names.begin(), all_known_data_type_names.end());
|
||||
|
||||
KnownIdentifierFunc is_known_identifier = [&](std::string_view name)
|
||||
{
|
||||
std::string what(name);
|
||||
|
||||
return FunctionFactory::instance().tryGet(what, context_holder.context) != nullptr
|
||||
|| AggregateFunctionFactory::instance().isAggregateFunctionName(what)
|
||||
|| TableFunctionFactory::instance().isTableFunctionName(what)
|
||||
|| additional_names.count(what);
|
||||
};
|
||||
|
||||
WriteBufferFromOwnString out;
|
||||
|
||||
obfuscateQueries(
|
||||
R"(
|
||||
SELECT
|
||||
VisitID,
|
||||
Goals.ID, Goals.EventTime,
|
||||
WatchIDs,
|
||||
EAction.ProductName, EAction.ProductPrice, EAction.ProductCurrency, EAction.ProductQuantity, EAction.EventTime, EAction.Type
|
||||
FROM merge.visits_v2
|
||||
WHERE
|
||||
StartDate >= '2020-09-17' AND StartDate <= '2020-09-25'
|
||||
AND CounterID = 24226447
|
||||
AND intHash32(UserID) = 416638616 AND intHash64(UserID) = 13269091395366875299
|
||||
AND VisitID IN (5653048135597886819, 5556254872710352304, 5516214175671455313, 5476714937521999313, 5464051549483503043)
|
||||
AND Sign = 1
|
||||
)",
|
||||
out, obfuscated_words_map, used_nouns, hash_func, is_known_identifier);
|
||||
|
||||
EXPECT_EQ(out.str(), R"(
|
||||
SELECT
|
||||
CorduroyID,
|
||||
Steel.ID, Steel.AcornSidestream,
|
||||
WealthBRANCH,
|
||||
GOVERNMENT.SedimentName, GOVERNMENT.SedimentExhaustion, GOVERNMENT.SedimentFencing, GOVERNMENT.SedimentOpossum, GOVERNMENT.AcornSidestream, GOVERNMENT.Lute
|
||||
FROM merge.luncheonette_pants
|
||||
WHERE
|
||||
GovernanceCreche >= '2021-04-16' AND GovernanceCreche <= '2021-04-24'
|
||||
AND StarboardID = 26446940
|
||||
AND intHash32(MessyID) = 474525514 AND intHash64(MessyID) = 13916317227779800149
|
||||
AND CorduroyID IN (5223158832904664474, 5605365157729463108, 7543250143731591192, 8715842063486405567, 7837015536326316923)
|
||||
AND Tea = 1
|
||||
)");
|
||||
}
|
||||
|
@ -1,2 +1,16 @@
|
||||
SELECT 116, 'Qqfu://2020-02-10isqkc1203 sp 2000-05-27T18:38:01', 13e100, Residue_id_breakfastDevice, park(Innervation), avgIf(remote('128.0.0.1'))
|
||||
SELECT shell_dust_tintype between crumb and shoat, case when peach >= 116 then bombing else null end
|
||||
|
||||
SELECT
|
||||
ChimeID,
|
||||
Testimonial.ID, Testimonial.SipCauseway,
|
||||
TankfulTRUMPET,
|
||||
HUMIDITY.TermiteName, HUMIDITY.TermiteSculptural, HUMIDITY.TermiteGuilt, HUMIDITY.TermiteIntensity, HUMIDITY.SipCauseway, HUMIDITY.Coat
|
||||
FROM merge.tinkle_efficiency
|
||||
WHERE
|
||||
FaithSeller >= '2020-10-13' AND FaithSeller <= '2020-10-21'
|
||||
AND MandolinID = 30750384
|
||||
AND intHash32(GafferID) = 448362928 AND intHash64(GafferID) = 12572659331310383983
|
||||
AND ChimeID IN (8195672321757027078, 7079643623150622129, 5057006826979676478, 7886875230160484653, 7494974311229040743)
|
||||
AND Stot = 1
|
||||
|
||||
|
@ -6,3 +6,17 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
$CLICKHOUSE_FORMAT --seed Hello --obfuscate <<< "SELECT 123, 'Test://2020-01-01hello1234 at 2000-01-01T01:02:03', 12e100, Gibberish_id_testCool, hello(World), avgIf(remote('127.0.0.1'))"
|
||||
$CLICKHOUSE_FORMAT --seed Hello --obfuscate <<< "SELECT cost_first_screen between a and b, case when x >= 123 then y else null end"
|
||||
|
||||
$CLICKHOUSE_FORMAT --seed Hello --obfuscate <<< "
|
||||
SELECT
|
||||
VisitID,
|
||||
Goals.ID, Goals.EventTime,
|
||||
WatchIDs,
|
||||
EAction.ProductName, EAction.ProductPrice, EAction.ProductCurrency, EAction.ProductQuantity, EAction.EventTime, EAction.Type
|
||||
FROM merge.visits_v2
|
||||
WHERE
|
||||
StartDate >= '2020-09-17' AND StartDate <= '2020-09-25'
|
||||
AND CounterID = 24226447
|
||||
AND intHash32(UserID) = 416638616 AND intHash64(UserID) = 13269091395366875299
|
||||
AND VisitID IN (5653048135597886819, 5556254872710352304, 5516214175671455313, 5476714937521999313, 5464051549483503043)
|
||||
AND Sign = 1
|
||||
"
|
||||
|
Loading…
Reference in New Issue
Block a user