mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge pull request #40120 from evillique/obfuscator-schema-inference
Add schema inference to `clickhouse-obfuscator`
This commit is contained in:
commit
95847775b6
@ -24,6 +24,7 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Formats/registerFormats.h>
|
||||
#include <Formats/ReadSchemaUtils.h>
|
||||
#include <Processors/Formats/IInputFormat.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <Processors/Executors/PullingPipelineExecutor.h>
|
||||
@ -38,6 +39,7 @@
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <Compression/CompressedReadBuffer.h>
|
||||
#include <Compression/CompressedWriteBuffer.h>
|
||||
#include <Interpreters/parseColumnsListForTableFunction.h>
|
||||
#include <memory>
|
||||
#include <cmath>
|
||||
#include <unistd.h>
|
||||
@ -1239,7 +1241,6 @@ try
|
||||
|
||||
if (options.count("help")
|
||||
|| !options.count("seed")
|
||||
|| !options.count("structure")
|
||||
|| !options.count("input-format")
|
||||
|| !options.count("output-format"))
|
||||
{
|
||||
@ -1259,7 +1260,11 @@ try
|
||||
|
||||
UInt64 seed = sipHash64(options["seed"].as<std::string>());
|
||||
|
||||
std::string structure = options["structure"].as<std::string>();
|
||||
std::string structure;
|
||||
|
||||
if (options.count("structure"))
|
||||
structure = options["structure"].as<std::string>();
|
||||
|
||||
std::string input_format = options["input-format"].as<std::string>();
|
||||
std::string output_format = options["output-format"].as<std::string>();
|
||||
|
||||
@ -1287,32 +1292,51 @@ try
|
||||
markov_model_params.determinator_sliding_window_size = options["determinator-sliding-window-size"].as<UInt64>();
|
||||
|
||||
/// Create the header block
|
||||
std::vector<std::string> structure_vals;
|
||||
boost::split(structure_vals, structure, boost::algorithm::is_any_of(" ,"), boost::algorithm::token_compress_on);
|
||||
|
||||
if (structure_vals.size() % 2 != 0)
|
||||
throw Exception("Odd number of elements in section structure: must be a list of name type pairs", ErrorCodes::LOGICAL_ERROR);
|
||||
SharedContextHolder shared_context = Context::createShared();
|
||||
auto context = Context::createGlobal(shared_context.get());
|
||||
auto context_const = WithContext(context).getContext();
|
||||
context->makeGlobalContext();
|
||||
|
||||
Block header;
|
||||
const DataTypeFactory & data_type_factory = DataTypeFactory::instance();
|
||||
|
||||
for (size_t i = 0, size = structure_vals.size(); i < size; i += 2)
|
||||
ColumnsDescription schema_columns;
|
||||
|
||||
if (structure.empty())
|
||||
{
|
||||
ReadBufferIterator read_buffer_iterator = [&](ColumnsDescription &)
|
||||
{
|
||||
auto file = std::make_unique<ReadBufferFromFileDescriptor>(STDIN_FILENO);
|
||||
|
||||
/// stdin must be seekable
|
||||
auto res = lseek(file->getFD(), 0, SEEK_SET);
|
||||
if (-1 == res)
|
||||
throwFromErrno("Input must be seekable file (it will be read twice).", ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
|
||||
|
||||
return file;
|
||||
};
|
||||
|
||||
schema_columns = readSchemaFromFormat(input_format, {}, read_buffer_iterator, false, context_const);
|
||||
}
|
||||
else
|
||||
{
|
||||
schema_columns = parseColumnsListFromString(structure, context_const);
|
||||
}
|
||||
|
||||
auto schema_columns_info = schema_columns.getOrdinary();
|
||||
|
||||
for (auto & info : schema_columns_info)
|
||||
{
|
||||
ColumnWithTypeAndName column;
|
||||
column.name = structure_vals[i];
|
||||
column.type = data_type_factory.get(structure_vals[i + 1]);
|
||||
column.name = info.name;
|
||||
column.type = info.type;
|
||||
column.column = column.type->createColumn();
|
||||
header.insert(std::move(column));
|
||||
}
|
||||
|
||||
SharedContextHolder shared_context = Context::createShared();
|
||||
auto context = Context::createGlobal(shared_context.get());
|
||||
context->makeGlobalContext();
|
||||
|
||||
ReadBufferFromFileDescriptor file_in(STDIN_FILENO);
|
||||
WriteBufferFromFileDescriptor file_out(STDOUT_FILENO);
|
||||
|
||||
if (load_from_file.empty())
|
||||
if (load_from_file.empty() || structure.empty())
|
||||
{
|
||||
/// stdin must be seekable
|
||||
auto res = lseek(file_in.getFD(), 0, SEEK_SET);
|
||||
|
@ -0,0 +1,4 @@
|
||||
403489
|
||||
1000 320 171 23
|
||||
2500 597 332 14
|
||||
2500 597 332 14
|
28
tests/queries/1_stateful/00175_obfuscator_schema_inference.sh
Executable file
28
tests/queries/1_stateful/00175_obfuscator_schema_inference.sh
Executable file
@ -0,0 +1,28 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
# Compared to explicitly specifying the structure of the input,
|
||||
# schema inference adds Nullable(T) to all types, so the model and the results
|
||||
# are a bit different from test '00175_obfuscator_schema_inference.sh'
|
||||
|
||||
$CLICKHOUSE_CLIENT --max_threads 1 --query="SELECT URL, Title, SearchPhrase FROM test.hits LIMIT 1000" > "${CLICKHOUSE_TMP}"/data.tsv
|
||||
|
||||
# Test obfuscator without saving the model
|
||||
$CLICKHOUSE_OBFUSCATOR --input-format TSV --output-format TSV --seed hello --limit 2500 < "${CLICKHOUSE_TMP}"/data.tsv > "${CLICKHOUSE_TMP}"/data2500.tsv 2>/dev/null
|
||||
|
||||
# Test obfuscator with saving the model
|
||||
$CLICKHOUSE_OBFUSCATOR --input-format TSV --output-format TSV --seed hello --limit 0 --save "${CLICKHOUSE_TMP}"/model.bin < "${CLICKHOUSE_TMP}"/data.tsv 2>/dev/null
|
||||
wc -c < "${CLICKHOUSE_TMP}"/model.bin
|
||||
$CLICKHOUSE_OBFUSCATOR --input-format TSV --output-format TSV --seed hello --limit 2500 --load "${CLICKHOUSE_TMP}"/model.bin < "${CLICKHOUSE_TMP}"/data.tsv > "${CLICKHOUSE_TMP}"/data2500_load_from_model.tsv 2>/dev/null
|
||||
rm "${CLICKHOUSE_TMP}"/model.bin
|
||||
|
||||
$CLICKHOUSE_LOCAL --structure "URL String, Title String, SearchPhrase String" --input-format TSV --output-format TSV --query "SELECT count(), uniq(URL), uniq(Title), uniq(SearchPhrase) FROM table" < "${CLICKHOUSE_TMP}"/data.tsv
|
||||
$CLICKHOUSE_LOCAL --structure "URL String, Title String, SearchPhrase String" --input-format TSV --output-format TSV --query "SELECT count(), uniq(URL), uniq(Title), uniq(SearchPhrase) FROM table" < "${CLICKHOUSE_TMP}"/data2500.tsv
|
||||
$CLICKHOUSE_LOCAL --structure "URL String, Title String, SearchPhrase String" --input-format TSV --output-format TSV --query "SELECT count(), uniq(URL), uniq(Title), uniq(SearchPhrase) FROM table" < "${CLICKHOUSE_TMP}"/data2500_load_from_model.tsv
|
||||
|
||||
rm "${CLICKHOUSE_TMP}"/data.tsv
|
||||
rm "${CLICKHOUSE_TMP}"/data2500.tsv
|
||||
rm "${CLICKHOUSE_TMP}"/data2500_load_from_model.tsv
|
Loading…
Reference in New Issue
Block a user