mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
dbms: added missing files [#CONV-2944].
This commit is contained in:
parent
476e8446e6
commit
c2c98e01b7
26
dbms/include/DB/DataStreams/TabSeparatedBlockOutputStream.h
Normal file
26
dbms/include/DB/DataStreams/TabSeparatedBlockOutputStream.h
Normal file
@ -0,0 +1,26 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Пишет данные в tab-separated файл, но по столбцам, блоками.
|
||||
* Блоки разделены двойным переводом строки.
|
||||
* На каждой строке блока - данные одного столбца.
|
||||
*/
|
||||
class TabSeparatedBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
TabSeparatedBlockOutputStream(WriteBuffer & ostr_) : ostr(ostr_) {}
|
||||
|
||||
/** Записать блок.
|
||||
*/
|
||||
void write(const Block & block);
|
||||
|
||||
private:
|
||||
WriteBuffer & ostr;
|
||||
};
|
||||
|
||||
}
|
289
dbms/include/DB/Functions/FunctionsComparison.h
Normal file
289
dbms/include/DB/Functions/FunctionsComparison.h
Normal file
@ -0,0 +1,289 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/NumberFormatter.h>
|
||||
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypesNumberVariable.h>
|
||||
#include <DB/DataTypes/DataTypeDateTime.h>
|
||||
#include <DB/DataTypes/DataTypeDate.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
#include <DB/DataTypes/DataTypeFixedString.h>
|
||||
#include <DB/Functions/IFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Функции сравнения: ==, !=, <, >, <=, >=.
|
||||
* Функции сравнения возвращают всегда 0 или 1 (UInt8).
|
||||
*
|
||||
* Сравнивать можно следующие типы:
|
||||
* - числа;
|
||||
* - строки и фиксированные строки;
|
||||
* - даты;
|
||||
* - даты-с-временем;
|
||||
* внутри каждой группы, но не из разных групп.
|
||||
*/
|
||||
|
||||
template<typename A, typename B>
|
||||
struct EqualsImpl
|
||||
{
|
||||
typedef UInt8 ResultType;
|
||||
|
||||
static void num_vector_num_vector(const std::vector<A> & a, const std::vector<B> & b, std::vector<ResultType> & c)
|
||||
{
|
||||
size_t size = a.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
c[i] = a[i] == b[i];
|
||||
}
|
||||
|
||||
static void num_vector_num_constant(const std::vector<A> & a, B b, std::vector<ResultType> & c)
|
||||
{
|
||||
size_t size = a.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
c[i] = a[i] == b;
|
||||
}
|
||||
|
||||
static void num_constant_num_vector(A a, const std::vector<B> & b, std::vector<ResultType> & c)
|
||||
{
|
||||
size_t size = b.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
c[i] = a == b[i];
|
||||
}
|
||||
|
||||
static void num_constant_num_constant(A a, B b, ResultType & c)
|
||||
{
|
||||
c = a == b;
|
||||
}
|
||||
|
||||
static void str_vector_str_vector(const std::vector<A> & a, const std::vector<B> & b, std::vector<ResultType> & c)
|
||||
{
|
||||
size_t size = a.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
c[i] = a[i] == b[i];
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <template <typename, typename> class Impl, typename Name>
|
||||
class FunctionComparison : public IFunction
|
||||
{
|
||||
private:
|
||||
template <typename T0, typename T1>
|
||||
bool checkRightType(const DataTypes & arguments, DataTypes & types_res) const
|
||||
{
|
||||
if (dynamic_cast<const T1 *>(&*arguments[1]))
|
||||
{
|
||||
types_res.push_back(new typename DataTypeFromFieldType<
|
||||
typename Impl<typename T0::FieldType, typename T1::FieldType>::ResultType>::Type);
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename T0>
|
||||
bool checkLeftType(const DataTypes & arguments, DataTypes & types_res) const
|
||||
{
|
||||
if (dynamic_cast<const T0 *>(&*arguments[0]))
|
||||
{
|
||||
if ( checkRightType<T0, DataTypeUInt8>(arguments, types_res)
|
||||
|| checkRightType<T0, DataTypeUInt16>(arguments, types_res)
|
||||
|| checkRightType<T0, DataTypeUInt32>(arguments, types_res)
|
||||
|| checkRightType<T0, DataTypeUInt64>(arguments, types_res)
|
||||
|| checkRightType<T0, DataTypeInt8>(arguments, types_res)
|
||||
|| checkRightType<T0, DataTypeInt16>(arguments, types_res)
|
||||
|| checkRightType<T0, DataTypeInt32>(arguments, types_res)
|
||||
|| checkRightType<T0, DataTypeInt64>(arguments, types_res)
|
||||
|| checkRightType<T0, DataTypeFloat32>(arguments, types_res)
|
||||
|| checkRightType<T0, DataTypeFloat64>(arguments, types_res)
|
||||
|| checkRightType<T0, DataTypeVarUInt>(arguments, types_res)
|
||||
|| checkRightType<T0, DataTypeVarInt>(arguments, types_res))
|
||||
return true;
|
||||
else
|
||||
throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
template <typename T0, typename T1>
|
||||
bool executeRightType(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & result, const ColumnVector<T0> * col_left)
|
||||
{
|
||||
if (ColumnVector<T1> * col_right = dynamic_cast<ColumnVector<T1> *>(&*block.getByPosition(arguments[1]).column))
|
||||
{
|
||||
typedef typename Impl<T0, T1>::ResultType ResultType;
|
||||
|
||||
ColumnVector<ResultType> * col_res = new ColumnVector<ResultType>;
|
||||
block.getByPosition(result[0]).column = col_res;
|
||||
|
||||
typename ColumnVector<ResultType>::Container_t & vec_res = col_res->getData();
|
||||
vec_res.resize(col_left->getData().size());
|
||||
Impl<T0, T1>::vector_vector(col_left->getData(), col_right->getData(), vec_res);
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (ColumnConst<T1> * col_right = dynamic_cast<ColumnConst<T1> *>(&*block.getByPosition(arguments[1]).column))
|
||||
{
|
||||
typedef typename Impl<T0, T1>::ResultType ResultType;
|
||||
|
||||
ColumnVector<ResultType> * col_res = new ColumnVector<ResultType>;
|
||||
block.getByPosition(result[0]).column = col_res;
|
||||
|
||||
typename ColumnVector<ResultType>::Container_t & vec_res = col_res->getData();
|
||||
vec_res.resize(col_left->getData().size());
|
||||
Impl<T0, T1>::vector_constant(col_left->getData(), col_right->getData(), vec_res);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename T0, typename T1>
|
||||
bool executeConstRightType(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & result, const ColumnConst<T0> * col_left)
|
||||
{
|
||||
if (ColumnVector<T1> * col_right = dynamic_cast<ColumnVector<T1> *>(&*block.getByPosition(arguments[1]).column))
|
||||
{
|
||||
typedef typename Impl<T0, T1>::ResultType ResultType;
|
||||
|
||||
ColumnVector<ResultType> * col_res = new ColumnVector<ResultType>;
|
||||
block.getByPosition(result[0]).column = col_res;
|
||||
|
||||
typename ColumnVector<ResultType>::Container_t & vec_res = col_res->getData();
|
||||
vec_res.resize(col_left->size());
|
||||
Impl<T0, T1>::constant_vector(col_left->getData(), col_right->getData(), vec_res);
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (ColumnConst<T1> * col_right = dynamic_cast<ColumnConst<T1> *>(&*block.getByPosition(arguments[1]).column))
|
||||
{
|
||||
typedef typename Impl<T0, T1>::ResultType ResultType;
|
||||
|
||||
ResultType res = 0;
|
||||
Impl<T0, T1>::constant_constant(col_left->getData(), col_right->getData(), res);
|
||||
|
||||
ColumnConst<ResultType> * col_res = new ColumnConst<ResultType>(col_left->size(), res);
|
||||
block.getByPosition(result[0]).column = col_res;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename T0>
|
||||
bool executeLeftType(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & result)
|
||||
{
|
||||
if (ColumnVector<T0> * col_left = dynamic_cast<ColumnVector<T0> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
if ( executeRightType<T0, UInt8>(block, arguments, result, col_left)
|
||||
|| executeRightType<T0, UInt16>(block, arguments, result, col_left)
|
||||
|| executeRightType<T0, UInt32>(block, arguments, result, col_left)
|
||||
|| executeRightType<T0, UInt64>(block, arguments, result, col_left)
|
||||
|| executeRightType<T0, Int8>(block, arguments, result, col_left)
|
||||
|| executeRightType<T0, Int16>(block, arguments, result, col_left)
|
||||
|| executeRightType<T0, Int32>(block, arguments, result, col_left)
|
||||
|| executeRightType<T0, Int64>(block, arguments, result, col_left)
|
||||
|| executeRightType<T0, Float32>(block, arguments, result, col_left)
|
||||
|| executeRightType<T0, Float64>(block, arguments, result, col_left))
|
||||
return true;
|
||||
else
|
||||
throw Exception("Illegal column of second argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else if (ColumnConst<T0> * col_left = dynamic_cast<ColumnConst<T0> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
if ( executeConstRightType<T0, UInt8>(block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, UInt16>(block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, UInt32>(block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, UInt64>(block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, Int8>(block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, Int16>(block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, Int32>(block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, Int64>(block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, Float32>(block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, Float64>(block, arguments, result, col_left))
|
||||
return true;
|
||||
else
|
||||
throw Exception("Illegal column of second argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
public:
|
||||
/// Получить все имена функции.
|
||||
String getName() const
|
||||
{
|
||||
return Name::get();
|
||||
}
|
||||
|
||||
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypes getReturnTypes(const DataTypes & arguments) const
|
||||
{
|
||||
if (arguments.size() != 2)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 2.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
DataTypes types_res;
|
||||
|
||||
if (!( checkLeftType<DataTypeUInt8>(arguments, types_res)
|
||||
|| checkLeftType<DataTypeUInt16>(arguments, types_res)
|
||||
|| checkLeftType<DataTypeUInt32>(arguments, types_res)
|
||||
|| checkLeftType<DataTypeUInt64>(arguments, types_res)
|
||||
|| checkLeftType<DataTypeInt8>(arguments, types_res)
|
||||
|| checkLeftType<DataTypeInt16>(arguments, types_res)
|
||||
|| checkLeftType<DataTypeInt32>(arguments, types_res)
|
||||
|| checkLeftType<DataTypeInt64>(arguments, types_res)
|
||||
|| checkLeftType<DataTypeFloat32>(arguments, types_res)
|
||||
|| checkLeftType<DataTypeFloat64>(arguments, types_res)
|
||||
|| checkLeftType<DataTypeVarUInt>(arguments, types_res)
|
||||
|| checkLeftType<DataTypeVarInt>(arguments, types_res)))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return types_res;
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & result)
|
||||
{
|
||||
if (result.size() != 1)
|
||||
throw Exception("Wrong number of result columns in function " + getName() + ", should be 1.",
|
||||
ErrorCodes::ILLEGAL_NUMBER_OF_RESULT_COLUMNS);
|
||||
|
||||
if (!( executeLeftType<UInt8>(block, arguments, result)
|
||||
|| executeLeftType<UInt16>(block, arguments, result)
|
||||
|| executeLeftType<UInt32>(block, arguments, result)
|
||||
|| executeLeftType<UInt64>(block, arguments, result)
|
||||
|| executeLeftType<Int8>(block, arguments, result)
|
||||
|| executeLeftType<Int16>(block, arguments, result)
|
||||
|| executeLeftType<Int32>(block, arguments, result)
|
||||
|| executeLeftType<Int64>(block, arguments, result)
|
||||
|| executeLeftType<Float32>(block, arguments, result)
|
||||
|| executeLeftType<Float64>(block, arguments, result)))
|
||||
throw Exception("Illegal column of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
struct NamePlus { static const char * get() { return "plus"; } };
|
||||
struct NameMinus { static const char * get() { return "minus"; } };
|
||||
struct NameMultiply { static const char * get() { return "multiply"; } };
|
||||
struct NameDivideFloating { static const char * get() { return "divide"; } };
|
||||
struct NameDivideIntegral { static const char * get() { return "div"; } };
|
||||
struct NameModulo { static const char * get() { return "modulo"; } };
|
||||
|
||||
typedef FunctionBinaryArithmetic<PlusImpl, NamePlus> FunctionPlus;
|
||||
typedef FunctionBinaryArithmetic<MinusImpl, NameMinus> FunctionMinus;
|
||||
typedef FunctionBinaryArithmetic<MultiplyImpl, NameMultiply> FunctionMultiply;
|
||||
typedef FunctionBinaryArithmetic<DivideFloatingImpl, NameDivideFloating> FunctionDivideFloating;
|
||||
typedef FunctionBinaryArithmetic<DivideIntegralImpl, NameDivideIntegral> FunctionDivideIntegral;
|
||||
typedef FunctionBinaryArithmetic<ModuloImpl, NameModulo> FunctionModulo;
|
||||
|
||||
|
||||
}
|
26
dbms/src/DataStreams/TabSeparatedBlockOutputStream.cpp
Normal file
26
dbms/src/DataStreams/TabSeparatedBlockOutputStream.cpp
Normal file
@ -0,0 +1,26 @@
|
||||
#include <DB/DataStreams/TabSeparatedBlockOutputStream.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void TabSeparatedBlockOutputStream::write(const Block & block)
|
||||
{
|
||||
size_t columns = block.columns();
|
||||
for (size_t i = 0; i < columns; ++i)
|
||||
{
|
||||
const ColumnWithNameAndType & col = block.getByPosition(i);
|
||||
|
||||
size_t rows = block.rows();
|
||||
for (size_t j = 0; j < rows; ++j)
|
||||
{
|
||||
if (j != 0)
|
||||
ostr.write('\t');
|
||||
col.type->serializeTextEscaped((*col.column)[j], ostr);
|
||||
}
|
||||
ostr.write('\n');
|
||||
}
|
||||
ostr.write('\n');
|
||||
}
|
||||
|
||||
}
|
131
dbms/src/DataStreams/tests/block_tab_separated_streams.cpp
Normal file
131
dbms/src/DataStreams/tests/block_tab_separated_streams.cpp
Normal file
@ -0,0 +1,131 @@
|
||||
#include <map>
|
||||
#include <list>
|
||||
#include <iostream>
|
||||
|
||||
#include <boost/assign/list_inserter.hpp>
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/IO/ReadBufferFromIStream.h>
|
||||
#include <DB/IO/WriteBufferFromOStream.h>
|
||||
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
#include <DB/DataTypes/DataTypeFixedString.h>
|
||||
#include <DB/DataTypes/DataTypeDateTime.h>
|
||||
|
||||
#include <DB/DataStreams/TabSeparatedRowInputStream.h>
|
||||
#include <DB/DataStreams/TabSeparatedBlockOutputStream.h>
|
||||
#include <DB/DataStreams/copyData.h>
|
||||
|
||||
#include <DB/Storages/StorageLog.h>
|
||||
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
{
|
||||
try
|
||||
{
|
||||
typedef std::pair<std::string, SharedPtr<DB::IDataType> > NameAndTypePair;
|
||||
typedef std::list<NameAndTypePair> NamesAndTypesList;
|
||||
|
||||
NamesAndTypesList names_and_types_list;
|
||||
|
||||
boost::assign::push_back(names_and_types_list)
|
||||
("WatchID", new DB::DataTypeUInt64)
|
||||
("JavaEnable", new DB::DataTypeUInt8)
|
||||
("Title", new DB::DataTypeString)
|
||||
("GoodEvent", new DB::DataTypeUInt32)
|
||||
("EventTime", new DB::DataTypeDateTime)
|
||||
("CounterID", new DB::DataTypeUInt32)
|
||||
("ClientIP", new DB::DataTypeUInt32)
|
||||
("RegionID", new DB::DataTypeUInt32)
|
||||
("UniqID", new DB::DataTypeUInt64)
|
||||
("CounterClass", new DB::DataTypeUInt8)
|
||||
("OS", new DB::DataTypeUInt8)
|
||||
("UserAgent", new DB::DataTypeUInt8)
|
||||
("URL", new DB::DataTypeString)
|
||||
("Referer", new DB::DataTypeString)
|
||||
("Refresh", new DB::DataTypeUInt8)
|
||||
("ResolutionWidth", new DB::DataTypeUInt16)
|
||||
("ResolutionHeight", new DB::DataTypeUInt16)
|
||||
("ResolutionDepth", new DB::DataTypeUInt8)
|
||||
("FlashMajor", new DB::DataTypeUInt8)
|
||||
("FlashMinor", new DB::DataTypeUInt8)
|
||||
("FlashMinor2", new DB::DataTypeString)
|
||||
("NetMajor", new DB::DataTypeUInt8)
|
||||
("NetMinor", new DB::DataTypeUInt8)
|
||||
("UserAgentMajor", new DB::DataTypeUInt16)
|
||||
("UserAgentMinor", new DB::DataTypeFixedString(2))
|
||||
("CookieEnable", new DB::DataTypeUInt8)
|
||||
("JavascriptEnable", new DB::DataTypeUInt8)
|
||||
("IsMobile", new DB::DataTypeUInt8)
|
||||
("MobilePhone", new DB::DataTypeUInt8)
|
||||
("MobilePhoneModel", new DB::DataTypeString)
|
||||
("Params", new DB::DataTypeString)
|
||||
("IPNetworkID", new DB::DataTypeUInt32)
|
||||
("TraficSourceID", new DB::DataTypeInt8)
|
||||
("SearchEngineID", new DB::DataTypeUInt16)
|
||||
("SearchPhrase", new DB::DataTypeString)
|
||||
("AdvEngineID", new DB::DataTypeUInt8)
|
||||
("IsArtifical", new DB::DataTypeUInt8)
|
||||
("WindowClientWidth", new DB::DataTypeUInt16)
|
||||
("WindowClientHeight", new DB::DataTypeUInt16)
|
||||
("ClientTimeZone", new DB::DataTypeInt16)
|
||||
("ClientEventTime", new DB::DataTypeDateTime)
|
||||
("SilverlightVersion1", new DB::DataTypeUInt8)
|
||||
("SilverlightVersion2", new DB::DataTypeUInt8)
|
||||
("SilverlightVersion3", new DB::DataTypeUInt32)
|
||||
("SilverlightVersion4", new DB::DataTypeUInt16)
|
||||
("PageCharset", new DB::DataTypeString)
|
||||
("CodeVersion", new DB::DataTypeUInt32)
|
||||
("IsLink", new DB::DataTypeUInt8)
|
||||
("IsDownload", new DB::DataTypeUInt8)
|
||||
("IsNotBounce", new DB::DataTypeUInt8)
|
||||
("FUniqID", new DB::DataTypeUInt64)
|
||||
("OriginalURL", new DB::DataTypeString)
|
||||
("HID", new DB::DataTypeUInt32)
|
||||
("IsOldCounter", new DB::DataTypeUInt8)
|
||||
("IsEvent", new DB::DataTypeUInt8)
|
||||
("IsParameter", new DB::DataTypeUInt8)
|
||||
("DontCountHits", new DB::DataTypeUInt8)
|
||||
("WithHash", new DB::DataTypeUInt8)
|
||||
;
|
||||
|
||||
SharedPtr<DB::DataTypes> data_types = new DB::DataTypes;
|
||||
|
||||
for (NamesAndTypesList::const_iterator it = names_and_types_list.begin(); it != names_and_types_list.end(); ++it)
|
||||
data_types->push_back(it->second);
|
||||
|
||||
/// создаём описание, как читать данные из tab separated дампа
|
||||
|
||||
DB::Block sample;
|
||||
for (NamesAndTypesList::const_iterator it = names_and_types_list.begin(); it != names_and_types_list.end(); ++it)
|
||||
{
|
||||
DB::ColumnWithNameAndType elem;
|
||||
elem.name = it->first;
|
||||
elem.type = it->second;
|
||||
elem.column = elem.type->createColumn();
|
||||
sample.insert(elem);
|
||||
}
|
||||
|
||||
/// читаем данные из строчного tsv файла и одновременно пишем в блочный tsv файл
|
||||
{
|
||||
DB::ReadBufferFromIStream in_buf(std::cin);
|
||||
DB::WriteBufferFromOStream out_buf(std::cout);
|
||||
|
||||
DB::TabSeparatedRowInputStream in(in_buf, data_types);
|
||||
DB::TabSeparatedBlockOutputStream out(out_buf);
|
||||
DB::copyData(in, out, sample);
|
||||
}
|
||||
}
|
||||
catch (const DB::Exception & e)
|
||||
{
|
||||
std::cerr << e.what() << ", " << e.message() << std::endl;
|
||||
return 1;
|
||||
}
|
||||
|
||||
return 0;
|
||||
}
|
71
dbms/src/Storages/IStorage.cpp
Normal file
71
dbms/src/Storages/IStorage.cpp
Normal file
@ -0,0 +1,71 @@
|
||||
#include <set>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
static std::string listOfColumns(const NamesAndTypes & available_columns)
|
||||
{
|
||||
std::stringstream s;
|
||||
for (NamesAndTypes::const_iterator it = available_columns.begin(); it != available_columns.end(); ++it)
|
||||
{
|
||||
if (it != available_columns.begin())
|
||||
s << ", ";
|
||||
s << it->first;
|
||||
}
|
||||
return s.str();
|
||||
}
|
||||
|
||||
|
||||
void IStorage::check(const Names & column_names) const
|
||||
{
|
||||
const NamesAndTypes & available_columns = getColumns();
|
||||
|
||||
if (column_names.empty())
|
||||
throw Exception("Empty list of columns queried for table " + getTableName()
|
||||
+ ". There are columns: " + listOfColumns(available_columns),
|
||||
ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED);
|
||||
|
||||
typedef std::set<std::string> UniqueStrings;
|
||||
UniqueStrings unique_names;
|
||||
|
||||
for (Names::const_iterator it = column_names.begin(); it != column_names.end(); ++it)
|
||||
{
|
||||
if (available_columns.end() == available_columns.find(*it))
|
||||
throw Exception("There is no column with name " + *it + " in table " + getTableName()
|
||||
+ ". There are columns: " + listOfColumns(available_columns),
|
||||
ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
|
||||
|
||||
if (unique_names.end() != unique_names.find(*it))
|
||||
throw Exception("Column " + *it + " queried more than once in table " + getTableName(),
|
||||
ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE);
|
||||
unique_names.insert(*it);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void IStorage::check(const Block & block) const
|
||||
{
|
||||
const NamesAndTypes & available_columns = getColumns();
|
||||
|
||||
for (size_t i = 0; i < block.columns(); ++i)
|
||||
{
|
||||
const ColumnWithNameAndType & column = block.getByPosition(i);
|
||||
|
||||
NamesAndTypes::const_iterator it = available_columns.find(column.name);
|
||||
if (available_columns.end() == it)
|
||||
throw Exception("There is no column with name " + column.name + " in table " + getTableName()
|
||||
+ ". There are columns: " + listOfColumns(available_columns),
|
||||
ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
|
||||
|
||||
if (column.type->getName() != it->second->getName())
|
||||
throw Exception("Type mismatch for column " + column.name + " in table " + getTableName()
|
||||
+ ". Column has type " + it->second->getName() + ", got type " + column.type->getName(),
|
||||
ErrorCodes::TYPE_MISMATCH);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
}
|
Loading…
Reference in New Issue
Block a user