Removed useless namespace [#CONV-2807].

This commit is contained in:
Alexey Milovidov 2013-08-11 03:40:14 +00:00
parent 3e0046011a
commit be67c3e3d6
26 changed files with 162 additions and 165 deletions

View File

@ -1,6 +1,6 @@
#pragma once
#include <Yandex/Common.h> /// Yandex::VisitID_t
#include <Yandex/Common.h> /// VisitID_t
#include <DB/Core/Field.h>
@ -26,7 +26,7 @@ inline Field toField(const mysqlxx::DateTime & x)
return toField(static_cast<UInt32>(static_cast<time_t>(x)));
}
inline Field toField(const Yandex::VisitID_t & x)
inline Field toField(const VisitID_t & x)
{
return toField(static_cast<UInt64>(x));
}

View File

@ -20,12 +20,12 @@ public:
void serializeText(const Field & field, WriteBuffer & ostr) const
{
writeDateText(Yandex::DayNum_t(get<UInt64>(field)), ostr);
writeDateText(DayNum_t(get<UInt64>(field)), ostr);
}
void deserializeText(Field & field, ReadBuffer & istr) const
{
Yandex::DayNum_t x;
DayNum_t x;
readDateText(x, istr);
field = static_cast<UInt64>(x);
}

View File

@ -70,7 +70,7 @@ struct ConvertImpl<DataTypeDate, DataTypeDateTime, Name>
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
{
typedef DataTypeDate::FieldType FromFieldType;
Yandex::DateLUTSingleton & date_lut = Yandex::DateLUTSingleton::instance();
DateLUTSingleton & date_lut = DateLUTSingleton::instance();
if (const ColumnVector<FromFieldType> * col_from = dynamic_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
{
@ -84,12 +84,12 @@ struct ConvertImpl<DataTypeDate, DataTypeDateTime, Name>
for (size_t i = 0; i < size; ++i)
{
vec_to[i] = date_lut.fromDayNum(Yandex::DayNum_t(vec_from[i]));
vec_to[i] = date_lut.fromDayNum(DayNum_t(vec_from[i]));
}
}
else if (const ColumnConst<FromFieldType> * col_from = dynamic_cast<const ColumnConst<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
{
block.getByPosition(result).column = new ColumnConst<ToFieldType>(col_from->size(), date_lut.fromDayNum(Yandex::DayNum_t(col_from->getData())));
block.getByPosition(result).column = new ColumnConst<ToFieldType>(col_from->size(), date_lut.fromDayNum(DayNum_t(col_from->getData())));
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
@ -109,7 +109,7 @@ struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name>
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
{
Yandex::DateLUTSingleton & date_lut = Yandex::DateLUTSingleton::instance();
DateLUTSingleton & date_lut = DateLUTSingleton::instance();
if (const ColumnVector<FromFieldType> * col_from = dynamic_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
{
@ -139,7 +139,7 @@ struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name>
/** Преобразование чисел, дат, дат-с-временем в строки: через форматирование.
*/
template <typename DataType> void formatImpl(typename DataType::FieldType x, WriteBuffer & wb) { writeText(x, wb); }
template <> inline void formatImpl<DataTypeDate>(DataTypeDate::FieldType x, WriteBuffer & wb) { writeDateText(Yandex::DayNum_t(x), wb); }
template <> inline void formatImpl<DataTypeDate>(DataTypeDate::FieldType x, WriteBuffer & wb) { writeDateText(DayNum_t(x), wb); }
template <> inline void formatImpl<DataTypeDateTime>(DataTypeDateTime::FieldType x, WriteBuffer & wb) { writeDateTimeText(x, wb); }
template <typename FromDataType, typename Name>
@ -192,7 +192,7 @@ template <typename DataType> void parseImpl(typename DataType::FieldType & x, Re
template <> inline void parseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb)
{
Yandex::DayNum_t tmp(0);
DayNum_t tmp(0);
readDateText(tmp, rb);
x = tmp;
}

View File

@ -48,32 +48,32 @@ namespace DB
struct ToYearImpl
{
static inline UInt16 execute(UInt32 t, Yandex::DateLUTSingleton & date_lut) { return date_lut.toYear(t); }
static inline UInt16 execute(UInt16 d, Yandex::DateLUTSingleton & date_lut) { return date_lut.toYear(Yandex::DayNum_t(d)); }
static inline UInt16 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toYear(t); }
static inline UInt16 execute(UInt16 d, DateLUTSingleton & date_lut) { return date_lut.toYear(DayNum_t(d)); }
};
struct ToMonthImpl
{
static inline UInt8 execute(UInt32 t, Yandex::DateLUTSingleton & date_lut) { return date_lut.toMonth(t); }
static inline UInt8 execute(UInt16 d, Yandex::DateLUTSingleton & date_lut) { return date_lut.toMonth(Yandex::DayNum_t(d)); }
static inline UInt8 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toMonth(t); }
static inline UInt8 execute(UInt16 d, DateLUTSingleton & date_lut) { return date_lut.toMonth(DayNum_t(d)); }
};
struct ToDayOfMonthImpl
{
static inline UInt8 execute(UInt32 t, Yandex::DateLUTSingleton & date_lut) { return date_lut.toDayOfMonth(t); }
static inline UInt8 execute(UInt16 d, Yandex::DateLUTSingleton & date_lut) { return date_lut.toDayOfMonth(Yandex::DayNum_t(d)); }
static inline UInt8 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toDayOfMonth(t); }
static inline UInt8 execute(UInt16 d, DateLUTSingleton & date_lut) { return date_lut.toDayOfMonth(DayNum_t(d)); }
};
struct ToDayOfWeekImpl
{
static inline UInt8 execute(UInt32 t, Yandex::DateLUTSingleton & date_lut) { return date_lut.toDayOfWeek(t); }
static inline UInt8 execute(UInt16 d, Yandex::DateLUTSingleton & date_lut) { return date_lut.toDayOfWeek(Yandex::DayNum_t(d)); }
static inline UInt8 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toDayOfWeek(t); }
static inline UInt8 execute(UInt16 d, DateLUTSingleton & date_lut) { return date_lut.toDayOfWeek(DayNum_t(d)); }
};
struct ToHourImpl
{
static inline UInt8 execute(UInt32 t, Yandex::DateLUTSingleton & date_lut) { return date_lut.toHourInaccurate(t); }
static inline UInt8 execute(UInt16 d, Yandex::DateLUTSingleton & date_lut)
static inline UInt8 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toHourInaccurate(t); }
static inline UInt8 execute(UInt16 d, DateLUTSingleton & date_lut)
{
throw Exception("Illegal type Date of argument for function toHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
@ -81,8 +81,8 @@ struct ToHourImpl
struct ToMinuteImpl
{
static inline UInt8 execute(UInt32 t, Yandex::DateLUTSingleton & date_lut) { return date_lut.toMinuteInaccurate(t); }
static inline UInt8 execute(UInt16 d, Yandex::DateLUTSingleton & date_lut)
static inline UInt8 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toMinuteInaccurate(t); }
static inline UInt8 execute(UInt16 d, DateLUTSingleton & date_lut)
{
throw Exception("Illegal type Date of argument for function toMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
@ -90,8 +90,8 @@ struct ToMinuteImpl
struct ToSecondImpl
{
static inline UInt8 execute(UInt32 t, Yandex::DateLUTSingleton & date_lut) { return date_lut.toSecondInaccurate(t); }
static inline UInt8 execute(UInt16 d, Yandex::DateLUTSingleton & date_lut)
static inline UInt8 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toSecondInaccurate(t); }
static inline UInt8 execute(UInt16 d, DateLUTSingleton & date_lut)
{
throw Exception("Illegal type Date of argument for function toSecond", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
@ -99,28 +99,28 @@ struct ToSecondImpl
struct ToMondayImpl
{
static inline UInt16 execute(UInt32 t, Yandex::DateLUTSingleton & date_lut) { return date_lut.toFirstDayNumOfWeek(date_lut.toDayNum(t)); }
static inline UInt16 execute(UInt16 d, Yandex::DateLUTSingleton & date_lut) { return date_lut.toFirstDayNumOfWeek(Yandex::DayNum_t(d)); }
static inline UInt16 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toFirstDayNumOfWeek(date_lut.toDayNum(t)); }
static inline UInt16 execute(UInt16 d, DateLUTSingleton & date_lut) { return date_lut.toFirstDayNumOfWeek(DayNum_t(d)); }
};
struct ToStartOfMonthImpl
{
static inline UInt16 execute(UInt32 t, Yandex::DateLUTSingleton & date_lut) { return date_lut.toFirstDayNumOfMonth(date_lut.toDayNum(t)); }
static inline UInt16 execute(UInt16 d, Yandex::DateLUTSingleton & date_lut) { return date_lut.toFirstDayNumOfMonth(Yandex::DayNum_t(d)); }
static inline UInt16 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toFirstDayNumOfMonth(date_lut.toDayNum(t)); }
static inline UInt16 execute(UInt16 d, DateLUTSingleton & date_lut) { return date_lut.toFirstDayNumOfMonth(DayNum_t(d)); }
};
struct ToStartOfYearImpl
{
static inline UInt16 execute(UInt32 t, Yandex::DateLUTSingleton & date_lut) { return date_lut.toFirstDayNumOfYear(date_lut.toDayNum(t)); }
static inline UInt16 execute(UInt16 d, Yandex::DateLUTSingleton & date_lut) { return date_lut.toFirstDayNumOfYear(Yandex::DayNum_t(d)); }
static inline UInt16 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toFirstDayNumOfYear(date_lut.toDayNum(t)); }
static inline UInt16 execute(UInt16 d, DateLUTSingleton & date_lut) { return date_lut.toFirstDayNumOfYear(DayNum_t(d)); }
};
struct ToTimeImpl
{
/// При переводе во время, дату будем приравнивать к 1970-01-02.
static inline UInt32 execute(UInt32 t, Yandex::DateLUTSingleton & date_lut) { return date_lut.toTimeInaccurate(t) + 86400; }
static inline UInt32 execute(UInt16 d, Yandex::DateLUTSingleton & date_lut)
static inline UInt32 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toTimeInaccurate(t) + 86400; }
static inline UInt32 execute(UInt16 d, DateLUTSingleton & date_lut)
{
throw Exception("Illegal type Date of argument for function toTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
@ -128,8 +128,8 @@ struct ToTimeImpl
struct ToStartOfMinuteImpl
{
static inline UInt32 execute(UInt32 t, Yandex::DateLUTSingleton & date_lut) { return date_lut.toStartOfMinuteInaccurate(t); }
static inline UInt32 execute(UInt16 d, Yandex::DateLUTSingleton & date_lut)
static inline UInt32 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toStartOfMinuteInaccurate(t); }
static inline UInt32 execute(UInt16 d, DateLUTSingleton & date_lut)
{
throw Exception("Illegal type Date of argument for function toStartOfMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
@ -137,8 +137,8 @@ struct ToStartOfMinuteImpl
struct ToStartOfHourImpl
{
static inline UInt32 execute(UInt32 t, Yandex::DateLUTSingleton & date_lut) { return date_lut.toStartOfHourInaccurate(t); }
static inline UInt32 execute(UInt16 d, Yandex::DateLUTSingleton & date_lut)
static inline UInt32 execute(UInt32 t, DateLUTSingleton & date_lut) { return date_lut.toStartOfHourInaccurate(t); }
static inline UInt32 execute(UInt16 d, DateLUTSingleton & date_lut)
{
throw Exception("Illegal type Date of argument for function toStartOfHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
@ -150,7 +150,7 @@ struct DateTimeTransformImpl
{
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
{
Yandex::DateLUTSingleton & date_lut = Yandex::DateLUTSingleton::instance();
DateLUTSingleton & date_lut = DateLUTSingleton::instance();
if (const ColumnVector<FromType> * col_from = dynamic_cast<const ColumnVector<FromType> *>(&*block.getByPosition(arguments[0]).column))
{

View File

@ -323,7 +323,7 @@ void readBackQuotedString(String & s, ReadBuffer & buf);
/// в формате YYYY-MM-DD
inline void readDateText(Yandex::DayNum_t & date, ReadBuffer & buf)
inline void readDateText(DayNum_t & date, ReadBuffer & buf)
{
char s[10];
size_t size = buf.read(s, 10);
@ -337,7 +337,7 @@ inline void readDateText(Yandex::DayNum_t & date, ReadBuffer & buf)
UInt8 month = (s[5] - '0') * 10 + (s[6] - '0');
UInt8 day = (s[8] - '0') * 10 + (s[9] - '0');
date = Yandex::DateLUTSingleton::instance().makeDayNum(year, month, day);
date = DateLUTSingleton::instance().makeDayNum(year, month, day);
}
inline void readDateText(mysqlxx::Date & date, ReadBuffer & buf)
@ -378,7 +378,7 @@ inline void readDateTimeText(time_t & datetime, ReadBuffer & buf)
if (unlikely(year == 0))
datetime = 0;
else
datetime = Yandex::DateLUTSingleton::instance().makeDateTime(year, month, day, hour, minute, second);
datetime = DateLUTSingleton::instance().makeDateTime(year, month, day, hour, minute, second);
}
inline void readDateTimeText(mysqlxx::DateTime & datetime, ReadBuffer & buf)
@ -415,7 +415,7 @@ inline void readBinary(Float64 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(String & x, ReadBuffer & buf) { readStringBinary(x, buf); }
inline void readBinary(bool & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(Yandex::VisitID_t & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(VisitID_t & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(mysqlxx::Date & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(mysqlxx::DateTime & x, ReadBuffer & buf) { readPODBinary(x, buf); }
@ -434,7 +434,7 @@ inline void readText(Float64 & x, ReadBuffer & buf) { readFloatText(x, buf); }
inline void readText(String & x, ReadBuffer & buf) { readEscapedString(x, buf); }
inline void readText(bool & x, ReadBuffer & buf) { readBoolText(x, buf); }
inline void readText(Yandex::VisitID_t & x, ReadBuffer & buf) { readIntText(x, buf); }
inline void readText(VisitID_t & x, ReadBuffer & buf) { readIntText(x, buf); }
inline void readText(mysqlxx::Date & x, ReadBuffer & buf) { readDateText(x, buf); }
inline void readText(mysqlxx::DateTime & x, ReadBuffer & buf) { readDateTimeText(x, buf); }
@ -453,7 +453,7 @@ inline void readQuoted(Float64 & x, ReadBuffer & buf) { readFloatText(x, buf); }
inline void readQuoted(String & x, ReadBuffer & buf) { readQuotedString(x, buf); }
inline void readQuoted(bool & x, ReadBuffer & buf) { readBoolText(x, buf); }
inline void readQuoted(Yandex::VisitID_t & x, ReadBuffer & buf) { readIntText(x, buf); }
inline void readQuoted(VisitID_t & x, ReadBuffer & buf) { readIntText(x, buf); }
inline void readQuoted(mysqlxx::Date & x, ReadBuffer & buf)
{
@ -484,7 +484,7 @@ inline void readDoubleQuoted(Float64 & x, ReadBuffer & buf) { readFloatText(x,
inline void readDoubleQuoted(String & x, ReadBuffer & buf) { readDoubleQuotedString(x, buf); }
inline void readDoubleQuoted(bool & x, ReadBuffer & buf) { readBoolText(x, buf); }
inline void readDoubleQuoted(Yandex::VisitID_t & x, ReadBuffer & buf) { readIntText(x, buf); }
inline void readDoubleQuoted(VisitID_t & x, ReadBuffer & buf) { readIntText(x, buf); }
inline void readDoubleQuoted(mysqlxx::Date & x, ReadBuffer & buf)
{

View File

@ -294,7 +294,7 @@ inline void writeProbablyBackQuotedString(const String & s, WriteBuffer & buf)
/// в формате YYYY-MM-DD
inline void writeDateText(Yandex::DayNum_t date, WriteBuffer & buf)
inline void writeDateText(DayNum_t date, WriteBuffer & buf)
{
char s[10] = {'0', '0', '0', '0', '-', '0', '0', '-', '0', '0'};
@ -304,7 +304,7 @@ inline void writeDateText(Yandex::DayNum_t date, WriteBuffer & buf)
return;
}
const Yandex::DateLUT::Values & values = Yandex::DateLUTSingleton::instance().getValues(date);
const DateLUT::Values & values = DateLUTSingleton::instance().getValues(date);
s[0] += values.year / 1000;
s[1] += (values.year / 100) % 10;
@ -346,8 +346,8 @@ inline void writeDateTimeText(time_t datetime, WriteBuffer & buf)
return;
}
Yandex::DateLUTSingleton & date_lut = Yandex::DateLUTSingleton::instance();
const Yandex::DateLUT::Values & values = date_lut.getValues(datetime);
DateLUTSingleton & date_lut = DateLUTSingleton::instance();
const DateLUT::Values & values = date_lut.getValues(datetime);
s[0] += values.year / 1000;
s[1] += (values.year / 100) % 10;
@ -429,7 +429,7 @@ inline void writeBinary(const Float64 & x, WriteBuffer & buf) { writePODBinary(
inline void writeBinary(const String & x, WriteBuffer & buf) { writeStringBinary(x, buf); }
inline void writeBinary(const bool & x, WriteBuffer & buf) { writePODBinary(x, buf); }
inline void writeBinary(const Yandex::VisitID_t & x, WriteBuffer & buf) { writePODBinary(static_cast<const UInt64 &>(x), buf); }
inline void writeBinary(const VisitID_t & x, WriteBuffer & buf) { writePODBinary(static_cast<const UInt64 &>(x), buf); }
inline void writeBinary(const mysqlxx::Date & x, WriteBuffer & buf) { writePODBinary(x, buf); }
inline void writeBinary(const mysqlxx::DateTime & x, WriteBuffer & buf) { writePODBinary(x, buf); }
@ -448,7 +448,7 @@ inline void writeText(const Float64 & x, WriteBuffer & buf) { writeFloatText(x,
inline void writeText(const String & x, WriteBuffer & buf) { writeEscapedString(x, buf); }
inline void writeText(const bool & x, WriteBuffer & buf) { writeBoolText(x, buf); }
inline void writeText(const Yandex::VisitID_t & x, WriteBuffer & buf) { writeIntText(static_cast<const UInt64 &>(x), buf); }
inline void writeText(const VisitID_t & x, WriteBuffer & buf) { writeIntText(static_cast<const UInt64 &>(x), buf); }
inline void writeText(const mysqlxx::Date & x, WriteBuffer & buf) { writeDateText(x, buf); }
inline void writeText(const mysqlxx::DateTime & x, WriteBuffer & buf) { writeDateTimeText(x, buf); }
@ -467,7 +467,7 @@ inline void writeQuoted(const Float64 & x, WriteBuffer & buf) { writeFloatText(
inline void writeQuoted(const String & x, WriteBuffer & buf) { writeQuotedString(x, buf); }
inline void writeQuoted(const bool & x, WriteBuffer & buf) { writeBoolText(x, buf); }
inline void writeQuoted(const Yandex::VisitID_t & x, WriteBuffer & buf)
inline void writeQuoted(const VisitID_t & x, WriteBuffer & buf)
{
writeIntText(static_cast<const UInt64 &>(x), buf);
}
@ -501,7 +501,7 @@ inline void writeDoubleQuoted(const Float64 & x, WriteBuffer & buf) { writeFloa
inline void writeDoubleQuoted(const String & x, WriteBuffer & buf) { writeDoubleQuotedString(x, buf); }
inline void writeDoubleQuoted(const bool & x, WriteBuffer & buf) { writeBoolText(x, buf); }
inline void writeDoubleQuoted(const Yandex::VisitID_t & x, WriteBuffer & buf)
inline void writeDoubleQuoted(const VisitID_t & x, WriteBuffer & buf)
{
writeIntText(static_cast<const UInt64 &>(x), buf);
}

View File

@ -22,10 +22,10 @@ using Poco::SharedPtr;
class Dictionaries
{
private:
Yandex::MultiVersion<RegionsHierarchy> regions_hierarchy;
Yandex::MultiVersion<TechDataHierarchy> tech_data_hierarchy;
Yandex::MultiVersion<CategoriesHierarchy> categories_hierarchy;
Yandex::MultiVersion<RegionsNames> regions_names;
MultiVersion<RegionsHierarchy> regions_hierarchy;
MultiVersion<TechDataHierarchy> tech_data_hierarchy;
MultiVersion<CategoriesHierarchy> categories_hierarchy;
MultiVersion<RegionsNames> regions_names;
/// Периодичность обновления справочников, в секундах.
int reload_period;
@ -47,12 +47,12 @@ private:
{
LOG_INFO(log, "Loading dictionaries.");
Yandex::MultiVersion<TechDataHierarchy>::Version new_tech_data_hierarchy = new TechDataHierarchy;
Yandex::MultiVersion<RegionsHierarchy>::Version new_regions_hierarchy = new RegionsHierarchy;
MultiVersion<TechDataHierarchy>::Version new_tech_data_hierarchy = new TechDataHierarchy;
MultiVersion<RegionsHierarchy>::Version new_regions_hierarchy = new RegionsHierarchy;
new_regions_hierarchy->reload();
Yandex::MultiVersion<CategoriesHierarchy>::Version new_categories_hierarchy = new CategoriesHierarchy;
MultiVersion<CategoriesHierarchy>::Version new_categories_hierarchy = new CategoriesHierarchy;
new_categories_hierarchy->reload();
Yandex::MultiVersion<RegionsNames>::Version new_regions_names = new RegionsNames;
MultiVersion<RegionsNames>::Version new_regions_names = new RegionsNames;
new_regions_names->reload();
tech_data_hierarchy.set(new_tech_data_hierarchy);
@ -102,22 +102,22 @@ public:
reloading_thread.join();
}
Yandex::MultiVersion<RegionsHierarchy>::Version getRegionsHierarchy() const
MultiVersion<RegionsHierarchy>::Version getRegionsHierarchy() const
{
return regions_hierarchy.get();
}
Yandex::MultiVersion<TechDataHierarchy>::Version getTechDataHierarchy() const
MultiVersion<TechDataHierarchy>::Version getTechDataHierarchy() const
{
return tech_data_hierarchy.get();
}
Yandex::MultiVersion<CategoriesHierarchy>::Version getCategoriesHierarchy() const
MultiVersion<CategoriesHierarchy>::Version getCategoriesHierarchy() const
{
return categories_hierarchy.get();
}
Yandex::MultiVersion<RegionsNames>::Version getRegionsNames() const
MultiVersion<RegionsNames>::Version getRegionsNames() const
{
return regions_names.get();
}

View File

@ -16,7 +16,7 @@ public:
{
storage.check(block, true);
Yandex::DateLUTSingleton & date_lut = Yandex::DateLUTSingleton::instance();
DateLUTSingleton & date_lut = DateLUTSingleton::instance();
size_t rows = block.rows();
size_t columns = block.columns();
@ -40,8 +40,8 @@ public:
typedef std::map<UInt16, BlockWithDateInterval> BlocksByMonth;
BlocksByMonth blocks_by_month;
UInt16 min_month = date_lut.toFirstDayNumOfMonth(Yandex::DayNum_t(min_date));
UInt16 max_month = date_lut.toFirstDayNumOfMonth(Yandex::DayNum_t(max_date));
UInt16 min_month = date_lut.toFirstDayNumOfMonth(DayNum_t(min_date));
UInt16 max_month = date_lut.toFirstDayNumOfMonth(DayNum_t(max_date));
/// Типичный случай - когда месяц один (ничего разделять не нужно).
if (min_month == max_month)
@ -50,7 +50,7 @@ public:
{
for (size_t i = 0; i < rows; ++i)
{
UInt16 month = date_lut.toFirstDayNumOfMonth(Yandex::DayNum_t(dates[i]));
UInt16 month = date_lut.toFirstDayNumOfMonth(DayNum_t(dates[i]));
BlockWithDateInterval & block_for_month = blocks_by_month[month];
if (!block_for_month.block)
@ -91,14 +91,14 @@ private:
void writePart(Block & block, UInt16 min_date, UInt16 max_date)
{
Yandex::DateLUTSingleton & date_lut = Yandex::DateLUTSingleton::instance();
DateLUTSingleton & date_lut = DateLUTSingleton::instance();
size_t rows = block.rows();
size_t columns = block.columns();
UInt64 part_id = storage.increment.get(true);
String part_name = storage.getPartName(
Yandex::DayNum_t(min_date), Yandex::DayNum_t(max_date),
DayNum_t(min_date), DayNum_t(max_date),
part_id, part_id, 0);
String part_tmp_path = storage.full_path + "tmp_" + part_name + "/";
@ -159,8 +159,8 @@ private:
Poco::ScopedLock<Poco::FastMutex> lock_all(storage.all_data_parts_mutex);
StorageMergeTree::DataPartPtr new_data_part = new StorageMergeTree::DataPart(storage);
new_data_part->left_date = Yandex::DayNum_t(min_date);
new_data_part->right_date = Yandex::DayNum_t(max_date);
new_data_part->left_date = DayNum_t(min_date);
new_data_part->right_date = DayNum_t(max_date);
new_data_part->left = part_id;
new_data_part->right = part_id;
new_data_part->level = 0;

View File

@ -16,7 +16,7 @@ public:
: storage(storage_), marks_count(0), index_offset(0)
{
part_name = storage.getPartName(
Yandex::DayNum_t(min_date), Yandex::DayNum_t(max_date),
DayNum_t(min_date), DayNum_t(max_date),
min_part_id, max_part_id, level);
part_tmp_path = storage.full_path + "tmp_" + part_name + "/";

View File

@ -194,8 +194,8 @@ private:
DataPart(StorageMergeTree & storage_) : storage(storage_), currently_merging(false) {}
StorageMergeTree & storage;
Yandex::DayNum_t left_date;
Yandex::DayNum_t right_date;
DayNum_t left_date;
DayNum_t right_date;
UInt64 left;
UInt64 right;
/// Уровень игнорируется. Использовался предыдущей эвристикой слияния.
@ -205,8 +205,8 @@ private:
size_t size; /// в количестве засечек.
time_t modification_time;
Yandex::DayNum_t left_month;
Yandex::DayNum_t right_month;
DayNum_t left_month;
DayNum_t right_month;
/// Смотреть и изменять это поле следует под залоченным data_parts_mutex.
bool currently_merging;
@ -300,7 +300,7 @@ private:
const String & sign_column_ = "",
const StorageMergeTreeSettings & settings_ = StorageMergeTreeSettings());
static String getPartName(Yandex::DayNum_t left_date, Yandex::DayNum_t right_date, UInt64 left_id, UInt64 right_id, UInt64 level);
static String getPartName(DayNum_t left_date, DayNum_t right_date, UInt64 left_id, UInt64 right_id, UInt64 level);
BlockInputStreams spreadMarkRangesAmongThreads(RangesInDataParts parts, size_t threads, const Names & column_names, size_t max_block_size);
BlockInputStreams spreadMarkRangesAmongThreadsFinal(RangesInDataParts parts, size_t threads, const Names & column_names, size_t max_block_size);

View File

@ -300,7 +300,7 @@ private:
Poco::File(history_file).createFile();
/// Инициализируем DateLUT, чтобы потраченное время не отображалось, как время, потраченное на запрос.
Yandex::DateLUTSingleton::instance();
DateLUTSingleton::instance();
loop();

View File

@ -30,7 +30,7 @@ int main(int argc, char ** argv)
Logger::root().setLevel("trace");
/// Заранее инициализируем DateLUT, чтобы первая инициализация потом не влияла на измеряемую скорость выполнения.
Yandex::DateLUTSingleton::instance();
DateLUTSingleton::instance();
DB::Context context;

View File

@ -316,18 +316,18 @@ std::string QueryConverter::convertDateRange(time_t date_first, time_t date_last
{
WriteBufferFromString first_buf(first_str);
WriteBufferFromString last_buf(last_str);
writeDateText(Yandex::DateLUTSingleton::instance().toDayNum(date_first), first_buf);
writeDateText(Yandex::DateLUTSingleton::instance().toDayNum(date_last), last_buf);
writeDateText(DateLUTSingleton::instance().toDayNum(date_first), first_buf);
writeDateText(DateLUTSingleton::instance().toDayNum(date_last), last_buf);
}
return "StartDate >= toDate('" + first_str + "') AND StartDate <= toDate('" + last_str + "')";
}
std::string QueryConverter::convertCounterID(Yandex::CounterID_t CounterID)
std::string QueryConverter::convertCounterID(CounterID_t CounterID)
{
return "CounterID == " + toString(CounterID);
}
std::string QueryConverter::getTableName(Yandex::CounterID_t CounterID)
std::string QueryConverter::getTableName(CounterID_t CounterID)
{
if (CounterID == 0)
return table_for_all_counters;

View File

@ -36,9 +36,9 @@ private:
/// <dates> => SELECT ... WHERE x
std::string convertDateRange(time_t date_first, time_t date_last);
/// <counter_id> => SELECT ... WHERE x
std::string convertCounterID(Yandex::CounterID_t CounterID);
std::string convertCounterID(CounterID_t CounterID);
std::string getTableName(Yandex::CounterID_t CounterID);
std::string getTableName(CounterID_t CounterID);
std::string getHavingSection();
void fillFormattedAttributeMap();

View File

@ -85,15 +85,15 @@ QueryParseResult QueryParser::parse(std::istream & s)
result.CounterID = DB::parse<unsigned>(getValueOfOneTextElement(result.query, CounterID_element_name));
int time_zone_diff = 0;
result.date_first = Yandex::Time2Date(Poco::DateTimeParser::parse(
result.date_first = Time2Date(Poco::DateTimeParser::parse(
getValueOfOneTextElement(result.query, date_first_element_name), time_zone_diff).timestamp().epochTime());
result.date_last = Yandex::Time2Date(Poco::DateTimeParser::parse(
result.date_last = Time2Date(Poco::DateTimeParser::parse(
getValueOfOneTextElement(result.query, date_last_element_name), time_zone_diff).timestamp().epochTime());
if (result.date_first > result.date_last)
throw Exception("First date is bigger than last date.", ErrorCodes::FIRST_DATE_IS_BIGGER_THAN_LAST_DATE);
Yandex::DateLUTSingleton & date_lut = Yandex::DateLUTSingleton::instance();
DateLUTSingleton & date_lut = DateLUTSingleton::instance();
result.days = 1 + date_lut.toDayNum(result.date_last) - date_lut.toDayNum(result.date_first);
result.cut_date_last = false;
@ -176,7 +176,7 @@ QueryParseResult QueryParser::parse(std::istream & s)
result.limit = DB::parse<unsigned>(limit_nodes->item(0)->innerText());
LOG_DEBUG(log, "CounterID: " << result.CounterID
<< ", dates: " << Yandex::Date2Str(result.date_first) << " - " << Yandex::Date2Str(result.date_last));
<< ", dates: " << Date2Str(result.date_first) << " - " << Date2Str(result.date_last));
/// получаем список имён атрибутов
Poco::AutoPtr<Poco::XML::NodeList> attributes = result.query->getElementsByTagName("attribute");

View File

@ -84,7 +84,7 @@ struct QueryParseResult
Poco::AutoPtr<Poco::XML::Document> query;
Yandex::CounterID_t CounterID;
CounterID_t CounterID;
time_t date_first;
time_t date_last;
unsigned days;

View File

@ -86,7 +86,7 @@ int Server::main(const std::vector<std::string> & args)
/// Заранее инициализируем DateLUT, чтобы первая инициализация потом не влияла на измеряемую скорость выполнения.
LOG_DEBUG(log, "Initializing DateLUT.");
Yandex::DateLUTSingleton::instance();
DateLUTSingleton::instance();
LOG_TRACE(log, "Initialized DateLUT.");
/** Контекст содержит всё, что влияет на обработку запроса:

View File

@ -457,15 +457,15 @@ void StorageMergeTree::createPositiveSignCondition(ExpressionActionsPtr & out_ex
}
String StorageMergeTree::getPartName(Yandex::DayNum_t left_date, Yandex::DayNum_t right_date, UInt64 left_id, UInt64 right_id, UInt64 level)
String StorageMergeTree::getPartName(DayNum_t left_date, DayNum_t right_date, UInt64 left_id, UInt64 right_id, UInt64 level)
{
Yandex::DateLUTSingleton & date_lut = Yandex::DateLUTSingleton::instance();
DateLUTSingleton & date_lut = DateLUTSingleton::instance();
/// Имя директории для куска иммет вид: YYYYMMDD_YYYYMMDD_N_N_L.
String res;
{
unsigned left_date_id = Yandex::Date2OrderedIdentifier(date_lut.fromDayNum(left_date));
unsigned right_date_id = Yandex::Date2OrderedIdentifier(date_lut.fromDayNum(right_date));
unsigned left_date_id = Date2OrderedIdentifier(date_lut.fromDayNum(left_date));
unsigned right_date_id = Date2OrderedIdentifier(date_lut.fromDayNum(right_date));
WriteBufferFromString wb(res);
@ -491,7 +491,7 @@ void StorageMergeTree::loadDataParts()
Poco::ScopedLock<Poco::FastMutex> lock(data_parts_mutex);
Poco::ScopedLock<Poco::FastMutex> lock_all(all_data_parts_mutex);
Yandex::DateLUTSingleton & date_lut = Yandex::DateLUTSingleton::instance();
DateLUTSingleton & date_lut = DateLUTSingleton::instance();
data_parts.clear();
Poco::DirectoryIterator end;
@ -504,8 +504,8 @@ void StorageMergeTree::loadDataParts()
continue;
DataPartPtr part = new DataPart(*this);
part->left_date = date_lut.toDayNum(Yandex::OrderedIdentifier2Date(file_name.substr(matches[1].offset, matches[1].length)));
part->right_date = date_lut.toDayNum(Yandex::OrderedIdentifier2Date(file_name.substr(matches[2].offset, matches[2].length)));
part->left_date = date_lut.toDayNum(OrderedIdentifier2Date(file_name.substr(matches[1].offset, matches[1].length)));
part->right_date = date_lut.toDayNum(OrderedIdentifier2Date(file_name.substr(matches[2].offset, matches[2].length)));
part->left = parse<UInt64>(file_name.substr(matches[3].offset, matches[3].length));
part->right = parse<UInt64>(file_name.substr(matches[4].offset, matches[4].length));
part->level = parse<UInt32>(file_name.substr(matches[5].offset, matches[5].length));
@ -679,7 +679,7 @@ bool StorageMergeTree::selectPartsToMerge(std::vector<DataPartPtr> & parts, bool
Poco::ScopedLock<Poco::FastMutex> lock(data_parts_mutex);
Yandex::DateLUTSingleton & date_lut = Yandex::DateLUTSingleton::instance();
DateLUTSingleton & date_lut = DateLUTSingleton::instance();
size_t min_max = -1U;
size_t min_min = -1U;
@ -687,8 +687,8 @@ bool StorageMergeTree::selectPartsToMerge(std::vector<DataPartPtr> & parts, bool
DataParts::iterator best_begin;
bool found = false;
Yandex::DayNum_t now_day = date_lut.toDayNum(time(0));
Yandex::DayNum_t now_month = date_lut.toFirstDayNumOfMonth(now_day);
DayNum_t now_day = date_lut.toDayNum(time(0));
DayNum_t now_month = date_lut.toFirstDayNumOfMonth(now_day);
/// Сколько кусков, начиная с текущего, можно включить в валидный отрезок, начинающийся левее текущего куска.
/// Нужно для определения максимальности по включению.
@ -724,7 +724,7 @@ bool StorageMergeTree::selectPartsToMerge(std::vector<DataPartPtr> & parts, bool
size_t cur_sum = first_part->size;
int cur_len = 1;
Yandex::DayNum_t month = first_part->left_month;
DayNum_t month = first_part->left_month;
UInt64 cur_id = first_part->right;
/// Этот месяц кончился хотя бы день назад.
@ -817,7 +817,7 @@ void StorageMergeTree::mergeParts(std::vector<DataPartPtr> parts)
for (NamesAndTypesList::const_iterator it = columns->begin(); it != columns->end(); ++it)
all_column_names.push_back(it->first);
Yandex::DateLUTSingleton & date_lut = Yandex::DateLUTSingleton::instance();
DateLUTSingleton & date_lut = DateLUTSingleton::instance();
StorageMergeTree::DataPartPtr new_data_part = new DataPart(*this);
new_data_part->left_date = std::numeric_limits<UInt16>::max();

View File

@ -2,45 +2,42 @@
#include <Yandex/DateLUT.h>
namespace Yandex
DateLUT::DateLUT()
{
DateLUT::DateLUT()
/** Дополнительный вызов Time2Date для случая, когда в 1981-1984 году в России,
* 1 апреля начиналось в час ночи, не в полночь.
*/
for (time_t t = Time2Date(DATE_LUT_MIN);
t <= DATE_LUT_MAX;
t = Time2Date(TimeDayShift(t)))
{
/** Дополнительный вызов Time2Date для случая, когда в 1981-1984 году в России,
* 1 апреля начиналось в час ночи, не в полночь.
*/
for (time_t t = Yandex::Time2Date(DATE_LUT_MIN);
t <= DATE_LUT_MAX;
t = Yandex::Time2Date(Yandex::TimeDayShift(t)))
{
Values values;
Values values;
struct tm tm;
localtime_r(&t, &tm);
struct tm tm;
localtime_r(&t, &tm);
values.year = tm.tm_year + 1900;
values.month = tm.tm_mon + 1;
values.day_of_week = tm.tm_wday == 0 ? 7 : tm.tm_wday;
values.day_of_month = tm.tm_mday;
values.year = tm.tm_year + 1900;
values.month = tm.tm_mon + 1;
values.day_of_week = tm.tm_wday == 0 ? 7 : tm.tm_wday;
values.day_of_month = tm.tm_mday;
tm.tm_hour = 0;
tm.tm_min = 0;
tm.tm_sec = 0;
tm.tm_isdst = -1;
values.date = mktime(&tm);
tm.tm_hour = 0;
tm.tm_min = 0;
tm.tm_sec = 0;
tm.tm_isdst = -1;
lut.push_back(values);
}
values.date = mktime(&tm);
/// Заполняем lookup таблицу для годов
memset(years_lut, 0, DATE_LUT_YEARS * sizeof(years_lut[0]));
for (size_t day = 0; day < lut.size() && lut[day].year <= DATE_LUT_MAX_YEAR; ++day)
{
if (lut[day].month == 1 && lut[day].day_of_month == 1)
years_lut[lut[day].year - DATE_LUT_MIN_YEAR] = day;
}
offset_at_start_of_epoch = 86400 - lut[findIndex(86400)].date;
lut.push_back(values);
}
/// Заполняем lookup таблицу для годов
memset(years_lut, 0, DATE_LUT_YEARS * sizeof(years_lut[0]));
for (size_t day = 0; day < lut.size() && lut[day].year <= DATE_LUT_MAX_YEAR; ++day)
{
if (lut[day].month == 1 && lut[day].day_of_month == 1)
years_lut[lut[day].year - DATE_LUT_MIN_YEAR] = day;
}
offset_at_start_of_epoch = 86400 - lut[findIndex(86400)].date;
}

View File

@ -6,11 +6,11 @@
void loop(time_t begin, time_t end, int step)
{
Yandex::DateLUTSingleton & date_lut = Yandex::DateLUTSingleton::instance();
DateLUTSingleton & date_lut = DateLUTSingleton::instance();
for (time_t t = begin; t < end; t += step)
std::cout << Yandex::Time2Sql(t)
<< ", " << Yandex::Time2Sql(date_lut.toTimeInaccurate(t))
std::cout << Time2Sql(t)
<< ", " << Time2Sql(date_lut.toTimeInaccurate(t))
<< ", " << date_lut.toHourInaccurate(t)
<< std::endl;
}
@ -18,8 +18,8 @@ void loop(time_t begin, time_t end, int step)
int main(int argc, char ** argv)
{
loop(Yandex::OrderedIdentifier2Date(20101031), Yandex::OrderedIdentifier2Date(20101101), 15 * 60);
loop(Yandex::OrderedIdentifier2Date(20100328), Yandex::OrderedIdentifier2Date(20100330), 15 * 60);
loop(OrderedIdentifier2Date(20101031), OrderedIdentifier2Date(20101101), 15 * 60);
loop(OrderedIdentifier2Date(20100328), OrderedIdentifier2Date(20100330), 15 * 60);
return 0;
}

View File

@ -8,15 +8,15 @@
void loop(time_t begin, time_t end, int step)
{
Yandex::DateLUTSingleton & date_lut = Yandex::DateLUTSingleton::instance();
DateLUTSingleton & date_lut = DateLUTSingleton::instance();
for (time_t t = begin; t < end; t += step)
{
time_t t2 = date_lut.makeDateTime(date_lut.toYear(t), date_lut.toMonth(t), date_lut.toDayOfMonth(t),
date_lut.toHourInaccurate(t), date_lut.toMinute(t), date_lut.toSecond(t));
std::string s1 = Yandex::Time2Sql(t);
std::string s2 = Yandex::Time2Sql(t2);
std::string s1 = Time2Sql(t);
std::string s2 = Time2Sql(t2);
std::cerr << s1 << ", " << s2 << std::endl;
@ -28,8 +28,8 @@ void loop(time_t begin, time_t end, int step)
int main(int argc, char ** argv)
{
loop(Yandex::OrderedIdentifier2Date(20101031), Yandex::OrderedIdentifier2Date(20101101), 15 * 60);
loop(Yandex::OrderedIdentifier2Date(20100328), Yandex::OrderedIdentifier2Date(20100330), 15 * 60);
loop(OrderedIdentifier2Date(20101031), OrderedIdentifier2Date(20101101), 15 * 60);
loop(OrderedIdentifier2Date(20100328), OrderedIdentifier2Date(20100330), 15 * 60);
return 0;
}

View File

@ -6,7 +6,7 @@
typedef std::string T;
typedef Yandex::MultiVersion<T> MV;
typedef MultiVersion<T> MV;
typedef std::vector<T> Results;

View File

@ -30,8 +30,8 @@ private:
void init(time_t time)
{
Yandex::DateLUTSingleton & date_lut = Yandex::DateLUTSingleton::instance();
const Yandex::DateLUT::Values & values = date_lut.getValues(time);
DateLUTSingleton & date_lut = DateLUTSingleton::instance();
const DateLUT::Values & values = date_lut.getValues(time);
m_year = values.year;
m_month = values.month;
@ -65,9 +65,9 @@ public:
init(time);
}
Date(Yandex::DayNum_t day_num)
Date(DayNum_t day_num)
{
const Yandex::DateLUT::Values & values = Yandex::DateLUTSingleton::instance().getValues(day_num);
const DateLUT::Values & values = DateLUTSingleton::instance().getValues(day_num);
m_year = values.year;
m_month = values.month;
m_day = values.day_of_month;
@ -114,15 +114,15 @@ public:
operator time_t() const
{
return Yandex::DateLUTSingleton::instance().makeDate(m_year, m_month, m_day);
return DateLUTSingleton::instance().makeDate(m_year, m_month, m_day);
}
Yandex::DayNum_t getDayNum() const
DayNum_t getDayNum() const
{
return Yandex::DateLUTSingleton::instance().makeDayNum(m_year, m_month, m_day);
return DateLUTSingleton::instance().makeDayNum(m_year, m_month, m_day);
}
operator Yandex::DayNum_t() const
operator DayNum_t() const
{
return getDayNum();
}

View File

@ -43,8 +43,8 @@ private:
return;
}
Yandex::DateLUTSingleton & date_lut = Yandex::DateLUTSingleton::instance();
const Yandex::DateLUT::Values & values = date_lut.getValues(time);
DateLUTSingleton & date_lut = DateLUTSingleton::instance();
const DateLUT::Values & values = date_lut.getValues(time);
m_year = values.year;
m_month = values.month;
@ -124,7 +124,7 @@ public:
{
return m_year == 0
? 0
: Yandex::DateLUTSingleton::instance().makeDateTime(m_year, m_month, m_day, m_hour, m_minute, m_second);
: DateLUTSingleton::instance().makeDateTime(m_year, m_month, m_day, m_hour, m_minute, m_second);
}
unsigned short year() const { return m_year; }

View File

@ -145,7 +145,7 @@ private:
time_t getDateTimeImpl() const
{
Yandex::DateLUTSingleton & date_lut = Yandex::DateLUTSingleton::instance();
DateLUTSingleton & date_lut = DateLUTSingleton::instance();
if (m_length == 10)
{
@ -173,7 +173,7 @@ private:
time_t getDateImpl() const
{
Yandex::DateLUTSingleton & date_lut = Yandex::DateLUTSingleton::instance();
DateLUTSingleton & date_lut = DateLUTSingleton::instance();
if (m_length == 10 || m_length == 19)
{
@ -216,7 +216,7 @@ private:
return getDateImpl();
else
{
Yandex::DateLUTSingleton & date_lut = Yandex::DateLUTSingleton::instance();
DateLUTSingleton & date_lut = DateLUTSingleton::instance();
return date_lut.toDate(getIntImpl());
}
}
@ -413,7 +413,7 @@ template <> inline std::string String::get<std::string >() const { return ge
template <> inline Date String::get<Date >() const { return getDate(); }
template <> inline DateTime String::get<DateTime >() const { return getDateTime(); }
template <> inline Yandex::VisitID_t String::get<Yandex::VisitID_t >() const { return Yandex::VisitID_t(getUInt()); }
template <> inline VisitID_t String::get<VisitID_t >() const { return VisitID_t(getUInt()); }
template <typename T> inline T String::get() const { return T(*this); }

View File

@ -23,8 +23,8 @@ int main(int argc, char ** argv)
std::cerr << row[1] << ", " << row["d"]
<< ", " << row[1].getDate()
<< ", " << row[1].getDateTime()
<< ", " << Yandex::Date2Str(row[1].getDate())
<< ", " << Yandex::Time2Str(row[1].getDateTime())
<< ", " << Date2Str(row[1].getDate())
<< ", " << Time2Str(row[1].getDateTime())
<< std::endl
<< mysqlxx::escape << row[1].getDate() << ", " << mysqlxx::escape << row[1].getDateTime() << std::endl
<< mysqlxx::quote << row[1].getDate() << ", " << mysqlxx::quote << row[1].getDateTime() << std::endl