Get dictionary and check access rights only once per each call of any function reading external dictionaries.

This commit is contained in:
Vitaly Baranov 2020-05-14 17:03:06 +03:00
parent 96d2132566
commit 201c88f64f
3 changed files with 290 additions and 229 deletions

View File

@ -54,6 +54,7 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
}
/** Functions that use plug-ins (external) dictionaries_loader.
*
* Get the value of the attribute of the specified type.
@ -69,6 +70,51 @@ namespace ErrorCodes
*/
class FunctionDictHelper
{
public:
FunctionDictHelper(const Context & context_) : context(context_), external_loader(context.getExternalDictionariesLoader()) {}
std::shared_ptr<const IDictionaryBase> getDictionary(const String & dictionary_name)
{
auto dict = std::atomic_load(&dictionary);
if (dict)
return dict;
dict = external_loader.getDictionary(dictionary_name);
context.checkAccess(AccessType::dictGet, dict->getDatabaseOrNoDatabaseTag(), dict->getName());
std::atomic_store(&dictionary, dict);
return dict;
}
std::shared_ptr<const IDictionaryBase> getDictionary(const ColumnWithTypeAndName & column)
{
const auto dict_name_col = checkAndGetColumnConst<ColumnString>(column.column.get());
return getDictionary(dict_name_col->getValue<String>());
}
bool isDictGetFunctionInjective(const Block & sample_block)
{
if (sample_block.columns() != 3 && sample_block.columns() != 4)
throw Exception{"Function dictGet... takes 3 or 4 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto dict_name_col = checkAndGetColumnConst<ColumnString>(sample_block.getByPosition(0).column.get());
if (!dict_name_col)
throw Exception{"First argument of function dictGet... must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(sample_block.getByPosition(1).column.get());
if (!attr_name_col)
throw Exception{"Second argument of function dictGet... must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
return getDictionary(dict_name_col->getValue<String>())->isInjective(attr_name_col->getValue<String>());
}
private:
const Context & context;
const ExternalDictionariesLoader & external_loader;
mutable std::shared_ptr<const IDictionaryBase> dictionary;
};
class FunctionDictHas final : public IFunction
{
public:
@ -76,12 +122,10 @@ public:
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionDictHas>(context.getExternalDictionariesLoader(), context);
return std::make_shared<FunctionDictHas>(context);
}
FunctionDictHas(const ExternalDictionariesLoader & dictionaries_loader_, const Context & context_)
: dictionaries_loader(dictionaries_loader_)
, context(context_) {}
FunctionDictHas(const Context & context_) : helper(context_) {}
String getName() const override { return name; }
@ -109,10 +153,6 @@ private:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
throw Exception{"First argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
/** Do not require existence of the dictionary if the function is called for empty block.
* This is needed to allow successful query analysis on a server,
* that is the initiator of a distributed query,
@ -127,28 +167,26 @@ private:
return;
}
auto dict = dictionaries_loader.getDictionary(dict_name_col->getValue<String>());
const auto dict_ptr = dict.get();
context.checkAccess(AccessType::dictGet, dict_ptr->getDatabaseOrNoDatabaseTag(), dict_ptr->getName());
auto dict = helper.getDictionary(block.getByPosition(arguments[0]));
if (!executeDispatchSimple<FlatDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchSimple<HashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchSimple<CacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
if (!executeDispatchSimple<FlatDictionary>(block, arguments, result, dict) &&
!executeDispatchSimple<HashedDictionary>(block, arguments, result, dict) &&
!executeDispatchSimple<CacheDictionary>(block, arguments, result, dict) &&
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict) &&
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict) &&
#if !defined(ARCADIA_BUILD)
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict) &&
#endif
!executeDispatchComplex<SimplePolygonDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchSimple<DirectDictionary>(block, arguments, result, dict_ptr))
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
!executeDispatchComplex<SimplePolygonDictionary>(block, arguments, result, dict) &&
!executeDispatchSimple<DirectDictionary>(block, arguments, result, dict))
throw Exception{"Unsupported dictionary type " + dict->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
}
template <typename DictionaryType>
bool executeDispatchSimple(
Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * dictionary)
Block & block, const ColumnNumbers & arguments, const size_t result, const std::shared_ptr<const IDictionaryBase> & dict_ptr)
{
const auto dict = typeid_cast<const DictionaryType *>(dictionary);
const auto dict = typeid_cast<const DictionaryType *>(dict_ptr.get());
if (!dict)
return false;
@ -169,9 +207,9 @@ private:
template <typename DictionaryType>
bool executeDispatchComplex(
Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * dictionary)
Block & block, const ColumnNumbers & arguments, const size_t result, const std::shared_ptr<const IDictionaryBase> & dict_ptr)
{
const auto dict = typeid_cast<const DictionaryType *>(dictionary);
const auto dict = typeid_cast<const DictionaryType *>(dict_ptr.get());
if (!dict)
return false;
@ -193,28 +231,11 @@ private:
return true;
}
const ExternalDictionariesLoader & dictionaries_loader;
const Context & context;
private:
mutable FunctionDictHelper helper;
};
static bool isDictGetFunctionInjective(const ExternalDictionariesLoader & dictionaries_loader, const Block & sample_block)
{
if (sample_block.columns() != 3 && sample_block.columns() != 4)
throw Exception{"Function dictGet... takes 3 or 4 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto dict_name_col = checkAndGetColumnConst<ColumnString>(sample_block.getByPosition(0).column.get());
if (!dict_name_col)
throw Exception{"First argument of function dictGet... must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(sample_block.getByPosition(1).column.get());
if (!attr_name_col)
throw Exception{"Second argument of function dictGet... must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
return dictionaries_loader.getDictionary(dict_name_col->getValue<String>())->isInjective(attr_name_col->getValue<String>());
}
/** For ColumnVector. Either returns a reference to internal data,
* or convert it to T type, stores the result in backup_storage and returns a reference to it.
*/
@ -229,12 +250,10 @@ public:
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionDictGetString>(context.getExternalDictionariesLoader(), context);
return std::make_shared<FunctionDictGetString>(context);
}
FunctionDictGetString(const ExternalDictionariesLoader & dictionaries_loader_, const Context & context_)
: dictionaries_loader(dictionaries_loader_)
, context(context_) {}
FunctionDictGetString(const Context & context_) : helper(context_) {}
String getName() const override { return name; }
@ -247,7 +266,7 @@ private:
bool isInjective(const Block & sample_block) const override
{
return isDictGetFunctionInjective(dictionaries_loader, sample_block);
return helper.isDictGetFunctionInjective(sample_block);
}
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
@ -290,10 +309,6 @@ private:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
throw Exception{"First argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
if (input_rows_count == 0)
{
auto & elem = block.getByPosition(result);
@ -301,29 +316,27 @@ private:
return;
}
auto dict = dictionaries_loader.getDictionary(dict_name_col->getValue<String>());
const auto dict_ptr = dict.get();
context.checkAccess(AccessType::dictGet, dict_ptr->getDatabaseOrNoDatabaseTag(), dict_ptr->getName());
auto dict = helper.getDictionary(block.getByPosition(arguments[0]));
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<DirectDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict) &&
!executeDispatch<DirectDictionary>(block, arguments, result, dict) &&
!executeDispatch<HashedDictionary>(block, arguments, result, dict) &&
!executeDispatch<CacheDictionary>(block, arguments, result, dict) &&
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict) &&
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict) &&
#if !defined(ARCADIA_BUILD)
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict) &&
#endif
!executeDispatchComplex<SimplePolygonDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchRange<RangeHashedDictionary>(block, arguments, result, dict_ptr))
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
!executeDispatchComplex<SimplePolygonDictionary>(block, arguments, result, dict) &&
!executeDispatchRange<RangeHashedDictionary>(block, arguments, result, dict))
throw Exception{"Unsupported dictionary type " + dict->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
}
template <typename DictionaryType>
bool executeDispatch(
Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * dictionary)
Block & block, const ColumnNumbers & arguments, const size_t result, const std::shared_ptr<const IDictionaryBase> & dict_ptr)
{
const auto dict = typeid_cast<const DictionaryType *>(dictionary);
const auto dict = typeid_cast<const DictionaryType *>(dict_ptr.get());
if (!dict)
return false;
@ -352,9 +365,9 @@ private:
template <typename DictionaryType>
bool executeDispatchComplex(
Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * dictionary)
Block & block, const ColumnNumbers & arguments, const size_t result, const std::shared_ptr<const IDictionaryBase> & dict_ptr)
{
const auto dict = typeid_cast<const DictionaryType *>(dictionary);
const auto dict = typeid_cast<const DictionaryType *>(dict_ptr.get());
if (!dict)
return false;
@ -389,9 +402,9 @@ private:
template <typename DictionaryType>
bool executeDispatchRange(
Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * dictionary)
Block & block, const ColumnNumbers & arguments, const size_t result, const std::shared_ptr<const IDictionaryBase> & dict_ptr)
{
const auto dict = typeid_cast<const DictionaryType *>(dictionary);
const auto dict = typeid_cast<const DictionaryType *>(dict_ptr.get());
if (!dict)
return false;
@ -420,8 +433,8 @@ private:
return true;
}
const ExternalDictionariesLoader & dictionaries_loader;
const Context & context;
private:
mutable FunctionDictHelper helper;
};
@ -432,12 +445,10 @@ public:
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionDictGetStringOrDefault>(context.getExternalDictionariesLoader(), context);
return std::make_shared<FunctionDictGetStringOrDefault>(context);
}
FunctionDictGetStringOrDefault(const ExternalDictionariesLoader & dictionaries_loader_, const Context & context_)
: dictionaries_loader(dictionaries_loader_)
, context(context_) {}
FunctionDictGetStringOrDefault(const Context & context_) : helper(context_) {}
String getName() const override { return name; }
@ -475,10 +486,6 @@ private:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
throw Exception{"First argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
if (input_rows_count == 0)
{
auto & elem = block.getByPosition(result);
@ -486,28 +493,26 @@ private:
return;
}
auto dict = dictionaries_loader.getDictionary(dict_name_col->getValue<String>());
const auto dict_ptr = dict.get();
context.checkAccess(AccessType::dictGet, dict_ptr->getDatabaseOrNoDatabaseTag(), dict_ptr->getName());
auto dict = helper.getDictionary(block.getByPosition(arguments[0]));
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<DirectDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict) &&
!executeDispatch<DirectDictionary>(block, arguments, result, dict) &&
!executeDispatch<HashedDictionary>(block, arguments, result, dict) &&
!executeDispatch<CacheDictionary>(block, arguments, result, dict) &&
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict) &&
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict) &&
#if !defined(ARCADIA_BUILD)
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict) &&
#endif
!executeDispatchComplex<SimplePolygonDictionary>(block, arguments, result, dict_ptr))
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
!executeDispatchComplex<SimplePolygonDictionary>(block, arguments, result, dict))
throw Exception{"Unsupported dictionary type " + dict->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
}
template <typename DictionaryType>
bool executeDispatch(
Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * dictionary)
Block & block, const ColumnNumbers & arguments, const size_t result, const std::shared_ptr<const IDictionaryBase> & dict_ptr)
{
const auto dict = typeid_cast<const DictionaryType *>(dictionary);
const auto dict = typeid_cast<const DictionaryType *>(dict_ptr.get());
if (!dict)
return false;
@ -530,7 +535,7 @@ private:
template <typename DictionaryType>
void executeDispatch(
Block & block, const ColumnNumbers & arguments, const size_t result, const DictionaryType * dictionary,
Block & block, const ColumnNumbers & arguments, const size_t result, const DictionaryType * dict,
const std::string & attr_name, const ColumnUInt64 * id_col)
{
const auto default_col_untyped = block.getByPosition(arguments[3]).column.get();
@ -540,7 +545,7 @@ private:
/// vector ids, vector defaults
auto out = ColumnString::create();
const auto & ids = id_col->getData();
dictionary->getString(attr_name, ids, default_col, out.get());
dict->getString(attr_name, ids, default_col, out.get());
block.getByPosition(result).column = std::move(out);
}
else if (const auto default_col_const = checkAndGetColumnConstStringOrFixedString(default_col_untyped))
@ -549,7 +554,7 @@ private:
auto out = ColumnString::create();
const auto & ids = id_col->getData();
String def = default_col_const->getValue<String>();
dictionary->getString(attr_name, ids, def, out.get());
dict->getString(attr_name, ids, def, out.get());
block.getByPosition(result).column = std::move(out);
}
else
@ -558,7 +563,7 @@ private:
template <typename DictionaryType>
void executeDispatch(
Block & block, const ColumnNumbers & arguments, const size_t result, const DictionaryType * dictionary,
Block & block, const ColumnNumbers & arguments, const size_t result, const DictionaryType * dict,
const std::string & attr_name, const ColumnConst * id_col)
{
const auto default_col_untyped = block.getByPosition(arguments[3]).column.get();
@ -568,11 +573,11 @@ private:
/// const ids, vector defaults
const PaddedPODArray<UInt64> ids(1, id_col->getValue<UInt64>());
PaddedPODArray<UInt8> flags(1);
dictionary->has(ids, flags);
dict->has(ids, flags);
if (flags.front())
{
auto out = ColumnString::create();
dictionary->getString(attr_name, ids, String(), out.get());
dict->getString(attr_name, ids, String(), out.get());
block.getByPosition(result).column = DataTypeString().createColumnConst(id_col->size(), out->getDataAt(0).toString());
}
else
@ -584,7 +589,7 @@ private:
const PaddedPODArray<UInt64> ids(1, id_col->getValue<UInt64>());
auto out = ColumnString::create();
String def = default_col_const->getValue<String>();
dictionary->getString(attr_name, ids, def, out.get());
dict->getString(attr_name, ids, def, out.get());
block.getByPosition(result).column = DataTypeString().createColumnConst(id_col->size(), out->getDataAt(0).toString());
}
else
@ -593,9 +598,9 @@ private:
template <typename DictionaryType>
bool executeDispatchComplex(
Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * dictionary)
Block & block, const ColumnNumbers & arguments, const size_t result, const std::shared_ptr<const IDictionaryBase> & dict_ptr)
{
const auto dict = typeid_cast<const DictionaryType *>(dictionary);
const auto dict = typeid_cast<const DictionaryType *>(dict_ptr.get());
if (!dict)
return false;
@ -631,8 +636,7 @@ private:
return true;
}
const ExternalDictionariesLoader & dictionaries_loader;
const Context & context;
mutable FunctionDictHelper helper;
};
@ -755,12 +759,11 @@ public:
static FunctionPtr create(const Context & context, UInt32 dec_scale = 0)
{
return std::make_shared<FunctionDictGet>(context.getExternalDictionariesLoader(), context, dec_scale);
return std::make_shared<FunctionDictGet>(context, dec_scale);
}
FunctionDictGet(const ExternalDictionariesLoader & dictionaries_loader_, const Context & context_, UInt32 dec_scale = 0)
: dictionaries_loader(dictionaries_loader_)
, context(context_)
FunctionDictGet(const Context & context_, UInt32 dec_scale = 0)
: helper(context_)
, decimal_scale(dec_scale)
{}
@ -775,7 +778,7 @@ private:
bool isInjective(const Block & sample_block) const override
{
return isDictGetFunctionInjective(dictionaries_loader, sample_block);
return helper.isDictGetFunctionInjective(sample_block);
}
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
@ -816,10 +819,6 @@ private:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
throw Exception{"First argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
if (input_rows_count == 0)
{
auto & elem = block.getByPosition(result);
@ -827,29 +826,26 @@ private:
return;
}
auto dict = dictionaries_loader.getDictionary(dict_name_col->getValue<String>());
const auto dict_ptr = dict.get();
context.checkAccess(AccessType::dictGet, dict_ptr->getDatabaseOrNoDatabaseTag(), dict_ptr->getName());
auto dict = helper.getDictionary(block.getByPosition(arguments[0]));
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<DirectDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict) &&
!executeDispatch<DirectDictionary>(block, arguments, result, dict) &&
!executeDispatch<HashedDictionary>(block, arguments, result, dict) &&
!executeDispatch<CacheDictionary>(block, arguments, result, dict) &&
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict) &&
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict) &&
#if !defined(ARCADIA_BUILD)
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict) &&
#endif
!executeDispatchComplex<SimplePolygonDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchRange<RangeHashedDictionary>(block, arguments, result, dict_ptr))
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
!executeDispatchComplex<SimplePolygonDictionary>(block, arguments, result, dict) &&
!executeDispatchRange<RangeHashedDictionary>(block, arguments, result, dict))
throw Exception{"Unsupported dictionary type " + dict->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
}
template <typename DictionaryType>
bool executeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result,
const IDictionaryBase * dictionary)
bool executeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result, const std::shared_ptr<const IDictionaryBase> & dict_ptr)
{
const auto dict = typeid_cast<const DictionaryType *>(dictionary);
const auto dict = typeid_cast<const DictionaryType *>(dict_ptr.get());
if (!dict)
return false;
@ -903,9 +899,9 @@ private:
template <typename DictionaryType>
bool executeDispatchComplex(
Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * dictionary)
Block & block, const ColumnNumbers & arguments, const size_t result, const std::shared_ptr<const IDictionaryBase> & dict_ptr)
{
const auto dict = typeid_cast<const DictionaryType *>(dictionary);
const auto dict = typeid_cast<const DictionaryType *>(dict_ptr.get());
if (!dict)
return false;
@ -946,9 +942,9 @@ private:
template <typename DictionaryType>
bool executeDispatchRange(
Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * dictionary)
Block & block, const ColumnNumbers & arguments, const size_t result, const std::shared_ptr<const IDictionaryBase> & dict_ptr)
{
const auto dict = typeid_cast<const DictionaryType *>(dictionary);
const auto dict = typeid_cast<const DictionaryType *>(dict_ptr.get());
if (!dict)
return false;
@ -982,8 +978,7 @@ private:
return true;
}
const ExternalDictionariesLoader & dictionaries_loader;
const Context & context;
mutable FunctionDictHelper helper;
UInt32 decimal_scale;
};
@ -1033,12 +1028,11 @@ public:
static FunctionPtr create(const Context & context, UInt32 dec_scale = 0)
{
return std::make_shared<FunctionDictGetOrDefault>(context.getExternalDictionariesLoader(), context, dec_scale);
return std::make_shared<FunctionDictGetOrDefault>(context, dec_scale);
}
FunctionDictGetOrDefault(const ExternalDictionariesLoader & dictionaries_loader_, const Context & context_, UInt32 dec_scale = 0)
: dictionaries_loader(dictionaries_loader_)
, context(context_)
FunctionDictGetOrDefault(const Context & context_, UInt32 dec_scale = 0)
: helper(context_)
, decimal_scale(dec_scale)
{}
@ -1079,10 +1073,6 @@ private:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
throw Exception{"First argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
if (input_rows_count == 0)
{
auto & elem = block.getByPosition(result);
@ -1090,28 +1080,25 @@ private:
return;
}
auto dict = dictionaries_loader.getDictionary(dict_name_col->getValue<String>());
const auto dict_ptr = dict.get();
context.checkAccess(AccessType::dictGet, dict_ptr->getDatabaseOrNoDatabaseTag(), dict_ptr->getName());
auto dict = helper.getDictionary(block.getByPosition(arguments[0]));
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<DirectDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict) &&
!executeDispatch<DirectDictionary>(block, arguments, result, dict) &&
!executeDispatch<HashedDictionary>(block, arguments, result, dict) &&
!executeDispatch<CacheDictionary>(block, arguments, result, dict) &&
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict) &&
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict) &&
#if !defined(ARCADIA_BUILD)
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict) &&
#endif
!executeDispatchComplex<SimplePolygonDictionary>(block, arguments, result, dict_ptr))
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
!executeDispatchComplex<SimplePolygonDictionary>(block, arguments, result, dict))
throw Exception{"Unsupported dictionary type " + dict->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
}
template <typename DictionaryType>
bool executeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result,
const IDictionaryBase * dictionary)
bool executeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result, const std::shared_ptr<const IDictionaryBase> & dict_ptr)
{
const auto dict = typeid_cast<const DictionaryType *>(dictionary);
const auto dict = typeid_cast<const DictionaryType *>(dict_ptr.get());
if (!dict)
return false;
@ -1134,7 +1121,7 @@ private:
template <typename DictionaryType>
void executeDispatch(
Block & block, const ColumnNumbers & arguments, const size_t result, const DictionaryType * dictionary,
Block & block, const ColumnNumbers & arguments, const size_t result, const DictionaryType * dict,
const std::string & attr_name, const ColumnUInt64 * id_col)
{
const auto default_col_untyped = block.getByPosition(arguments[3]).column.get();
@ -1150,7 +1137,7 @@ private:
const auto & ids = id_col->getData();
auto & data = out->getData();
const auto & defs = default_col->getData();
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, defs, data);
DictGetTraits<DataType>::getOrDefault(dict, attr_name, ids, defs, data);
block.getByPosition(result).column = std::move(out);
}
else if (const auto default_col_const = checkAndGetColumnConst<ColVec>(default_col_untyped))
@ -1164,7 +1151,7 @@ private:
const auto & ids = id_col->getData();
auto & data = out->getData();
const auto def = default_col_const->template getValue<Type>();
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, def, data);
DictGetTraits<DataType>::getOrDefault(dict, attr_name, ids, def, data);
block.getByPosition(result).column = std::move(out);
}
else
@ -1173,7 +1160,7 @@ private:
template <typename DictionaryType>
void executeDispatch(
Block & block, const ColumnNumbers & arguments, const size_t result, const DictionaryType * dictionary,
Block & block, const ColumnNumbers & arguments, const size_t result, const DictionaryType * dict,
const std::string & attr_name, const ColumnConst * id_col)
{
const auto default_col_untyped = block.getByPosition(arguments[3]).column.get();
@ -1183,13 +1170,13 @@ private:
/// const ids, vector defaults
const PaddedPODArray<UInt64> ids(1, id_col->getValue<UInt64>());
PaddedPODArray<UInt8> flags(1);
dictionary->has(ids, flags);
dict->has(ids, flags);
if (flags.front())
{
if constexpr (IsDataTypeDecimal<DataType>)
{
DecimalPaddedPODArray<Type> data(1, decimal_scale);
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, Type(), data);
DictGetTraits<DataType>::getOrDefault(dict, attr_name, ids, Type(), data);
block.getByPosition(result).column =
DataType(DataType::maxPrecision(), decimal_scale).createColumnConst(
id_col->size(), toField(data.front(), decimal_scale));
@ -1197,7 +1184,7 @@ private:
else
{
PaddedPODArray<Type> data(1);
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, Type(), data);
DictGetTraits<DataType>::getOrDefault(dict, attr_name, ids, Type(), data);
block.getByPosition(result).column = DataType().createColumnConst(id_col->size(), toField(data.front()));
}
}
@ -1213,7 +1200,7 @@ private:
{
DecimalPaddedPODArray<Type> data(1, decimal_scale);
const auto & def = default_col_const->template getValue<Type>();
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, def, data);
DictGetTraits<DataType>::getOrDefault(dict, attr_name, ids, def, data);
block.getByPosition(result).column =
DataType(DataType::maxPrecision(), decimal_scale).createColumnConst(
id_col->size(), toField(data.front(), decimal_scale));
@ -1222,7 +1209,7 @@ private:
{
PaddedPODArray<Type> data(1);
const auto & def = default_col_const->template getValue<Type>();
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, def, data);
DictGetTraits<DataType>::getOrDefault(dict, attr_name, ids, def, data);
block.getByPosition(result).column = DataType().createColumnConst(id_col->size(), toField(data.front()));
}
}
@ -1232,9 +1219,9 @@ private:
template <typename DictionaryType>
bool executeDispatchComplex(
Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * dictionary)
Block & block, const ColumnNumbers & arguments, const size_t result, const std::shared_ptr<const IDictionaryBase> & dict_ptr)
{
const auto dict = typeid_cast<const DictionaryType *>(dictionary);
const auto dict = typeid_cast<const DictionaryType *>(dict_ptr.get());
if (!dict)
return false;
@ -1282,8 +1269,7 @@ private:
return true;
}
const ExternalDictionariesLoader & dictionaries_loader;
const Context & context;
mutable FunctionDictHelper helper;
UInt32 decimal_scale;
};
@ -1330,10 +1316,10 @@ public:
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionDictGetNoType>(context.getExternalDictionariesLoader(), context);
return std::make_shared<FunctionDictGetNoType>(context);
}
FunctionDictGetNoType(const ExternalDictionariesLoader & dictionaries_loader_, const Context & context_) : dictionaries_loader(dictionaries_loader_), context(context_) {}
FunctionDictGetNoType(const Context & context_) : context(context_), helper(context_) {}
String getName() const override { return name; }
@ -1346,7 +1332,7 @@ private:
bool isInjective(const Block & sample_block) const override
{
return isDictGetFunctionInjective(dictionaries_loader, sample_block);
return helper.isDictGetFunctionInjective(sample_block);
}
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
@ -1386,7 +1372,7 @@ private:
+ ", must be convertible to " + TypeName<Int64>::get() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
auto dict = dictionaries_loader.getDictionary(dict_name);
auto dict = helper.getDictionary(dict_name);
const DictionaryStructure & structure = dict->getStructure();
for (const auto idx : ext::range(0, structure.attributes.size()))
@ -1466,8 +1452,8 @@ private:
}
private:
const ExternalDictionariesLoader & dictionaries_loader;
const Context & context;
mutable FunctionDictHelper helper;
mutable FunctionPtr impl; // underlying function used by dictGet function without explicit type info
};
@ -1479,10 +1465,10 @@ public:
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionDictGetNoTypeOrDefault>(context.getExternalDictionariesLoader(), context);
return std::make_shared<FunctionDictGetNoTypeOrDefault>(context);
}
FunctionDictGetNoTypeOrDefault(const ExternalDictionariesLoader & dictionaries_loader_, const Context & context_) : dictionaries_loader(dictionaries_loader_), context(context_) {}
FunctionDictGetNoTypeOrDefault(const Context & context_) : context(context_), helper(context_) {}
String getName() const override { return name; }
@ -1494,7 +1480,7 @@ private:
bool isInjective(const Block & sample_block) const override
{
return isDictGetFunctionInjective(dictionaries_loader, sample_block);
return helper.isDictGetFunctionInjective(sample_block);
}
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
@ -1522,7 +1508,7 @@ private:
throw Exception{"Illegal type " + arguments[2].type->getName() + " of third argument of function " + getName()
+ ", must be UInt64 or tuple(...).", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
auto dict = dictionaries_loader.getDictionary(dict_name);
auto dict = helper.getDictionary(dict_name);
const DictionaryStructure & structure = dict->getStructure();
for (const auto idx : ext::range(0, structure.attributes.size()))
@ -1608,8 +1594,8 @@ private:
}
private:
const ExternalDictionariesLoader & dictionaries_loader;
const Context & context;
mutable FunctionDictHelper helper;
mutable FunctionPtr impl; // underlying function used by dictGet function without explicit type info
};
@ -1622,12 +1608,10 @@ public:
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionDictGetHierarchy>(context.getExternalDictionariesLoader(), context);
return std::make_shared<FunctionDictGetHierarchy>(context);
}
FunctionDictGetHierarchy(const ExternalDictionariesLoader & dictionaries_loader_, const Context & context_)
: dictionaries_loader(dictionaries_loader_)
, context(context_) {}
FunctionDictGetHierarchy(const Context & context_) : helper(context_) {}
String getName() const override { return name; }
@ -1655,10 +1639,6 @@ private:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
throw Exception{"First argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
if (input_rows_count == 0)
{
auto & elem = block.getByPosition(result);
@ -1666,22 +1646,20 @@ private:
return;
}
auto dict = dictionaries_loader.getDictionary(dict_name_col->getValue<String>());
const auto dict_ptr = dict.get();
context.checkAccess(AccessType::dictGet, dict_ptr->getDatabaseOrNoDatabaseTag(), dict_ptr->getName());
auto dict = helper.getDictionary(block.getByPosition(arguments[0]));
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<DirectDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr))
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict) &&
!executeDispatch<DirectDictionary>(block, arguments, result, dict) &&
!executeDispatch<HashedDictionary>(block, arguments, result, dict) &&
!executeDispatch<CacheDictionary>(block, arguments, result, dict))
throw Exception{"Unsupported dictionary type " + dict->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
}
template <typename DictionaryType>
bool executeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result,
const IDictionaryBase * dictionary)
const std::shared_ptr<const IDictionaryBase> & dict_ptr)
{
const auto dict = typeid_cast<const DictionaryType *>(dictionary);
const auto dict = typeid_cast<const DictionaryType *>(dict_ptr.get());
if (!dict)
return false;
@ -1772,8 +1750,7 @@ private:
return true;
}
const ExternalDictionariesLoader & dictionaries_loader;
const Context & context;
mutable FunctionDictHelper helper;
};
@ -1784,12 +1761,11 @@ public:
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionDictIsIn>(context.getExternalDictionariesLoader(), context);
return std::make_shared<FunctionDictIsIn>(context);
}
FunctionDictIsIn(const ExternalDictionariesLoader & dictionaries_loader_, const Context & context_)
: dictionaries_loader(dictionaries_loader_)
, context(context_) {}
FunctionDictIsIn(const Context & context_)
: helper(context_) {}
String getName() const override { return name; }
@ -1820,10 +1796,6 @@ private:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
throw Exception{"First argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
if (input_rows_count == 0)
{
auto & elem = block.getByPosition(result);
@ -1831,22 +1803,20 @@ private:
return;
}
auto dict = dictionaries_loader.getDictionary(dict_name_col->getValue<String>());
const auto dict_ptr = dict.get();
context.checkAccess(AccessType::dictGet, dict_ptr->getDatabaseOrNoDatabaseTag(), dict_ptr->getName());
auto dict = helper.getDictionary(block.getByPosition(arguments[0]));
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr)
&& !executeDispatch<DirectDictionary>(block, arguments, result, dict_ptr)
&& !executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr)
&& !executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr))
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict)
&& !executeDispatch<DirectDictionary>(block, arguments, result, dict)
&& !executeDispatch<HashedDictionary>(block, arguments, result, dict)
&& !executeDispatch<CacheDictionary>(block, arguments, result, dict))
throw Exception{"Unsupported dictionary type " + dict->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
}
template <typename DictionaryType>
bool executeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result,
const IDictionaryBase * dictionary)
const std::shared_ptr<const IDictionaryBase> & dict_ptr)
{
const auto dict = typeid_cast<const DictionaryType *>(dictionary);
const auto dict = typeid_cast<const DictionaryType *>(dict_ptr.get());
if (!dict)
return false;
@ -1868,7 +1838,7 @@ private:
}
template <typename DictionaryType>
bool execute(Block & block, const size_t result, const DictionaryType * dictionary,
bool execute(Block & block, const size_t result, const DictionaryType * dict,
const ColumnUInt64 * child_id_col, const IColumn * ancestor_id_col_untyped)
{
if (const auto ancestor_id_col = checkAndGetColumn<ColumnUInt64>(ancestor_id_col_untyped))
@ -1881,7 +1851,7 @@ private:
const auto size = child_id_col->size();
data.resize(size);
dictionary->isInVectorVector(child_ids, ancestor_ids, data);
dict->isInVectorVector(child_ids, ancestor_ids, data);
block.getByPosition(result).column = std::move(out);
}
else if (const auto ancestor_id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(ancestor_id_col_untyped))
@ -1894,7 +1864,7 @@ private:
const auto size = child_id_col->size();
data.resize(size);
dictionary->isInVectorConstant(child_ids, ancestor_id, data);
dict->isInVectorConstant(child_ids, ancestor_id, data);
block.getByPosition(result).column = std::move(out);
}
else
@ -1907,7 +1877,7 @@ private:
}
template <typename DictionaryType>
bool execute(Block & block, const size_t result, const DictionaryType * dictionary,
bool execute(Block & block, const size_t result, const DictionaryType * dict,
const ColumnConst * child_id_col, const IColumn * ancestor_id_col_untyped)
{
if (const auto ancestor_id_col = checkAndGetColumn<ColumnUInt64>(ancestor_id_col_untyped))
@ -1920,7 +1890,7 @@ private:
const auto size = child_id_col->size();
data.resize(size);
dictionary->isInConstantVector(child_id, ancestor_ids, data);
dict->isInConstantVector(child_id, ancestor_ids, data);
block.getByPosition(result).column = std::move(out);
}
else if (const auto ancestor_id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(ancestor_id_col_untyped))
@ -1929,7 +1899,7 @@ private:
const auto ancestor_id = ancestor_id_col_const->getValue<UInt64>();
UInt8 res = 0;
dictionary->isInConstantConstant(child_id, ancestor_id, res);
dict->isInConstantConstant(child_id, ancestor_id, res);
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(child_id_col->size(), res);
}
else
@ -1939,8 +1909,7 @@ private:
return true;
}
const ExternalDictionariesLoader & dictionaries_loader;
const Context & context;
mutable FunctionDictHelper helper;
};

View File

@ -0,0 +1,92 @@
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance('instance')
@pytest.fixture(scope="module", autouse=True)
def started_cluster():
try:
cluster.start()
instance.query("CREATE USER mira")
instance.query("CREATE TABLE test_table(x Int32, y Int32) ENGINE=Log")
instance.query("INSERT INTO test_table VALUES (5,6)")
yield cluster
finally:
cluster.shutdown()
@pytest.fixture(autouse=True)
def clear_after_test():
try:
yield
finally:
instance.query("CREATE USER OR REPLACE mira")
instance.query("DROP DICTIONARY IF EXISTS test_dict")
create_query = """
CREATE DICTIONARY test_dict(x Int32, y Int32) PRIMARY KEY x
LAYOUT(FLAT())
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'test_table' DB 'default'))
LIFETIME(0)
"""
drop_query = "DROP DICTIONARY test_dict"
def test_create():
assert instance.query("SHOW GRANTS FOR mira") == ""
assert "Not enough privileges" in instance.query_and_get_error(create_query, user="mira")
instance.query("GRANT CREATE DICTIONARY ON *.* TO mira")
instance.query(create_query, user="mira")
instance.query(drop_query)
instance.query("REVOKE CREATE DICTIONARY ON *.* FROM mira")
assert instance.query("SHOW GRANTS FOR mira") == ""
assert "Not enough privileges" in instance.query_and_get_error(create_query, user="mira")
instance.query("GRANT CREATE DICTIONARY ON default.* TO mira")
instance.query(create_query, user="mira")
instance.query(drop_query)
instance.query("REVOKE CREATE DICTIONARY ON default.* FROM mira")
assert instance.query("SHOW GRANTS FOR mira") == ""
assert "Not enough privileges" in instance.query_and_get_error(create_query, user="mira")
instance.query("GRANT CREATE DICTIONARY ON default.test_dict TO mira")
instance.query(create_query, user="mira")
def test_drop():
instance.query(create_query)
assert instance.query("SHOW GRANTS FOR mira") == ""
assert "Not enough privileges" in instance.query_and_get_error(drop_query, user="mira")
instance.query("GRANT DROP DICTIONARY ON *.* TO mira")
instance.query(drop_query, user="mira")
instance.query(create_query)
def test_dictget():
instance.query(create_query)
dictget_query = "SELECT dictGet('default.test_dict', 'y', toUInt64(5))"
instance.query(dictget_query) == "6\n"
assert "Not enough privileges" in instance.query_and_get_error(dictget_query, user='mira')
instance.query("GRANT dictGet ON default.test_dict TO mira")
instance.query(dictget_query, user='mira') == "6\n"
dictget_query = "SELECT dictGet('default.test_dict', 'y', toUInt64(1))"
instance.query(dictget_query) == "0\n"
instance.query(dictget_query, user='mira') == "0\n"
instance.query("REVOKE dictGet ON *.* FROM mira")
assert "Not enough privileges" in instance.query_and_get_error(dictget_query, user='mira')