提交 749a3013 编写于 作者: I Ivan Blinkov

Merge branch 'master' of github.com:yandex/ClickHouse

......@@ -94,7 +94,8 @@ void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & reques
{
schema_name = params.get("schema");
LOG_TRACE(log, "Will fetch info for table '" << schema_name + "." + table_name << "'");
} else
}
else
LOG_TRACE(log, "Will fetch info for table '" << table_name << "'");
LOG_TRACE(log, "Got connection str '" << connection_string << "'");
......
#pragma once
#include <Core/Types.h>
#include <type_traits>
#include <Core/Types.h>
#include <Common/UInt128.h>
namespace DB
{
......@@ -146,6 +148,7 @@ template <typename A> struct ResultOfBitNot
* UInt<x>, Int<y> -> Int<max(x*2, y)>
* Float<x>, [U]Int<y> -> Float<max(x, y*2)>
* Decimal<x>, Decimal<y> -> Decimal<max(x,y)>
* UUID, UUID -> UUID
* UInt64 , Int<x> -> Error
* Float<x>, [U]Int64 -> Error
*/
......@@ -168,7 +171,9 @@ struct ResultOfIf
? max(sizeof(A), sizeof(B)) * 2
: max(sizeof(A), sizeof(B))>::Type;
using Type = std::conditional_t<!IsDecimalNumber<A> && !IsDecimalNumber<B>, ConstructedType,
using ConstructedWithUUID = std::conditional_t<std::is_same_v<A, UInt128> && std::is_same_v<B, UInt128>, A, ConstructedType>;
using Type = std::conditional_t<!IsDecimalNumber<A> && !IsDecimalNumber<B>, ConstructedWithUUID,
std::conditional_t<IsDecimalNumber<A> && IsDecimalNumber<B>, std::conditional_t<(sizeof(A) > sizeof(B)), A, B>, Error>>;
};
......
......@@ -5,6 +5,7 @@
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypeNullable.h>
#include <Common/typeid_cast.h>
......@@ -20,57 +21,48 @@ void ExternalResultDescription::init(const Block & sample_block_)
{
sample_block = sample_block_;
const auto num_columns = sample_block.columns();
types.reserve(num_columns);
names.reserve(num_columns);
sample_columns.reserve(num_columns);
types.reserve(sample_block.columns());
for (const auto idx : ext::range(0, num_columns))
for (auto & elem : sample_block)
{
const auto & column = sample_block.safeGetByPosition(idx);
const auto type = column.type.get();
/// If default value for column was not provided, use default from data type.
if (elem.column->empty())
elem.column = elem.type->createColumnConstWithDefaultValue(1)->convertToFullColumnIfConst();
bool is_nullable = elem.type->isNullable();
DataTypePtr type_not_nullable = removeNullable(elem.type);
const IDataType * type = type_not_nullable.get();
if (typeid_cast<const DataTypeUInt8 *>(type))
types.push_back(ValueType::UInt8);
types.emplace_back(ValueType::UInt8, is_nullable);
else if (typeid_cast<const DataTypeUInt16 *>(type))
types.push_back(ValueType::UInt16);
types.emplace_back(ValueType::UInt16, is_nullable);
else if (typeid_cast<const DataTypeUInt32 *>(type))
types.push_back(ValueType::UInt32);
types.emplace_back(ValueType::UInt32, is_nullable);
else if (typeid_cast<const DataTypeUInt64 *>(type))
types.push_back(ValueType::UInt64);
types.emplace_back(ValueType::UInt64, is_nullable);
else if (typeid_cast<const DataTypeInt8 *>(type))
types.push_back(ValueType::Int8);
types.emplace_back(ValueType::Int8, is_nullable);
else if (typeid_cast<const DataTypeInt16 *>(type))
types.push_back(ValueType::Int16);
types.emplace_back(ValueType::Int16, is_nullable);
else if (typeid_cast<const DataTypeInt32 *>(type))
types.push_back(ValueType::Int32);
types.emplace_back(ValueType::Int32, is_nullable);
else if (typeid_cast<const DataTypeInt64 *>(type))
types.push_back(ValueType::Int64);
types.emplace_back(ValueType::Int64, is_nullable);
else if (typeid_cast<const DataTypeFloat32 *>(type))
types.push_back(ValueType::Float32);
types.emplace_back(ValueType::Float32, is_nullable);
else if (typeid_cast<const DataTypeFloat64 *>(type))
types.push_back(ValueType::Float64);
types.emplace_back(ValueType::Float64, is_nullable);
else if (typeid_cast<const DataTypeString *>(type))
types.push_back(ValueType::String);
types.emplace_back(ValueType::String, is_nullable);
else if (typeid_cast<const DataTypeDate *>(type))
types.push_back(ValueType::Date);
types.emplace_back(ValueType::Date, is_nullable);
else if (typeid_cast<const DataTypeDateTime *>(type))
types.push_back(ValueType::DateTime);
types.emplace_back(ValueType::DateTime, is_nullable);
else if (typeid_cast<const DataTypeUUID *>(type))
types.push_back(ValueType::UUID);
types.emplace_back(ValueType::UUID, is_nullable);
else
throw Exception{"Unsupported type " + type->getName(), ErrorCodes::UNKNOWN_TYPE};
names.emplace_back(column.name);
sample_columns.emplace_back(column.column);
/// If default value for column was not provided, use default from data type.
if (sample_columns.back()->empty())
{
MutableColumnPtr mutable_column = (*std::move(sample_columns.back())).mutate();
column.type->insertDefaultInto(*mutable_column);
sample_columns.back() = std::move(mutable_column);
}
}
}
......
......@@ -25,13 +25,11 @@ struct ExternalResultDescription
String,
Date,
DateTime,
UUID
UUID,
};
Block sample_block;
std::vector<ValueType> types;
std::vector<std::string> names;
Columns sample_columns;
std::vector<std::pair<ValueType, bool /* is_nullable */>> types;
void init(const Block & sample_block_);
};
......
......@@ -14,6 +14,7 @@
#include <Dictionaries/MongoDBBlockInputStream.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnNullable.h>
#include <Common/FieldVisitors.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
......@@ -179,13 +180,22 @@ Block MongoDBBlockInputStream::readImpl()
for (const auto idx : ext::range(0, size))
{
const auto & name = description.names[idx];
const auto & name = description.sample_block.getByPosition(idx).name;
const Poco::MongoDB::Element::Ptr value = document->get(name);
if (value.isNull() || value->type() == Poco::MongoDB::ElementTraits<Poco::MongoDB::NullValue>::TypeId)
insertDefaultValue(*columns[idx], *description.sample_columns[idx]);
insertDefaultValue(*columns[idx], *description.sample_block.getByPosition(idx).column);
else
insertValue(*columns[idx], description.types[idx], *value, name);
{
if (description.types[idx].second)
{
ColumnNullable & column_nullable = static_cast<ColumnNullable &>(*columns[idx]);
insertValue(column_nullable.getNestedColumn(), description.types[idx].first, *value, name);
column_nullable.getNullMapData().emplace_back(0);
}
else
insertValue(*columns[idx], description.types[idx].first, *value, name);
}
}
}
......
......@@ -32,7 +32,7 @@ public:
String getName() const override { return "MongoDB"; }
Block getHeader() const override { return description.sample_block; }
Block getHeader() const override { return description.sample_block.cloneEmpty(); }
private:
Block readImpl() override;
......
......@@ -4,6 +4,7 @@
#include <Dictionaries/MySQLBlockInputStream.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnNullable.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <ext/range.h>
......@@ -82,9 +83,18 @@ Block MySQLBlockInputStream::readImpl()
{
const auto value = row[idx];
if (!value.isNull())
insertValue(*columns[idx], description.types[idx], value);
{
if (description.types[idx].second)
{
ColumnNullable & column_nullable = static_cast<ColumnNullable &>(*columns[idx]);
insertValue(column_nullable.getNestedColumn(), description.types[idx].first, value);
column_nullable.getNullMapData().emplace_back(0);
}
else
insertValue(*columns[idx], description.types[idx].first, value);
}
else
insertDefaultValue(*columns[idx], *description.sample_columns[idx]);
insertDefaultValue(*columns[idx], *description.sample_block.getByPosition(idx).column);
}
++num_rows;
......
......@@ -21,7 +21,7 @@ public:
String getName() const override { return "MySQL"; }
Block getHeader() const override { return description.sample_block; }
Block getHeader() const override { return description.sample_block.cloneEmpty(); }
private:
Block readImpl() override;
......
......@@ -2,6 +2,7 @@
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnNullable.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
......@@ -91,9 +92,18 @@ Block ODBCBlockInputStream::readImpl()
const Poco::Dynamic::Var & value = row[idx];
if (!value.isEmpty())
insertValue(*columns[idx], description.types[idx], value);
{
if (description.types[idx].second)
{
ColumnNullable & column_nullable = static_cast<ColumnNullable &>(*columns[idx]);
insertValue(column_nullable.getNestedColumn(), description.types[idx].first, value);
column_nullable.getNullMapData().emplace_back(0);
}
else
insertValue(*columns[idx], description.types[idx].first, value);
}
else
insertDefaultValue(*columns[idx], *description.sample_columns[idx]);
insertDefaultValue(*columns[idx], *description.sample_block.getByPosition(idx).column);
}
++iterator;
......
......@@ -24,7 +24,7 @@ public:
String getName() const override { return "ODBC"; }
Block getHeader() const override { return description.sample_block; }
Block getHeader() const override { return description.sample_block.cloneEmpty(); }
private:
Block readImpl() override;
......
......@@ -230,6 +230,11 @@ public:
static FunctionPtr create(const Context &) { return std::make_shared<FunctionIf>(); }
private:
template <typename T0, typename T1>
static constexpr bool allow_arrays =
!IsDecimalNumber<T0> && !IsDecimalNumber<T1> &&
!std::is_same_v<T0, UInt128> && !std::is_same_v<T1, UInt128>;
template <typename T0, typename T1>
static UInt32 decimalScale(Block & block [[maybe_unused]], const ColumnNumbers & arguments [[maybe_unused]])
{
......@@ -314,7 +319,7 @@ private:
{
if constexpr (std::is_same_v<NumberTraits::Error, typename NumberTraits::ResultOfIf<T0, T1>::Type>)
return false;
else if constexpr (!IsDecimalNumber<T0> && !IsDecimalNumber<T1>)
else if constexpr (allow_arrays<T0, T1>)
{
using ResultType = typename NumberTraits::ResultOfIf<T0, T1>::Type;
......@@ -370,7 +375,7 @@ private:
{
if constexpr (std::is_same_v<NumberTraits::Error, typename NumberTraits::ResultOfIf<T0, T1>::Type>)
return false;
else if constexpr (!IsDecimalNumber<T0> && !IsDecimalNumber<T1>)
else if constexpr (allow_arrays<T0, T1>)
{
using ResultType = typename NumberTraits::ResultOfIf<T0, T1>::Type;
......@@ -978,6 +983,7 @@ public:
bool executed_with_nums = callOnBasicTypes<true, true, true, true>(left_id, right_id, call);
if (!( executed_with_nums
|| executeTyped<UInt128, UInt128>(cond_col, block, arguments, result, input_rows_count)
|| executeString(cond_col, block, arguments, result)
|| executeGenericArray(cond_col, block, arguments, result)
|| executeTuple(block, arguments, result, input_rows_count)))
......
value vs value
0 1 1 Int8 Int8 Int8
0 1 1 Int8 Int16 Int16
0 1 1 Int8 Int32 Int32
0 1 1 Int8 Int64 Int64
0 1 1 Int8 UInt8 Int16
0 1 1 Int8 UInt16 Int32
0 1 1 Int8 UInt32 Int64
0 1 1 Int8 Float32 Float32
0 1 1 Int8 Float64 Float64
0 1 1 Int16 Int8 Int16
0 1 1 Int16 Int16 Int16
0 1 1 Int16 Int32 Int32
0 1 1 Int16 Int64 Int64
0 1 1 Int16 UInt8 Int16
0 1 1 Int16 UInt16 Int32
0 1 1 Int16 UInt32 Int64
0 1 1 Int16 Float32 Float32
0 1 1 Int16 Float64 Float64
0 1 1 Int32 Int8 Int32
0 1 1 Int32 Int16 Int32
0 1 1 Int32 Int32 Int32
0 1 1 Int32 Int64 Int64
0 1 1 Int32 UInt8 Int32
0 1 1 Int32 UInt16 Int32
0 1 1 Int32 UInt32 Int64
0 1 1 Int32 Float32 Float64
0 1 1 Int32 Float64 Float64
0 1 1 Int64 Int8 Int64
0 1 1 Int64 Int16 Int64
0 1 1 Int64 Int32 Int64
0 1 1 Int64 Int64 Int64
0 1 1 Int64 UInt8 Int64
0 1 1 Int64 UInt16 Int64
0 1 1 Int64 UInt32 Int64
0 1 1 UInt8 Int8 Int16
0 1 1 UInt8 Int16 Int16
0 1 1 UInt8 Int32 Int32
0 1 1 UInt8 Int64 Int64
0 1 1 UInt8 UInt8 UInt8
0 1 1 UInt8 UInt16 UInt16
0 1 1 UInt8 UInt32 UInt32
0 1 1 UInt8 UInt64 UInt64
0 1 1 UInt8 Float32 Float32
0 1 1 UInt8 Float64 Float64
0 1 1 UInt16 Int8 Int32
0 1 1 UInt16 Int16 Int32
0 1 1 UInt16 Int32 Int32
0 1 1 UInt16 Int64 Int64
0 1 1 UInt16 UInt8 UInt16
0 1 1 UInt16 UInt16 UInt16
0 1 1 UInt16 UInt32 UInt32
0 1 1 UInt16 UInt64 UInt64
0 1 1 UInt16 Float32 Float32
0 1 1 UInt16 Float64 Float64
0 1 1 UInt32 Int8 Int64
0 1 1 UInt32 Int16 Int64
0 1 1 UInt32 Int32 Int64
0 1 1 UInt32 Int64 Int64
0 1 1 UInt32 UInt8 UInt32
0 1 1 UInt32 UInt16 UInt32
0 1 1 UInt32 UInt32 UInt32
0 1 1 UInt32 UInt64 UInt64
0 1 1 UInt32 Float32 Float64
0 1 1 UInt32 Float64 Float64
0 1 1 UInt64 UInt8 UInt64
0 1 1 UInt64 UInt16 UInt64
0 1 1 UInt64 UInt32 UInt64
0 1 1 UInt64 UInt64 UInt64
0000-00-00 1970-01-02 1970-01-02 Date Date Date
0000-00-00 1970-01-01 03:00:01 1970-01-01 03:00:01 Date DateTime DateTime
0000-00-00 00:00:00 1970-01-02 1970-01-01 03:00:01 DateTime Date DateTime
0000-00-00 00:00:00 1970-01-01 03:00:01 1970-01-01 03:00:01 DateTime DateTime DateTime
00000000-0000-0000-0000-000000000000 00000000-0000-0001-0000-000000000000 00000000-0000-0001-0000-000000000000 UUID UUID UUID
column vs value
0 1 1 Int8 Int8 Int8
0 1 1 Int8 Int16 Int16
0 1 1 Int8 Int32 Int32
0 1 1 Int8 Int64 Int64
0 1 1 Int8 UInt8 Int16
0 1 1 Int8 UInt16 Int32
0 1 1 Int8 UInt32 Int64
0 1 1 Int8 Float32 Float32
0 1 1 Int8 Float64 Float64
0 1 1 Int16 Int8 Int16
0 1 1 Int16 Int16 Int16
0 1 1 Int16 Int32 Int32
0 1 1 Int16 Int64 Int64
0 1 1 Int16 UInt8 Int16
0 1 1 Int16 UInt16 Int32
0 1 1 Int16 UInt32 Int64
0 1 1 Int16 Float32 Float32
0 1 1 Int16 Float64 Float64
0 1 1 Int32 Int8 Int32
0 1 1 Int32 Int16 Int32
0 1 1 Int32 Int32 Int32
0 1 1 Int32 Int64 Int64
0 1 1 Int32 UInt8 Int32
0 1 1 Int32 UInt16 Int32
0 1 1 Int32 UInt32 Int64
0 1 1 Int32 Float32 Float64
0 1 1 Int32 Float64 Float64
0 1 1 Int64 Int8 Int64
0 1 1 Int64 Int16 Int64
0 1 1 Int64 Int32 Int64
0 1 1 Int64 Int64 Int64
0 1 1 Int64 UInt8 Int64
0 1 1 Int64 UInt16 Int64
0 1 1 Int64 UInt32 Int64
0 1 1 UInt8 Int8 Int16
0 1 1 UInt8 Int16 Int16
0 1 1 UInt8 Int32 Int32
0 1 1 UInt8 Int64 Int64
0 1 1 UInt8 UInt8 UInt8
0 1 1 UInt8 UInt16 UInt16
0 1 1 UInt8 UInt32 UInt32
0 1 1 UInt8 UInt64 UInt64
0 1 1 UInt8 Float32 Float32
0 1 1 UInt8 Float64 Float64
0 1 1 UInt16 Int8 Int32
0 1 1 UInt16 Int16 Int32
0 1 1 UInt16 Int32 Int32
0 1 1 UInt16 Int64 Int64
0 1 1 UInt16 UInt8 UInt16
0 1 1 UInt16 UInt16 UInt16
0 1 1 UInt16 UInt32 UInt32
0 1 1 UInt16 UInt64 UInt64
0 1 1 UInt16 Float32 Float32
0 1 1 UInt16 Float64 Float64
0 1 1 UInt32 Int8 Int64
0 1 1 UInt32 Int16 Int64
0 1 1 UInt32 Int32 Int64
0 1 1 UInt32 Int64 Int64
0 1 1 UInt32 UInt8 UInt32
0 1 1 UInt32 UInt16 UInt32
0 1 1 UInt32 UInt32 UInt32
0 1 1 UInt32 UInt64 UInt64
0 1 1 UInt32 Float32 Float64
0 1 1 UInt32 Float64 Float64
0 1 1 UInt64 UInt8 UInt64
0 1 1 UInt64 UInt16 UInt64
0 1 1 UInt64 UInt32 UInt64
0 1 1 UInt64 UInt64 UInt64
0000-00-00 1970-01-02 1970-01-02 Date Date Date
0000-00-00 1970-01-01 03:00:01 1970-01-01 03:00:01 Date DateTime DateTime
0000-00-00 00:00:00 1970-01-02 1970-01-01 03:00:01 DateTime Date DateTime
0000-00-00 00:00:00 1970-01-01 03:00:01 1970-01-01 03:00:01 DateTime DateTime DateTime
00000000-0000-0000-0000-000000000000 00000000-0000-0001-0000-000000000000 00000000-0000-0001-0000-000000000000 UUID UUID UUID
此差异已折叠。
CREATE DATABASE IF NOT EXISTS test;
DROP TABLE IF EXISTS test.or_expr_bug;
CREATE TABLE test.or_expr_bug (a UInt64, b UInt64) ENGINE = Memory;
INSERT INTO test.or_expr_bug VALUES(1,21),(1,22),(1,23),(2,21),(2,22),(2,23),(3,21),(3,22),(3,23);
SELECT count(*) FROM test.or_expr_bug WHERE (a=1 OR a=2 OR a=3) AND (b=21 OR b=22 OR b=23);
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册