add experimental settings for Object type

This commit is contained in:
Anton Popov 2022-03-16 16:51:23 +00:00
parent 0ba78c3c3a
commit 2ced42ed41
21 changed files with 58 additions and 19 deletions

View File

@ -567,6 +567,7 @@ class IColumn;
/** Experimental functions */ \
M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \
M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \
M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \
M(String, insert_deduplication_token, "", "If not empty, used for duplicate detection instead of data digest", 0) \
// End of COMMON_SETTINGS
// Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS.

View File

@ -13,7 +13,6 @@ void registerFileSegmentationEngineCSV(FormatFactory & factory);
void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory);
void registerFileSegmentationEngineRegexp(FormatFactory & factory);
void registerFileSegmentationEngineJSONAsString(FormatFactory & factory);
void registerFileSegmentationEngineJSONAsObject(FormatFactory & factory);
void registerFileSegmentationEngineJSONCompactEachRow(FormatFactory & factory);
/// Formats for both input/output.
@ -121,7 +120,6 @@ void registerFormats()
registerFileSegmentationEngineJSONEachRow(factory);
registerFileSegmentationEngineRegexp(factory);
registerFileSegmentationEngineJSONAsString(factory);
registerFileSegmentationEngineJSONAsObject(factory);
registerFileSegmentationEngineJSONCompactEachRow(factory);
registerInputFormatNative(factory);

View File

@ -53,6 +53,7 @@
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <DataTypes/ObjectUtils.h>
#include <Databases/DatabaseFactory.h>
#include <Databases/DatabaseReplicated.h>
@ -733,11 +734,26 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat
{
String message = "Cannot create table with column '" + name_and_type_pair.name + "' which type is '"
+ type + "' because experimental geo types are not allowed. "
+ "Set setting allow_experimental_geo_types = 1 in order to allow it.";
+ "Set setting allow_experimental_geo_types = 1 in order to allow it";
throw Exception(message, ErrorCodes::ILLEGAL_COLUMN);
}
}
}
if (!create.attach && !settings.allow_experimental_object_type)
{
for (const auto & [name, type] : properties.columns.getAllPhysical())
{
if (isObject(type))
{
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
"Cannot create table with column '{}' which type is '{}' "
"because experimental Object type is not allowed. "
"Set setting allow_experimental_object_type = 1 in order to allow it",
name, type->getName());
}
}
}
}
String InterpreterCreateQuery::getTableEngineName(DefaultTableEngine default_table_engine)
@ -1230,6 +1246,14 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
/// we can safely destroy the object without a call to "shutdown", because there is guarantee
/// that no background threads/similar resources remain after exception from "startup".
if (!res->supportsDynamicSubcolumns() && hasObjectColumns(res->getInMemoryMetadataPtr()->getColumns()))
{
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
"Cannot create table with column of type Object, "
"because storage {} doesn't support dynamic subcolumns",
res->getName());
}
res->startup();
return true;
}

View File

@ -240,11 +240,6 @@ void registerInputFormatJSONAsObject(FormatFactory & factory)
});
}
void registerFileSegmentationEngineJSONAsObject(FormatFactory & factory)
{
factory.registerFileSegmentationEngine("JSONAsObject", &fileSegmentationEngineJSONEachRow);
}
void registerNonTrivialPrefixAndSuffixCheckerJSONAsObject(FormatFactory & factory)
{
factory.registerNonTrivialPrefixAndSuffixChecker("JSONAsObject", nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl);

View File

@ -1,5 +1,7 @@
-- Tags: no-fasttest
SET allow_experimental_object_type = 1;
DROP TABLE IF EXISTS t_json;
CREATE TABLE t_json(id UInt64, data Object('JSON'))
@ -79,3 +81,5 @@ WHERE table = 't_json' AND database = currentDatabase() AND active AND column =
ORDER BY name;
DROP TABLE IF EXISTS t_json;
CREATE TABLE t_json(id UInt64, data Object('JSON')) ENGINE = Log; -- { serverError 44 }

View File

@ -1,5 +1,7 @@
-- Tags: no-fasttest
SET allow_experimental_object_type = 1;
DROP TABLE IF EXISTS t_json_2;
CREATE TABLE t_json_2(id UInt64, data Object('JSON'))

View File

@ -2,6 +2,8 @@
{% for engine in ["ReplicatedMergeTree('/clickhouse/tables/{database}/test_01825_3/t_json_3', 'r1') ORDER BY tuple()", "Memory"] -%}
SET allow_experimental_object_type = 1;
DROP TABLE IF EXISTS t_json_3;
CREATE TABLE t_json_3(id UInt64, data JSON)

View File

@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t_json_4"
$CLICKHOUSE_CLIENT -q "CREATE TABLE t_json_4(id UInt64, data JSON) \
ENGINE = MergeTree ORDER BY tuple()"
ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_object_type 1
echo '{"id": 1, "data": {"k1": "v1"}}, {"id": 2, "data": {"k1": [1, 2]}}' \
| $CLICKHOUSE_CLIENT -q "INSERT INTO t_json_4 FORMAT JSONEachRow" 2>&1 | grep -o -m1 "Code: 645"

View File

@ -1,5 +1,7 @@
-- Tags: no-fasttest
SET allow_experimental_object_type = 1;
SELECT '{"a": {"b": 1, "c": 2}}'::JSON AS s;
SELECT '{"a": {"b": 1, "c": 2}}'::JSON AS s format JSONEachRow;

View File

@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t_json_6;"
$CLICKHOUSE_CLIENT -q "CREATE TABLE t_json_6 (data JSON) ENGINE = MergeTree ORDER BY tuple();"
$CLICKHOUSE_CLIENT -q "CREATE TABLE t_json_6 (data JSON) ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_object_type 1
cat <<EOF | $CLICKHOUSE_CLIENT -q "INSERT INTO t_json_6 FORMAT JSONAsObject"
{

View File

@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t_json_7;"
$CLICKHOUSE_CLIENT -q "CREATE TABLE t_json_7 (data JSON) ENGINE = MergeTree ORDER BY tuple();"
$CLICKHOUSE_CLIENT -q "CREATE TABLE t_json_7 (data JSON) ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_object_type 1
cat <<EOF | $CLICKHOUSE_CLIENT -q "INSERT INTO t_json_7 FORMAT JSONAsObject"
{

View File

@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t_json_8"
$CLICKHOUSE_CLIENT -q "CREATE TABLE t_json_8 (data JSON) ENGINE = MergeTree ORDER BY tuple()"
$CLICKHOUSE_CLIENT -q "CREATE TABLE t_json_8 (data JSON) ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_object_type 1
cat <<EOF | $CLICKHOUSE_CLIENT -q "INSERT INTO t_json_8 FORMAT JSONAsObject"
{

View File

@ -7,9 +7,9 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS btc"
${CLICKHOUSE_CLIENT} -q "CREATE TABLE btc (data JSON) ENGINE = MergeTree ORDER BY tuple()"
${CLICKHOUSE_CLIENT} -q "CREATE TABLE btc (data JSON) ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_object_type 1
cat $CUR_DIR/data_json/btc_transactions.json | ${CLICKHOUSE_CLIENT} -q "INSERT INTO btc FORMAT JSONAsObject" --input_format_parallel_parsing 0
cat $CUR_DIR/data_json/btc_transactions.json | ${CLICKHOUSE_CLIENT} -q "INSERT INTO btc FORMAT JSONAsObject"
${CLICKHOUSE_CLIENT} -q "SELECT count() FROM btc WHERE NOT ignore(*)"
${CLICKHOUSE_CLIENT} -q "DESC btc SETTINGS describe_extend_object_types = 1"

View File

@ -1,5 +1,8 @@
-- Tags: no-fasttest
SET allow_experimental_object_type = 1;
DROP TABLE IF EXISTS t_json_desc;
CREATE TABLE t_json_desc (data JSON) ENGINE = MergeTree ORDER BY tuple();

View File

@ -1,5 +1,7 @@
-- Tags: no-fasttest
SET allow_experimental_object_type = 1;
DROP TABLE IF EXISTS t_json_local;
DROP TABLE IF EXISTS t_json_dist;

View File

@ -1,5 +1,7 @@
-- Tags: no-fasttest
SET allow_experimental_object_type = 1;
DROP TABLE IF EXISTS t_json_field;
CREATE TABLE t_json_field (id UInt32, data JSON)

View File

@ -7,9 +7,9 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS ghdata"
${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata (data JSON) ENGINE = MergeTree ORDER BY tuple()"
${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata (data JSON) ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_object_type 1
cat $CUR_DIR/data_json/ghdata_sample.json | ${CLICKHOUSE_CLIENT} -q "INSERT INTO ghdata FORMAT JSONAsObject" --input_format_parallel_parsing 0
cat $CUR_DIR/data_json/ghdata_sample.json | ${CLICKHOUSE_CLIENT} -q "INSERT INTO ghdata FORMAT JSONAsObject"
${CLICKHOUSE_CLIENT} -q "SELECT count() FROM ghdata WHERE NOT ignore(*)"

View File

@ -1,5 +1,7 @@
-- Tags: no-fasttest
SET allow_experimental_object_type = 1;
DROP TABLE IF EXISTS type_json_src;
DROP TABLE IF EXISTS type_json_dst;

View File

@ -7,9 +7,9 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS nbagames"
${CLICKHOUSE_CLIENT} -q "CREATE TABLE nbagames (data JSON) ENGINE = MergeTree ORDER BY tuple()"
${CLICKHOUSE_CLIENT} -q "CREATE TABLE nbagames (data JSON) ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_object_type 1
cat $CUR_DIR/data_json/nbagames_sample.json | ${CLICKHOUSE_CLIENT} -q "INSERT INTO nbagames FORMAT JSONAsObject" --input_format_parallel_parsing 0
cat $CUR_DIR/data_json/nbagames_sample.json | ${CLICKHOUSE_CLIENT} -q "INSERT INTO nbagames FORMAT JSONAsObject"
${CLICKHOUSE_CLIENT} -q "SELECT count() FROM nbagames WHERE NOT ignore(*)"
${CLICKHOUSE_CLIENT} -q "DESC nbagames SETTINGS describe_extend_object_types = 1"

View File

@ -1,5 +1,7 @@
-- Tags: no-fasttest
SET allow_experimental_object_type = 1;
DROP TABLE IF EXISTS t_json_null;
CREATE TABLE t_json_null(id UInt64, data Object(Nullable('JSON')))

View File

@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t_json_race"
$CLICKHOUSE_CLIENT -q "CREATE TABLE t_json_race (data JSON) ENGINE = MergeTree ORDER BY tuple()"
$CLICKHOUSE_CLIENT -q "CREATE TABLE t_json_race (data JSON) ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_object_type 1
function test_case()
{