mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-19 06:01:57 +00:00
Merge
This commit is contained in:
commit
058bb9dff3
@ -119,38 +119,7 @@ public:
|
|||||||
getOffsets().push_back((getOffsets().size() == 0 ? 0 : getOffsets().back()) + elems);
|
getOffsets().push_back((getOffsets().size() == 0 ? 0 : getOffsets().back()) + elems);
|
||||||
}
|
}
|
||||||
|
|
||||||
ColumnPtr cut(size_t start, size_t length) const override
|
ColumnPtr cut(size_t start, size_t length) const override;
|
||||||
{
|
|
||||||
if (length == 0)
|
|
||||||
return new ColumnArray(data);
|
|
||||||
|
|
||||||
if (start + length > getOffsets().size())
|
|
||||||
throw Exception("Parameter out of bound in IColumnArray::cut() method.",
|
|
||||||
ErrorCodes::PARAMETER_OUT_OF_BOUND);
|
|
||||||
|
|
||||||
size_t nested_offset = offsetAt(start);
|
|
||||||
size_t nested_length = getOffsets()[start + length - 1] - nested_offset;
|
|
||||||
|
|
||||||
ColumnArray * res_ = new ColumnArray(data);
|
|
||||||
ColumnPtr res = res_;
|
|
||||||
|
|
||||||
res_->data = data->cut(nested_offset, nested_length);
|
|
||||||
Offsets_t & res_offsets = res_->getOffsets();
|
|
||||||
|
|
||||||
if (start == 0)
|
|
||||||
{
|
|
||||||
res_offsets.assign(getOffsets().begin(), getOffsets().begin() + length);
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
res_offsets.resize(length);
|
|
||||||
|
|
||||||
for (size_t i = 0; i < length; ++i)
|
|
||||||
res_offsets[i] = getOffsets()[start + i] - nested_offset;
|
|
||||||
}
|
|
||||||
|
|
||||||
return res;
|
|
||||||
}
|
|
||||||
|
|
||||||
void insert(const Field & x) override
|
void insert(const Field & x) override
|
||||||
{
|
{
|
||||||
@ -178,82 +147,9 @@ public:
|
|||||||
getOffsets().push_back(getOffsets().size() == 0 ? 0 : getOffsets().back());
|
getOffsets().push_back(getOffsets().size() == 0 ? 0 : getOffsets().back());
|
||||||
}
|
}
|
||||||
|
|
||||||
ColumnPtr filter(const Filter & filt) const override
|
ColumnPtr filter(const Filter & filt) const override;
|
||||||
{
|
|
||||||
size_t size = getOffsets().size();
|
|
||||||
if (size != filt.size())
|
|
||||||
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
|
||||||
|
|
||||||
if (size == 0)
|
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||||
return new ColumnArray(data);
|
|
||||||
|
|
||||||
/// Не слишком оптимально. Можно сделать специализацию для массивов известных типов.
|
|
||||||
Filter nested_filt(getOffsets().back());
|
|
||||||
for (size_t i = 0; i < size; ++i)
|
|
||||||
{
|
|
||||||
if (filt[i])
|
|
||||||
memset(&nested_filt[offsetAt(i)], 1, sizeAt(i));
|
|
||||||
else
|
|
||||||
memset(&nested_filt[offsetAt(i)], 0, sizeAt(i));
|
|
||||||
}
|
|
||||||
|
|
||||||
ColumnArray * res_ = new ColumnArray(data);
|
|
||||||
ColumnPtr res = res_;
|
|
||||||
res_->data = data->filter(nested_filt);
|
|
||||||
|
|
||||||
Offsets_t & res_offsets = res_->getOffsets();
|
|
||||||
res_offsets.reserve(size);
|
|
||||||
|
|
||||||
size_t current_offset = 0;
|
|
||||||
for (size_t i = 0; i < size; ++i)
|
|
||||||
{
|
|
||||||
if (filt[i])
|
|
||||||
{
|
|
||||||
current_offset += sizeAt(i);
|
|
||||||
res_offsets.push_back(current_offset);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return res;
|
|
||||||
}
|
|
||||||
|
|
||||||
ColumnPtr permute(const Permutation & perm, size_t limit) const override
|
|
||||||
{
|
|
||||||
size_t size = getOffsets().size();
|
|
||||||
|
|
||||||
if (limit == 0)
|
|
||||||
limit = size;
|
|
||||||
else
|
|
||||||
limit = std::min(size, limit);
|
|
||||||
|
|
||||||
if (perm.size() < limit)
|
|
||||||
throw Exception("Size of permutation is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
|
||||||
|
|
||||||
if (limit == 0)
|
|
||||||
return new ColumnArray(data);
|
|
||||||
|
|
||||||
Permutation nested_perm(getOffsets().back());
|
|
||||||
|
|
||||||
ColumnArray * res_ = new ColumnArray(data->cloneEmpty());
|
|
||||||
ColumnPtr res = res_;
|
|
||||||
|
|
||||||
Offsets_t & res_offsets = res_->getOffsets();
|
|
||||||
res_offsets.resize(limit);
|
|
||||||
size_t current_offset = 0;
|
|
||||||
|
|
||||||
for (size_t i = 0; i < limit; ++i)
|
|
||||||
{
|
|
||||||
for (size_t j = 0; j < sizeAt(perm[i]); ++j)
|
|
||||||
nested_perm[current_offset + j] = offsetAt(perm[i]) + j;
|
|
||||||
current_offset += sizeAt(perm[i]);
|
|
||||||
res_offsets[i] = current_offset;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (current_offset != 0)
|
|
||||||
res_->data = data->permute(nested_perm, current_offset);
|
|
||||||
|
|
||||||
return res;
|
|
||||||
}
|
|
||||||
|
|
||||||
int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override
|
int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override
|
||||||
{
|
{
|
||||||
@ -290,31 +186,7 @@ public:
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
void getPermutation(bool reverse, size_t limit, Permutation & res) const override
|
void getPermutation(bool reverse, size_t limit, Permutation & res) const override;
|
||||||
{
|
|
||||||
size_t s = size();
|
|
||||||
if (limit >= s)
|
|
||||||
limit = 0;
|
|
||||||
|
|
||||||
res.resize(s);
|
|
||||||
for (size_t i = 0; i < s; ++i)
|
|
||||||
res[i] = i;
|
|
||||||
|
|
||||||
if (limit)
|
|
||||||
{
|
|
||||||
if (reverse)
|
|
||||||
std::partial_sort(res.begin(), res.begin() + limit, res.end(), less<false>(*this));
|
|
||||||
else
|
|
||||||
std::partial_sort(res.begin(), res.begin() + limit, res.end(), less<true>(*this));
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
if (reverse)
|
|
||||||
std::sort(res.begin(), res.end(), less<false>(*this));
|
|
||||||
else
|
|
||||||
std::sort(res.begin(), res.end(), less<true>(*this));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
void reserve(size_t n) override
|
void reserve(size_t n) override
|
||||||
{
|
{
|
||||||
@ -364,24 +236,7 @@ public:
|
|||||||
const ColumnPtr & getOffsetsColumn() const { return offsets; }
|
const ColumnPtr & getOffsetsColumn() const { return offsets; }
|
||||||
|
|
||||||
|
|
||||||
ColumnPtr replicate(const Offsets_t & replicate_offsets) const override
|
ColumnPtr replicate(const Offsets_t & replicate_offsets) const override;
|
||||||
{
|
|
||||||
/// Не получается реализовать в общем случае.
|
|
||||||
|
|
||||||
if (typeid_cast<const ColumnUInt8 *>(&*data)) return replicate<UInt8>(replicate_offsets);
|
|
||||||
if (typeid_cast<const ColumnUInt16 *>(&*data)) return replicate<UInt16>(replicate_offsets);
|
|
||||||
if (typeid_cast<const ColumnUInt32 *>(&*data)) return replicate<UInt32>(replicate_offsets);
|
|
||||||
if (typeid_cast<const ColumnUInt64 *>(&*data)) return replicate<UInt64>(replicate_offsets);
|
|
||||||
if (typeid_cast<const ColumnInt8 *>(&*data)) return replicate<Int8>(replicate_offsets);
|
|
||||||
if (typeid_cast<const ColumnInt16 *>(&*data)) return replicate<Int16>(replicate_offsets);
|
|
||||||
if (typeid_cast<const ColumnInt32 *>(&*data)) return replicate<Int32>(replicate_offsets);
|
|
||||||
if (typeid_cast<const ColumnInt64 *>(&*data)) return replicate<Int64>(replicate_offsets);
|
|
||||||
if (typeid_cast<const ColumnFloat32 *>(&*data)) return replicate<Float32>(replicate_offsets);
|
|
||||||
if (typeid_cast<const ColumnFloat64 *>(&*data)) return replicate<Float64>(replicate_offsets);
|
|
||||||
if (typeid_cast<const ColumnString *>(&*data)) return replicateString(replicate_offsets);
|
|
||||||
|
|
||||||
throw Exception("Replication of column " + getName() + " is not implemented.", ErrorCodes::NOT_IMPLEMENTED);
|
|
||||||
}
|
|
||||||
|
|
||||||
private:
|
private:
|
||||||
ColumnPtr data;
|
ColumnPtr data;
|
||||||
@ -393,129 +248,17 @@ private:
|
|||||||
|
|
||||||
/// Размножить значения, если вложенный столбец - ColumnArray<T>.
|
/// Размножить значения, если вложенный столбец - ColumnArray<T>.
|
||||||
template <typename T>
|
template <typename T>
|
||||||
ColumnPtr replicate(const Offsets_t & replicate_offsets) const
|
ColumnPtr replicate(const Offsets_t & replicate_offsets) const;
|
||||||
{
|
|
||||||
size_t col_size = size();
|
|
||||||
if (col_size != replicate_offsets.size())
|
|
||||||
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
|
||||||
|
|
||||||
ColumnPtr res = cloneEmpty();
|
|
||||||
|
|
||||||
if (0 == col_size)
|
|
||||||
return res;
|
|
||||||
|
|
||||||
ColumnArray & res_ = typeid_cast<ColumnArray &>(*res);
|
|
||||||
|
|
||||||
const typename ColumnVector<T>::Container_t & cur_data = typeid_cast<const ColumnVector<T> &>(*data).getData();
|
|
||||||
const Offsets_t & cur_offsets = getOffsets();
|
|
||||||
|
|
||||||
typename ColumnVector<T>::Container_t & res_data = typeid_cast<ColumnVector<T> &>(res_.getData()).getData();
|
|
||||||
Offsets_t & res_offsets = res_.getOffsets();
|
|
||||||
|
|
||||||
res_data.reserve(data->size() / col_size * replicate_offsets.back());
|
|
||||||
res_offsets.reserve(replicate_offsets.back());
|
|
||||||
|
|
||||||
Offset_t prev_replicate_offset = 0;
|
|
||||||
Offset_t prev_data_offset = 0;
|
|
||||||
Offset_t current_new_offset = 0;
|
|
||||||
|
|
||||||
for (size_t i = 0; i < col_size; ++i)
|
|
||||||
{
|
|
||||||
size_t size_to_replicate = replicate_offsets[i] - prev_replicate_offset;
|
|
||||||
size_t value_size = cur_offsets[i] - prev_data_offset;
|
|
||||||
|
|
||||||
for (size_t j = 0; j < size_to_replicate; ++j)
|
|
||||||
{
|
|
||||||
current_new_offset += value_size;
|
|
||||||
res_offsets.push_back(current_new_offset);
|
|
||||||
|
|
||||||
res_data.resize(res_data.size() + value_size);
|
|
||||||
memcpy(&res_data[res_data.size() - value_size], &cur_data[prev_data_offset], value_size * sizeof(T));
|
|
||||||
}
|
|
||||||
|
|
||||||
prev_replicate_offset = replicate_offsets[i];
|
|
||||||
prev_data_offset = cur_offsets[i];
|
|
||||||
}
|
|
||||||
|
|
||||||
return res;
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Размножить значения, если вложенный столбец - ColumnString. Код слишком сложный.
|
/// Размножить значения, если вложенный столбец - ColumnString. Код слишком сложный.
|
||||||
ColumnPtr replicateString(const Offsets_t & replicate_offsets) const
|
ColumnPtr replicateString(const Offsets_t & replicate_offsets) const;
|
||||||
{
|
|
||||||
size_t col_size = size();
|
|
||||||
if (col_size != replicate_offsets.size())
|
|
||||||
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
|
||||||
|
|
||||||
ColumnPtr res = cloneEmpty();
|
/** Неконстантные массивы константных значений - довольно редкое явление.
|
||||||
|
* Большинство функций не умеет с ними работать, и не создаёт такие столбцы в качестве результата.
|
||||||
if (0 == col_size)
|
* Исключение - функция replicate (см. FunctionsMiscellaneous.h), которая имеет служебное значение для реализации лямбда-функций.
|
||||||
return res;
|
* Только ради неё сделана реализация метода replicate для ColumnArray(ColumnConst).
|
||||||
|
*/
|
||||||
ColumnArray & res_ = typeid_cast<ColumnArray &>(*res);
|
ColumnPtr replicateConst(const Offsets_t & replicate_offsets) const;
|
||||||
|
|
||||||
const ColumnString & cur_string = typeid_cast<const ColumnString &>(*data);
|
|
||||||
const ColumnString::Chars_t & cur_chars = cur_string.getChars();
|
|
||||||
const Offsets_t & cur_string_offsets = cur_string.getOffsets();
|
|
||||||
const Offsets_t & cur_offsets = getOffsets();
|
|
||||||
|
|
||||||
ColumnString::Chars_t & res_chars = typeid_cast<ColumnString &>(res_.getData()).getChars();
|
|
||||||
Offsets_t & res_string_offsets = typeid_cast<ColumnString &>(res_.getData()).getOffsets();
|
|
||||||
Offsets_t & res_offsets = res_.getOffsets();
|
|
||||||
|
|
||||||
res_chars.reserve(cur_chars.size() / col_size * replicate_offsets.back());
|
|
||||||
res_string_offsets.reserve(cur_string_offsets.size() / col_size * replicate_offsets.back());
|
|
||||||
res_offsets.reserve(replicate_offsets.back());
|
|
||||||
|
|
||||||
Offset_t prev_replicate_offset = 0;
|
|
||||||
|
|
||||||
Offset_t prev_cur_offset = 0;
|
|
||||||
Offset_t prev_cur_string_offset = 0;
|
|
||||||
|
|
||||||
Offset_t current_res_offset = 0;
|
|
||||||
Offset_t current_res_string_offset = 0;
|
|
||||||
|
|
||||||
for (size_t i = 0; i < col_size; ++i)
|
|
||||||
{
|
|
||||||
/// Насколько размножить массив.
|
|
||||||
size_t size_to_replicate = replicate_offsets[i] - prev_replicate_offset;
|
|
||||||
/// Количество строк в массиве.
|
|
||||||
size_t value_size = cur_offsets[i] - prev_cur_offset;
|
|
||||||
|
|
||||||
size_t sum_chars_size = 0;
|
|
||||||
|
|
||||||
for (size_t j = 0; j < size_to_replicate; ++j)
|
|
||||||
{
|
|
||||||
current_res_offset += value_size;
|
|
||||||
res_offsets.push_back(current_res_offset);
|
|
||||||
|
|
||||||
sum_chars_size = 0;
|
|
||||||
|
|
||||||
size_t prev_cur_string_offset_local = prev_cur_string_offset;
|
|
||||||
for (size_t k = 0; k < value_size; ++k)
|
|
||||||
{
|
|
||||||
/// Размер одной строки.
|
|
||||||
size_t chars_size = cur_string_offsets[k + prev_cur_offset] - prev_cur_string_offset_local;
|
|
||||||
|
|
||||||
current_res_string_offset += chars_size;
|
|
||||||
res_string_offsets.push_back(current_res_string_offset);
|
|
||||||
|
|
||||||
/// Копирование символов одной строки.
|
|
||||||
res_chars.resize(res_chars.size() + chars_size);
|
|
||||||
memcpy(&res_chars[res_chars.size() - chars_size], &cur_chars[prev_cur_string_offset_local], chars_size);
|
|
||||||
|
|
||||||
sum_chars_size += chars_size;
|
|
||||||
prev_cur_string_offset_local += chars_size;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
prev_replicate_offset = replicate_offsets[i];
|
|
||||||
prev_cur_offset = cur_offsets[i];
|
|
||||||
prev_cur_string_offset += sum_chars_size;
|
|
||||||
}
|
|
||||||
|
|
||||||
return res;
|
|
||||||
}
|
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
@ -262,6 +262,32 @@ struct BitShiftRightImpl
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
template<typename A, typename B>
|
||||||
|
struct LeastImpl
|
||||||
|
{
|
||||||
|
typedef typename NumberTraits::ResultOfIf<A, B>::Type ResultType;
|
||||||
|
|
||||||
|
template <typename Result = ResultType>
|
||||||
|
static inline Result apply(A a, B b)
|
||||||
|
{
|
||||||
|
/** gcc 4.9.2 успешно векторизует цикл из этой функции. */
|
||||||
|
return static_cast<Result>(a) < static_cast<Result>(b) ? static_cast<Result>(a) : static_cast<Result>(b);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
template<typename A, typename B>
|
||||||
|
struct GreatestImpl
|
||||||
|
{
|
||||||
|
typedef typename NumberTraits::ResultOfIf<A, B>::Type ResultType;
|
||||||
|
|
||||||
|
template <typename Result = ResultType>
|
||||||
|
static inline Result apply(A a, B b)
|
||||||
|
{
|
||||||
|
return static_cast<Result>(a) > static_cast<Result>(b) ? static_cast<Result>(a) : static_cast<Result>(b);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
template<typename A>
|
template<typename A>
|
||||||
struct NegateImpl
|
struct NegateImpl
|
||||||
{
|
{
|
||||||
@ -319,6 +345,12 @@ template <typename T> using Else = T;
|
|||||||
/// Used to indicate undefined operation
|
/// Used to indicate undefined operation
|
||||||
struct InvalidType;
|
struct InvalidType;
|
||||||
|
|
||||||
|
template <>
|
||||||
|
struct DataTypeFromFieldType<NumberTraits::Error>
|
||||||
|
{
|
||||||
|
using Type = InvalidType;
|
||||||
|
};
|
||||||
|
|
||||||
template <typename DataType> struct IsIntegral { static constexpr auto value = false; };
|
template <typename DataType> struct IsIntegral { static constexpr auto value = false; };
|
||||||
template <> struct IsIntegral<DataTypeUInt8> { static constexpr auto value = true; };
|
template <> struct IsIntegral<DataTypeUInt8> { static constexpr auto value = true; };
|
||||||
template <> struct IsIntegral<DataTypeUInt16> { static constexpr auto value = true; };
|
template <> struct IsIntegral<DataTypeUInt16> { static constexpr auto value = true; };
|
||||||
@ -342,11 +374,13 @@ template <typename DataType> struct IsDateOrDateTime { static constexpr auto val
|
|||||||
template <> struct IsDateOrDateTime<DataTypeDate> { static constexpr auto value = true; };
|
template <> struct IsDateOrDateTime<DataTypeDate> { static constexpr auto value = true; };
|
||||||
template <> struct IsDateOrDateTime<DataTypeDateTime> { static constexpr auto value = true; };
|
template <> struct IsDateOrDateTime<DataTypeDateTime> { static constexpr auto value = true; };
|
||||||
|
|
||||||
/** Returns appropriate result type for binary operator on dates:
|
/** Returns appropriate result type for binary operator on dates (or datetimes):
|
||||||
* Date + Integral -> Date
|
* Date + Integral -> Date
|
||||||
* Integral + Date -> Date
|
* Integral + Date -> Date
|
||||||
* Date - Date -> Int32
|
* Date - Date -> Int32
|
||||||
* Date - Integral -> Date
|
* Date - Integral -> Date
|
||||||
|
* least(Date, Date) -> Date
|
||||||
|
* greatest(Date, Date) -> Date
|
||||||
* All other operations are not defined and return InvalidType, operations on
|
* All other operations are not defined and return InvalidType, operations on
|
||||||
* distinct date types are also undefined (e.g. DataTypeDate - DataTypeDateTime) */
|
* distinct date types are also undefined (e.g. DataTypeDate - DataTypeDateTime) */
|
||||||
template <template <typename, typename> class Operation, typename LeftDataType, typename RightDataType>
|
template <template <typename, typename> class Operation, typename LeftDataType, typename RightDataType>
|
||||||
@ -387,7 +421,13 @@ struct DateBinaryOperationTraits
|
|||||||
Else<InvalidType>
|
Else<InvalidType>
|
||||||
>
|
>
|
||||||
>,
|
>,
|
||||||
Else<InvalidType>
|
Else<
|
||||||
|
If<std::is_same<T0, T1>::value
|
||||||
|
&& (std::is_same<Op, LeastImpl<T0, T1>>::value || std::is_same<Op, GreatestImpl<T0, T1>>::value),
|
||||||
|
Then<LeftDataType>,
|
||||||
|
Else<InvalidType>
|
||||||
|
>
|
||||||
|
>
|
||||||
>
|
>
|
||||||
>
|
>
|
||||||
>;
|
>;
|
||||||
@ -479,35 +519,27 @@ private:
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// Overload for date operations
|
/// Overload for date operations
|
||||||
template <typename LeftDataType, typename RightDataType, typename ColumnType,
|
template <typename LeftDataType, typename RightDataType, typename ColumnType>
|
||||||
typename std::enable_if<IsDateOrDateTime<LeftDataType>::value || IsDateOrDateTime<RightDataType>::value>::type * = nullptr>
|
|
||||||
bool executeRightType(Block & block, const ColumnNumbers & arguments, const size_t result, const ColumnType * col_left)
|
bool executeRightType(Block & block, const ColumnNumbers & arguments, const size_t result, const ColumnType * col_left)
|
||||||
{
|
{
|
||||||
if (!typeid_cast<const RightDataType *>(block.getByPosition(arguments[1]).type.get()))
|
if (!typeid_cast<const RightDataType *>(block.getByPosition(arguments[1]).type.get()))
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
using ResultDataType = typename DateBinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
|
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
|
||||||
|
|
||||||
return executeRightTypeDispatch<LeftDataType, RightDataType, ResultDataType>(
|
return executeRightTypeDispatch<LeftDataType, RightDataType, ResultDataType>(
|
||||||
block, arguments, result, col_left);
|
block, arguments, result, col_left);
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Overload for numeric operations
|
|
||||||
template <typename LeftDataType, typename RightDataType, typename ColumnType,
|
|
||||||
typename T0 = typename LeftDataType::FieldType, typename T1 = typename RightDataType::FieldType,
|
|
||||||
typename std::enable_if<IsNumeric<LeftDataType>::value && IsNumeric<RightDataType>::value>::type * = nullptr>
|
|
||||||
bool executeRightType(Block & block, const ColumnNumbers & arguments, const size_t result, const ColumnType * col_left)
|
|
||||||
{
|
|
||||||
return executeRightTypeImpl<T0, T1>(block, arguments, result, col_left);
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Overload for InvalidType
|
/// Overload for InvalidType
|
||||||
template <typename LeftDataType, typename RightDataType, typename ResultDataType, typename ColumnType,
|
template <typename LeftDataType, typename RightDataType, typename ResultDataType, typename ColumnType,
|
||||||
typename std::enable_if<std::is_same<ResultDataType, InvalidType>::value>::type * = nullptr>
|
typename std::enable_if<std::is_same<ResultDataType, InvalidType>::value>::type * = nullptr>
|
||||||
bool executeRightTypeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result,
|
bool executeRightTypeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result,
|
||||||
const ColumnType * col_left)
|
const ColumnType * col_left)
|
||||||
{
|
{
|
||||||
return false;
|
throw Exception("Types " + TypeName<typename LeftDataType::FieldType>::get()
|
||||||
|
+ " and " + TypeName<typename LeftDataType::FieldType>::get()
|
||||||
|
+ " are incompatible for function " + getName() + " or not upscaleable to common type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Overload for well-defined operations
|
/// Overload for well-defined operations
|
||||||
@ -527,7 +559,7 @@ private:
|
|||||||
template <typename T0, typename T1, typename ResultType = typename Op<T0, T1>::ResultType>
|
template <typename T0, typename T1, typename ResultType = typename Op<T0, T1>::ResultType>
|
||||||
bool executeRightTypeImpl(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnVector<T0> * col_left)
|
bool executeRightTypeImpl(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnVector<T0> * col_left)
|
||||||
{
|
{
|
||||||
if (auto col_right = typeid_cast<ColumnVector<T1> *>(block.getByPosition(arguments[1]).column.get()))
|
if (auto col_right = typeid_cast<const ColumnVector<T1> *>(block.getByPosition(arguments[1]).column.get()))
|
||||||
{
|
{
|
||||||
auto col_res = new ColumnVector<ResultType>;
|
auto col_res = new ColumnVector<ResultType>;
|
||||||
block.getByPosition(result).column = col_res;
|
block.getByPosition(result).column = col_res;
|
||||||
@ -538,7 +570,7 @@ private:
|
|||||||
|
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
else if (auto col_right = typeid_cast<ColumnConst<T1> *>(block.getByPosition(arguments[1]).column.get()))
|
else if (auto col_right = typeid_cast<const ColumnConst<T1> *>(block.getByPosition(arguments[1]).column.get()))
|
||||||
{
|
{
|
||||||
auto col_res = new ColumnVector<ResultType>;
|
auto col_res = new ColumnVector<ResultType>;
|
||||||
block.getByPosition(result).column = col_res;
|
block.getByPosition(result).column = col_res;
|
||||||
@ -550,14 +582,14 @@ private:
|
|||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
return false;
|
throw Exception("Logical error: unexpected type of column", ErrorCodes::LOGICAL_ERROR);
|
||||||
}
|
}
|
||||||
|
|
||||||
/// ColumnConst overload
|
/// ColumnConst overload
|
||||||
template <typename T0, typename T1, typename ResultType = typename Op<T0, T1>::ResultType>
|
template <typename T0, typename T1, typename ResultType = typename Op<T0, T1>::ResultType>
|
||||||
bool executeRightTypeImpl(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnConst<T0> * col_left)
|
bool executeRightTypeImpl(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnConst<T0> * col_left)
|
||||||
{
|
{
|
||||||
if (auto col_right = typeid_cast<ColumnVector<T1> *>(block.getByPosition(arguments[1]).column.get()))
|
if (auto col_right = typeid_cast<const ColumnVector<T1> *>(block.getByPosition(arguments[1]).column.get()))
|
||||||
{
|
{
|
||||||
auto col_res = new ColumnVector<ResultType>;
|
auto col_res = new ColumnVector<ResultType>;
|
||||||
block.getByPosition(result).column = col_res;
|
block.getByPosition(result).column = col_res;
|
||||||
@ -568,7 +600,7 @@ private:
|
|||||||
|
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
else if (auto col_right = typeid_cast<ColumnConst<T1> *>(block.getByPosition(arguments[1]).column.get()))
|
else if (auto col_right = typeid_cast<const ColumnConst<T1> *>(block.getByPosition(arguments[1]).column.get()))
|
||||||
{
|
{
|
||||||
ResultType res = 0;
|
ResultType res = 0;
|
||||||
BinaryOperationImpl<T0, T1, Op<T0, T1>, ResultType>::constant_constant(col_left->getData(), col_right->getData(), res);
|
BinaryOperationImpl<T0, T1, Op<T0, T1>, ResultType>::constant_constant(col_left->getData(), col_right->getData(), res);
|
||||||
@ -582,26 +614,12 @@ private:
|
|||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename LeftDataType,
|
template <typename LeftDataType>
|
||||||
typename std::enable_if<IsDateOrDateTime<LeftDataType>::value>::type * = nullptr>
|
|
||||||
bool executeLeftType(Block & block, const ColumnNumbers & arguments, const size_t result)
|
bool executeLeftType(Block & block, const ColumnNumbers & arguments, const size_t result)
|
||||||
{
|
{
|
||||||
if (!typeid_cast<const LeftDataType *>(block.getByPosition(arguments[0]).type.get()))
|
if (!typeid_cast<const LeftDataType *>(block.getByPosition(arguments[0]).type.get()))
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
return executeLeftTypeDispatch<LeftDataType>(block, arguments, result);
|
|
||||||
}
|
|
||||||
|
|
||||||
template <typename LeftDataType,
|
|
||||||
typename std::enable_if<IsNumeric<LeftDataType>::value>::type * = nullptr>
|
|
||||||
bool executeLeftType(Block & block, const ColumnNumbers & arguments, const size_t result)
|
|
||||||
{
|
|
||||||
return executeLeftTypeDispatch<LeftDataType>(block, arguments, result);
|
|
||||||
}
|
|
||||||
|
|
||||||
template <typename LeftDataType>
|
|
||||||
bool executeLeftTypeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result)
|
|
||||||
{
|
|
||||||
using T0 = typename LeftDataType::FieldType;
|
using T0 = typename LeftDataType::FieldType;
|
||||||
|
|
||||||
if ( executeLeftTypeImpl<LeftDataType, ColumnVector<T0>>(block, arguments, result)
|
if ( executeLeftTypeImpl<LeftDataType, ColumnVector<T0>>(block, arguments, result)
|
||||||
@ -614,7 +632,7 @@ private:
|
|||||||
template <typename LeftDataType, typename ColumnType>
|
template <typename LeftDataType, typename ColumnType>
|
||||||
bool executeLeftTypeImpl(Block & block, const ColumnNumbers & arguments, const size_t result)
|
bool executeLeftTypeImpl(Block & block, const ColumnNumbers & arguments, const size_t result)
|
||||||
{
|
{
|
||||||
if (auto col_left = typeid_cast<ColumnType *>(block.getByPosition(arguments[0]).column.get()))
|
if (auto col_left = typeid_cast<const ColumnType *>(block.getByPosition(arguments[0]).column.get()))
|
||||||
{
|
{
|
||||||
if ( executeRightType<LeftDataType, DataTypeDate>(block, arguments, result, col_left)
|
if ( executeRightType<LeftDataType, DataTypeDate>(block, arguments, result, col_left)
|
||||||
|| executeRightType<LeftDataType, DataTypeDateTime>(block, arguments, result, col_left)
|
|| executeRightType<LeftDataType, DataTypeDateTime>(block, arguments, result, col_left)
|
||||||
@ -718,7 +736,7 @@ private:
|
|||||||
template <typename T0>
|
template <typename T0>
|
||||||
bool executeType(Block & block, const ColumnNumbers & arguments, size_t result)
|
bool executeType(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||||
{
|
{
|
||||||
if (ColumnVector<T0> * col = typeid_cast<ColumnVector<T0> *>(&*block.getByPosition(arguments[0]).column))
|
if (const ColumnVector<T0> * col = typeid_cast<const ColumnVector<T0> *>(&*block.getByPosition(arguments[0]).column))
|
||||||
{
|
{
|
||||||
typedef typename Op<T0>::ResultType ResultType;
|
typedef typename Op<T0>::ResultType ResultType;
|
||||||
|
|
||||||
@ -731,7 +749,7 @@ private:
|
|||||||
|
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
else if (ColumnConst<T0> * col = typeid_cast<ColumnConst<T0> *>(&*block.getByPosition(arguments[0]).column))
|
else if (const ColumnConst<T0> * col = typeid_cast<const ColumnConst<T0> *>(&*block.getByPosition(arguments[0]).column))
|
||||||
{
|
{
|
||||||
typedef typename Op<T0>::ResultType ResultType;
|
typedef typename Op<T0>::ResultType ResultType;
|
||||||
|
|
||||||
@ -815,6 +833,8 @@ struct NameBitXor { static constexpr auto name = "bitXor"; };
|
|||||||
struct NameBitNot { static constexpr auto name = "bitNot"; };
|
struct NameBitNot { static constexpr auto name = "bitNot"; };
|
||||||
struct NameBitShiftLeft { static constexpr auto name = "bitShiftLeft"; };
|
struct NameBitShiftLeft { static constexpr auto name = "bitShiftLeft"; };
|
||||||
struct NameBitShiftRight { static constexpr auto name = "bitShiftRight"; };
|
struct NameBitShiftRight { static constexpr auto name = "bitShiftRight"; };
|
||||||
|
struct NameLeast { static constexpr auto name = "least"; };
|
||||||
|
struct NameGreatest { static constexpr auto name = "greatest"; };
|
||||||
|
|
||||||
typedef FunctionBinaryArithmetic<PlusImpl, NamePlus> FunctionPlus;
|
typedef FunctionBinaryArithmetic<PlusImpl, NamePlus> FunctionPlus;
|
||||||
typedef FunctionBinaryArithmetic<MinusImpl, NameMinus> FunctionMinus;
|
typedef FunctionBinaryArithmetic<MinusImpl, NameMinus> FunctionMinus;
|
||||||
@ -831,7 +851,8 @@ typedef FunctionBinaryArithmetic<BitXorImpl, NameBitXor> FunctionBitXor;
|
|||||||
typedef FunctionUnaryArithmetic<BitNotImpl, NameBitNot> FunctionBitNot;
|
typedef FunctionUnaryArithmetic<BitNotImpl, NameBitNot> FunctionBitNot;
|
||||||
typedef FunctionBinaryArithmetic<BitShiftLeftImpl, NameBitShiftLeft> FunctionBitShiftLeft;
|
typedef FunctionBinaryArithmetic<BitShiftLeftImpl, NameBitShiftLeft> FunctionBitShiftLeft;
|
||||||
typedef FunctionBinaryArithmetic<BitShiftRightImpl, NameBitShiftRight> FunctionBitShiftRight;
|
typedef FunctionBinaryArithmetic<BitShiftRightImpl, NameBitShiftRight> FunctionBitShiftRight;
|
||||||
|
typedef FunctionBinaryArithmetic<LeastImpl, NameLeast> FunctionLeast;
|
||||||
|
typedef FunctionBinaryArithmetic<GreatestImpl, NameGreatest> FunctionGreatest;
|
||||||
|
|
||||||
|
|
||||||
/// Оптимизации для целочисленного деления на константу.
|
/// Оптимизации для целочисленного деления на константу.
|
||||||
|
@ -69,7 +69,7 @@ struct ConvertImpl<DataTypeDate, DataTypeDateTime, Name>
|
|||||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||||
{
|
{
|
||||||
typedef DataTypeDate::FieldType FromFieldType;
|
typedef DataTypeDate::FieldType FromFieldType;
|
||||||
DateLUT & date_lut = DateLUT::instance();
|
const auto & date_lut = DateLUT::instance();
|
||||||
|
|
||||||
if (const ColumnVector<FromFieldType> * col_from = typeid_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
if (const ColumnVector<FromFieldType> * col_from = typeid_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
||||||
{
|
{
|
||||||
@ -108,7 +108,7 @@ struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name>
|
|||||||
|
|
||||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||||
{
|
{
|
||||||
DateLUT & date_lut = DateLUT::instance();
|
const auto & date_lut = DateLUT::instance();
|
||||||
|
|
||||||
if (const ColumnVector<FromFieldType> * col_from = typeid_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
if (const ColumnVector<FromFieldType> * col_from = typeid_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
||||||
{
|
{
|
||||||
@ -149,7 +149,7 @@ struct ConvertImpl32Or64ToDate
|
|||||||
typedef DataTypeDate::FieldType ToFieldType;
|
typedef DataTypeDate::FieldType ToFieldType;
|
||||||
|
|
||||||
template <typename To, typename From>
|
template <typename To, typename From>
|
||||||
static To convert(const From & from, const DateLUT & date_lut)
|
static To convert(const From & from, const DateLUTImpl & date_lut)
|
||||||
{
|
{
|
||||||
return from < 0xFFFF
|
return from < 0xFFFF
|
||||||
? from
|
? from
|
||||||
@ -158,7 +158,7 @@ struct ConvertImpl32Or64ToDate
|
|||||||
|
|
||||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||||
{
|
{
|
||||||
DateLUT & date_lut = DateLUT::instance();
|
const auto & date_lut = DateLUT::instance();
|
||||||
|
|
||||||
if (const ColumnVector<FromFieldType> * col_from
|
if (const ColumnVector<FromFieldType> * col_from
|
||||||
= typeid_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
= typeid_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
||||||
@ -242,6 +242,251 @@ struct ConvertImpl<FromDataType, DataTypeString, Name>
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
namespace details { namespace {
|
||||||
|
|
||||||
|
/** Пусть source_timestamp представляет дату и время в исходном часовом поясе соответствующем
|
||||||
|
* объекту from_date_lut. Эта функция возвращает timestamp представлящий те же дату и время
|
||||||
|
* в часовом поясе соответствующем объекту to_date_lut.
|
||||||
|
*/
|
||||||
|
time_t convertTimestamp(time_t source_timestamp, const DateLUTImpl & from_date_lut, const DateLUTImpl & to_date_lut)
|
||||||
|
{
|
||||||
|
if (&from_date_lut == &to_date_lut)
|
||||||
|
return source_timestamp;
|
||||||
|
else
|
||||||
|
{
|
||||||
|
const auto & values = from_date_lut.getValues(source_timestamp);
|
||||||
|
return to_date_lut.makeDateTime(values.year, values.month, values.day_of_month,
|
||||||
|
from_date_lut.toHourInaccurate(source_timestamp),
|
||||||
|
from_date_lut.toMinuteInaccurate(source_timestamp),
|
||||||
|
from_date_lut.toSecondInaccurate(source_timestamp));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Функции для преобразования даты + времени в строку.
|
||||||
|
*/
|
||||||
|
struct DateTimeToStringConverter
|
||||||
|
{
|
||||||
|
using FromFieldType = typename DataTypeDateTime::FieldType;
|
||||||
|
|
||||||
|
static void vector_vector(const PODArray<FromFieldType> & vec_from, const ColumnString::Chars_t & data,
|
||||||
|
const ColumnString::Offsets_t & offsets, ColumnString & vec_to)
|
||||||
|
{
|
||||||
|
const auto & local_date_lut = DateLUT::instance();
|
||||||
|
|
||||||
|
ColumnString::Chars_t & data_to = vec_to.getChars();
|
||||||
|
ColumnString::Offsets_t & offsets_to = vec_to.getOffsets();
|
||||||
|
size_t size = vec_from.size();
|
||||||
|
data_to.resize(size * 2);
|
||||||
|
offsets_to.resize(size);
|
||||||
|
|
||||||
|
WriteBufferFromVector<ColumnString::Chars_t> write_buffer(data_to);
|
||||||
|
|
||||||
|
ColumnString::Offset_t prev_offset = 0;
|
||||||
|
|
||||||
|
for (size_t i = 0; i < size; ++i)
|
||||||
|
{
|
||||||
|
ColumnString::Offset_t cur_offset = offsets[i];
|
||||||
|
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||||
|
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||||
|
|
||||||
|
auto ti = convertTimestamp(vec_from[i], remote_date_lut, local_date_lut);
|
||||||
|
formatImpl<DataTypeDateTime>(ti, write_buffer);
|
||||||
|
writeChar(0, write_buffer);
|
||||||
|
offsets_to[i] = write_buffer.count();
|
||||||
|
|
||||||
|
prev_offset = cur_offset;
|
||||||
|
}
|
||||||
|
data_to.resize(write_buffer.count());
|
||||||
|
}
|
||||||
|
|
||||||
|
static void vector_constant(const PODArray<FromFieldType> & vec_from, const std::string & data,
|
||||||
|
ColumnString & vec_to)
|
||||||
|
{
|
||||||
|
const auto & local_date_lut = DateLUT::instance();
|
||||||
|
const auto & remote_date_lut = DateLUT::instance(data);
|
||||||
|
|
||||||
|
ColumnString::Chars_t & data_to = vec_to.getChars();
|
||||||
|
ColumnString::Offsets_t & offsets_to = vec_to.getOffsets();
|
||||||
|
size_t size = vec_from.size();
|
||||||
|
data_to.resize(size * 2);
|
||||||
|
offsets_to.resize(size);
|
||||||
|
|
||||||
|
WriteBufferFromVector<ColumnString::Chars_t> write_buffer(data_to);
|
||||||
|
|
||||||
|
for (size_t i = 0; i < size; ++i)
|
||||||
|
{
|
||||||
|
auto ti = convertTimestamp(vec_from[i], remote_date_lut, local_date_lut);
|
||||||
|
formatImpl<DataTypeDateTime>(ti, write_buffer);
|
||||||
|
writeChar(0, write_buffer);
|
||||||
|
offsets_to[i] = write_buffer.count();
|
||||||
|
}
|
||||||
|
data_to.resize(write_buffer.count());
|
||||||
|
}
|
||||||
|
|
||||||
|
static void vector_constant(const PODArray<FromFieldType> & vec_from, ColumnString & vec_to)
|
||||||
|
{
|
||||||
|
ColumnString::Chars_t & data_to = vec_to.getChars();
|
||||||
|
ColumnString::Offsets_t & offsets_to = vec_to.getOffsets();
|
||||||
|
size_t size = vec_from.size();
|
||||||
|
data_to.resize(size * 2);
|
||||||
|
offsets_to.resize(size);
|
||||||
|
|
||||||
|
WriteBufferFromVector<ColumnString::Chars_t> write_buffer(data_to);
|
||||||
|
|
||||||
|
for (size_t i = 0; i < size; ++i)
|
||||||
|
{
|
||||||
|
formatImpl<DataTypeDateTime>(vec_from[i], write_buffer);
|
||||||
|
writeChar(0, write_buffer);
|
||||||
|
offsets_to[i] = write_buffer.count();
|
||||||
|
}
|
||||||
|
data_to.resize(write_buffer.count());
|
||||||
|
}
|
||||||
|
|
||||||
|
static void constant_vector(FromFieldType from, const ColumnString::Chars_t & data,
|
||||||
|
const ColumnString::Offsets_t & offsets,
|
||||||
|
ColumnString & vec_to)
|
||||||
|
{
|
||||||
|
const auto & local_date_lut = DateLUT::instance();
|
||||||
|
|
||||||
|
ColumnString::Chars_t & data_to = vec_to.getChars();
|
||||||
|
ColumnString::Offsets_t & offsets_to = vec_to.getOffsets();
|
||||||
|
size_t size = offsets.size();
|
||||||
|
data_to.resize(size * 2);
|
||||||
|
offsets_to.resize(size);
|
||||||
|
|
||||||
|
WriteBufferFromVector<ColumnString::Chars_t> write_buffer(data_to);
|
||||||
|
|
||||||
|
ColumnString::Offset_t prev_offset = 0;
|
||||||
|
|
||||||
|
for (size_t i = 0; i < size; ++i)
|
||||||
|
{
|
||||||
|
ColumnString::Offset_t cur_offset = offsets[i];
|
||||||
|
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||||
|
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||||
|
|
||||||
|
auto ti = convertTimestamp(from, remote_date_lut, local_date_lut);
|
||||||
|
formatImpl<DataTypeDateTime>(ti, write_buffer);
|
||||||
|
writeChar(0, write_buffer);
|
||||||
|
offsets_to[i] = write_buffer.count();
|
||||||
|
|
||||||
|
prev_offset = cur_offset;
|
||||||
|
}
|
||||||
|
data_to.resize(write_buffer.count());
|
||||||
|
}
|
||||||
|
|
||||||
|
static void constant_constant(FromFieldType from, const std::string & data, std::string & to)
|
||||||
|
{
|
||||||
|
const auto & local_date_lut = DateLUT::instance();
|
||||||
|
const auto & remote_date_lut = DateLUT::instance(data);
|
||||||
|
|
||||||
|
std::vector<char> buf;
|
||||||
|
WriteBufferFromVector<std::vector<char> > write_buffer(buf);
|
||||||
|
auto ti = convertTimestamp(from, remote_date_lut, local_date_lut);
|
||||||
|
formatImpl<DataTypeDateTime>(ti, write_buffer);
|
||||||
|
to = std::string(&buf[0], write_buffer.count());
|
||||||
|
}
|
||||||
|
|
||||||
|
static void constant_constant(FromFieldType from, std::string & to)
|
||||||
|
{
|
||||||
|
std::vector<char> buf;
|
||||||
|
WriteBufferFromVector<std::vector<char> > write_buffer(buf);
|
||||||
|
formatImpl<DataTypeDateTime>(from, write_buffer);
|
||||||
|
to = std::string(&buf[0], write_buffer.count());
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
}}
|
||||||
|
|
||||||
|
template<typename Name>
|
||||||
|
struct ConvertImpl<DataTypeDateTime, DataTypeString, Name>
|
||||||
|
{
|
||||||
|
using Op = details::DateTimeToStringConverter;
|
||||||
|
using FromFieldType = Op::FromFieldType;
|
||||||
|
|
||||||
|
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||||
|
{
|
||||||
|
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
|
||||||
|
const auto * sources = typeid_cast<const ColumnVector<FromFieldType> *>(&*source_col);
|
||||||
|
const auto * const_source = typeid_cast<const ColumnConst<FromFieldType> *>(&*source_col);
|
||||||
|
|
||||||
|
if (arguments.size() == 1)
|
||||||
|
{
|
||||||
|
if (sources)
|
||||||
|
{
|
||||||
|
ColumnString * col_to = new ColumnString;
|
||||||
|
block.getByPosition(result).column = col_to;
|
||||||
|
|
||||||
|
auto & vec_from = sources->getData();
|
||||||
|
auto & vec_to = *col_to;
|
||||||
|
|
||||||
|
Op::vector_constant(vec_from, vec_to);
|
||||||
|
}
|
||||||
|
else if (const_source)
|
||||||
|
{
|
||||||
|
std::string res;
|
||||||
|
Op::constant_constant(const_source->getData(), res);
|
||||||
|
block.getByPosition(result).column = new ColumnConstString(const_source->size(), res);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||||
|
+ " of first argument of function " + Name::name,
|
||||||
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
else if (arguments.size() == 2)
|
||||||
|
{
|
||||||
|
const ColumnPtr time_zone_col = block.getByPosition(arguments[1]).column;
|
||||||
|
const auto * time_zones = typeid_cast<const ColumnString *>(&*time_zone_col);
|
||||||
|
const auto * const_time_zone = typeid_cast<const ColumnConstString *>(&*time_zone_col);
|
||||||
|
|
||||||
|
if (sources)
|
||||||
|
{
|
||||||
|
ColumnString * col_to = new ColumnString;
|
||||||
|
block.getByPosition(result).column = col_to;
|
||||||
|
|
||||||
|
auto & vec_from = sources->getData();
|
||||||
|
auto & vec_to = *col_to;
|
||||||
|
|
||||||
|
if (time_zones)
|
||||||
|
Op::vector_vector(vec_from, time_zones->getChars(), time_zones->getOffsets(), vec_to);
|
||||||
|
else if (const_time_zone)
|
||||||
|
Op::vector_constant(vec_from, const_time_zone->getData(), vec_to);
|
||||||
|
else
|
||||||
|
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||||
|
+ " of second argument of function " + Name::name,
|
||||||
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
}
|
||||||
|
else if (const_source)
|
||||||
|
{
|
||||||
|
if (time_zones)
|
||||||
|
{
|
||||||
|
ColumnString * col_to = new ColumnString;
|
||||||
|
block.getByPosition(result).column = col_to;
|
||||||
|
auto & vec_to = *col_to;
|
||||||
|
|
||||||
|
Op::constant_vector(const_source->getData(), time_zones->getChars(), time_zones->getOffsets(), vec_to);
|
||||||
|
}
|
||||||
|
else if (const_time_zone)
|
||||||
|
{
|
||||||
|
std::string res;
|
||||||
|
Op::constant_constant(const_source->getData(), const_time_zone->getData(), res);
|
||||||
|
block.getByPosition(result).column = new ColumnConstString(const_source->size(), res);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||||
|
+ " of second argument of function " + Name::name,
|
||||||
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||||
|
+ " of first argument of function " + Name::name,
|
||||||
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
throw Exception("Internal error.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
/** Преобразование строк в числа, даты, даты-с-временем: через парсинг.
|
/** Преобразование строк в числа, даты, даты-с-временем: через парсинг.
|
||||||
*/
|
*/
|
||||||
@ -304,6 +549,228 @@ struct ConvertImpl<DataTypeString, ToDataType, Name>
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
namespace details { namespace {
|
||||||
|
|
||||||
|
/** Функции для преобразования строк в timestamp.
|
||||||
|
*/
|
||||||
|
struct StringToTimestampConverter
|
||||||
|
{
|
||||||
|
using ToFieldType = typename DataTypeInt32::FieldType;
|
||||||
|
|
||||||
|
static void vector_vector(const ColumnString::Chars_t & vec_from, const ColumnString::Chars_t & data,
|
||||||
|
const ColumnString::Offsets_t & offsets, PODArray<ToFieldType> & vec_to)
|
||||||
|
{
|
||||||
|
const auto & local_date_lut = DateLUT::instance();
|
||||||
|
ReadBuffer read_buffer(const_cast<char *>(reinterpret_cast<const char *>(&vec_from[0])), vec_from.size(), 0);
|
||||||
|
|
||||||
|
ColumnString::Offset_t prev_offset = 0;
|
||||||
|
|
||||||
|
char zero = 0;
|
||||||
|
for (size_t i = 0; i < vec_to.size(); ++i)
|
||||||
|
{
|
||||||
|
DataTypeDateTime::FieldType x = 0;
|
||||||
|
parseImpl<DataTypeDateTime>(x, read_buffer);
|
||||||
|
|
||||||
|
ColumnString::Offset_t cur_offset = offsets[i];
|
||||||
|
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||||
|
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||||
|
|
||||||
|
auto ti = convertTimestamp(x, local_date_lut, remote_date_lut);
|
||||||
|
|
||||||
|
vec_to[i] = ti;
|
||||||
|
readChar(zero, read_buffer);
|
||||||
|
if (zero != 0)
|
||||||
|
throw Exception("Cannot parse from string.", ErrorCodes::CANNOT_PARSE_NUMBER);
|
||||||
|
|
||||||
|
prev_offset = cur_offset;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static void vector_constant(const ColumnString::Chars_t & vec_from, const std::string & data,
|
||||||
|
PODArray<ToFieldType> & vec_to)
|
||||||
|
{
|
||||||
|
const auto & local_date_lut = DateLUT::instance();
|
||||||
|
const auto & remote_date_lut = DateLUT::instance(data);
|
||||||
|
ReadBuffer read_buffer(const_cast<char *>(reinterpret_cast<const char *>(&vec_from[0])), vec_from.size(), 0);
|
||||||
|
|
||||||
|
char zero = 0;
|
||||||
|
for (size_t i = 0; i < vec_to.size(); ++i)
|
||||||
|
{
|
||||||
|
DataTypeDateTime::FieldType x = 0;
|
||||||
|
parseImpl<DataTypeDateTime>(x, read_buffer);
|
||||||
|
|
||||||
|
auto ti = convertTimestamp(x, local_date_lut, remote_date_lut);
|
||||||
|
|
||||||
|
vec_to[i] = ti;
|
||||||
|
readChar(zero, read_buffer);
|
||||||
|
if (zero != 0)
|
||||||
|
throw Exception("Cannot parse from string.", ErrorCodes::CANNOT_PARSE_NUMBER);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static void vector_constant(const ColumnString::Chars_t & vec_from, PODArray<ToFieldType> & vec_to)
|
||||||
|
{
|
||||||
|
ReadBuffer read_buffer(const_cast<char *>(reinterpret_cast<const char *>(&vec_from[0])), vec_from.size(), 0);
|
||||||
|
|
||||||
|
char zero = 0;
|
||||||
|
for (size_t i = 0; i < vec_to.size(); ++i)
|
||||||
|
{
|
||||||
|
DataTypeDateTime::FieldType x = 0;
|
||||||
|
parseImpl<DataTypeDateTime>(x, read_buffer);
|
||||||
|
vec_to[i] = x;
|
||||||
|
readChar(zero, read_buffer);
|
||||||
|
if (zero != 0)
|
||||||
|
throw Exception("Cannot parse from string.", ErrorCodes::CANNOT_PARSE_NUMBER);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static void constant_vector(const std::string & from, const ColumnString::Chars_t & data,
|
||||||
|
const ColumnString::Offsets_t & offsets, PODArray<ToFieldType> & vec_to)
|
||||||
|
{
|
||||||
|
const auto & local_date_lut = DateLUT::instance();
|
||||||
|
|
||||||
|
ReadBufferFromString read_buffer(from);
|
||||||
|
DataTypeDateTime::FieldType x = 0;
|
||||||
|
parseImpl<DataTypeDateTime>(x, read_buffer);
|
||||||
|
|
||||||
|
ColumnString::Offset_t prev_offset = 0;
|
||||||
|
|
||||||
|
for (size_t i = 0; i < offsets.size(); ++i)
|
||||||
|
{
|
||||||
|
ColumnString::Offset_t cur_offset = offsets[i];
|
||||||
|
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||||
|
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||||
|
|
||||||
|
auto ti = convertTimestamp(x, local_date_lut, remote_date_lut);
|
||||||
|
|
||||||
|
vec_to[i] = ti;
|
||||||
|
prev_offset = cur_offset;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static void constant_constant(const std::string & from, const std::string & data, ToFieldType & to)
|
||||||
|
{
|
||||||
|
const auto & local_date_lut = DateLUT::instance();
|
||||||
|
const auto & remote_date_lut = DateLUT::instance(data);
|
||||||
|
|
||||||
|
ReadBufferFromString read_buffer(from);
|
||||||
|
DataTypeDateTime::FieldType x = 0;
|
||||||
|
parseImpl<DataTypeDateTime>(x, read_buffer);
|
||||||
|
|
||||||
|
to = convertTimestamp(x, local_date_lut, remote_date_lut);
|
||||||
|
}
|
||||||
|
|
||||||
|
static void constant_constant(const std::string & from, ToFieldType & to)
|
||||||
|
{
|
||||||
|
ReadBufferFromString read_buffer(from);
|
||||||
|
DataTypeDateTime::FieldType x = 0;
|
||||||
|
parseImpl<DataTypeDateTime>(x, read_buffer);
|
||||||
|
to = x;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
}}
|
||||||
|
|
||||||
|
struct NameToUnixTimestamp { static constexpr auto name = "toUnixTimestamp"; };
|
||||||
|
|
||||||
|
template<>
|
||||||
|
struct ConvertImpl<DataTypeString, DataTypeInt32, NameToUnixTimestamp>
|
||||||
|
{
|
||||||
|
using Op = details::StringToTimestampConverter;
|
||||||
|
using ToFieldType = Op::ToFieldType;
|
||||||
|
|
||||||
|
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||||
|
{
|
||||||
|
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
|
||||||
|
const auto * sources = typeid_cast<const ColumnString *>(&*source_col);
|
||||||
|
const auto * const_source = typeid_cast<const ColumnConstString *>(&*source_col);
|
||||||
|
|
||||||
|
if (arguments.size() == 1)
|
||||||
|
{
|
||||||
|
if (sources)
|
||||||
|
{
|
||||||
|
auto * col_to = new ColumnVector<ToFieldType>;
|
||||||
|
block.getByPosition(result).column = col_to;
|
||||||
|
|
||||||
|
auto & vec_from = sources->getChars();
|
||||||
|
auto & vec_to = col_to->getData();
|
||||||
|
size_t size = sources->size();
|
||||||
|
vec_to.resize(size);
|
||||||
|
|
||||||
|
Op::vector_constant(vec_from, vec_to);
|
||||||
|
}
|
||||||
|
else if (const_source)
|
||||||
|
{
|
||||||
|
ToFieldType res;
|
||||||
|
Op::constant_constant(const_source->getData(), res);
|
||||||
|
block.getByPosition(result).column = new ColumnConst<ToFieldType>(const_source->size(), res);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||||
|
+ " of first argument of function " + NameToUnixTimestamp::name,
|
||||||
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
else if (arguments.size() == 2)
|
||||||
|
{
|
||||||
|
const ColumnPtr time_zone_col = block.getByPosition(arguments[1]).column;
|
||||||
|
const auto * time_zones = typeid_cast<const ColumnString *>(&*time_zone_col);
|
||||||
|
const auto * const_time_zone = typeid_cast<const ColumnConstString *>(&*time_zone_col);
|
||||||
|
|
||||||
|
if (sources)
|
||||||
|
{
|
||||||
|
auto * col_to = new ColumnVector<ToFieldType>;
|
||||||
|
block.getByPosition(result).column = col_to;
|
||||||
|
|
||||||
|
auto & vec_from = sources->getChars();
|
||||||
|
auto & vec_to = col_to->getData();
|
||||||
|
size_t size = sources->size();
|
||||||
|
vec_to.resize(size);
|
||||||
|
|
||||||
|
if (time_zones)
|
||||||
|
Op::vector_vector(vec_from, time_zones->getChars(), time_zones->getOffsets(), vec_to);
|
||||||
|
else if (const_time_zone)
|
||||||
|
Op::vector_constant(vec_from, const_time_zone->getData(), vec_to);
|
||||||
|
else
|
||||||
|
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||||
|
+ " of second argument of function " + NameToUnixTimestamp::name,
|
||||||
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
}
|
||||||
|
else if (const_source)
|
||||||
|
{
|
||||||
|
if (time_zones)
|
||||||
|
{
|
||||||
|
auto * col_to = new ColumnVector<ToFieldType>;
|
||||||
|
block.getByPosition(result).column = col_to;
|
||||||
|
|
||||||
|
auto & vec_to = col_to->getData();
|
||||||
|
vec_to.resize(time_zones->getOffsets().size());
|
||||||
|
|
||||||
|
Op::constant_vector(const_source->getData(), time_zones->getChars(), time_zones->getOffsets(), vec_to);
|
||||||
|
}
|
||||||
|
else if (const_time_zone)
|
||||||
|
{
|
||||||
|
ToFieldType res;
|
||||||
|
Op::constant_constant(const_source->getData(), const_time_zone->getData(), res);
|
||||||
|
block.getByPosition(result).column = new ColumnConst<ToFieldType>(const_source->size(), res);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||||
|
+ " of second argument of function " + NameToUnixTimestamp::name,
|
||||||
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||||
|
+ " of first argument of function " + NameToUnixTimestamp::name,
|
||||||
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
throw Exception("Internal error.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
/** Если типы совпадают - просто скопируем ссылку на столбец.
|
/** Если типы совпадают - просто скопируем ссылку на столбец.
|
||||||
*/
|
*/
|
||||||
template <typename Name>
|
template <typename Name>
|
||||||
@ -437,12 +904,7 @@ public:
|
|||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||||
{
|
{
|
||||||
if (arguments.size() != 1)
|
return getReturnTypeImpl(arguments);
|
||||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
|
||||||
+ toString(arguments.size()) + ", should be 1.",
|
|
||||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
||||||
|
|
||||||
return new ToDataType;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Выполнить функцию над блоком.
|
/// Выполнить функцию над блоком.
|
||||||
@ -468,6 +930,73 @@ public:
|
|||||||
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
|
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
|
||||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
template<typename ToDataType2 = ToDataType, typename Name2 = Name>
|
||||||
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments,
|
||||||
|
typename std::enable_if<!(std::is_same<ToDataType2, DataTypeString>::value || std::is_same<Name2, NameToUnixTimestamp>::value), void>::type * = nullptr) const
|
||||||
|
{
|
||||||
|
if (arguments.size() != 1)
|
||||||
|
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||||
|
+ toString(arguments.size()) + ", should be 1.",
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
return new ToDataType;
|
||||||
|
}
|
||||||
|
|
||||||
|
template<typename ToDataType2 = ToDataType, typename Name2 = Name>
|
||||||
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments,
|
||||||
|
typename std::enable_if<std::is_same<ToDataType2, DataTypeString>::value>::type * = nullptr) const
|
||||||
|
{
|
||||||
|
if ((arguments.size() < 1) || (arguments.size() > 2))
|
||||||
|
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||||
|
+ toString(arguments.size()) + ", should be 1 or 2.",
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
if (typeid_cast<const DataTypeDateTime *>(&*arguments[0]) == nullptr)
|
||||||
|
{
|
||||||
|
if (arguments.size() != 1)
|
||||||
|
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||||
|
+ toString(arguments.size()) + ", should be 1.",
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
}
|
||||||
|
else if ((arguments.size()) == 2 && typeid_cast<const DataTypeString *>(&*arguments[1]) == nullptr)
|
||||||
|
{
|
||||||
|
throw Exception{
|
||||||
|
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(),
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
return new ToDataType2;
|
||||||
|
}
|
||||||
|
|
||||||
|
template<typename ToDataType2 = ToDataType, typename Name2 = Name>
|
||||||
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments,
|
||||||
|
typename std::enable_if<std::is_same<Name2, NameToUnixTimestamp>::value, void>::type * = nullptr) const
|
||||||
|
{
|
||||||
|
if ((arguments.size() < 1) || (arguments.size() > 2))
|
||||||
|
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||||
|
+ toString(arguments.size()) + ", should be 1 or 2.",
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
if (typeid_cast<const DataTypeString *>(&*arguments[0]) == nullptr)
|
||||||
|
{
|
||||||
|
if (arguments.size() != 1)
|
||||||
|
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||||
|
+ toString(arguments.size()) + ", should be 1.",
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
}
|
||||||
|
else if ((arguments.size()) == 2 && typeid_cast<const DataTypeString *>(&*arguments[1]) == nullptr)
|
||||||
|
{
|
||||||
|
throw Exception{
|
||||||
|
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(),
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
return new ToDataType2;
|
||||||
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
@ -628,6 +1157,6 @@ typedef FunctionConvert<DataTypeFloat64, NameToFloat64> FunctionToFloat64;
|
|||||||
typedef FunctionConvert<DataTypeDate, NameToDate> FunctionToDate;
|
typedef FunctionConvert<DataTypeDate, NameToDate> FunctionToDate;
|
||||||
typedef FunctionConvert<DataTypeDateTime, NameToDateTime> FunctionToDateTime;
|
typedef FunctionConvert<DataTypeDateTime, NameToDateTime> FunctionToDateTime;
|
||||||
typedef FunctionConvert<DataTypeString, NameToString> FunctionToString;
|
typedef FunctionConvert<DataTypeString, NameToString> FunctionToString;
|
||||||
|
typedef FunctionConvert<DataTypeInt32, NameToUnixTimestamp> FunctionToUnixTimestamp;
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -4,12 +4,15 @@
|
|||||||
#include <DB/DataTypes/DataTypeDate.h>
|
#include <DB/DataTypes/DataTypeDate.h>
|
||||||
#include <DB/DataTypes/DataTypeDateTime.h>
|
#include <DB/DataTypes/DataTypeDateTime.h>
|
||||||
#include <DB/DataTypes/DataTypeArray.h>
|
#include <DB/DataTypes/DataTypeArray.h>
|
||||||
|
#include <DB/DataTypes/DataTypeString.h>
|
||||||
|
|
||||||
#include <DB/Columns/ColumnConst.h>
|
#include <DB/Columns/ColumnConst.h>
|
||||||
#include <DB/Columns/ColumnArray.h>
|
#include <DB/Columns/ColumnArray.h>
|
||||||
|
#include <DB/Columns/ColumnFixedString.h>
|
||||||
|
|
||||||
#include <DB/Functions/IFunction.h>
|
#include <DB/Functions/IFunction.h>
|
||||||
|
|
||||||
|
#include <type_traits>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -48,32 +51,32 @@ namespace DB
|
|||||||
|
|
||||||
struct ToYearImpl
|
struct ToYearImpl
|
||||||
{
|
{
|
||||||
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toYear(t); }
|
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(t); }
|
||||||
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toYear(DayNum_t(d)); }
|
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(DayNum_t(d)); }
|
||||||
};
|
};
|
||||||
|
|
||||||
struct ToMonthImpl
|
struct ToMonthImpl
|
||||||
{
|
{
|
||||||
static inline UInt8 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toMonth(t); }
|
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toMonth(t); }
|
||||||
static inline UInt8 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toMonth(DayNum_t(d)); }
|
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toMonth(DayNum_t(d)); }
|
||||||
};
|
};
|
||||||
|
|
||||||
struct ToDayOfMonthImpl
|
struct ToDayOfMonthImpl
|
||||||
{
|
{
|
||||||
static inline UInt8 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toDayOfMonth(t); }
|
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfMonth(t); }
|
||||||
static inline UInt8 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toDayOfMonth(DayNum_t(d)); }
|
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfMonth(DayNum_t(d)); }
|
||||||
};
|
};
|
||||||
|
|
||||||
struct ToDayOfWeekImpl
|
struct ToDayOfWeekImpl
|
||||||
{
|
{
|
||||||
static inline UInt8 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toDayOfWeek(t); }
|
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfWeek(t); }
|
||||||
static inline UInt8 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toDayOfWeek(DayNum_t(d)); }
|
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfWeek(DayNum_t(d)); }
|
||||||
};
|
};
|
||||||
|
|
||||||
struct ToHourImpl
|
struct ToHourImpl
|
||||||
{
|
{
|
||||||
static inline UInt8 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toHourInaccurate(t); }
|
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toHourInaccurate(t); }
|
||||||
static inline UInt8 execute(UInt16 d, DateLUT & date_lut)
|
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||||
{
|
{
|
||||||
throw Exception("Illegal type Date of argument for function toHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
throw Exception("Illegal type Date of argument for function toHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
}
|
}
|
||||||
@ -81,8 +84,8 @@ struct ToHourImpl
|
|||||||
|
|
||||||
struct ToMinuteImpl
|
struct ToMinuteImpl
|
||||||
{
|
{
|
||||||
static inline UInt8 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toMinuteInaccurate(t); }
|
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toMinuteInaccurate(t); }
|
||||||
static inline UInt8 execute(UInt16 d, DateLUT & date_lut)
|
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||||
{
|
{
|
||||||
throw Exception("Illegal type Date of argument for function toMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
throw Exception("Illegal type Date of argument for function toMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
}
|
}
|
||||||
@ -90,8 +93,8 @@ struct ToMinuteImpl
|
|||||||
|
|
||||||
struct ToSecondImpl
|
struct ToSecondImpl
|
||||||
{
|
{
|
||||||
static inline UInt8 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toSecondInaccurate(t); }
|
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toSecondInaccurate(t); }
|
||||||
static inline UInt8 execute(UInt16 d, DateLUT & date_lut)
|
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||||
{
|
{
|
||||||
throw Exception("Illegal type Date of argument for function toSecond", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
throw Exception("Illegal type Date of argument for function toSecond", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
}
|
}
|
||||||
@ -99,34 +102,49 @@ struct ToSecondImpl
|
|||||||
|
|
||||||
struct ToMondayImpl
|
struct ToMondayImpl
|
||||||
{
|
{
|
||||||
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toFirstDayNumOfWeek(date_lut.toDayNum(t)); }
|
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfWeek(remote_date_lut.toDayNum(t)); }
|
||||||
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toFirstDayNumOfWeek(DayNum_t(d)); }
|
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfWeek(DayNum_t(d)); }
|
||||||
};
|
};
|
||||||
|
|
||||||
struct ToStartOfMonthImpl
|
struct ToStartOfMonthImpl
|
||||||
{
|
{
|
||||||
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toFirstDayNumOfMonth(date_lut.toDayNum(t)); }
|
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfMonth(remote_date_lut.toDayNum(t)); }
|
||||||
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toFirstDayNumOfMonth(DayNum_t(d)); }
|
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfMonth(DayNum_t(d)); }
|
||||||
};
|
};
|
||||||
|
|
||||||
struct ToStartOfQuarterImpl
|
struct ToStartOfQuarterImpl
|
||||||
{
|
{
|
||||||
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toFirstDayNumOfQuarter(date_lut.toDayNum(t)); }
|
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfQuarter(remote_date_lut.toDayNum(t)); }
|
||||||
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toFirstDayNumOfQuarter(DayNum_t(d)); }
|
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfQuarter(DayNum_t(d)); }
|
||||||
};
|
};
|
||||||
|
|
||||||
struct ToStartOfYearImpl
|
struct ToStartOfYearImpl
|
||||||
{
|
{
|
||||||
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toFirstDayNumOfYear(date_lut.toDayNum(t)); }
|
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfYear(remote_date_lut.toDayNum(t)); }
|
||||||
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toFirstDayNumOfYear(DayNum_t(d)); }
|
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfYear(DayNum_t(d)); }
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
struct ToTimeImpl
|
struct ToTimeImpl
|
||||||
{
|
{
|
||||||
/// При переводе во время, дату будем приравнивать к 1970-01-02.
|
/// При переводе во время, дату будем приравнивать к 1970-01-02.
|
||||||
static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toTimeInaccurate(t) + 86400; }
|
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||||
static inline UInt32 execute(UInt16 d, DateLUT & date_lut)
|
{
|
||||||
|
time_t remote_ts = remote_date_lut.toTimeInaccurate(t) + 86400;
|
||||||
|
|
||||||
|
if (&remote_date_lut == &local_date_lut)
|
||||||
|
return remote_ts;
|
||||||
|
else
|
||||||
|
{
|
||||||
|
const auto & values = remote_date_lut.getValues(remote_ts);
|
||||||
|
return local_date_lut.makeDateTime(values.year, values.month, values.day_of_month,
|
||||||
|
remote_date_lut.toHourInaccurate(remote_ts),
|
||||||
|
remote_date_lut.toMinuteInaccurate(remote_ts),
|
||||||
|
remote_date_lut.toSecondInaccurate(remote_ts));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||||
{
|
{
|
||||||
throw Exception("Illegal type Date of argument for function toTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
throw Exception("Illegal type Date of argument for function toTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
}
|
}
|
||||||
@ -134,8 +152,8 @@ struct ToTimeImpl
|
|||||||
|
|
||||||
struct ToStartOfMinuteImpl
|
struct ToStartOfMinuteImpl
|
||||||
{
|
{
|
||||||
static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toStartOfMinuteInaccurate(t); }
|
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toStartOfMinuteInaccurate(t); }
|
||||||
static inline UInt32 execute(UInt16 d, DateLUT & date_lut)
|
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||||
{
|
{
|
||||||
throw Exception("Illegal type Date of argument for function toStartOfMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
throw Exception("Illegal type Date of argument for function toStartOfMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
}
|
}
|
||||||
@ -143,8 +161,8 @@ struct ToStartOfMinuteImpl
|
|||||||
|
|
||||||
struct ToStartOfFiveMinuteImpl
|
struct ToStartOfFiveMinuteImpl
|
||||||
{
|
{
|
||||||
static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toStartOfFiveMinuteInaccurate(t); }
|
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toStartOfFiveMinuteInaccurate(t); }
|
||||||
static inline UInt32 execute(UInt16 d, DateLUT & date_lut)
|
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||||
{
|
{
|
||||||
throw Exception("Illegal type Date of argument for function toStartOfFiveMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
throw Exception("Illegal type Date of argument for function toStartOfFiveMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
}
|
}
|
||||||
@ -152,8 +170,8 @@ struct ToStartOfFiveMinuteImpl
|
|||||||
|
|
||||||
struct ToStartOfHourImpl
|
struct ToStartOfHourImpl
|
||||||
{
|
{
|
||||||
static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toStartOfHourInaccurate(t); }
|
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toStartOfHourInaccurate(t); }
|
||||||
static inline UInt32 execute(UInt16 d, DateLUT & date_lut)
|
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||||
{
|
{
|
||||||
throw Exception("Illegal type Date of argument for function toStartOfHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
throw Exception("Illegal type Date of argument for function toStartOfHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
}
|
}
|
||||||
@ -161,33 +179,33 @@ struct ToStartOfHourImpl
|
|||||||
|
|
||||||
struct ToRelativeYearNumImpl
|
struct ToRelativeYearNumImpl
|
||||||
{
|
{
|
||||||
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toYear(t); }
|
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(t); }
|
||||||
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toYear(DayNum_t(d)); }
|
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(DayNum_t(d)); }
|
||||||
};
|
};
|
||||||
|
|
||||||
struct ToRelativeMonthNumImpl
|
struct ToRelativeMonthNumImpl
|
||||||
{
|
{
|
||||||
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toRelativeMonthNum(t); }
|
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeMonthNum(t); }
|
||||||
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toRelativeMonthNum(DayNum_t(d)); }
|
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeMonthNum(DayNum_t(d)); }
|
||||||
};
|
};
|
||||||
|
|
||||||
struct ToRelativeWeekNumImpl
|
struct ToRelativeWeekNumImpl
|
||||||
{
|
{
|
||||||
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toRelativeWeekNum(t); }
|
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeWeekNum(t); }
|
||||||
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toRelativeWeekNum(DayNum_t(d)); }
|
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeWeekNum(DayNum_t(d)); }
|
||||||
};
|
};
|
||||||
|
|
||||||
struct ToRelativeDayNumImpl
|
struct ToRelativeDayNumImpl
|
||||||
{
|
{
|
||||||
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toDayNum(t); }
|
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toDayNum(t); }
|
||||||
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return static_cast<DayNum_t>(d); }
|
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return static_cast<DayNum_t>(d); }
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
struct ToRelativeHourNumImpl
|
struct ToRelativeHourNumImpl
|
||||||
{
|
{
|
||||||
static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toRelativeHourNum(t); }
|
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeHourNum(t); }
|
||||||
static inline UInt32 execute(UInt16 d, DateLUT & date_lut)
|
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||||
{
|
{
|
||||||
throw Exception("Illegal type Date of argument for function toRelativeHourNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
throw Exception("Illegal type Date of argument for function toRelativeHourNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
}
|
}
|
||||||
@ -195,8 +213,8 @@ struct ToRelativeHourNumImpl
|
|||||||
|
|
||||||
struct ToRelativeMinuteNumImpl
|
struct ToRelativeMinuteNumImpl
|
||||||
{
|
{
|
||||||
static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toRelativeMinuteNum(t); }
|
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeMinuteNum(t); }
|
||||||
static inline UInt32 execute(UInt16 d, DateLUT & date_lut)
|
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||||
{
|
{
|
||||||
throw Exception("Illegal type Date of argument for function toRelativeMinuteNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
throw Exception("Illegal type Date of argument for function toRelativeMinuteNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
}
|
}
|
||||||
@ -204,46 +222,173 @@ struct ToRelativeMinuteNumImpl
|
|||||||
|
|
||||||
struct ToRelativeSecondNumImpl
|
struct ToRelativeSecondNumImpl
|
||||||
{
|
{
|
||||||
static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return t; }
|
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return t; }
|
||||||
static inline UInt32 execute(UInt16 d, DateLUT & date_lut)
|
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||||
{
|
{
|
||||||
throw Exception("Illegal type Date of argument for function toRelativeSecondNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
throw Exception("Illegal type Date of argument for function toRelativeSecondNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
template<typename FromType, typename ToType, typename Transform>
|
||||||
|
struct Transformer
|
||||||
|
{
|
||||||
|
static void vector_vector(const PODArray<FromType> & vec_from, const ColumnString::Chars_t & data,
|
||||||
|
const ColumnString::Offsets_t & offsets, PODArray<ToType> & vec_to)
|
||||||
|
{
|
||||||
|
const auto & local_date_lut = DateLUT::instance();
|
||||||
|
ColumnString::Offset_t prev_offset = 0;
|
||||||
|
|
||||||
|
for (size_t i = 0; i < vec_from.size(); ++i)
|
||||||
|
{
|
||||||
|
ColumnString::Offset_t cur_offset = offsets[i];
|
||||||
|
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||||
|
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||||
|
vec_to[i] = Transform::execute(vec_from[i], remote_date_lut, local_date_lut);
|
||||||
|
prev_offset = cur_offset;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static void vector_constant(const PODArray<FromType> & vec_from, const std::string & data,
|
||||||
|
PODArray<ToType> & vec_to)
|
||||||
|
{
|
||||||
|
const auto & local_date_lut = DateLUT::instance();
|
||||||
|
const auto & remote_date_lut = DateLUT::instance(data);
|
||||||
|
for (size_t i = 0; i < vec_from.size(); ++i)
|
||||||
|
vec_to[i] = Transform::execute(vec_from[i], remote_date_lut, local_date_lut);
|
||||||
|
}
|
||||||
|
|
||||||
|
static void vector_constant(const PODArray<FromType> & vec_from, PODArray<ToType> & vec_to)
|
||||||
|
{
|
||||||
|
const auto & local_date_lut = DateLUT::instance();
|
||||||
|
for (size_t i = 0; i < vec_from.size(); ++i)
|
||||||
|
vec_to[i] = Transform::execute(vec_from[i], local_date_lut, local_date_lut);
|
||||||
|
}
|
||||||
|
|
||||||
|
static void constant_vector(const FromType & from, const ColumnString::Chars_t & data,
|
||||||
|
const ColumnString::Offsets_t & offsets, PODArray<ToType> & vec_to)
|
||||||
|
{
|
||||||
|
const auto & local_date_lut = DateLUT::instance();
|
||||||
|
ColumnString::Offset_t prev_offset = 0;
|
||||||
|
|
||||||
|
for (size_t i = 0; i < offsets.size(); ++i)
|
||||||
|
{
|
||||||
|
ColumnString::Offset_t cur_offset = offsets[i];
|
||||||
|
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||||
|
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||||
|
vec_to[i] = Transform::execute(from, remote_date_lut, local_date_lut);
|
||||||
|
prev_offset = cur_offset;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static void constant_constant(const FromType & from, const std::string & data, ToType & to)
|
||||||
|
{
|
||||||
|
const auto & local_date_lut = DateLUT::instance();
|
||||||
|
const auto & remote_date_lut = DateLUT::instance(data);
|
||||||
|
to = Transform::execute(from, remote_date_lut, local_date_lut);
|
||||||
|
}
|
||||||
|
|
||||||
|
static void constant_constant(const FromType & from, ToType & to)
|
||||||
|
{
|
||||||
|
const auto & local_date_lut = DateLUT::instance();
|
||||||
|
to = Transform::execute(from, local_date_lut, local_date_lut);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
template <typename FromType, typename ToType, typename Transform, typename Name>
|
template <typename FromType, typename ToType, typename Transform, typename Name>
|
||||||
struct DateTimeTransformImpl
|
struct DateTimeTransformImpl
|
||||||
{
|
{
|
||||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||||
{
|
{
|
||||||
DateLUT & date_lut = DateLUT::instance();
|
using Op = Transformer<FromType, ToType, Transform>;
|
||||||
|
|
||||||
if (const ColumnVector<FromType> * col_from = typeid_cast<const ColumnVector<FromType> *>(&*block.getByPosition(arguments[0]).column))
|
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
|
||||||
|
const auto * sources = typeid_cast<const ColumnVector<FromType> *>(&*source_col);
|
||||||
|
const auto * const_source = typeid_cast<const ColumnConst<FromType> *>(&*source_col);
|
||||||
|
|
||||||
|
if (arguments.size() == 1)
|
||||||
{
|
{
|
||||||
ColumnVector<ToType> * col_to = new ColumnVector<ToType>;
|
if (sources)
|
||||||
block.getByPosition(result).column = col_to;
|
{
|
||||||
|
auto * col_to = new ColumnVector<ToType>;
|
||||||
|
block.getByPosition(result).column = col_to;
|
||||||
|
|
||||||
const typename ColumnVector<FromType>::Container_t & vec_from = col_from->getData();
|
auto & vec_from = sources->getData();
|
||||||
typename ColumnVector<ToType>::Container_t & vec_to = col_to->getData();
|
auto & vec_to = col_to->getData();
|
||||||
size_t size = vec_from.size();
|
size_t size = vec_from.size();
|
||||||
vec_to.resize(size);
|
vec_to.resize(size);
|
||||||
|
|
||||||
for (size_t i = 0; i < size; ++i)
|
Op::vector_constant(vec_from, vec_to);
|
||||||
vec_to[i] = Transform::execute(vec_from[i], date_lut);
|
}
|
||||||
|
else if (const_source)
|
||||||
|
{
|
||||||
|
ToType res;
|
||||||
|
Op::constant_constant(const_source->getData(), res);
|
||||||
|
block.getByPosition(result).column = new ColumnConst<ToType>(const_source->size(), res);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||||
|
+ " of first argument of function " + Name::name,
|
||||||
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
else if (const ColumnConst<FromType> * col_from = typeid_cast<const ColumnConst<FromType> *>(&*block.getByPosition(arguments[0]).column))
|
else if (arguments.size() == 2)
|
||||||
{
|
{
|
||||||
block.getByPosition(result).column = new ColumnConst<ToType>(col_from->size(), Transform::execute(col_from->getData(), date_lut));
|
const ColumnPtr time_zone_col = block.getByPosition(arguments[1]).column;
|
||||||
|
const auto * time_zones = typeid_cast<const ColumnString *>(&*time_zone_col);
|
||||||
|
const auto * const_time_zone = typeid_cast<const ColumnConstString *>(&*time_zone_col);
|
||||||
|
|
||||||
|
if (sources)
|
||||||
|
{
|
||||||
|
auto * col_to = new ColumnVector<ToType>;
|
||||||
|
block.getByPosition(result).column = col_to;
|
||||||
|
|
||||||
|
auto & vec_from = sources->getData();
|
||||||
|
auto & vec_to = col_to->getData();
|
||||||
|
vec_to.resize(vec_from.size());
|
||||||
|
|
||||||
|
if (time_zones)
|
||||||
|
Op::vector_vector(vec_from, time_zones->getChars(), time_zones->getOffsets(), vec_to);
|
||||||
|
else if (const_time_zone)
|
||||||
|
Op::vector_constant(vec_from, const_time_zone->getData(), vec_to);
|
||||||
|
else
|
||||||
|
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||||
|
+ " of second argument of function " + Name::name,
|
||||||
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
}
|
||||||
|
else if (const_source)
|
||||||
|
{
|
||||||
|
if (time_zones)
|
||||||
|
{
|
||||||
|
auto * col_to = new ColumnVector<ToType>;
|
||||||
|
block.getByPosition(result).column = col_to;
|
||||||
|
|
||||||
|
auto & vec_to = col_to->getData();
|
||||||
|
vec_to.resize(time_zones->getOffsets().size());
|
||||||
|
|
||||||
|
Op::constant_vector(const_source->getData(), time_zones->getChars(), time_zones->getOffsets(), vec_to);
|
||||||
|
}
|
||||||
|
else if (const_time_zone)
|
||||||
|
{
|
||||||
|
ToType res;
|
||||||
|
Op::constant_constant(const_source->getData(), const_time_zone->getData(), res);
|
||||||
|
block.getByPosition(result).column = new ColumnConst<ToType>(const_source->size(), res);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||||
|
+ " of second argument of function " + Name::name,
|
||||||
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||||
|
+ " of first argument of function " + Name::name,
|
||||||
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
throw Exception("Internal error.", ErrorCodes::LOGICAL_ERROR);
|
||||||
+ " of first argument of function " + Name::name,
|
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
template <typename ToDataType, typename Transform, typename Name>
|
template <typename ToDataType, typename Transform, typename Name>
|
||||||
class FunctionDateOrDateTimeToSomething : public IFunction
|
class FunctionDateOrDateTimeToSomething : public IFunction
|
||||||
{
|
{
|
||||||
@ -257,15 +402,9 @@ public:
|
|||||||
return name;
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
DataTypePtr getReturnType(const DataTypes & arguments) const override
|
||||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
|
||||||
{
|
{
|
||||||
if (arguments.size() != 1)
|
return getReturnTypeImpl(arguments);
|
||||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
|
||||||
+ toString(arguments.size()) + ", should be 1.",
|
|
||||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
||||||
|
|
||||||
return new ToDataType;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Выполнить функцию над блоком.
|
/// Выполнить функцию над блоком.
|
||||||
@ -281,6 +420,54 @@ public:
|
|||||||
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
|
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
|
||||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
|
|
||||||
|
template<typename ToDataType2 = ToDataType, typename Transform2 = Transform>
|
||||||
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments,
|
||||||
|
typename std::enable_if<
|
||||||
|
!(std::is_same<ToDataType2, DataTypeDate>::value
|
||||||
|
|| (std::is_same<ToDataType2, DataTypeDateTime>::value && std::is_same<Transform2, ToTimeImpl>::value))
|
||||||
|
, void>::type * = nullptr) const
|
||||||
|
{
|
||||||
|
if (arguments.size() != 1)
|
||||||
|
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||||
|
+ toString(arguments.size()) + ", should be 1.",
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
return new ToDataType;
|
||||||
|
}
|
||||||
|
|
||||||
|
template<typename ToDataType2 = ToDataType, typename Transform2 = Transform>
|
||||||
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments,
|
||||||
|
typename std::enable_if<
|
||||||
|
std::is_same<ToDataType2, DataTypeDate>::value
|
||||||
|
|| (std::is_same<ToDataType2, DataTypeDateTime>::value && std::is_same<Transform2, ToTimeImpl>::value)
|
||||||
|
, void>::type * = nullptr) const
|
||||||
|
{
|
||||||
|
if ((arguments.size() < 1) || (arguments.size() > 2))
|
||||||
|
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||||
|
+ toString(arguments.size()) + ", should be 1 or 2.",
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
if (typeid_cast<const DataTypeDateTime *>(&*arguments[0]) == nullptr)
|
||||||
|
{
|
||||||
|
if (arguments.size() != 1)
|
||||||
|
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||||
|
+ toString(arguments.size()) + ", should be 1.",
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
}
|
||||||
|
else if ((arguments.size()) == 2 && typeid_cast<const DataTypeString *>(&*arguments[1]) == nullptr)
|
||||||
|
{
|
||||||
|
throw Exception{
|
||||||
|
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(),
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
return new ToDataType;
|
||||||
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
@ -725,7 +725,11 @@ public:
|
|||||||
column_first_array = column_array;
|
column_first_array = column_array;
|
||||||
}
|
}
|
||||||
|
|
||||||
temp_block.insert(ColumnWithNameAndType(column_array->getDataPtr(), argument_type, argument_name));
|
temp_block.insert(ColumnWithNameAndType(
|
||||||
|
column_array->getDataPtr(),
|
||||||
|
argument_type,
|
||||||
|
argument_name));
|
||||||
|
|
||||||
argument_names.insert(argument_name);
|
argument_names.insert(argument_name);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -745,6 +749,7 @@ public:
|
|||||||
|
|
||||||
replicated_column.name = name;
|
replicated_column.name = name;
|
||||||
replicated_column.column = typeid_cast<ColumnArray &>(*replicated_column.column).getDataPtr();
|
replicated_column.column = typeid_cast<ColumnArray &>(*replicated_column.column).getDataPtr();
|
||||||
|
replicated_column.type = typeid_cast<const DataTypeArray &>(*replicated_column.type).getNestedType(),
|
||||||
temp_block.insert(replicated_column);
|
temp_block.insert(replicated_column);
|
||||||
|
|
||||||
++prerequisite_index;
|
++prerequisite_index;
|
||||||
|
@ -332,7 +332,7 @@ inline void writeDateText(DayNum_t date, WriteBuffer & buf)
|
|||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
const DateLUT::Values & values = DateLUT::instance().getValues(date);
|
const auto & values = DateLUT::instance().getValues(date);
|
||||||
|
|
||||||
s[0] += values.year / 1000;
|
s[0] += values.year / 1000;
|
||||||
s[1] += (values.year / 100) % 10;
|
s[1] += (values.year / 100) % 10;
|
||||||
@ -364,7 +364,8 @@ inline void writeDateText(mysqlxx::Date date, WriteBuffer & buf)
|
|||||||
|
|
||||||
|
|
||||||
/// в формате YYYY-MM-DD HH:MM:SS, согласно текущему часовому поясу
|
/// в формате YYYY-MM-DD HH:MM:SS, согласно текущему часовому поясу
|
||||||
inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, char date_delimeter = '-', char time_delimeter = ':')
|
template <char date_delimeter = '-', char time_delimeter = ':'>
|
||||||
|
inline void writeDateTimeText(time_t datetime, WriteBuffer & buf)
|
||||||
{
|
{
|
||||||
char s[19] = {'0', '0', '0', '0', date_delimeter, '0', '0', date_delimeter, '0', '0', ' ', '0', '0', time_delimeter, '0', '0', time_delimeter, '0', '0'};
|
char s[19] = {'0', '0', '0', '0', date_delimeter, '0', '0', date_delimeter, '0', '0', ' ', '0', '0', time_delimeter, '0', '0', time_delimeter, '0', '0'};
|
||||||
|
|
||||||
@ -374,8 +375,8 @@ inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, char date_deli
|
|||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
DateLUT & date_lut = DateLUT::instance();
|
const auto & date_lut = DateLUT::instance();
|
||||||
const DateLUT::Values & values = date_lut.getValues(datetime);
|
const auto & values = date_lut.getValues(datetime);
|
||||||
|
|
||||||
s[0] += values.year / 1000;
|
s[0] += values.year / 1000;
|
||||||
s[1] += (values.year / 100) % 10;
|
s[1] += (values.year / 100) % 10;
|
||||||
@ -400,7 +401,8 @@ inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, char date_deli
|
|||||||
buf.write(s, 19);
|
buf.write(s, 19);
|
||||||
}
|
}
|
||||||
|
|
||||||
inline void writeDateTimeText(mysqlxx::DateTime datetime, WriteBuffer & buf, char date_delimeter = '-', char time_delimeter = ':')
|
template <char date_delimeter = '-', char time_delimeter = ':'>
|
||||||
|
inline void writeDateTimeText(mysqlxx::DateTime datetime, WriteBuffer & buf)
|
||||||
{
|
{
|
||||||
char s[19] = {'0', '0', '0', '0', date_delimeter, '0', '0', date_delimeter, '0', '0', ' ', '0', '0', time_delimeter, '0', '0', time_delimeter, '0', '0'};
|
char s[19] = {'0', '0', '0', '0', date_delimeter, '0', '0', date_delimeter, '0', '0', ' ', '0', '0', time_delimeter, '0', '0', time_delimeter, '0', '0'};
|
||||||
|
|
||||||
|
@ -678,9 +678,15 @@ public:
|
|||||||
std::string getModePrefix() const;
|
std::string getModePrefix() const;
|
||||||
|
|
||||||
bool supportsSampling() const { return !!sampling_expression; }
|
bool supportsSampling() const { return !!sampling_expression; }
|
||||||
bool supportsFinal() const { return !sign_column.empty(); }
|
|
||||||
bool supportsPrewhere() const { return true; }
|
bool supportsPrewhere() const { return true; }
|
||||||
|
|
||||||
|
bool supportsFinal() const
|
||||||
|
{
|
||||||
|
return mode == Mode::Collapsing
|
||||||
|
|| mode == Mode::Summing
|
||||||
|
|| mode == Mode::Aggregating;
|
||||||
|
}
|
||||||
|
|
||||||
UInt64 getMaxDataPartIndex();
|
UInt64 getMaxDataPartIndex();
|
||||||
|
|
||||||
std::string getTableName() const override
|
std::string getTableName() const override
|
||||||
|
@ -1007,7 +1007,7 @@ private:
|
|||||||
text.resize(embedded_stack_trace_pos);
|
text.resize(embedded_stack_trace_pos);
|
||||||
|
|
||||||
std::cerr << "Received exception from server:" << std::endl
|
std::cerr << "Received exception from server:" << std::endl
|
||||||
<< "Code: " << e.code() << ". " << text;
|
<< "Code: " << e.code() << ". " << text << std::endl;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
334
dbms/src/Columns/ColumnArray.cpp
Normal file
334
dbms/src/Columns/ColumnArray.cpp
Normal file
@ -0,0 +1,334 @@
|
|||||||
|
#include <DB/Columns/ColumnArray.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
|
||||||
|
ColumnPtr ColumnArray::cut(size_t start, size_t length) const
|
||||||
|
{
|
||||||
|
if (length == 0)
|
||||||
|
return new ColumnArray(data);
|
||||||
|
|
||||||
|
if (start + length > getOffsets().size())
|
||||||
|
throw Exception("Parameter out of bound in IColumnArray::cut() method.",
|
||||||
|
ErrorCodes::PARAMETER_OUT_OF_BOUND);
|
||||||
|
|
||||||
|
size_t nested_offset = offsetAt(start);
|
||||||
|
size_t nested_length = getOffsets()[start + length - 1] - nested_offset;
|
||||||
|
|
||||||
|
ColumnArray * res_ = new ColumnArray(data);
|
||||||
|
ColumnPtr res = res_;
|
||||||
|
|
||||||
|
res_->data = data->cut(nested_offset, nested_length);
|
||||||
|
Offsets_t & res_offsets = res_->getOffsets();
|
||||||
|
|
||||||
|
if (start == 0)
|
||||||
|
{
|
||||||
|
res_offsets.assign(getOffsets().begin(), getOffsets().begin() + length);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
res_offsets.resize(length);
|
||||||
|
|
||||||
|
for (size_t i = 0; i < length; ++i)
|
||||||
|
res_offsets[i] = getOffsets()[start + i] - nested_offset;
|
||||||
|
}
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
ColumnPtr ColumnArray::filter(const Filter & filt) const
|
||||||
|
{
|
||||||
|
size_t size = getOffsets().size();
|
||||||
|
if (size != filt.size())
|
||||||
|
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
if (size == 0)
|
||||||
|
return new ColumnArray(data);
|
||||||
|
|
||||||
|
/// Не слишком оптимально. Можно сделать специализацию для массивов известных типов.
|
||||||
|
Filter nested_filt(getOffsets().back());
|
||||||
|
for (size_t i = 0; i < size; ++i)
|
||||||
|
{
|
||||||
|
if (filt[i])
|
||||||
|
memset(&nested_filt[offsetAt(i)], 1, sizeAt(i));
|
||||||
|
else
|
||||||
|
memset(&nested_filt[offsetAt(i)], 0, sizeAt(i));
|
||||||
|
}
|
||||||
|
|
||||||
|
ColumnArray * res_ = new ColumnArray(data);
|
||||||
|
ColumnPtr res = res_;
|
||||||
|
res_->data = data->filter(nested_filt);
|
||||||
|
|
||||||
|
Offsets_t & res_offsets = res_->getOffsets();
|
||||||
|
res_offsets.reserve(size);
|
||||||
|
|
||||||
|
size_t current_offset = 0;
|
||||||
|
for (size_t i = 0; i < size; ++i)
|
||||||
|
{
|
||||||
|
if (filt[i])
|
||||||
|
{
|
||||||
|
current_offset += sizeAt(i);
|
||||||
|
res_offsets.push_back(current_offset);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
ColumnPtr ColumnArray::permute(const Permutation & perm, size_t limit) const
|
||||||
|
{
|
||||||
|
size_t size = getOffsets().size();
|
||||||
|
|
||||||
|
if (limit == 0)
|
||||||
|
limit = size;
|
||||||
|
else
|
||||||
|
limit = std::min(size, limit);
|
||||||
|
|
||||||
|
if (perm.size() < limit)
|
||||||
|
throw Exception("Size of permutation is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
if (limit == 0)
|
||||||
|
return new ColumnArray(data);
|
||||||
|
|
||||||
|
Permutation nested_perm(getOffsets().back());
|
||||||
|
|
||||||
|
ColumnArray * res_ = new ColumnArray(data->cloneEmpty());
|
||||||
|
ColumnPtr res = res_;
|
||||||
|
|
||||||
|
Offsets_t & res_offsets = res_->getOffsets();
|
||||||
|
res_offsets.resize(limit);
|
||||||
|
size_t current_offset = 0;
|
||||||
|
|
||||||
|
for (size_t i = 0; i < limit; ++i)
|
||||||
|
{
|
||||||
|
for (size_t j = 0; j < sizeAt(perm[i]); ++j)
|
||||||
|
nested_perm[current_offset + j] = offsetAt(perm[i]) + j;
|
||||||
|
current_offset += sizeAt(perm[i]);
|
||||||
|
res_offsets[i] = current_offset;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (current_offset != 0)
|
||||||
|
res_->data = data->permute(nested_perm, current_offset);
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void ColumnArray::getPermutation(bool reverse, size_t limit, Permutation & res) const
|
||||||
|
{
|
||||||
|
size_t s = size();
|
||||||
|
if (limit >= s)
|
||||||
|
limit = 0;
|
||||||
|
|
||||||
|
res.resize(s);
|
||||||
|
for (size_t i = 0; i < s; ++i)
|
||||||
|
res[i] = i;
|
||||||
|
|
||||||
|
if (limit)
|
||||||
|
{
|
||||||
|
if (reverse)
|
||||||
|
std::partial_sort(res.begin(), res.begin() + limit, res.end(), less<false>(*this));
|
||||||
|
else
|
||||||
|
std::partial_sort(res.begin(), res.begin() + limit, res.end(), less<true>(*this));
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
if (reverse)
|
||||||
|
std::sort(res.begin(), res.end(), less<false>(*this));
|
||||||
|
else
|
||||||
|
std::sort(res.begin(), res.end(), less<true>(*this));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
ColumnPtr ColumnArray::replicate(const Offsets_t & replicate_offsets) const
|
||||||
|
{
|
||||||
|
/// Не получается реализовать в общем случае.
|
||||||
|
|
||||||
|
if (typeid_cast<const ColumnUInt8 *>(&*data)) return replicate<UInt8>(replicate_offsets);
|
||||||
|
if (typeid_cast<const ColumnUInt16 *>(&*data)) return replicate<UInt16>(replicate_offsets);
|
||||||
|
if (typeid_cast<const ColumnUInt32 *>(&*data)) return replicate<UInt32>(replicate_offsets);
|
||||||
|
if (typeid_cast<const ColumnUInt64 *>(&*data)) return replicate<UInt64>(replicate_offsets);
|
||||||
|
if (typeid_cast<const ColumnInt8 *>(&*data)) return replicate<Int8>(replicate_offsets);
|
||||||
|
if (typeid_cast<const ColumnInt16 *>(&*data)) return replicate<Int16>(replicate_offsets);
|
||||||
|
if (typeid_cast<const ColumnInt32 *>(&*data)) return replicate<Int32>(replicate_offsets);
|
||||||
|
if (typeid_cast<const ColumnInt64 *>(&*data)) return replicate<Int64>(replicate_offsets);
|
||||||
|
if (typeid_cast<const ColumnFloat32 *>(&*data)) return replicate<Float32>(replicate_offsets);
|
||||||
|
if (typeid_cast<const ColumnFloat64 *>(&*data)) return replicate<Float64>(replicate_offsets);
|
||||||
|
if (typeid_cast<const ColumnString *>(&*data)) return replicateString(replicate_offsets);
|
||||||
|
if (dynamic_cast<const IColumnConst *>(&*data)) return replicateConst(replicate_offsets);
|
||||||
|
|
||||||
|
throw Exception("Replication of column " + getName() + " is not implemented.", ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
ColumnPtr ColumnArray::replicate(const Offsets_t & replicate_offsets) const
|
||||||
|
{
|
||||||
|
size_t col_size = size();
|
||||||
|
if (col_size != replicate_offsets.size())
|
||||||
|
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
ColumnPtr res = cloneEmpty();
|
||||||
|
|
||||||
|
if (0 == col_size)
|
||||||
|
return res;
|
||||||
|
|
||||||
|
ColumnArray & res_ = typeid_cast<ColumnArray &>(*res);
|
||||||
|
|
||||||
|
const typename ColumnVector<T>::Container_t & cur_data = typeid_cast<const ColumnVector<T> &>(*data).getData();
|
||||||
|
const Offsets_t & cur_offsets = getOffsets();
|
||||||
|
|
||||||
|
typename ColumnVector<T>::Container_t & res_data = typeid_cast<ColumnVector<T> &>(res_.getData()).getData();
|
||||||
|
Offsets_t & res_offsets = res_.getOffsets();
|
||||||
|
|
||||||
|
res_data.reserve(data->size() / col_size * replicate_offsets.back());
|
||||||
|
res_offsets.reserve(replicate_offsets.back());
|
||||||
|
|
||||||
|
Offset_t prev_replicate_offset = 0;
|
||||||
|
Offset_t prev_data_offset = 0;
|
||||||
|
Offset_t current_new_offset = 0;
|
||||||
|
|
||||||
|
for (size_t i = 0; i < col_size; ++i)
|
||||||
|
{
|
||||||
|
size_t size_to_replicate = replicate_offsets[i] - prev_replicate_offset;
|
||||||
|
size_t value_size = cur_offsets[i] - prev_data_offset;
|
||||||
|
|
||||||
|
for (size_t j = 0; j < size_to_replicate; ++j)
|
||||||
|
{
|
||||||
|
current_new_offset += value_size;
|
||||||
|
res_offsets.push_back(current_new_offset);
|
||||||
|
|
||||||
|
res_data.resize(res_data.size() + value_size);
|
||||||
|
memcpy(&res_data[res_data.size() - value_size], &cur_data[prev_data_offset], value_size * sizeof(T));
|
||||||
|
}
|
||||||
|
|
||||||
|
prev_replicate_offset = replicate_offsets[i];
|
||||||
|
prev_data_offset = cur_offsets[i];
|
||||||
|
}
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
ColumnPtr ColumnArray::replicateString(const Offsets_t & replicate_offsets) const
|
||||||
|
{
|
||||||
|
size_t col_size = size();
|
||||||
|
if (col_size != replicate_offsets.size())
|
||||||
|
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
ColumnPtr res = cloneEmpty();
|
||||||
|
|
||||||
|
if (0 == col_size)
|
||||||
|
return res;
|
||||||
|
|
||||||
|
ColumnArray & res_ = typeid_cast<ColumnArray &>(*res);
|
||||||
|
|
||||||
|
const ColumnString & cur_string = typeid_cast<const ColumnString &>(*data);
|
||||||
|
const ColumnString::Chars_t & cur_chars = cur_string.getChars();
|
||||||
|
const Offsets_t & cur_string_offsets = cur_string.getOffsets();
|
||||||
|
const Offsets_t & cur_offsets = getOffsets();
|
||||||
|
|
||||||
|
ColumnString::Chars_t & res_chars = typeid_cast<ColumnString &>(res_.getData()).getChars();
|
||||||
|
Offsets_t & res_string_offsets = typeid_cast<ColumnString &>(res_.getData()).getOffsets();
|
||||||
|
Offsets_t & res_offsets = res_.getOffsets();
|
||||||
|
|
||||||
|
res_chars.reserve(cur_chars.size() / col_size * replicate_offsets.back());
|
||||||
|
res_string_offsets.reserve(cur_string_offsets.size() / col_size * replicate_offsets.back());
|
||||||
|
res_offsets.reserve(replicate_offsets.back());
|
||||||
|
|
||||||
|
Offset_t prev_replicate_offset = 0;
|
||||||
|
|
||||||
|
Offset_t prev_cur_offset = 0;
|
||||||
|
Offset_t prev_cur_string_offset = 0;
|
||||||
|
|
||||||
|
Offset_t current_res_offset = 0;
|
||||||
|
Offset_t current_res_string_offset = 0;
|
||||||
|
|
||||||
|
for (size_t i = 0; i < col_size; ++i)
|
||||||
|
{
|
||||||
|
/// Насколько размножить массив.
|
||||||
|
size_t size_to_replicate = replicate_offsets[i] - prev_replicate_offset;
|
||||||
|
/// Количество строк в массиве.
|
||||||
|
size_t value_size = cur_offsets[i] - prev_cur_offset;
|
||||||
|
|
||||||
|
size_t sum_chars_size = 0;
|
||||||
|
|
||||||
|
for (size_t j = 0; j < size_to_replicate; ++j)
|
||||||
|
{
|
||||||
|
current_res_offset += value_size;
|
||||||
|
res_offsets.push_back(current_res_offset);
|
||||||
|
|
||||||
|
sum_chars_size = 0;
|
||||||
|
|
||||||
|
size_t prev_cur_string_offset_local = prev_cur_string_offset;
|
||||||
|
for (size_t k = 0; k < value_size; ++k)
|
||||||
|
{
|
||||||
|
/// Размер одной строки.
|
||||||
|
size_t chars_size = cur_string_offsets[k + prev_cur_offset] - prev_cur_string_offset_local;
|
||||||
|
|
||||||
|
current_res_string_offset += chars_size;
|
||||||
|
res_string_offsets.push_back(current_res_string_offset);
|
||||||
|
|
||||||
|
/// Копирование символов одной строки.
|
||||||
|
res_chars.resize(res_chars.size() + chars_size);
|
||||||
|
memcpy(&res_chars[res_chars.size() - chars_size], &cur_chars[prev_cur_string_offset_local], chars_size);
|
||||||
|
|
||||||
|
sum_chars_size += chars_size;
|
||||||
|
prev_cur_string_offset_local += chars_size;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
prev_replicate_offset = replicate_offsets[i];
|
||||||
|
prev_cur_offset = cur_offsets[i];
|
||||||
|
prev_cur_string_offset += sum_chars_size;
|
||||||
|
}
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
ColumnPtr ColumnArray::replicateConst(const Offsets_t & replicate_offsets) const
|
||||||
|
{
|
||||||
|
size_t col_size = size();
|
||||||
|
if (col_size != replicate_offsets.size())
|
||||||
|
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
if (0 == col_size)
|
||||||
|
return cloneEmpty();
|
||||||
|
|
||||||
|
const Offsets_t & cur_offsets = getOffsets();
|
||||||
|
|
||||||
|
ColumnOffsets_t * res_column_offsets = new ColumnOffsets_t;
|
||||||
|
ColumnPtr res_column_offsets_holder = res_column_offsets;
|
||||||
|
Offsets_t & res_offsets = res_column_offsets->getData();
|
||||||
|
res_offsets.reserve(replicate_offsets.back());
|
||||||
|
|
||||||
|
Offset_t prev_replicate_offset = 0;
|
||||||
|
Offset_t prev_data_offset = 0;
|
||||||
|
Offset_t current_new_offset = 0;
|
||||||
|
|
||||||
|
for (size_t i = 0; i < col_size; ++i)
|
||||||
|
{
|
||||||
|
size_t size_to_replicate = replicate_offsets[i] - prev_replicate_offset;
|
||||||
|
size_t value_size = cur_offsets[i] - prev_data_offset;
|
||||||
|
|
||||||
|
for (size_t j = 0; j < size_to_replicate; ++j)
|
||||||
|
{
|
||||||
|
current_new_offset += value_size;
|
||||||
|
res_offsets.push_back(current_new_offset);
|
||||||
|
}
|
||||||
|
|
||||||
|
prev_replicate_offset = replicate_offsets[i];
|
||||||
|
prev_data_offset = cur_offsets[i];
|
||||||
|
}
|
||||||
|
|
||||||
|
return new ColumnArray(getData().cloneResized(current_new_offset), res_column_offsets_holder);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
}
|
@ -21,6 +21,8 @@ void registerFunctionsArithmetic(FunctionFactory & factory)
|
|||||||
factory.registerFunction<FunctionBitNot>();
|
factory.registerFunction<FunctionBitNot>();
|
||||||
factory.registerFunction<FunctionBitShiftLeft>();
|
factory.registerFunction<FunctionBitShiftLeft>();
|
||||||
factory.registerFunction<FunctionBitShiftRight>();
|
factory.registerFunction<FunctionBitShiftRight>();
|
||||||
|
factory.registerFunction<FunctionLeast>();
|
||||||
|
factory.registerFunction<FunctionGreatest>();
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -20,6 +20,7 @@ void registerFunctionsConversion(FunctionFactory & factory)
|
|||||||
factory.registerFunction<FunctionToDateTime>();
|
factory.registerFunction<FunctionToDateTime>();
|
||||||
factory.registerFunction<FunctionToString>();
|
factory.registerFunction<FunctionToString>();
|
||||||
factory.registerFunction<FunctionToFixedString>();
|
factory.registerFunction<FunctionToFixedString>();
|
||||||
|
factory.registerFunction<FunctionToUnixTimestamp>();
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -210,7 +210,7 @@ void QueryLog::flush()
|
|||||||
{
|
{
|
||||||
LOG_TRACE(log, "Flushing query log");
|
LOG_TRACE(log, "Flushing query log");
|
||||||
|
|
||||||
DateLUT & date_lut = DateLUT::instance();
|
const auto & date_lut = DateLUT::instance();
|
||||||
|
|
||||||
Block block = createBlock();
|
Block block = createBlock();
|
||||||
|
|
||||||
|
@ -95,7 +95,7 @@ QueryParseResult QueryParser::parse(std::istream & s)
|
|||||||
if (result.date_first > result.date_last)
|
if (result.date_first > result.date_last)
|
||||||
throw Exception("First date is bigger than last date.", ErrorCodes::FIRST_DATE_IS_BIGGER_THAN_LAST_DATE);
|
throw Exception("First date is bigger than last date.", ErrorCodes::FIRST_DATE_IS_BIGGER_THAN_LAST_DATE);
|
||||||
|
|
||||||
DateLUT & date_lut = DateLUT::instance();
|
const auto & date_lut = DateLUT::instance();
|
||||||
result.days = 1 + date_lut.toDayNum(result.date_last) - date_lut.toDayNum(result.date_first);
|
result.days = 1 + date_lut.toDayNum(result.date_last) - date_lut.toDayNum(result.date_first);
|
||||||
|
|
||||||
result.cut_date_last = false;
|
result.cut_date_last = false;
|
||||||
|
@ -105,7 +105,7 @@ size_t ActiveDataPartSet::size() const
|
|||||||
|
|
||||||
String ActiveDataPartSet::getPartName(DayNum_t left_date, DayNum_t right_date, UInt64 left_id, UInt64 right_id, UInt64 level)
|
String ActiveDataPartSet::getPartName(DayNum_t left_date, DayNum_t right_date, UInt64 left_id, UInt64 right_id, UInt64 level)
|
||||||
{
|
{
|
||||||
DateLUT & date_lut = DateLUT::instance();
|
const auto & date_lut = DateLUT::instance();
|
||||||
|
|
||||||
/// Имя директории для куска иммет вид: YYYYMMDD_YYYYMMDD_N_N_L.
|
/// Имя директории для куска иммет вид: YYYYMMDD_YYYYMMDD_N_N_L.
|
||||||
String res;
|
String res;
|
||||||
@ -153,7 +153,7 @@ void ActiveDataPartSet::parsePartName(const String & file_name, Part & part, con
|
|||||||
|
|
||||||
const Poco::RegularExpression::MatchVec & matches = *matches_p;
|
const Poco::RegularExpression::MatchVec & matches = *matches_p;
|
||||||
|
|
||||||
DateLUT & date_lut = DateLUT::instance();
|
const auto & date_lut = DateLUT::instance();
|
||||||
|
|
||||||
part.left_date = date_lut.YYYYMMDDToDayNum(parse<UInt32>(file_name.substr(matches[1].offset, matches[1].length)));
|
part.left_date = date_lut.YYYYMMDDToDayNum(parse<UInt32>(file_name.substr(matches[1].offset, matches[1].length)));
|
||||||
part.right_date = date_lut.YYYYMMDDToDayNum(parse<UInt32>(file_name.substr(matches[2].offset, matches[2].length)));
|
part.right_date = date_lut.YYYYMMDDToDayNum(parse<UInt32>(file_name.substr(matches[2].offset, matches[2].length)));
|
||||||
|
@ -46,7 +46,7 @@ bool MergeTreeDataMerger::selectPartsToMerge(MergeTreeData::DataPartsVector & pa
|
|||||||
{
|
{
|
||||||
MergeTreeData::DataParts data_parts = data.getDataParts();
|
MergeTreeData::DataParts data_parts = data.getDataParts();
|
||||||
|
|
||||||
DateLUT & date_lut = DateLUT::instance();
|
const auto & date_lut = DateLUT::instance();
|
||||||
|
|
||||||
size_t min_max = -1U;
|
size_t min_max = -1U;
|
||||||
size_t min_min = -1U;
|
size_t min_min = -1U;
|
||||||
|
@ -8,6 +8,8 @@
|
|||||||
#include <DB/DataStreams/AddingConstColumnBlockInputStream.h>
|
#include <DB/DataStreams/AddingConstColumnBlockInputStream.h>
|
||||||
#include <DB/DataStreams/CreatingSetsBlockInputStream.h>
|
#include <DB/DataStreams/CreatingSetsBlockInputStream.h>
|
||||||
#include <DB/DataStreams/NullBlockInputStream.h>
|
#include <DB/DataStreams/NullBlockInputStream.h>
|
||||||
|
#include <DB/DataStreams/SummingSortedBlockInputStream.h>
|
||||||
|
#include <DB/DataStreams/AggregatingSortedBlockInputStream.h>
|
||||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||||
#include <DB/Common/VirtualColumnUtils.h>
|
#include <DB/Common/VirtualColumnUtils.h>
|
||||||
|
|
||||||
@ -292,7 +294,10 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
|
|||||||
/// Добавим столбцы, нужные для вычисления первичного ключа и знака.
|
/// Добавим столбцы, нужные для вычисления первичного ключа и знака.
|
||||||
std::vector<String> add_columns = data.getPrimaryExpression()->getRequiredColumns();
|
std::vector<String> add_columns = data.getPrimaryExpression()->getRequiredColumns();
|
||||||
column_names_to_read.insert(column_names_to_read.end(), add_columns.begin(), add_columns.end());
|
column_names_to_read.insert(column_names_to_read.end(), add_columns.begin(), add_columns.end());
|
||||||
column_names_to_read.push_back(data.sign_column);
|
|
||||||
|
if (!data.sign_column.empty())
|
||||||
|
column_names_to_read.push_back(data.sign_column);
|
||||||
|
|
||||||
std::sort(column_names_to_read.begin(), column_names_to_read.end());
|
std::sort(column_names_to_read.begin(), column_names_to_read.end());
|
||||||
column_names_to_read.erase(std::unique(column_names_to_read.begin(), column_names_to_read.end()), column_names_to_read.end());
|
column_names_to_read.erase(std::unique(column_names_to_read.begin(), column_names_to_read.end()), column_names_to_read.end());
|
||||||
|
|
||||||
@ -474,11 +479,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreadsFinal
|
|||||||
if (sum_marks > max_marks_to_use_cache)
|
if (sum_marks > max_marks_to_use_cache)
|
||||||
use_uncompressed_cache = false;
|
use_uncompressed_cache = false;
|
||||||
|
|
||||||
ExpressionActionsPtr sign_filter_expression;
|
BlockInputStreams to_merge;
|
||||||
String sign_filter_column;
|
|
||||||
createPositiveSignCondition(sign_filter_expression, sign_filter_column);
|
|
||||||
|
|
||||||
BlockInputStreams to_collapse;
|
|
||||||
|
|
||||||
for (size_t part_index = 0; part_index < parts.size(); ++part_index)
|
for (size_t part_index = 0; part_index < parts.size(); ++part_index)
|
||||||
{
|
{
|
||||||
@ -499,14 +500,51 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreadsFinal
|
|||||||
source_stream, new DataTypeUInt64, part.part_index_in_query, "_part_index");
|
source_stream, new DataTypeUInt64, part.part_index_in_query, "_part_index");
|
||||||
}
|
}
|
||||||
|
|
||||||
to_collapse.push_back(new ExpressionBlockInputStream(source_stream, data.getPrimaryExpression()));
|
to_merge.push_back(new ExpressionBlockInputStream(source_stream, data.getPrimaryExpression()));
|
||||||
}
|
}
|
||||||
|
|
||||||
BlockInputStreams res;
|
BlockInputStreams res;
|
||||||
if (to_collapse.size() == 1)
|
if (to_merge.size() == 1)
|
||||||
res.push_back(new FilterBlockInputStream(new ExpressionBlockInputStream(to_collapse[0], sign_filter_expression), sign_filter_column));
|
{
|
||||||
else if (to_collapse.size() > 1)
|
if (!data.sign_column.empty())
|
||||||
res.push_back(new CollapsingFinalBlockInputStream(to_collapse, data.getSortDescription(), data.sign_column));
|
{
|
||||||
|
ExpressionActionsPtr sign_filter_expression;
|
||||||
|
String sign_filter_column;
|
||||||
|
|
||||||
|
createPositiveSignCondition(sign_filter_expression, sign_filter_column);
|
||||||
|
|
||||||
|
res.push_back(new FilterBlockInputStream(new ExpressionBlockInputStream(to_merge[0], sign_filter_expression), sign_filter_column));
|
||||||
|
}
|
||||||
|
else
|
||||||
|
res = to_merge;
|
||||||
|
}
|
||||||
|
else if (to_merge.size() > 1)
|
||||||
|
{
|
||||||
|
BlockInputStreamPtr merged;
|
||||||
|
|
||||||
|
switch (data.mode)
|
||||||
|
{
|
||||||
|
case MergeTreeData::Ordinary:
|
||||||
|
throw Exception("Ordinary MergeTree doesn't support FINAL", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
case MergeTreeData::Collapsing:
|
||||||
|
merged = new CollapsingFinalBlockInputStream(to_merge, data.getSortDescription(), data.sign_column);
|
||||||
|
break;
|
||||||
|
|
||||||
|
case MergeTreeData::Summing:
|
||||||
|
merged = new SummingSortedBlockInputStream(to_merge, data.getSortDescription(), data.columns_to_sum, max_block_size);
|
||||||
|
break;
|
||||||
|
|
||||||
|
case MergeTreeData::Aggregating:
|
||||||
|
merged = new AggregatingSortedBlockInputStream(to_merge, data.getSortDescription(), max_block_size);
|
||||||
|
break;
|
||||||
|
|
||||||
|
case MergeTreeData::Unsorted:
|
||||||
|
throw Exception("UnsortedMergeTree doesn't support FINAL", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
|
||||||
|
res.push_back(merged);
|
||||||
|
}
|
||||||
|
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
@ -12,7 +12,7 @@ BlocksWithDateIntervals MergeTreeDataWriter::splitBlockIntoParts(const Block & b
|
|||||||
{
|
{
|
||||||
data.check(block, true);
|
data.check(block, true);
|
||||||
|
|
||||||
DateLUT & date_lut = DateLUT::instance();
|
const auto & date_lut = DateLUT::instance();
|
||||||
|
|
||||||
size_t rows = block.rows();
|
size_t rows = block.rows();
|
||||||
size_t columns = block.columns();
|
size_t columns = block.columns();
|
||||||
@ -77,7 +77,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithDa
|
|||||||
UInt16 min_date = block_with_dates.min_date;
|
UInt16 min_date = block_with_dates.min_date;
|
||||||
UInt16 max_date = block_with_dates.max_date;
|
UInt16 max_date = block_with_dates.max_date;
|
||||||
|
|
||||||
DateLUT & date_lut = DateLUT::instance();
|
const auto & date_lut = DateLUT::instance();
|
||||||
|
|
||||||
size_t part_size = (block.rows() + data.index_granularity - 1) / data.index_granularity;
|
size_t part_size = (block.rows() + data.index_granularity - 1) / data.index_granularity;
|
||||||
|
|
||||||
|
@ -2214,8 +2214,8 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params,
|
|||||||
static String getFakePartNameForDrop(const String & month_name, UInt64 left, UInt64 right)
|
static String getFakePartNameForDrop(const String & month_name, UInt64 left, UInt64 right)
|
||||||
{
|
{
|
||||||
/// Диапазон дат - весь месяц.
|
/// Диапазон дат - весь месяц.
|
||||||
DateLUT & lut = DateLUT::instance();
|
const auto & lut = DateLUT::instance();
|
||||||
time_t start_time = DateLUT::instance().YYYYMMDDToDate(parse<UInt32>(month_name + "01"));
|
time_t start_time = lut.YYYYMMDDToDate(parse<UInt32>(month_name + "01"));
|
||||||
DayNum_t left_date = lut.toDayNum(start_time);
|
DayNum_t left_date = lut.toDayNum(start_time);
|
||||||
DayNum_t right_date = DayNum_t(static_cast<size_t>(left_date) + lut.daysInMonth(start_time) - 1);
|
DayNum_t right_date = DayNum_t(static_cast<size_t>(left_date) + lut.daysInMonth(start_time) - 1);
|
||||||
|
|
||||||
|
85
dbms/tests/queries/0_stateless/00189_time_zones.reference
Normal file
85
dbms/tests/queries/0_stateless/00189_time_zones.reference
Normal file
@ -0,0 +1,85 @@
|
|||||||
|
2014-12-29
|
||||||
|
2014-12-22
|
||||||
|
2014-12-22
|
||||||
|
2014-12-29
|
||||||
|
2014-12-22
|
||||||
|
2014-12-01
|
||||||
|
2014-12-01
|
||||||
|
2014-12-01
|
||||||
|
2014-12-01
|
||||||
|
2014-12-01
|
||||||
|
2014-12-01
|
||||||
|
2014-12-01
|
||||||
|
2014-12-01
|
||||||
|
2014-12-01
|
||||||
|
2014-12-01
|
||||||
|
2014-07-01
|
||||||
|
2014-07-01
|
||||||
|
2014-07-01
|
||||||
|
2014-10-01
|
||||||
|
2014-07-01
|
||||||
|
1970-01-02 12:00:00 1970-01-02 12:00:00
|
||||||
|
1970-01-02 10:00:00 1970-01-02 11:00:00
|
||||||
|
1970-01-02 09:00:00 1970-01-02 10:00:00
|
||||||
|
1970-01-02 18:00:00 1970-01-02 18:00:00
|
||||||
|
1970-01-02 01:30:00 1970-01-02 01:30:00
|
||||||
|
2015-07-13 2015-07-01 2015-07-01 1970-01-02 19:30:00
|
||||||
|
2014-12-22 2014-12-01 2014-10-01 1970-01-02 21:00:00
|
||||||
|
2014-12-29 2015-01-01 2015-01-01 1970-01-02 12:00:00
|
||||||
|
2014-09-29 2014-09-01 2014-07-01 1970-01-02 21:50:00
|
||||||
|
2015-03-09 2015-03-01 2015-01-01 1970-01-02 02:00:00
|
||||||
|
2014-09-29 2014-09-01 2014-10-01 1970-01-02 11:20:00
|
||||||
|
2014-12-22 2014-12-01 2014-10-01 1970-01-02 13:30:00
|
||||||
|
2014-12-29 2015-01-01 2015-01-01 1970-01-02 01:30:00
|
||||||
|
2015-03-09 2015-03-01 2015-01-01 1970-01-02 02:00:00
|
||||||
|
2015-07-13 2015-07-01 2015-07-01 1970-01-02 02:00:00
|
||||||
|
2015-03-09 2015-03-01 2015-01-01 1970-01-02 19:30:00
|
||||||
|
2015-03-09 2015-03-01 2015-01-01 1970-01-02 10:30:00
|
||||||
|
2015-03-09 2015-03-01 2015-01-01 1970-01-02 13:30:00
|
||||||
|
2015-03-09 2015-03-01 2015-01-01 1970-01-02 11:30:00
|
||||||
|
2015-03-09 2015-03-01 2015-01-01 1970-01-02 02:00:00
|
||||||
|
2015-07-15 13:30:00
|
||||||
|
2015-07-15 12:30:00
|
||||||
|
2015-07-15 11:30:00
|
||||||
|
2015-07-15 19:30:00
|
||||||
|
2015-07-15 02:30:00
|
||||||
|
2015-07-15 19:30:00
|
||||||
|
2014-12-28 21:00:00
|
||||||
|
2015-01-01 12:00:00
|
||||||
|
2014-09-30 21:50:00
|
||||||
|
2015-03-15 02:30:00
|
||||||
|
2015-07-15 19:30:00
|
||||||
|
2015-07-15 11:30:00
|
||||||
|
2015-07-15 13:30:00
|
||||||
|
2015-07-15 12:30:00
|
||||||
|
2015-07-15 02:30:00
|
||||||
|
2014-09-30 20:50:00
|
||||||
|
2014-12-28 21:00:00
|
||||||
|
2015-01-01 09:00:00
|
||||||
|
2015-03-15 10:30:00
|
||||||
|
2015-07-15 11:30:00
|
||||||
|
1426415400
|
||||||
|
1426422600
|
||||||
|
1426426200
|
||||||
|
1426393800
|
||||||
|
1426455000
|
||||||
|
1426415400
|
||||||
|
1426415400
|
||||||
|
1426415400
|
||||||
|
1426415400
|
||||||
|
1426415400
|
||||||
|
1436934600
|
||||||
|
1419811200
|
||||||
|
1420102800
|
||||||
|
1412113800
|
||||||
|
1426455000
|
||||||
|
1426393800
|
||||||
|
1426426200
|
||||||
|
1426415400
|
||||||
|
1426422600
|
||||||
|
1426455000
|
||||||
|
1412113800
|
||||||
|
1419807600
|
||||||
|
1420110000
|
||||||
|
1426422600
|
||||||
|
1436959800
|
89
dbms/tests/queries/0_stateless/00189_time_zones.sql
Normal file
89
dbms/tests/queries/0_stateless/00189_time_zones.sql
Normal file
@ -0,0 +1,89 @@
|
|||||||
|
|
||||||
|
/* timestamp 1419800400 == 2014-12-29 00:00:00 (Europe/Moscow) */
|
||||||
|
/* timestamp 1412106600 == 2014-09-30 23:50:00 (Europe/Moscow) */
|
||||||
|
/* timestamp 1420102800 == 2015-01-01 12:00:00 (Europe/Moscow) */
|
||||||
|
/* timestamp 1428310800 == 2015-04-06 12:00:00 (Europe/Moscow) */
|
||||||
|
/* timestamp 1436956200 == 2015-07-15 13:30:00 (Europe/Moscow) */
|
||||||
|
/* timestamp 1426415400 == 2015-03-15 13:30:00 (Europe/Moscow) */
|
||||||
|
|
||||||
|
/* toMonday */
|
||||||
|
|
||||||
|
SELECT toMonday(toDateTime(1419800400), 'Europe/Moscow');
|
||||||
|
SELECT toMonday(toDateTime(1419800400), 'Europe/Paris');
|
||||||
|
SELECT toMonday(toDateTime(1419800400), 'Europe/London');
|
||||||
|
SELECT toMonday(toDateTime(1419800400), 'Asia/Tokyo');
|
||||||
|
SELECT toMonday(toDateTime(1419800400), 'Pacific/Pitcairn');
|
||||||
|
|
||||||
|
/* toStartOfMonth */
|
||||||
|
|
||||||
|
SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/Moscow');
|
||||||
|
SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/Paris');
|
||||||
|
SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/London');
|
||||||
|
SELECT toStartOfMonth(toDateTime(1419800400), 'Asia/Tokyo');
|
||||||
|
SELECT toStartOfMonth(toDateTime(1419800400), 'Pacific/Pitcairn');
|
||||||
|
|
||||||
|
/* toStartOfQuarter */
|
||||||
|
|
||||||
|
SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/Moscow');
|
||||||
|
SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/Paris');
|
||||||
|
SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/London');
|
||||||
|
SELECT toStartOfMonth(toDateTime(1419800400), 'Asia/Tokyo');
|
||||||
|
SELECT toStartOfMonth(toDateTime(1419800400), 'Pacific/Pitcairn');
|
||||||
|
|
||||||
|
/* toStartOfYear */
|
||||||
|
|
||||||
|
SELECT toStartOfQuarter(toDateTime(1412106600), 'Europe/Moscow');
|
||||||
|
SELECT toStartOfQuarter(toDateTime(1412106600), 'Europe/Paris');
|
||||||
|
SELECT toStartOfQuarter(toDateTime(1412106600), 'Europe/London');
|
||||||
|
SELECT toStartOfQuarter(toDateTime(1412106600), 'Asia/Tokyo');
|
||||||
|
SELECT toStartOfQuarter(toDateTime(1412106600), 'Pacific/Pitcairn');
|
||||||
|
|
||||||
|
/* toTime */
|
||||||
|
|
||||||
|
SELECT toTime(toDateTime(1420102800), 'Europe/Moscow'), toTime(toDateTime(1428310800), 'Europe/Moscow');
|
||||||
|
SELECT toTime(toDateTime(1420102800), 'Europe/Paris'), toTime(toDateTime(1428310800), 'Europe/Paris');
|
||||||
|
SELECT toTime(toDateTime(1420102800), 'Europe/London'), toTime(toDateTime(1428310800), 'Europe/London');
|
||||||
|
SELECT toTime(toDateTime(1420102800), 'Asia/Tokyo'), toTime(toDateTime(1428310800), 'Asia/Tokyo');
|
||||||
|
SELECT toTime(toDateTime(1420102800), 'Pacific/Pitcairn'), toTime(toDateTime(1428310800), 'Pacific/Pitcairn');
|
||||||
|
|
||||||
|
DROP TABLE IF EXISTS foo;
|
||||||
|
CREATE TABLE foo(x Int32, y String) ENGINE=Memory;
|
||||||
|
INSERT INTO foo(x, y) VALUES(1420102800, 'Europe/Moscow');
|
||||||
|
INSERT INTO foo(x, y) VALUES(1412106600, 'Europe/Paris');
|
||||||
|
INSERT INTO foo(x, y) VALUES(1419800400, 'Europe/London');
|
||||||
|
INSERT INTO foo(x, y) VALUES(1436956200, 'Asia/Tokyo');
|
||||||
|
INSERT INTO foo(x, y) VALUES(1426415400, 'Pacific/Pitcairn');
|
||||||
|
|
||||||
|
SELECT toMonday(toDateTime(x), y), toStartOfMonth(toDateTime(x), y), toStartOfQuarter(toDateTime(x), y), toTime(toDateTime(x), y) FROM foo ORDER BY y ASC;
|
||||||
|
SELECT toMonday(toDateTime(x), 'Europe/Paris'), toStartOfMonth(toDateTime(x), 'Europe/London'), toStartOfQuarter(toDateTime(x), 'Asia/Tokyo'), toTime(toDateTime(x), 'Pacific/Pitcairn') FROM foo ORDER BY x ASC;
|
||||||
|
SELECT toMonday(toDateTime(1426415400), y), toStartOfMonth(toDateTime(1426415400), y), toStartOfQuarter(toDateTime(1426415400), y), toTime(toDateTime(1426415400), y) FROM foo ORDER BY y ASC;
|
||||||
|
|
||||||
|
/* toString */
|
||||||
|
|
||||||
|
SELECT toString(toDateTime(1436956200), 'Europe/Moscow');
|
||||||
|
SELECT toString(toDateTime(1436956200), 'Europe/Paris');
|
||||||
|
SELECT toString(toDateTime(1436956200), 'Europe/London');
|
||||||
|
SELECT toString(toDateTime(1436956200), 'Asia/Tokyo');
|
||||||
|
SELECT toString(toDateTime(1436956200), 'Pacific/Pitcairn');
|
||||||
|
|
||||||
|
SELECT toString(toDateTime(x), y) FROM foo ORDER BY y ASC;
|
||||||
|
SELECT toString(toDateTime(1436956200), y) FROM foo ORDER BY y ASC;
|
||||||
|
SELECT toString(toDateTime(x), 'Europe/London') FROM foo ORDER BY x ASC;
|
||||||
|
|
||||||
|
/* toUnixTimestamp */
|
||||||
|
|
||||||
|
SELECT toUnixTimestamp(toString(toDateTime(1426415400)), 'Europe/Moscow');
|
||||||
|
SELECT toUnixTimestamp(toString(toDateTime(1426415400)), 'Europe/Paris');
|
||||||
|
SELECT toUnixTimestamp(toString(toDateTime(1426415400)), 'Europe/London');
|
||||||
|
SELECT toUnixTimestamp(toString(toDateTime(1426415400)), 'Asia/Tokyo');
|
||||||
|
SELECT toUnixTimestamp(toString(toDateTime(1426415400)), 'Pacific/Pitcairn');
|
||||||
|
|
||||||
|
SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Europe/Moscow'), 'Europe/Moscow');
|
||||||
|
SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Europe/Paris'), 'Europe/Paris');
|
||||||
|
SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Europe/London'), 'Europe/London');
|
||||||
|
SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Asia/Tokyo'), 'Asia/Tokyo');
|
||||||
|
SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Pacific/Pitcairn'), 'Pacific/Pitcairn');
|
||||||
|
|
||||||
|
SELECT toUnixTimestamp(toString(toDateTime(x)), y) FROM foo ORDER BY y ASC;
|
||||||
|
SELECT toUnixTimestamp(toString(toDateTime(1426415400)), y) FROM foo ORDER BY y ASC;
|
||||||
|
SELECT toUnixTimestamp(toString(toDateTime(x)), 'Europe/Paris') FROM foo ORDER BY x ASC;
|
@ -0,0 +1,8 @@
|
|||||||
|
['']
|
||||||
|
---
|
||||||
|
---
|
||||||
|
0 ['hello']
|
||||||
|
---
|
||||||
|
---
|
||||||
|
0 []
|
||||||
|
---
|
@ -0,0 +1,11 @@
|
|||||||
|
SELECT arrayFilter(x -> notEmpty(concat(x, 'hello')), ['']) ARRAY JOIN [0] AS elem, arrayMap(x -> concat(x, 'hello'), ['']) AS unused WHERE NOT ignore(elem);
|
||||||
|
SELECT '---';
|
||||||
|
SELECT arrayFilter(x -> x = 'hello', ['']) ARRAY JOIN [0] AS elem WHERE NOT ignore(elem) AND arrayExists(x -> x = 'hello', ['']);
|
||||||
|
SELECT '---';
|
||||||
|
SELECT arrayJoin([0]), replicate('hello', [1]) WHERE NOT ignore(replicate('hello', [1]));
|
||||||
|
SELECT '---';
|
||||||
|
SELECT arrayJoin([0]), replicate('hello', emptyArrayString()) ARRAY JOIN emptyArrayString() AS unused WHERE NOT ignore(replicate('hello', emptyArrayString()));
|
||||||
|
SELECT '---';
|
||||||
|
SELECT arrayJoin([0]), replicate('hello', emptyArrayString()) WHERE NOT ignore(replicate('hello', emptyArrayString()));
|
||||||
|
SELECT '---';
|
||||||
|
SELECT replicate('hello', emptyArrayString()) ARRAY JOIN emptyArrayString() AS unused WHERE NOT ignore(replicate('hello', emptyArrayString()));
|
@ -0,0 +1,45 @@
|
|||||||
|
0 100
|
||||||
|
1 100
|
||||||
|
2 100
|
||||||
|
3 100
|
||||||
|
4 100
|
||||||
|
5 150
|
||||||
|
6 150
|
||||||
|
7 150
|
||||||
|
8 150
|
||||||
|
9 150
|
||||||
|
10 100
|
||||||
|
11 100
|
||||||
|
12 100
|
||||||
|
13 100
|
||||||
|
14 100
|
||||||
|
0 100
|
||||||
|
1 100
|
||||||
|
2 100
|
||||||
|
3 100
|
||||||
|
4 100
|
||||||
|
5 150
|
||||||
|
6 150
|
||||||
|
7 150
|
||||||
|
8 150
|
||||||
|
9 150
|
||||||
|
10 100
|
||||||
|
11 100
|
||||||
|
12 100
|
||||||
|
13 100
|
||||||
|
14 100
|
||||||
|
0 100
|
||||||
|
1 100
|
||||||
|
2 100
|
||||||
|
3 100
|
||||||
|
4 100
|
||||||
|
5 150
|
||||||
|
6 150
|
||||||
|
7 150
|
||||||
|
8 150
|
||||||
|
9 150
|
||||||
|
10 100
|
||||||
|
11 100
|
||||||
|
12 100
|
||||||
|
13 100
|
||||||
|
14 100
|
@ -0,0 +1,14 @@
|
|||||||
|
DROP TABLE IF EXISTS test.aggregating;
|
||||||
|
CREATE TABLE test.aggregating (d Date DEFAULT '2000-01-01', k UInt64, u AggregateFunction(uniq, UInt64)) ENGINE = AggregatingMergeTree(d, k, 8192);
|
||||||
|
|
||||||
|
INSERT INTO test.aggregating SELECT intDiv(number, 100) AS k, uniqState(toUInt64(number % 100)) AS u FROM (SELECT * FROM system.numbers LIMIT 1000) GROUP BY k;
|
||||||
|
INSERT INTO test.aggregating SELECT intDiv(number, 100) AS k, uniqState(toUInt64(number % 100) + 50) AS u FROM (SELECT * FROM system.numbers LIMIT 500, 1000) GROUP BY k;
|
||||||
|
|
||||||
|
SELECT k, finalizeAggregation(u) FROM test.aggregating FINAL;
|
||||||
|
|
||||||
|
OPTIMIZE TABLE test.aggregating;
|
||||||
|
|
||||||
|
SELECT k, finalizeAggregation(u) FROM test.aggregating;
|
||||||
|
SELECT k, finalizeAggregation(u) FROM test.aggregating FINAL;
|
||||||
|
|
||||||
|
DROP TABLE test.aggregating;
|
@ -0,0 +1,12 @@
|
|||||||
|
1 2 1 2 1 2 2 UInt8
|
||||||
|
1.1 2 1.1 2 1.1 2 2 Float64
|
||||||
|
-1 2 -1 2 -1 2 2 Int16
|
||||||
|
1 2 1 2 1 2 2 Float64
|
||||||
|
1 2000 1 2000 1 2000 2000 UInt16
|
||||||
|
1 200000 1 200000 1 200000 200000 UInt32
|
||||||
|
1 20000000000 1 20000000000 1 20000000000 20000000000 UInt64
|
||||||
|
123 123 123 123 123 123 123 UInt8
|
||||||
|
2010-01-02 2011-02-03 2010-01-02 2011-02-03 2010-01-02 2011-02-03 2011-02-03 Date
|
||||||
|
2010-01-02 03:04:05 2011-02-03 04:05:06 2010-01-02 03:04:05 2011-02-03 04:05:06 2010-01-02 03:04:05 2011-02-03 04:05:06 2011-02-03 04:05:06 DateTime
|
||||||
|
10
|
||||||
|
9
|
12
dbms/tests/queries/0_stateless/00192_least_greatest.sql
Normal file
12
dbms/tests/queries/0_stateless/00192_least_greatest.sql
Normal file
@ -0,0 +1,12 @@
|
|||||||
|
SELECT 1 AS x, 2 AS y, least(x, y), greatest(x, y), least(x, materialize(y)), greatest(materialize(x), y), greatest(materialize(x), materialize(y)), toTypeName(least(x, y));
|
||||||
|
SELECT 1.1 AS x, 2 AS y, least(x, y), greatest(x, y), least(x, materialize(y)), greatest(materialize(x), y), greatest(materialize(x), materialize(y)), toTypeName(least(x, y));
|
||||||
|
SELECT -1 AS x, 2 AS y, least(x, y), greatest(x, y), least(x, materialize(y)), greatest(materialize(x), y), greatest(materialize(x), materialize(y)), toTypeName(least(x, y));
|
||||||
|
SELECT 1.0 AS x, 2.0 AS y, least(x, y), greatest(x, y), least(x, materialize(y)), greatest(materialize(x), y), greatest(materialize(x), materialize(y)), toTypeName(least(x, y));
|
||||||
|
SELECT 1 AS x, 2000 AS y, least(x, y), greatest(x, y), least(x, materialize(y)), greatest(materialize(x), y), greatest(materialize(x), materialize(y)), toTypeName(least(x, y));
|
||||||
|
SELECT 1 AS x, 200000 AS y, least(x, y), greatest(x, y), least(x, materialize(y)), greatest(materialize(x), y), greatest(materialize(x), materialize(y)), toTypeName(least(x, y));
|
||||||
|
SELECT 1 AS x, 20000000000 AS y, least(x, y), greatest(x, y), least(x, materialize(y)), greatest(materialize(x), y), greatest(materialize(x), materialize(y)), toTypeName(least(x, y));
|
||||||
|
SELECT 123 AS x, 123 AS y, least(x, y), greatest(x, y), least(x, materialize(y)), greatest(materialize(x), y), greatest(materialize(x), materialize(y)), toTypeName(least(x, y));
|
||||||
|
SELECT toDate('2010-01-02') AS x, toDate('2011-02-03') AS y, least(x, y), greatest(x, y), least(x, materialize(y)), greatest(materialize(x), y), greatest(materialize(x), materialize(y)), toTypeName(least(x, y));
|
||||||
|
SELECT toDateTime('2010-01-02 03:04:05') AS x, toDateTime('2011-02-03 04:05:06') AS y, least(x, y), greatest(x, y), least(x, materialize(y)), greatest(materialize(x), y), greatest(materialize(x), materialize(y)), toTypeName(least(x, y));
|
||||||
|
SELECT greatest(now(), now() + 10) - now();
|
||||||
|
SELECT greatest(today(), yesterday() + 10) - today();
|
@ -1,57 +1,109 @@
|
|||||||
#include <cstring>
|
|
||||||
#include <Yandex/DateLUT.h>
|
#include <Yandex/DateLUT.h>
|
||||||
#include <Poco/Exception.h>
|
|
||||||
|
|
||||||
|
#include <unicode/timezone.h>
|
||||||
|
#include <unicode/unistr.h>
|
||||||
|
|
||||||
DateLUT::DateLUT()
|
DateLUT::DateLUT()
|
||||||
{
|
{
|
||||||
size_t i = 0;
|
using namespace icu;
|
||||||
time_t start_of_day = DATE_LUT_MIN;
|
|
||||||
|
|
||||||
do
|
std::unique_ptr<TimeZone> tz(TimeZone::createDefault());
|
||||||
|
if (tz == nullptr)
|
||||||
|
throw Poco::Exception("Failed to determine the host time zone.");
|
||||||
|
|
||||||
|
UnicodeString u_out;
|
||||||
|
tz->getID(u_out);
|
||||||
|
std::string default_time_zone;
|
||||||
|
u_out.toUTF8String(default_time_zone);
|
||||||
|
|
||||||
|
std::unique_ptr<StringEnumeration> time_zone_ids(TimeZone::createEnumeration());
|
||||||
|
if (time_zone_ids == nullptr)
|
||||||
|
throw Poco::Exception("Failed to query the list of time zones.");
|
||||||
|
|
||||||
|
UErrorCode status = U_ZERO_ERROR;
|
||||||
|
const UnicodeString * zone_id = time_zone_ids->snext(status);
|
||||||
|
if (zone_id == nullptr)
|
||||||
|
throw Poco::Exception("No time zone available.");
|
||||||
|
|
||||||
|
std::vector<UnicodeString> time_zones;
|
||||||
|
while ((zone_id != nullptr) && (status == U_ZERO_ERROR))
|
||||||
{
|
{
|
||||||
if (i > DATE_LUT_MAX_DAY_NUM)
|
time_zones.push_back(*zone_id);
|
||||||
throw Poco::Exception("Cannot create DateLUT: i > DATE_LUT_MAX_DAY_NUM.");
|
zone_id = time_zone_ids->snext(status);
|
||||||
|
|
||||||
tm time_descr;
|
|
||||||
localtime_r(&start_of_day, &time_descr);
|
|
||||||
|
|
||||||
time_descr.tm_hour = 0;
|
|
||||||
time_descr.tm_min = 0;
|
|
||||||
time_descr.tm_sec = 0;
|
|
||||||
time_descr.tm_isdst = -1;
|
|
||||||
|
|
||||||
start_of_day = mktime(&time_descr);
|
|
||||||
|
|
||||||
Values & values = lut[i];
|
|
||||||
|
|
||||||
values.year = time_descr.tm_year + 1900;
|
|
||||||
values.month = time_descr.tm_mon + 1;
|
|
||||||
values.day_of_week = time_descr.tm_wday == 0 ? 7 : time_descr.tm_wday;
|
|
||||||
values.day_of_month = time_descr.tm_mday;
|
|
||||||
|
|
||||||
values.date = start_of_day;
|
|
||||||
|
|
||||||
/// Переходим на следующий день.
|
|
||||||
++time_descr.tm_mday;
|
|
||||||
|
|
||||||
/** Обратите внимание, что в 1981-1984 году в России,
|
|
||||||
* 1 апреля начиналось в час ночи, а не в полночь.
|
|
||||||
* Если здесь оставить час равным нулю, то прибавление единицы к дню, привело бы к 23 часам того же дня.
|
|
||||||
*/
|
|
||||||
time_descr.tm_hour = 12;
|
|
||||||
start_of_day = mktime(&time_descr);
|
|
||||||
|
|
||||||
++i;
|
|
||||||
} while (start_of_day <= DATE_LUT_MAX);
|
|
||||||
|
|
||||||
/// Заполняем lookup таблицу для годов
|
|
||||||
memset(years_lut, 0, DATE_LUT_YEARS * sizeof(years_lut[0]));
|
|
||||||
for (size_t day = 0; day < i && 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;
|
size_t group_id = 0;
|
||||||
|
|
||||||
|
for (const auto & time_zone : time_zones)
|
||||||
|
{
|
||||||
|
const UnicodeString & u_group_name = TimeZone::getEquivalentID(time_zone, 0);
|
||||||
|
std::string group_name;
|
||||||
|
|
||||||
|
if (u_group_name.isEmpty())
|
||||||
|
{
|
||||||
|
time_zone.toUTF8String(group_name);
|
||||||
|
|
||||||
|
auto res = time_zone_to_group.insert(std::make_pair(group_name, group_id));
|
||||||
|
if (!res.second)
|
||||||
|
throw Poco::Exception("Failed to initialize time zone information.");
|
||||||
|
++group_id;
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
u_group_name.toUTF8String(group_name);
|
||||||
|
|
||||||
|
auto it = time_zone_to_group.find(group_name);
|
||||||
|
if (it == time_zone_to_group.end())
|
||||||
|
{
|
||||||
|
auto count = TimeZone::countEquivalentIDs(time_zone);
|
||||||
|
if (count == 0)
|
||||||
|
throw Poco::Exception("Inconsistent time zone information.");
|
||||||
|
|
||||||
|
for (auto i = 0; i < count; ++i)
|
||||||
|
{
|
||||||
|
const UnicodeString & u_name = TimeZone::getEquivalentID(time_zone, i);
|
||||||
|
std::string name;
|
||||||
|
u_name.toUTF8String(name);
|
||||||
|
auto res = time_zone_to_group.insert(std::make_pair(name, group_id));
|
||||||
|
if (!res.second)
|
||||||
|
throw Poco::Exception("Failed to initialize time zone information.");
|
||||||
|
}
|
||||||
|
++group_id;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (group_id == 0)
|
||||||
|
throw Poco::Exception("Could not find any time zone information.");
|
||||||
|
|
||||||
|
date_lut_impl_list = std::make_unique<DateLUTImplList>(group_id);
|
||||||
|
|
||||||
|
/// Инициализация указателя на реализацию для часового пояса по-умолчанию.
|
||||||
|
auto it = time_zone_to_group.find(default_time_zone);
|
||||||
|
if (it == time_zone_to_group.end())
|
||||||
|
throw Poco::Exception("Failed to get default time zone information.");
|
||||||
|
default_group_id = it->second;
|
||||||
|
|
||||||
|
default_date_lut_impl = new DateLUTImpl(default_time_zone);
|
||||||
|
auto & wrapper = (*date_lut_impl_list)[default_group_id];
|
||||||
|
wrapper.store(default_date_lut_impl, std::memory_order_seq_cst);
|
||||||
|
}
|
||||||
|
|
||||||
|
const DateLUTImpl & DateLUT::getImplementation(const std::string & time_zone, size_t group_id) const
|
||||||
|
{
|
||||||
|
auto & wrapper = (*date_lut_impl_list)[group_id];
|
||||||
|
|
||||||
|
DateLUTImpl * tmp = wrapper.load(std::memory_order_acquire);
|
||||||
|
if (tmp == nullptr)
|
||||||
|
{
|
||||||
|
std::lock_guard<std::mutex> guard(mutex);
|
||||||
|
tmp = wrapper.load(std::memory_order_relaxed);
|
||||||
|
if (tmp == nullptr)
|
||||||
|
{
|
||||||
|
tmp = new DateLUTImpl(time_zone);
|
||||||
|
wrapper.store(tmp, std::memory_order_release);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return *tmp;
|
||||||
}
|
}
|
||||||
|
143
libs/libcommon/src/DateLUTImpl.cpp
Normal file
143
libs/libcommon/src/DateLUTImpl.cpp
Normal file
@ -0,0 +1,143 @@
|
|||||||
|
#include <Yandex/DateLUTImpl.h>
|
||||||
|
#include <Poco/Exception.h>
|
||||||
|
|
||||||
|
#include <memory>
|
||||||
|
#include <cstring>
|
||||||
|
#include <glib.h>
|
||||||
|
|
||||||
|
namespace details { namespace {
|
||||||
|
|
||||||
|
struct GTimeZoneUnref
|
||||||
|
{
|
||||||
|
void operator()(GTimeZone * tz) const
|
||||||
|
{
|
||||||
|
g_time_zone_unref(tz);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
using GTimeZonePtr = std::unique_ptr<GTimeZone, GTimeZoneUnref>;
|
||||||
|
|
||||||
|
struct GDateTimeUnref
|
||||||
|
{
|
||||||
|
void operator()(GDateTime * dt) const
|
||||||
|
{
|
||||||
|
g_date_time_unref(dt);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
using GDateTimePtr = std::unique_ptr<GDateTime, GDateTimeUnref>;
|
||||||
|
|
||||||
|
GTimeZonePtr createGTimeZone(const std::string & description)
|
||||||
|
{
|
||||||
|
GTimeZone * tz = g_time_zone_new(description.c_str());
|
||||||
|
if (tz == nullptr)
|
||||||
|
throw Poco::Exception("Failed to create GTimeZone object.");
|
||||||
|
|
||||||
|
return GTimeZonePtr(tz);
|
||||||
|
}
|
||||||
|
|
||||||
|
GDateTimePtr createGDateTime(time_t timestamp)
|
||||||
|
{
|
||||||
|
GDateTime * dt= g_date_time_new_from_unix_utc(timestamp);
|
||||||
|
if (dt == nullptr)
|
||||||
|
throw Poco::Exception("Failed to create GDateTime object.");
|
||||||
|
|
||||||
|
return GDateTimePtr(dt);
|
||||||
|
}
|
||||||
|
|
||||||
|
GDateTimePtr createGDateTime(const GTimeZonePtr & p_tz, const GDateTimePtr & p_dt)
|
||||||
|
{
|
||||||
|
GDateTime * dt = p_dt.get();
|
||||||
|
if (dt == nullptr)
|
||||||
|
throw Poco::Exception("Null pointer.");
|
||||||
|
|
||||||
|
gint year;
|
||||||
|
gint month;
|
||||||
|
gint day;
|
||||||
|
g_date_time_get_ymd(dt, &year, &month, &day);
|
||||||
|
|
||||||
|
GDateTime * local_dt = g_date_time_new(p_tz.get(), year, month, day,
|
||||||
|
g_date_time_get_hour(dt),
|
||||||
|
g_date_time_get_minute(dt),
|
||||||
|
g_date_time_get_second(dt));
|
||||||
|
if (local_dt == nullptr)
|
||||||
|
throw Poco::Exception("Failed to create GDateTime object.");
|
||||||
|
|
||||||
|
return GDateTimePtr(local_dt);
|
||||||
|
}
|
||||||
|
|
||||||
|
GDateTimePtr toNextDay(const GTimeZonePtr & p_tz, const GDateTimePtr & p_dt)
|
||||||
|
{
|
||||||
|
GDateTime * dt = p_dt.get();
|
||||||
|
if (dt == nullptr)
|
||||||
|
throw Poco::Exception("Null pointer.");
|
||||||
|
|
||||||
|
dt = g_date_time_add_days(dt, 1);
|
||||||
|
if (dt == nullptr)
|
||||||
|
throw Poco::Exception("Failed to create GDateTime object.");
|
||||||
|
|
||||||
|
GDateTimePtr p_next_dt = GDateTimePtr(dt);
|
||||||
|
GDateTime * next_dt = p_next_dt.get();
|
||||||
|
|
||||||
|
gint year;
|
||||||
|
gint month;
|
||||||
|
gint day;
|
||||||
|
g_date_time_get_ymd(next_dt, &year, &month, &day);
|
||||||
|
|
||||||
|
dt = g_date_time_new(p_tz.get(), year, month, day, 0, 0, 0);
|
||||||
|
if (dt == nullptr)
|
||||||
|
throw Poco::Exception("Failed to create GDateTime object.");
|
||||||
|
|
||||||
|
return GDateTimePtr(dt);
|
||||||
|
}
|
||||||
|
|
||||||
|
}}
|
||||||
|
|
||||||
|
DateLUTImpl::DateLUTImpl(const std::string & time_zone)
|
||||||
|
{
|
||||||
|
details::GTimeZonePtr p_tz = details::createGTimeZone(time_zone);
|
||||||
|
|
||||||
|
size_t i = 0;
|
||||||
|
time_t start_of_day = DATE_LUT_MIN;
|
||||||
|
|
||||||
|
details::GDateTimePtr p_dt = details::createGDateTime(start_of_day);
|
||||||
|
|
||||||
|
p_dt = details::createGDateTime(p_tz, p_dt);
|
||||||
|
|
||||||
|
do
|
||||||
|
{
|
||||||
|
if (i > DATE_LUT_MAX_DAY_NUM)
|
||||||
|
throw Poco::Exception("Cannot create DateLUTImpl: i > DATE_LUT_MAX_DAY_NUM.");
|
||||||
|
|
||||||
|
GDateTime * dt = p_dt.get();
|
||||||
|
|
||||||
|
start_of_day = g_date_time_to_unix(dt);
|
||||||
|
|
||||||
|
gint year;
|
||||||
|
gint month;
|
||||||
|
gint day;
|
||||||
|
g_date_time_get_ymd(dt, &year, &month, &day);
|
||||||
|
|
||||||
|
Values & values = lut[i];
|
||||||
|
values.year = year;
|
||||||
|
values.month = month;
|
||||||
|
values.day_of_month = day;
|
||||||
|
values.day_of_week = g_date_time_get_day_of_week(dt);
|
||||||
|
values.date = start_of_day;
|
||||||
|
|
||||||
|
/// Переходим на следующий день.
|
||||||
|
p_dt = details::toNextDay(p_tz, p_dt);
|
||||||
|
++i;
|
||||||
|
}
|
||||||
|
while (start_of_day <= DATE_LUT_MAX);
|
||||||
|
|
||||||
|
/// Заполняем lookup таблицу для годов
|
||||||
|
::memset(years_lut, 0, DATE_LUT_YEARS * sizeof(years_lut[0]));
|
||||||
|
for (size_t day = 0; day < i && 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 = g_time_zone_get_offset(p_tz.get(), g_time_zone_find_interval(p_tz.get(), G_TIME_TYPE_UNIVERSAL, 0));
|
||||||
|
}
|
@ -33,7 +33,7 @@ static time_t orderedIdentifierToDate(unsigned value)
|
|||||||
|
|
||||||
void loop(time_t begin, time_t end, int step)
|
void loop(time_t begin, time_t end, int step)
|
||||||
{
|
{
|
||||||
DateLUT & date_lut = DateLUT::instance();
|
const auto & date_lut = DateLUT::instance();
|
||||||
|
|
||||||
for (time_t t = begin; t < end; t += step)
|
for (time_t t = begin; t < end; t += step)
|
||||||
std::cout << toString(t)
|
std::cout << toString(t)
|
||||||
|
@ -35,7 +35,7 @@ static time_t orderedIdentifierToDate(unsigned value)
|
|||||||
|
|
||||||
void loop(time_t begin, time_t end, int step)
|
void loop(time_t begin, time_t end, int step)
|
||||||
{
|
{
|
||||||
DateLUT & date_lut = DateLUT::instance();
|
const auto & date_lut = DateLUT::instance();
|
||||||
|
|
||||||
for (time_t t = begin; t < end; t += step)
|
for (time_t t = begin; t < end; t += step)
|
||||||
{
|
{
|
||||||
|
@ -7,12 +7,12 @@ int main(int argc, char ** argv)
|
|||||||
/** В DateLUT был глюк - для времён из дня 1970-01-01, возвращался номер часа больше 23. */
|
/** В DateLUT был глюк - для времён из дня 1970-01-01, возвращался номер часа больше 23. */
|
||||||
static const time_t TIME = 66130;
|
static const time_t TIME = 66130;
|
||||||
|
|
||||||
DateLUT & date_lut = DateLUT::instance();
|
const auto & date_lut = DateLUT::instance();
|
||||||
|
|
||||||
std::cerr << date_lut.toHourInaccurate(TIME) << std::endl;
|
std::cerr << date_lut.toHourInaccurate(TIME) << std::endl;
|
||||||
std::cerr << date_lut.toDayNum(TIME) << std::endl;
|
std::cerr << date_lut.toDayNum(TIME) << std::endl;
|
||||||
|
|
||||||
const DateLUT::Values * values = reinterpret_cast<const DateLUT::Values *>(&date_lut);
|
const auto * values = reinterpret_cast<const DateLUTImpl::Values *>(&date_lut);
|
||||||
|
|
||||||
std::cerr << values[0].date << ", " << time_t(values[1].date - values[0].date) << std::endl;
|
std::cerr << values[0].date << ", " << time_t(values[1].date - values[0].date) << std::endl;
|
||||||
|
|
||||||
|
@ -29,8 +29,8 @@ private:
|
|||||||
|
|
||||||
void init(time_t time)
|
void init(time_t time)
|
||||||
{
|
{
|
||||||
DateLUT & date_lut = DateLUT::instance();
|
const auto & date_lut = DateLUT::instance();
|
||||||
const DateLUT::Values & values = date_lut.getValues(time);
|
const auto & values = date_lut.getValues(time);
|
||||||
|
|
||||||
m_year = values.year;
|
m_year = values.year;
|
||||||
m_month = values.month;
|
m_month = values.month;
|
||||||
@ -66,7 +66,7 @@ public:
|
|||||||
|
|
||||||
Date(DayNum_t day_num)
|
Date(DayNum_t day_num)
|
||||||
{
|
{
|
||||||
const DateLUT::Values & values = DateLUT::instance().getValues(day_num);
|
const auto & values = DateLUT::instance().getValues(day_num);
|
||||||
m_year = values.year;
|
m_year = values.year;
|
||||||
m_month = values.month;
|
m_month = values.month;
|
||||||
m_day = values.day_of_month;
|
m_day = values.day_of_month;
|
||||||
|
@ -43,8 +43,8 @@ private:
|
|||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
DateLUT & date_lut = DateLUT::instance();
|
const auto & date_lut = DateLUT::instance();
|
||||||
const DateLUT::Values & values = date_lut.getValues(time);
|
const auto & values = date_lut.getValues(time);
|
||||||
|
|
||||||
m_year = values.year;
|
m_year = values.year;
|
||||||
m_month = values.month;
|
m_month = values.month;
|
||||||
|
@ -145,7 +145,7 @@ private:
|
|||||||
|
|
||||||
time_t getDateTimeImpl() const
|
time_t getDateTimeImpl() const
|
||||||
{
|
{
|
||||||
DateLUT & date_lut = DateLUT::instance();
|
const auto & date_lut = DateLUT::instance();
|
||||||
|
|
||||||
if (m_length == 10)
|
if (m_length == 10)
|
||||||
{
|
{
|
||||||
@ -173,7 +173,7 @@ private:
|
|||||||
|
|
||||||
time_t getDateImpl() const
|
time_t getDateImpl() const
|
||||||
{
|
{
|
||||||
DateLUT & date_lut = DateLUT::instance();
|
const auto & date_lut = DateLUT::instance();
|
||||||
|
|
||||||
if (m_length == 10 || m_length == 19)
|
if (m_length == 10 || m_length == 19)
|
||||||
{
|
{
|
||||||
@ -216,7 +216,7 @@ private:
|
|||||||
return getDateImpl();
|
return getDateImpl();
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
DateLUT & date_lut = DateLUT::instance();
|
const auto & date_lut = DateLUT::instance();
|
||||||
return date_lut.toDate(getIntImpl());
|
return date_lut.toDate(getIntImpl());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1 +1 @@
|
|||||||
https://github.com/Cyan4973/zstd/tree/765207c54934d478488c236749b01c7d6fc63d70/
|
https://github.com/Cyan4973/zstd/tree/1eca5f52994434d3b0427c9014403cf01495f54a/
|
||||||
|
@ -241,14 +241,11 @@ typedef struct
|
|||||||
int deltaFindState;
|
int deltaFindState;
|
||||||
U16 maxState;
|
U16 maxState;
|
||||||
BYTE minBitsOut;
|
BYTE minBitsOut;
|
||||||
/* one byte padding */
|
/* one byte padding ; total 8 bytes */
|
||||||
} FSE_symbolCompressionTransform;
|
} FSE_symbolCompressionTransform;
|
||||||
|
|
||||||
typedef struct
|
typedef U32 CTable_max_t[FSE_CTABLE_SIZE_U32(FSE_MAX_TABLELOG, FSE_MAX_SYMBOL_VALUE)];
|
||||||
{
|
typedef U32 DTable_max_t[FSE_DTABLE_SIZE_U32(FSE_MAX_TABLELOG)];
|
||||||
U32 fakeTable[FSE_CTABLE_SIZE_U32(FSE_MAX_TABLELOG, FSE_MAX_SYMBOL_VALUE)]; /* compatible with FSE_compressU16() */
|
|
||||||
} CTable_max_t;
|
|
||||||
|
|
||||||
|
|
||||||
/****************************************************************
|
/****************************************************************
|
||||||
* Internal functions
|
* Internal functions
|
||||||
@ -299,13 +296,15 @@ static short FSE_abs(short a)
|
|||||||
/****************************************************************
|
/****************************************************************
|
||||||
* Header bitstream management
|
* Header bitstream management
|
||||||
****************************************************************/
|
****************************************************************/
|
||||||
size_t FSE_headerBound(unsigned maxSymbolValue, unsigned tableLog)
|
size_t FSE_NCountWriteBound(unsigned maxSymbolValue, unsigned tableLog)
|
||||||
{
|
{
|
||||||
size_t maxHeaderSize = (((maxSymbolValue+1) * tableLog) >> 3) + 1;
|
size_t maxHeaderSize = (((maxSymbolValue+1) * tableLog) >> 3) + 1;
|
||||||
return maxSymbolValue ? maxHeaderSize : FSE_MAX_HEADERSIZE;
|
return maxSymbolValue ? maxHeaderSize : FSE_MAX_HEADERSIZE;
|
||||||
}
|
}
|
||||||
|
|
||||||
static size_t FSE_writeHeader_generic (void* header, size_t headerBufferSize,
|
#ifndef __clang_analyzer__ /* clang static analyzer has difficulties with this function : seems to believe normalizedCounter is uninitialized */
|
||||||
|
|
||||||
|
static size_t FSE_writeNCount_generic (void* header, size_t headerBufferSize,
|
||||||
const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog,
|
const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog,
|
||||||
unsigned safeWrite)
|
unsigned safeWrite)
|
||||||
{
|
{
|
||||||
@ -341,9 +340,9 @@ static size_t FSE_writeHeader_generic (void* header, size_t headerBufferSize,
|
|||||||
while (charnum >= start+24)
|
while (charnum >= start+24)
|
||||||
{
|
{
|
||||||
start+=24;
|
start+=24;
|
||||||
bitStream += 0xFFFF<<bitCount;
|
bitStream += 0xFFFFU << bitCount;
|
||||||
if ((!safeWrite) && (out > oend-2)) return (size_t)-FSE_ERROR_GENERIC; /* Buffer overflow */
|
if ((!safeWrite) && (out > oend-2)) return (size_t)-FSE_ERROR_GENERIC; /* Buffer overflow */
|
||||||
out[0] = (BYTE)bitStream;
|
out[0] = (BYTE) bitStream;
|
||||||
out[1] = (BYTE)(bitStream>>8);
|
out[1] = (BYTE)(bitStream>>8);
|
||||||
out+=2;
|
out+=2;
|
||||||
bitStream>>=16;
|
bitStream>>=16;
|
||||||
@ -370,7 +369,7 @@ static size_t FSE_writeHeader_generic (void* header, size_t headerBufferSize,
|
|||||||
short count = normalizedCounter[charnum++];
|
short count = normalizedCounter[charnum++];
|
||||||
const short max = (short)((2*threshold-1)-remaining);
|
const short max = (short)((2*threshold-1)-remaining);
|
||||||
remaining -= FSE_abs(count);
|
remaining -= FSE_abs(count);
|
||||||
if (remaining<0) return (size_t)-FSE_ERROR_GENERIC;
|
if (remaining<1) return (size_t)-FSE_ERROR_GENERIC;
|
||||||
count++; /* +1 for extra accuracy */
|
count++; /* +1 for extra accuracy */
|
||||||
if (count>=threshold) count += max; /* [0..max[ [max..threshold[ (...) [threshold+max 2*threshold[ */
|
if (count>=threshold) count += max; /* [0..max[ [max..threshold[ (...) [threshold+max 2*threshold[ */
|
||||||
bitStream += count << bitCount;
|
bitStream += count << bitCount;
|
||||||
@ -400,24 +399,26 @@ static size_t FSE_writeHeader_generic (void* header, size_t headerBufferSize,
|
|||||||
|
|
||||||
return (out-ostart);
|
return (out-ostart);
|
||||||
}
|
}
|
||||||
|
#endif // __clang_analyzer__
|
||||||
|
|
||||||
|
|
||||||
size_t FSE_writeHeader (void* header, size_t headerBufferSize, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog)
|
size_t FSE_writeNCount (void* header, size_t headerBufferSize, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog)
|
||||||
{
|
{
|
||||||
if (tableLog > FSE_MAX_TABLELOG) return (size_t)-FSE_ERROR_GENERIC; /* Unsupported */
|
if (tableLog > FSE_MAX_TABLELOG) return (size_t)-FSE_ERROR_GENERIC; /* Unsupported */
|
||||||
if (tableLog < FSE_MIN_TABLELOG) return (size_t)-FSE_ERROR_GENERIC; /* Unsupported */
|
if (tableLog < FSE_MIN_TABLELOG) return (size_t)-FSE_ERROR_GENERIC; /* Unsupported */
|
||||||
|
|
||||||
if (headerBufferSize < FSE_headerBound(maxSymbolValue, tableLog))
|
if (headerBufferSize < FSE_NCountWriteBound(maxSymbolValue, tableLog))
|
||||||
return FSE_writeHeader_generic(header, headerBufferSize, normalizedCounter, maxSymbolValue, tableLog, 0);
|
return FSE_writeNCount_generic(header, headerBufferSize, normalizedCounter, maxSymbolValue, tableLog, 0);
|
||||||
|
|
||||||
return FSE_writeHeader_generic(header, headerBufferSize, normalizedCounter, maxSymbolValue, tableLog, 1);
|
return FSE_writeNCount_generic(header, headerBufferSize, normalizedCounter, maxSymbolValue, tableLog, 1);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
size_t FSE_readHeader (short* normalizedCounter, unsigned* maxSVPtr, unsigned* tableLogPtr,
|
size_t FSE_readNCount (short* normalizedCounter, unsigned* maxSVPtr, unsigned* tableLogPtr,
|
||||||
const void* headerBuffer, size_t hbSize)
|
const void* headerBuffer, size_t hbSize)
|
||||||
{
|
{
|
||||||
const BYTE* const istart = (const BYTE*) headerBuffer;
|
const BYTE* const istart = (const BYTE*) headerBuffer;
|
||||||
|
const BYTE* const iend = istart + hbSize;
|
||||||
const BYTE* ip = istart;
|
const BYTE* ip = istart;
|
||||||
int nbBits;
|
int nbBits;
|
||||||
int remaining;
|
int remaining;
|
||||||
@ -427,6 +428,7 @@ size_t FSE_readHeader (short* normalizedCounter, unsigned* maxSVPtr, unsigned* t
|
|||||||
unsigned charnum = 0;
|
unsigned charnum = 0;
|
||||||
int previous0 = 0;
|
int previous0 = 0;
|
||||||
|
|
||||||
|
if (hbSize < 4) return (size_t)-FSE_ERROR_srcSize_wrong;
|
||||||
bitStream = FSE_readLE32(ip);
|
bitStream = FSE_readLE32(ip);
|
||||||
nbBits = (bitStream & 0xF) + FSE_MIN_TABLELOG; /* extract tableLog */
|
nbBits = (bitStream & 0xF) + FSE_MIN_TABLELOG; /* extract tableLog */
|
||||||
if (nbBits > FSE_TABLELOG_ABSOLUTE_MAX) return (size_t)-FSE_ERROR_tableLog_tooLarge;
|
if (nbBits > FSE_TABLELOG_ABSOLUTE_MAX) return (size_t)-FSE_ERROR_tableLog_tooLarge;
|
||||||
@ -456,7 +458,7 @@ size_t FSE_readHeader (short* normalizedCounter, unsigned* maxSVPtr, unsigned* t
|
|||||||
}
|
}
|
||||||
n0 += bitStream & 3;
|
n0 += bitStream & 3;
|
||||||
bitCount += 2;
|
bitCount += 2;
|
||||||
if (n0 > *maxSVPtr) return (size_t)-FSE_ERROR_GENERIC;
|
if (n0 > *maxSVPtr) return (size_t)-FSE_ERROR_maxSymbolValue_tooSmall;
|
||||||
while (charnum < n0) normalizedCounter[charnum++] = 0;
|
while (charnum < n0) normalizedCounter[charnum++] = 0;
|
||||||
ip += bitCount>>3;
|
ip += bitCount>>3;
|
||||||
bitCount &= 7;
|
bitCount &= 7;
|
||||||
@ -488,16 +490,27 @@ size_t FSE_readHeader (short* normalizedCounter, unsigned* maxSVPtr, unsigned* t
|
|||||||
threshold >>= 1;
|
threshold >>= 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
ip += bitCount>>3;
|
{
|
||||||
bitCount &= 7;
|
const BYTE* itarget = ip + (bitCount>>3);
|
||||||
bitStream = FSE_readLE32(ip) >> bitCount;
|
if (itarget > iend - 4)
|
||||||
|
{
|
||||||
|
ip = iend - 4;
|
||||||
|
bitCount -= (int)(8 * (iend - 4 - ip));
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
ip = itarget;
|
||||||
|
bitCount &= 7;
|
||||||
|
}
|
||||||
|
bitStream = FSE_readLE32(ip) >> (bitCount & 31);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (remaining != 1) return (size_t)-FSE_ERROR_GENERIC;
|
if (remaining != 1) return (size_t)-FSE_ERROR_GENERIC;
|
||||||
*maxSVPtr = charnum-1;
|
*maxSVPtr = charnum-1;
|
||||||
|
|
||||||
ip += bitCount>0;
|
ip += (bitCount+7)>>3;
|
||||||
if ((size_t)(ip-istart) >= hbSize) return (size_t)-FSE_ERROR_srcSize_wrong; /* arguably a bit late , tbd */
|
if ((size_t)(ip-istart) > hbSize) return (size_t)-FSE_ERROR_srcSize_wrong;
|
||||||
return ip-istart;
|
return ip-istart;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -506,7 +519,7 @@ size_t FSE_readHeader (short* normalizedCounter, unsigned* maxSVPtr, unsigned* t
|
|||||||
* FSE Compression Code
|
* FSE Compression Code
|
||||||
****************************************************************/
|
****************************************************************/
|
||||||
/*
|
/*
|
||||||
CTable is a variable size structure which contains :
|
FSE_CTable[0] is a variable size structure which contains :
|
||||||
U16 tableLog;
|
U16 tableLog;
|
||||||
U16 maxSymbolValue;
|
U16 maxSymbolValue;
|
||||||
U16 nextStateNumber[1 << tableLog]; // This size is variable
|
U16 nextStateNumber[1 << tableLog]; // This size is variable
|
||||||
@ -523,17 +536,17 @@ size_t FSE_sizeof_CTable (unsigned maxSymbolValue, unsigned tableLog)
|
|||||||
return size;
|
return size;
|
||||||
}
|
}
|
||||||
|
|
||||||
void* FSE_createCTable (unsigned maxSymbolValue, unsigned tableLog)
|
FSE_CTable* FSE_createCTable (unsigned maxSymbolValue, unsigned tableLog)
|
||||||
{
|
{
|
||||||
size_t size;
|
size_t size;
|
||||||
if (tableLog > FSE_TABLELOG_ABSOLUTE_MAX) tableLog = FSE_TABLELOG_ABSOLUTE_MAX;
|
if (tableLog > FSE_TABLELOG_ABSOLUTE_MAX) tableLog = FSE_TABLELOG_ABSOLUTE_MAX;
|
||||||
size = FSE_CTABLE_SIZE_U32 (tableLog, maxSymbolValue) * sizeof(U32);
|
size = FSE_CTABLE_SIZE_U32 (tableLog, maxSymbolValue) * sizeof(U32);
|
||||||
return malloc(size);
|
return (FSE_CTable*)malloc(size);
|
||||||
}
|
}
|
||||||
|
|
||||||
void FSE_freeCTable (void* CTable)
|
void FSE_freeCTable (FSE_CTable* ct)
|
||||||
{
|
{
|
||||||
free(CTable);
|
free(ct);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -542,7 +555,7 @@ unsigned FSE_optimalTableLog(unsigned maxTableLog, size_t srcSize, unsigned maxS
|
|||||||
U32 tableLog = maxTableLog;
|
U32 tableLog = maxTableLog;
|
||||||
if (tableLog==0) tableLog = FSE_DEFAULT_TABLELOG;
|
if (tableLog==0) tableLog = FSE_DEFAULT_TABLELOG;
|
||||||
if ((FSE_highbit32((U32)(srcSize - 1)) - 2) < tableLog) tableLog = FSE_highbit32((U32)(srcSize - 1)) - 2; /* Accuracy can be reduced */
|
if ((FSE_highbit32((U32)(srcSize - 1)) - 2) < tableLog) tableLog = FSE_highbit32((U32)(srcSize - 1)) - 2; /* Accuracy can be reduced */
|
||||||
if ((FSE_highbit32(maxSymbolValue+1)+1) > tableLog) tableLog = FSE_highbit32(maxSymbolValue+1)+1; /* Need a minimum to safely represent all symbol values */
|
if ((FSE_highbit32(maxSymbolValue)+2) > tableLog) tableLog = FSE_highbit32(maxSymbolValue)+2; /* Need a minimum to safely represent all symbol values */
|
||||||
if (tableLog < FSE_MIN_TABLELOG) tableLog = FSE_MIN_TABLELOG;
|
if (tableLog < FSE_MIN_TABLELOG) tableLog = FSE_MIN_TABLELOG;
|
||||||
if (tableLog > FSE_MAX_TABLELOG) tableLog = FSE_MAX_TABLELOG;
|
if (tableLog > FSE_MAX_TABLELOG) tableLog = FSE_MAX_TABLELOG;
|
||||||
return tableLog;
|
return tableLog;
|
||||||
@ -671,7 +684,7 @@ static size_t FSE_normalizeM2(short* norm, U32 tableLog, const unsigned* count,
|
|||||||
U32 maxV = 0, maxC =0;
|
U32 maxV = 0, maxC =0;
|
||||||
for (s=0; s<=maxSymbolValue; s++)
|
for (s=0; s<=maxSymbolValue; s++)
|
||||||
if (count[s] > maxC) maxV=s, maxC=count[s];
|
if (count[s] > maxC) maxV=s, maxC=count[s];
|
||||||
norm[maxV] += ToDistribute;
|
norm[maxV] += (short)ToDistribute;
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -690,7 +703,7 @@ static size_t FSE_normalizeM2(short* norm, U32 tableLog, const unsigned* count,
|
|||||||
U32 weight = sEnd - sStart;
|
U32 weight = sEnd - sStart;
|
||||||
if (weight < 1)
|
if (weight < 1)
|
||||||
return (size_t)-FSE_ERROR_GENERIC;
|
return (size_t)-FSE_ERROR_GENERIC;
|
||||||
norm[s] = weight;
|
norm[s] = (short)weight;
|
||||||
tmpTotal = end;
|
tmpTotal = end;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -779,19 +792,18 @@ size_t FSE_normalizeCount (short* normalizedCounter, unsigned tableLog,
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/* fake CTable, for raw (uncompressed) input */
|
/* fake FSE_CTable, for raw (uncompressed) input */
|
||||||
size_t FSE_buildCTable_raw (void* CTable, unsigned nbBits)
|
size_t FSE_buildCTable_raw (FSE_CTable* ct, unsigned nbBits)
|
||||||
{
|
{
|
||||||
const unsigned tableSize = 1 << nbBits;
|
const unsigned tableSize = 1 << nbBits;
|
||||||
const unsigned tableMask = tableSize - 1;
|
const unsigned tableMask = tableSize - 1;
|
||||||
const unsigned maxSymbolValue = tableMask;
|
const unsigned maxSymbolValue = tableMask;
|
||||||
U16* tableU16 = ( (U16*) CTable) + 2;
|
U16* tableU16 = ( (U16*) ct) + 2;
|
||||||
FSE_symbolCompressionTransform* symbolTT = (FSE_symbolCompressionTransform*) ((((U32*)CTable)+1) + (tableSize>>1));
|
FSE_symbolCompressionTransform* symbolTT = (FSE_symbolCompressionTransform*) ((((U32*)ct)+1) + (tableSize>>1));
|
||||||
unsigned s;
|
unsigned s;
|
||||||
|
|
||||||
/* Sanity checks */
|
/* Sanity checks */
|
||||||
if (nbBits < 1) return (size_t)-FSE_ERROR_GENERIC; /* min size */
|
if (nbBits < 1) return (size_t)-FSE_ERROR_GENERIC; /* min size */
|
||||||
if (((size_t)CTable) & 3) return (size_t)-FSE_ERROR_GENERIC; /* Must be allocated of 4 bytes boundaries */
|
|
||||||
|
|
||||||
/* header */
|
/* header */
|
||||||
tableU16[-2] = (U16) nbBits;
|
tableU16[-2] = (U16) nbBits;
|
||||||
@ -813,15 +825,12 @@ size_t FSE_buildCTable_raw (void* CTable, unsigned nbBits)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/* fake CTable, for rle (100% always same symbol) input */
|
/* fake FSE_CTable, for rle (100% always same symbol) input */
|
||||||
size_t FSE_buildCTable_rle (void* CTable, BYTE symbolValue)
|
size_t FSE_buildCTable_rle (FSE_CTable* ct, BYTE symbolValue)
|
||||||
{
|
{
|
||||||
const unsigned tableSize = 1;
|
const unsigned tableSize = 1;
|
||||||
U16* tableU16 = ( (U16*) CTable) + 2;
|
U16* tableU16 = ( (U16*) ct) + 2;
|
||||||
FSE_symbolCompressionTransform* symbolTT = (FSE_symbolCompressionTransform*) ((U32*)CTable + 2);
|
FSE_symbolCompressionTransform* symbolTT = (FSE_symbolCompressionTransform*) ((U32*)ct + 2);
|
||||||
|
|
||||||
/* safety checks */
|
|
||||||
if (((size_t)CTable) & 3) return (size_t)-FSE_ERROR_GENERIC; /* Must be 4 bytes aligned */
|
|
||||||
|
|
||||||
/* header */
|
/* header */
|
||||||
tableU16[-2] = (U16) 0;
|
tableU16[-2] = (U16) 0;
|
||||||
@ -850,12 +859,12 @@ void FSE_initCStream(FSE_CStream_t* bitC, void* start)
|
|||||||
bitC->ptr = bitC->startPtr;
|
bitC->ptr = bitC->startPtr;
|
||||||
}
|
}
|
||||||
|
|
||||||
void FSE_initCState(FSE_CState_t* statePtr, const void* CTable)
|
void FSE_initCState(FSE_CState_t* statePtr, const FSE_CTable* ct)
|
||||||
{
|
{
|
||||||
const U32 tableLog = ( (U16*) CTable) [0];
|
const U32 tableLog = ( (const U16*) ct) [0];
|
||||||
statePtr->value = (ptrdiff_t)1<<tableLog;
|
statePtr->value = (ptrdiff_t)1<<tableLog;
|
||||||
statePtr->stateTable = ((const U16*) CTable) + 2;
|
statePtr->stateTable = ((const U16*) ct) + 2;
|
||||||
statePtr->symbolTT = (const U32*)CTable + 1 + (tableLog ? (1<<(tableLog-1)) : 1);
|
statePtr->symbolTT = (const FSE_symbolCompressionTransform*)((const U32*)ct + 1 + (tableLog ? (1<<(tableLog-1)) : 1));
|
||||||
statePtr->stateLog = tableLog;
|
statePtr->stateLog = tableLog;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -866,14 +875,14 @@ void FSE_addBits(FSE_CStream_t* bitC, size_t value, unsigned nbBits)
|
|||||||
bitC->bitPos += nbBits;
|
bitC->bitPos += nbBits;
|
||||||
}
|
}
|
||||||
|
|
||||||
void FSE_encodeByte(FSE_CStream_t* bitC, FSE_CState_t* statePtr, BYTE symbol)
|
void FSE_encodeSymbol(FSE_CStream_t* bitC, FSE_CState_t* statePtr, BYTE symbol)
|
||||||
{
|
{
|
||||||
const FSE_symbolCompressionTransform* const symbolTT = (const FSE_symbolCompressionTransform*) statePtr->symbolTT;
|
const FSE_symbolCompressionTransform symbolTT = ((const FSE_symbolCompressionTransform*)(statePtr->symbolTT))[symbol];
|
||||||
const U16* const stateTable = (const U16*) statePtr->stateTable;
|
const U16* const stateTable = (const U16*)(statePtr->stateTable);
|
||||||
int nbBitsOut = symbolTT[symbol].minBitsOut;
|
int nbBitsOut = symbolTT.minBitsOut;
|
||||||
nbBitsOut -= (int)((symbolTT[symbol].maxState - statePtr->value) >> 31);
|
nbBitsOut -= (int)((symbolTT.maxState - statePtr->value) >> 31);
|
||||||
FSE_addBits(bitC, statePtr->value, nbBitsOut);
|
FSE_addBits(bitC, statePtr->value, nbBitsOut);
|
||||||
statePtr->value = stateTable[ (statePtr->value >> nbBitsOut) + symbolTT[symbol].deltaFindState];
|
statePtr->value = stateTable[ (statePtr->value >> nbBitsOut) + symbolTT.deltaFindState];
|
||||||
}
|
}
|
||||||
|
|
||||||
void FSE_flushBits(FSE_CStream_t* bitC)
|
void FSE_flushBits(FSE_CStream_t* bitC)
|
||||||
@ -908,7 +917,7 @@ size_t FSE_closeCStream(FSE_CStream_t* bitC)
|
|||||||
|
|
||||||
size_t FSE_compress_usingCTable (void* dst, size_t dstSize,
|
size_t FSE_compress_usingCTable (void* dst, size_t dstSize,
|
||||||
const void* src, size_t srcSize,
|
const void* src, size_t srcSize,
|
||||||
const void* CTable)
|
const FSE_CTable* ct)
|
||||||
{
|
{
|
||||||
const BYTE* const istart = (const BYTE*) src;
|
const BYTE* const istart = (const BYTE*) src;
|
||||||
const BYTE* ip;
|
const BYTE* ip;
|
||||||
@ -921,7 +930,7 @@ size_t FSE_compress_usingCTable (void* dst, size_t dstSize,
|
|||||||
/* init */
|
/* init */
|
||||||
(void)dstSize; /* objective : ensure it fits into dstBuffer (Todo) */
|
(void)dstSize; /* objective : ensure it fits into dstBuffer (Todo) */
|
||||||
FSE_initCStream(&bitC, dst);
|
FSE_initCStream(&bitC, dst);
|
||||||
FSE_initCState(&CState1, CTable);
|
FSE_initCState(&CState1, ct);
|
||||||
CState2 = CState1;
|
CState2 = CState1;
|
||||||
|
|
||||||
ip=iend;
|
ip=iend;
|
||||||
@ -929,32 +938,32 @@ size_t FSE_compress_usingCTable (void* dst, size_t dstSize,
|
|||||||
/* join to even */
|
/* join to even */
|
||||||
if (srcSize & 1)
|
if (srcSize & 1)
|
||||||
{
|
{
|
||||||
FSE_encodeByte(&bitC, &CState1, *--ip);
|
FSE_encodeSymbol(&bitC, &CState1, *--ip);
|
||||||
FSE_flushBits(&bitC);
|
FSE_flushBits(&bitC);
|
||||||
}
|
}
|
||||||
|
|
||||||
/* join to mod 4 */
|
/* join to mod 4 */
|
||||||
if ((sizeof(size_t)*8 > FSE_MAX_TABLELOG*4+7 ) && (srcSize & 2)) /* test bit 2 */
|
if ((sizeof(size_t)*8 > FSE_MAX_TABLELOG*4+7 ) && (srcSize & 2)) /* test bit 2 */
|
||||||
{
|
{
|
||||||
FSE_encodeByte(&bitC, &CState2, *--ip);
|
FSE_encodeSymbol(&bitC, &CState2, *--ip);
|
||||||
FSE_encodeByte(&bitC, &CState1, *--ip);
|
FSE_encodeSymbol(&bitC, &CState1, *--ip);
|
||||||
FSE_flushBits(&bitC);
|
FSE_flushBits(&bitC);
|
||||||
}
|
}
|
||||||
|
|
||||||
/* 2 or 4 encoding per loop */
|
/* 2 or 4 encoding per loop */
|
||||||
while (ip>istart)
|
while (ip>istart)
|
||||||
{
|
{
|
||||||
FSE_encodeByte(&bitC, &CState2, *--ip);
|
FSE_encodeSymbol(&bitC, &CState2, *--ip);
|
||||||
|
|
||||||
if (sizeof(size_t)*8 < FSE_MAX_TABLELOG*2+7 ) /* this test must be static */
|
if (sizeof(size_t)*8 < FSE_MAX_TABLELOG*2+7 ) /* this test must be static */
|
||||||
FSE_flushBits(&bitC);
|
FSE_flushBits(&bitC);
|
||||||
|
|
||||||
FSE_encodeByte(&bitC, &CState1, *--ip);
|
FSE_encodeSymbol(&bitC, &CState1, *--ip);
|
||||||
|
|
||||||
if (sizeof(size_t)*8 > FSE_MAX_TABLELOG*4+7 ) /* this test must be static */
|
if (sizeof(size_t)*8 > FSE_MAX_TABLELOG*4+7 ) /* this test must be static */
|
||||||
{
|
{
|
||||||
FSE_encodeByte(&bitC, &CState2, *--ip);
|
FSE_encodeSymbol(&bitC, &CState2, *--ip);
|
||||||
FSE_encodeByte(&bitC, &CState1, *--ip);
|
FSE_encodeSymbol(&bitC, &CState1, *--ip);
|
||||||
}
|
}
|
||||||
|
|
||||||
FSE_flushBits(&bitC);
|
FSE_flushBits(&bitC);
|
||||||
@ -980,7 +989,7 @@ size_t FSE_compress2 (void* dst, size_t dstSize, const void* src, size_t srcSize
|
|||||||
|
|
||||||
U32 count[FSE_MAX_SYMBOL_VALUE+1];
|
U32 count[FSE_MAX_SYMBOL_VALUE+1];
|
||||||
S16 norm[FSE_MAX_SYMBOL_VALUE+1];
|
S16 norm[FSE_MAX_SYMBOL_VALUE+1];
|
||||||
CTable_max_t CTable;
|
CTable_max_t ct;
|
||||||
size_t errorCode;
|
size_t errorCode;
|
||||||
|
|
||||||
/* early out */
|
/* early out */
|
||||||
@ -990,7 +999,7 @@ size_t FSE_compress2 (void* dst, size_t dstSize, const void* src, size_t srcSize
|
|||||||
if (!tableLog) tableLog = FSE_DEFAULT_TABLELOG;
|
if (!tableLog) tableLog = FSE_DEFAULT_TABLELOG;
|
||||||
|
|
||||||
/* Scan input and build symbol stats */
|
/* Scan input and build symbol stats */
|
||||||
errorCode = FSE_count (count, ip, srcSize, &maxSymbolValue);
|
errorCode = FSE_count (count, &maxSymbolValue, ip, srcSize);
|
||||||
if (FSE_isError(errorCode)) return errorCode;
|
if (FSE_isError(errorCode)) return errorCode;
|
||||||
if (errorCode == srcSize) return 1;
|
if (errorCode == srcSize) return 1;
|
||||||
if (errorCode < (srcSize >> 7)) return 0; /* Heuristic : not compressible enough */
|
if (errorCode < (srcSize >> 7)) return 0; /* Heuristic : not compressible enough */
|
||||||
@ -1000,14 +1009,14 @@ size_t FSE_compress2 (void* dst, size_t dstSize, const void* src, size_t srcSize
|
|||||||
if (FSE_isError(errorCode)) return errorCode;
|
if (FSE_isError(errorCode)) return errorCode;
|
||||||
|
|
||||||
/* Write table description header */
|
/* Write table description header */
|
||||||
errorCode = FSE_writeHeader (op, FSE_MAX_HEADERSIZE, norm, maxSymbolValue, tableLog);
|
errorCode = FSE_writeNCount (op, FSE_MAX_HEADERSIZE, norm, maxSymbolValue, tableLog);
|
||||||
if (FSE_isError(errorCode)) return errorCode;
|
if (FSE_isError(errorCode)) return errorCode;
|
||||||
op += errorCode;
|
op += errorCode;
|
||||||
|
|
||||||
/* Compress */
|
/* Compress */
|
||||||
errorCode = FSE_buildCTable (&CTable, norm, maxSymbolValue, tableLog);
|
errorCode = FSE_buildCTable (ct, norm, maxSymbolValue, tableLog);
|
||||||
if (FSE_isError(errorCode)) return errorCode;
|
if (FSE_isError(errorCode)) return errorCode;
|
||||||
op += FSE_compress_usingCTable(op, oend - op, ip, srcSize, &CTable);
|
op += FSE_compress_usingCTable(op, oend - op, ip, srcSize, ct);
|
||||||
|
|
||||||
/* check compressibility */
|
/* check compressibility */
|
||||||
if ( (size_t)(op-ostart) >= srcSize-1 )
|
if ( (size_t)(op-ostart) >= srcSize-1 )
|
||||||
@ -1033,24 +1042,12 @@ typedef struct
|
|||||||
BYTE nbBits;
|
BYTE nbBits;
|
||||||
} FSE_decode_t; /* size == U32 */
|
} FSE_decode_t; /* size == U32 */
|
||||||
|
|
||||||
/* Specific corner case : RLE compression */
|
|
||||||
size_t FSE_decompressRLE(void* dst, size_t originalSize,
|
|
||||||
const void* cSrc, size_t cSrcSize)
|
|
||||||
{
|
|
||||||
if (cSrcSize != 1) return (size_t)-FSE_ERROR_srcSize_wrong;
|
|
||||||
memset(dst, *(BYTE*)cSrc, originalSize);
|
|
||||||
return originalSize;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
size_t FSE_buildDTable_rle (FSE_DTable* dt, BYTE symbolValue)
|
||||||
size_t FSE_buildDTable_rle (void* DTable, BYTE symbolValue)
|
|
||||||
{
|
{
|
||||||
U32* const base32 = (U32*)DTable;
|
U32* const base32 = (U32*)dt;
|
||||||
FSE_decode_t* const cell = (FSE_decode_t*)(base32 + 1);
|
FSE_decode_t* const cell = (FSE_decode_t*)(base32 + 1);
|
||||||
|
|
||||||
/* Sanity check */
|
|
||||||
if (((size_t)DTable) & 3) return (size_t)-FSE_ERROR_GENERIC; /* Must be allocated of 4 bytes boundaries */
|
|
||||||
|
|
||||||
base32[0] = 0;
|
base32[0] = 0;
|
||||||
|
|
||||||
cell->newState = 0;
|
cell->newState = 0;
|
||||||
@ -1061,9 +1058,9 @@ size_t FSE_buildDTable_rle (void* DTable, BYTE symbolValue)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
size_t FSE_buildDTable_raw (void* DTable, unsigned nbBits)
|
size_t FSE_buildDTable_raw (FSE_DTable* dt, unsigned nbBits)
|
||||||
{
|
{
|
||||||
U32* const base32 = (U32*)DTable;
|
U32* const base32 = (U32*)dt;
|
||||||
FSE_decode_t* dinfo = (FSE_decode_t*)(base32 + 1);
|
FSE_decode_t* dinfo = (FSE_decode_t*)(base32 + 1);
|
||||||
const unsigned tableSize = 1 << nbBits;
|
const unsigned tableSize = 1 << nbBits;
|
||||||
const unsigned tableMask = tableSize - 1;
|
const unsigned tableMask = tableSize - 1;
|
||||||
@ -1072,7 +1069,6 @@ size_t FSE_buildDTable_raw (void* DTable, unsigned nbBits)
|
|||||||
|
|
||||||
/* Sanity checks */
|
/* Sanity checks */
|
||||||
if (nbBits < 1) return (size_t)-FSE_ERROR_GENERIC; /* min size */
|
if (nbBits < 1) return (size_t)-FSE_ERROR_GENERIC; /* min size */
|
||||||
if (((size_t)DTable) & 3) return (size_t)-FSE_ERROR_GENERIC; /* Must be allocated of 4 bytes boundaries */
|
|
||||||
|
|
||||||
/* Build Decoding Table */
|
/* Build Decoding Table */
|
||||||
base32[0] = nbBits;
|
base32[0] = nbBits;
|
||||||
@ -1097,36 +1093,36 @@ size_t FSE_initDStream(FSE_DStream_t* bitD, const void* srcBuffer, size_t srcSiz
|
|||||||
{
|
{
|
||||||
if (srcSize < 1) return (size_t)-FSE_ERROR_srcSize_wrong;
|
if (srcSize < 1) return (size_t)-FSE_ERROR_srcSize_wrong;
|
||||||
|
|
||||||
if (srcSize >= sizeof(bitD_t))
|
if (srcSize >= sizeof(size_t))
|
||||||
{
|
{
|
||||||
U32 contain32;
|
U32 contain32;
|
||||||
bitD->start = (char*)srcBuffer;
|
bitD->start = (const char*)srcBuffer;
|
||||||
bitD->ptr = (char*)srcBuffer + srcSize - sizeof(bitD_t);
|
bitD->ptr = (const char*)srcBuffer + srcSize - sizeof(size_t);
|
||||||
bitD->bitContainer = FSE_readLEST(bitD->ptr);
|
bitD->bitContainer = FSE_readLEST(bitD->ptr);
|
||||||
contain32 = ((BYTE*)srcBuffer)[srcSize-1];
|
contain32 = ((const BYTE*)srcBuffer)[srcSize-1];
|
||||||
if (contain32 == 0) return (size_t)-FSE_ERROR_GENERIC; /* stop bit not present */
|
if (contain32 == 0) return (size_t)-FSE_ERROR_GENERIC; /* stop bit not present */
|
||||||
bitD->bitsConsumed = 8 - FSE_highbit32(contain32);
|
bitD->bitsConsumed = 8 - FSE_highbit32(contain32);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
U32 contain32;
|
U32 contain32;
|
||||||
bitD->start = (char*)srcBuffer;
|
bitD->start = (const char*)srcBuffer;
|
||||||
bitD->ptr = bitD->start;
|
bitD->ptr = bitD->start;
|
||||||
bitD->bitContainer = *(BYTE*)(bitD->start);
|
bitD->bitContainer = *(const BYTE*)(bitD->start);
|
||||||
switch(srcSize)
|
switch(srcSize)
|
||||||
{
|
{
|
||||||
case 7: bitD->bitContainer += (bitD_t)(((BYTE*)(bitD->start))[6]) << (sizeof(bitD_t)*8 - 16);
|
case 7: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[6]) << (sizeof(size_t)*8 - 16);
|
||||||
case 6: bitD->bitContainer += (bitD_t)(((BYTE*)(bitD->start))[5]) << (sizeof(bitD_t)*8 - 24);
|
case 6: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[5]) << (sizeof(size_t)*8 - 24);
|
||||||
case 5: bitD->bitContainer += (bitD_t)(((BYTE*)(bitD->start))[4]) << (sizeof(bitD_t)*8 - 32);
|
case 5: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[4]) << (sizeof(size_t)*8 - 32);
|
||||||
case 4: bitD->bitContainer += (bitD_t)(((BYTE*)(bitD->start))[3]) << 24;
|
case 4: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[3]) << 24;
|
||||||
case 3: bitD->bitContainer += (bitD_t)(((BYTE*)(bitD->start))[2]) << 16;
|
case 3: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[2]) << 16;
|
||||||
case 2: bitD->bitContainer += (bitD_t)(((BYTE*)(bitD->start))[1]) << 8;
|
case 2: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[1]) << 8;
|
||||||
default:;
|
default:;
|
||||||
}
|
}
|
||||||
contain32 = ((BYTE*)srcBuffer)[srcSize-1];
|
contain32 = ((const BYTE*)srcBuffer)[srcSize-1];
|
||||||
if (contain32 == 0) return (size_t)-FSE_ERROR_GENERIC; /* stop bit not present */
|
if (contain32 == 0) return (size_t)-FSE_ERROR_GENERIC; /* stop bit not present */
|
||||||
bitD->bitsConsumed = 8 - FSE_highbit32(contain32);
|
bitD->bitsConsumed = 8 - FSE_highbit32(contain32);
|
||||||
bitD->bitsConsumed += (U32)(sizeof(bitD_t) - srcSize)*8;
|
bitD->bitsConsumed += (U32)(sizeof(size_t) - srcSize)*8;
|
||||||
}
|
}
|
||||||
|
|
||||||
return srcSize;
|
return srcSize;
|
||||||
@ -1135,27 +1131,28 @@ size_t FSE_initDStream(FSE_DStream_t* bitD, const void* srcBuffer, size_t srcSiz
|
|||||||
|
|
||||||
/* FSE_readBits
|
/* FSE_readBits
|
||||||
* Read next n bits from the bitContainer.
|
* Read next n bits from the bitContainer.
|
||||||
* Use the fast variant *only* if n > 0.
|
* On 32-bits, don't read more than maxNbBits==25
|
||||||
* Note : for this function to work properly on 32-bits, don't read more than maxNbBits==25
|
* On 64-bits, don't read more than maxNbBits==57
|
||||||
|
* Use the fast variant *only* if n >= 1.
|
||||||
* return : value extracted.
|
* return : value extracted.
|
||||||
*/
|
*/
|
||||||
bitD_t FSE_readBits(FSE_DStream_t* bitD, U32 nbBits)
|
size_t FSE_readBits(FSE_DStream_t* bitD, U32 nbBits)
|
||||||
{
|
{
|
||||||
bitD_t value = ((bitD->bitContainer << bitD->bitsConsumed) >> 1) >> (((sizeof(bitD_t)*8)-1)-nbBits);
|
size_t value = ((bitD->bitContainer << (bitD->bitsConsumed & ((sizeof(size_t)*8)-1))) >> 1) >> (((sizeof(size_t)*8)-1)-nbBits);
|
||||||
bitD->bitsConsumed += nbBits;
|
bitD->bitsConsumed += nbBits;
|
||||||
return value;
|
return value;
|
||||||
}
|
}
|
||||||
|
|
||||||
bitD_t FSE_readBitsFast(FSE_DStream_t* bitD, U32 nbBits) /* only if nbBits >= 1 */
|
size_t FSE_readBitsFast(FSE_DStream_t* bitD, U32 nbBits) /* only if nbBits >= 1 !! */
|
||||||
{
|
{
|
||||||
bitD_t value = (bitD->bitContainer << bitD->bitsConsumed) >> ((sizeof(bitD_t)*8)-nbBits);
|
size_t value = (bitD->bitContainer << bitD->bitsConsumed) >> ((sizeof(size_t)*8)-nbBits);
|
||||||
bitD->bitsConsumed += nbBits;
|
bitD->bitsConsumed += nbBits;
|
||||||
return value;
|
return value;
|
||||||
}
|
}
|
||||||
|
|
||||||
unsigned FSE_reloadDStream(FSE_DStream_t* bitD)
|
unsigned FSE_reloadDStream(FSE_DStream_t* bitD)
|
||||||
{
|
{
|
||||||
if (bitD->ptr >= bitD->start + sizeof(bitD_t))
|
if (bitD->ptr >= bitD->start + sizeof(size_t))
|
||||||
{
|
{
|
||||||
bitD->ptr -= bitD->bitsConsumed >> 3;
|
bitD->ptr -= bitD->bitsConsumed >> 3;
|
||||||
bitD->bitsConsumed &= 7;
|
bitD->bitsConsumed &= 7;
|
||||||
@ -1164,8 +1161,8 @@ unsigned FSE_reloadDStream(FSE_DStream_t* bitD)
|
|||||||
}
|
}
|
||||||
if (bitD->ptr == bitD->start)
|
if (bitD->ptr == bitD->start)
|
||||||
{
|
{
|
||||||
if (bitD->bitsConsumed < sizeof(bitD_t)*8) return 1;
|
if (bitD->bitsConsumed < sizeof(size_t)*8) return 1;
|
||||||
if (bitD->bitsConsumed == sizeof(bitD_t)*8) return 2;
|
if (bitD->bitsConsumed == sizeof(size_t)*8) return 2;
|
||||||
return 3;
|
return 3;
|
||||||
}
|
}
|
||||||
{
|
{
|
||||||
@ -1180,9 +1177,9 @@ unsigned FSE_reloadDStream(FSE_DStream_t* bitD)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void FSE_initDState(FSE_DState_t* DStatePtr, FSE_DStream_t* bitD, const void* DTable)
|
void FSE_initDState(FSE_DState_t* DStatePtr, FSE_DStream_t* bitD, const FSE_DTable* dt)
|
||||||
{
|
{
|
||||||
const U32* const base32 = (const U32*)DTable;
|
const U32* const base32 = (const U32*)dt;
|
||||||
DStatePtr->state = FSE_readBits(bitD, base32[0]);
|
DStatePtr->state = FSE_readBits(bitD, base32[0]);
|
||||||
FSE_reloadDStream(bitD);
|
FSE_reloadDStream(bitD);
|
||||||
DStatePtr->table = base32 + 1;
|
DStatePtr->table = base32 + 1;
|
||||||
@ -1193,7 +1190,7 @@ BYTE FSE_decodeSymbol(FSE_DState_t* DStatePtr, FSE_DStream_t* bitD)
|
|||||||
const FSE_decode_t DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state];
|
const FSE_decode_t DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state];
|
||||||
const U32 nbBits = DInfo.nbBits;
|
const U32 nbBits = DInfo.nbBits;
|
||||||
BYTE symbol = DInfo.symbol;
|
BYTE symbol = DInfo.symbol;
|
||||||
bitD_t lowBits = FSE_readBits(bitD, nbBits);
|
size_t lowBits = FSE_readBits(bitD, nbBits);
|
||||||
|
|
||||||
DStatePtr->state = DInfo.newState + lowBits;
|
DStatePtr->state = DInfo.newState + lowBits;
|
||||||
return symbol;
|
return symbol;
|
||||||
@ -1204,7 +1201,7 @@ BYTE FSE_decodeSymbolFast(FSE_DState_t* DStatePtr, FSE_DStream_t* bitD)
|
|||||||
const FSE_decode_t DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state];
|
const FSE_decode_t DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state];
|
||||||
const U32 nbBits = DInfo.nbBits;
|
const U32 nbBits = DInfo.nbBits;
|
||||||
BYTE symbol = DInfo.symbol;
|
BYTE symbol = DInfo.symbol;
|
||||||
bitD_t lowBits = FSE_readBitsFast(bitD, nbBits);
|
size_t lowBits = FSE_readBitsFast(bitD, nbBits);
|
||||||
|
|
||||||
DStatePtr->state = DInfo.newState + lowBits;
|
DStatePtr->state = DInfo.newState + lowBits;
|
||||||
return symbol;
|
return symbol;
|
||||||
@ -1215,19 +1212,19 @@ BYTE FSE_decodeSymbolFast(FSE_DState_t* DStatePtr, FSE_DStream_t* bitD)
|
|||||||
|
|
||||||
unsigned FSE_endOfDStream(const FSE_DStream_t* bitD)
|
unsigned FSE_endOfDStream(const FSE_DStream_t* bitD)
|
||||||
{
|
{
|
||||||
return FSE_reloadDStream((FSE_DStream_t*)bitD)==2;
|
return ((bitD->ptr == bitD->start) && (bitD->bitsConsumed == sizeof(size_t)*8));
|
||||||
}
|
}
|
||||||
|
|
||||||
unsigned FSE_endOfDState(const FSE_DState_t* statePtr)
|
unsigned FSE_endOfDState(const FSE_DState_t* DStatePtr)
|
||||||
{
|
{
|
||||||
return statePtr->state == 0;
|
return DStatePtr->state == 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
FORCE_INLINE size_t FSE_decompress_usingDTable_generic(
|
FORCE_INLINE size_t FSE_decompress_usingDTable_generic(
|
||||||
void* dst, size_t maxDstSize,
|
void* dst, size_t maxDstSize,
|
||||||
const void* cSrc, size_t cSrcSize,
|
const void* cSrc, size_t cSrcSize,
|
||||||
const void* DTable, unsigned fast)
|
const FSE_DTable* dt, unsigned fast)
|
||||||
{
|
{
|
||||||
BYTE* const ostart = (BYTE*) dst;
|
BYTE* const ostart = (BYTE*) dst;
|
||||||
BYTE* op = ostart;
|
BYTE* op = ostart;
|
||||||
@ -1235,15 +1232,16 @@ FORCE_INLINE size_t FSE_decompress_usingDTable_generic(
|
|||||||
BYTE* const olimit = omax-3;
|
BYTE* const olimit = omax-3;
|
||||||
|
|
||||||
FSE_DStream_t bitD;
|
FSE_DStream_t bitD;
|
||||||
FSE_DState_t state1, state2;
|
FSE_DState_t state1;
|
||||||
|
FSE_DState_t state2;
|
||||||
size_t errorCode;
|
size_t errorCode;
|
||||||
|
|
||||||
/* Init */
|
/* Init */
|
||||||
errorCode = FSE_initDStream(&bitD, cSrc, cSrcSize); /* replaced last arg by maxCompressed Size */
|
errorCode = FSE_initDStream(&bitD, cSrc, cSrcSize); /* replaced last arg by maxCompressed Size */
|
||||||
if (FSE_isError(errorCode)) return errorCode;
|
if (FSE_isError(errorCode)) return errorCode;
|
||||||
|
|
||||||
FSE_initDState(&state1, &bitD, DTable);
|
FSE_initDState(&state1, &bitD, dt);
|
||||||
FSE_initDState(&state2, &bitD, DTable);
|
FSE_initDState(&state2, &bitD, dt);
|
||||||
|
|
||||||
|
|
||||||
/* 2 symbols per loop */
|
/* 2 symbols per loop */
|
||||||
@ -1251,12 +1249,12 @@ FORCE_INLINE size_t FSE_decompress_usingDTable_generic(
|
|||||||
{
|
{
|
||||||
*op++ = fast ? FSE_decodeSymbolFast(&state1, &bitD) : FSE_decodeSymbol(&state1, &bitD);
|
*op++ = fast ? FSE_decodeSymbolFast(&state1, &bitD) : FSE_decodeSymbol(&state1, &bitD);
|
||||||
|
|
||||||
if (FSE_MAX_TABLELOG*2+7 > sizeof(bitD_t)*8) /* This test must be static */
|
if (FSE_MAX_TABLELOG*2+7 > sizeof(size_t)*8) /* This test must be static */
|
||||||
FSE_reloadDStream(&bitD);
|
FSE_reloadDStream(&bitD);
|
||||||
|
|
||||||
*op++ = fast ? FSE_decodeSymbolFast(&state2, &bitD) : FSE_decodeSymbol(&state2, &bitD);
|
*op++ = fast ? FSE_decodeSymbolFast(&state2, &bitD) : FSE_decodeSymbol(&state2, &bitD);
|
||||||
|
|
||||||
if (FSE_MAX_TABLELOG*4+7 < sizeof(bitD_t)*8) /* This test must be static */
|
if (FSE_MAX_TABLELOG*4+7 < sizeof(size_t)*8) /* This test must be static */
|
||||||
{
|
{
|
||||||
*op++ = fast ? FSE_decodeSymbolFast(&state1, &bitD) : FSE_decodeSymbol(&state1, &bitD);
|
*op++ = fast ? FSE_decodeSymbolFast(&state1, &bitD) : FSE_decodeSymbol(&state1, &bitD);
|
||||||
*op++ = fast ? FSE_decodeSymbolFast(&state2, &bitD) : FSE_decodeSymbol(&state2, &bitD);
|
*op++ = fast ? FSE_decodeSymbolFast(&state2, &bitD) : FSE_decodeSymbol(&state2, &bitD);
|
||||||
@ -1264,14 +1262,15 @@ FORCE_INLINE size_t FSE_decompress_usingDTable_generic(
|
|||||||
}
|
}
|
||||||
|
|
||||||
/* tail */
|
/* tail */
|
||||||
|
/* note : FSE_reloadDStream(&bitD) >= 1; Ends at exactly 2 */
|
||||||
while (1)
|
while (1)
|
||||||
{
|
{
|
||||||
if ( (FSE_reloadDStream(&bitD)>2) || (op==omax) || (FSE_endOfDState(&state1) && FSE_endOfDStream(&bitD)) )
|
if ( (FSE_reloadDStream(&bitD)>2) || (op==omax) || (FSE_endOfDStream(&bitD) && (fast || FSE_endOfDState(&state1))) )
|
||||||
break;
|
break;
|
||||||
|
|
||||||
*op++ = fast ? FSE_decodeSymbolFast(&state1, &bitD) : FSE_decodeSymbol(&state1, &bitD);
|
*op++ = fast ? FSE_decodeSymbolFast(&state1, &bitD) : FSE_decodeSymbol(&state1, &bitD);
|
||||||
|
|
||||||
if ( (FSE_reloadDStream(&bitD)>2) || (op==omax) || (FSE_endOfDState(&state2) && FSE_endOfDStream(&bitD)) )
|
if ( (FSE_reloadDStream(&bitD)>2) || (op==omax) || (FSE_endOfDStream(&bitD) && (fast || FSE_endOfDState(&state2))) )
|
||||||
break;
|
break;
|
||||||
|
|
||||||
*op++ = fast ? FSE_decodeSymbolFast(&state2, &bitD) : FSE_decodeSymbol(&state2, &bitD);
|
*op++ = fast ? FSE_decodeSymbolFast(&state2, &bitD) : FSE_decodeSymbol(&state2, &bitD);
|
||||||
@ -1289,11 +1288,11 @@ FORCE_INLINE size_t FSE_decompress_usingDTable_generic(
|
|||||||
|
|
||||||
size_t FSE_decompress_usingDTable(void* dst, size_t originalSize,
|
size_t FSE_decompress_usingDTable(void* dst, size_t originalSize,
|
||||||
const void* cSrc, size_t cSrcSize,
|
const void* cSrc, size_t cSrcSize,
|
||||||
const void* DTable, size_t fastMode)
|
const FSE_DTable* dt, size_t fastMode)
|
||||||
{
|
{
|
||||||
/* select fast mode (static) */
|
/* select fast mode (static) */
|
||||||
if (fastMode) return FSE_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, DTable, 1);
|
if (fastMode) return FSE_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, dt, 1);
|
||||||
return FSE_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, DTable, 0);
|
return FSE_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, dt, 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -1302,25 +1301,25 @@ size_t FSE_decompress(void* dst, size_t maxDstSize, const void* cSrc, size_t cSr
|
|||||||
const BYTE* const istart = (const BYTE*)cSrc;
|
const BYTE* const istart = (const BYTE*)cSrc;
|
||||||
const BYTE* ip = istart;
|
const BYTE* ip = istart;
|
||||||
short counting[FSE_MAX_SYMBOL_VALUE+1];
|
short counting[FSE_MAX_SYMBOL_VALUE+1];
|
||||||
FSE_decode_t DTable[FSE_DTABLE_SIZE_U32(FSE_MAX_TABLELOG)];
|
DTable_max_t dt; /* Static analyzer seems unable to understand this table will be properly initialized later */
|
||||||
unsigned maxSymbolValue = FSE_MAX_SYMBOL_VALUE;
|
|
||||||
unsigned tableLog;
|
unsigned tableLog;
|
||||||
|
unsigned maxSymbolValue = FSE_MAX_SYMBOL_VALUE;
|
||||||
size_t errorCode, fastMode;
|
size_t errorCode, fastMode;
|
||||||
|
|
||||||
if (cSrcSize<2) return (size_t)-FSE_ERROR_srcSize_wrong; /* too small input size */
|
if (cSrcSize<2) return (size_t)-FSE_ERROR_srcSize_wrong; /* too small input size */
|
||||||
|
|
||||||
/* normal FSE decoding mode */
|
/* normal FSE decoding mode */
|
||||||
errorCode = FSE_readHeader (counting, &maxSymbolValue, &tableLog, istart, cSrcSize);
|
errorCode = FSE_readNCount (counting, &maxSymbolValue, &tableLog, istart, cSrcSize);
|
||||||
if (FSE_isError(errorCode)) return errorCode;
|
if (FSE_isError(errorCode)) return errorCode;
|
||||||
if (errorCode >= cSrcSize) return (size_t)-FSE_ERROR_srcSize_wrong; /* too small input size */
|
if (errorCode >= cSrcSize) return (size_t)-FSE_ERROR_srcSize_wrong; /* too small input size */
|
||||||
ip += errorCode;
|
ip += errorCode;
|
||||||
cSrcSize -= errorCode;
|
cSrcSize -= errorCode;
|
||||||
|
|
||||||
fastMode = FSE_buildDTable (DTable, counting, maxSymbolValue, tableLog);
|
fastMode = FSE_buildDTable (dt, counting, maxSymbolValue, tableLog);
|
||||||
if (FSE_isError(fastMode)) return fastMode;
|
if (FSE_isError(fastMode)) return fastMode;
|
||||||
|
|
||||||
/* always return, even if it is an error code */
|
/* always return, even if it is an error code */
|
||||||
return FSE_decompress_usingDTable (dst, maxDstSize, ip, cSrcSize, DTable, fastMode);
|
return FSE_decompress_usingDTable (dst, maxDstSize, ip, cSrcSize, dt, fastMode);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -1329,8 +1328,8 @@ size_t FSE_decompress(void* dst, size_t maxDstSize, const void* cSrc, size_t cSr
|
|||||||
/*
|
/*
|
||||||
2nd part of the file
|
2nd part of the file
|
||||||
designed to be included
|
designed to be included
|
||||||
for type-specific functions (template equivalent in C)
|
for type-specific functions (template emulation in C)
|
||||||
Objective is to write such functions only once, for better maintenance
|
Objective is to write these functions only once, for improved maintenance
|
||||||
*/
|
*/
|
||||||
|
|
||||||
/* safety checks */
|
/* safety checks */
|
||||||
@ -1348,7 +1347,8 @@ size_t FSE_decompress(void* dst, size_t maxDstSize, const void* cSrc, size_t cSr
|
|||||||
|
|
||||||
|
|
||||||
/* Function templates */
|
/* Function templates */
|
||||||
size_t FSE_FUNCTION_NAME(FSE_count_generic, FSE_FUNCTION_EXTENSION) (unsigned* count, const FSE_FUNCTION_TYPE* source, size_t sourceSize, unsigned* maxSymbolValuePtr, unsigned safe)
|
size_t FSE_FUNCTION_NAME(FSE_count_generic, FSE_FUNCTION_EXTENSION)
|
||||||
|
(unsigned* count, unsigned* maxSymbolValuePtr, const FSE_FUNCTION_TYPE* source, size_t sourceSize, unsigned safe)
|
||||||
{
|
{
|
||||||
const FSE_FUNCTION_TYPE* ip = source;
|
const FSE_FUNCTION_TYPE* ip = source;
|
||||||
const FSE_FUNCTION_TYPE* const iend = ip+sourceSize;
|
const FSE_FUNCTION_TYPE* const iend = ip+sourceSize;
|
||||||
@ -1422,46 +1422,45 @@ size_t FSE_FUNCTION_NAME(FSE_count_generic, FSE_FUNCTION_EXTENSION) (unsigned* c
|
|||||||
|
|
||||||
while (!count[maxSymbolValue]) maxSymbolValue--;
|
while (!count[maxSymbolValue]) maxSymbolValue--;
|
||||||
*maxSymbolValuePtr = maxSymbolValue;
|
*maxSymbolValuePtr = maxSymbolValue;
|
||||||
return (int)max;
|
return (size_t)max;
|
||||||
}
|
}
|
||||||
|
|
||||||
/* hidden fast variant (unsafe) */
|
/* hidden fast variant (unsafe) */
|
||||||
size_t FSE_FUNCTION_NAME(FSE_countFast, FSE_FUNCTION_EXTENSION) (unsigned* count, const FSE_FUNCTION_TYPE* source, size_t sourceSize, unsigned* maxSymbolValuePtr)
|
size_t FSE_FUNCTION_NAME(FSE_countFast, FSE_FUNCTION_EXTENSION)
|
||||||
|
(unsigned* count, unsigned* maxSymbolValuePtr, const FSE_FUNCTION_TYPE* source, size_t sourceSize)
|
||||||
{
|
{
|
||||||
return FSE_FUNCTION_NAME(FSE_count_generic, FSE_FUNCTION_EXTENSION) (count, source, sourceSize, maxSymbolValuePtr, 0);
|
return FSE_FUNCTION_NAME(FSE_count_generic, FSE_FUNCTION_EXTENSION) (count, maxSymbolValuePtr, source, sourceSize, 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
size_t FSE_FUNCTION_NAME(FSE_count, FSE_FUNCTION_EXTENSION) (unsigned* count, const FSE_FUNCTION_TYPE* source, size_t sourceSize, unsigned* maxSymbolValuePtr)
|
size_t FSE_FUNCTION_NAME(FSE_count, FSE_FUNCTION_EXTENSION)
|
||||||
|
(unsigned* count, unsigned* maxSymbolValuePtr, const FSE_FUNCTION_TYPE* source, size_t sourceSize)
|
||||||
{
|
{
|
||||||
if ((sizeof(FSE_FUNCTION_TYPE)==1) && (*maxSymbolValuePtr >= 255))
|
if ((sizeof(FSE_FUNCTION_TYPE)==1) && (*maxSymbolValuePtr >= 255))
|
||||||
{
|
{
|
||||||
*maxSymbolValuePtr = 255;
|
*maxSymbolValuePtr = 255;
|
||||||
return FSE_FUNCTION_NAME(FSE_count_generic, FSE_FUNCTION_EXTENSION) (count, source, sourceSize, maxSymbolValuePtr, 0);
|
return FSE_FUNCTION_NAME(FSE_count_generic, FSE_FUNCTION_EXTENSION) (count, maxSymbolValuePtr, source, sourceSize, 0);
|
||||||
}
|
}
|
||||||
return FSE_FUNCTION_NAME(FSE_count_generic, FSE_FUNCTION_EXTENSION) (count, source, sourceSize, maxSymbolValuePtr, 1);
|
return FSE_FUNCTION_NAME(FSE_count_generic, FSE_FUNCTION_EXTENSION) (count, maxSymbolValuePtr, source, sourceSize, 1);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
static U32 FSE_tableStep(U32 tableSize) { return (tableSize>>1) + (tableSize>>3) + 3; }
|
static U32 FSE_tableStep(U32 tableSize) { return (tableSize>>1) + (tableSize>>3) + 3; }
|
||||||
|
|
||||||
size_t FSE_FUNCTION_NAME(FSE_buildCTable, FSE_FUNCTION_EXTENSION)
|
size_t FSE_FUNCTION_NAME(FSE_buildCTable, FSE_FUNCTION_EXTENSION)
|
||||||
(void* CTable, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog)
|
(FSE_CTable* ct, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog)
|
||||||
{
|
{
|
||||||
const unsigned tableSize = 1 << tableLog;
|
const unsigned tableSize = 1 << tableLog;
|
||||||
const unsigned tableMask = tableSize - 1;
|
const unsigned tableMask = tableSize - 1;
|
||||||
U16* tableU16 = ( (U16*) CTable) + 2;
|
U16* tableU16 = ( (U16*) ct) + 2;
|
||||||
FSE_symbolCompressionTransform* symbolTT = (FSE_symbolCompressionTransform*) (((U32*)CTable) + 1 + (tableLog ? tableSize>>1 : 1) );
|
FSE_symbolCompressionTransform* symbolTT = (FSE_symbolCompressionTransform*) (((U32*)ct) + 1 + (tableLog ? tableSize>>1 : 1) );
|
||||||
const unsigned step = FSE_tableStep(tableSize);
|
const unsigned step = FSE_tableStep(tableSize);
|
||||||
unsigned cumul[FSE_MAX_SYMBOL_VALUE+2];
|
unsigned cumul[FSE_MAX_SYMBOL_VALUE+2];
|
||||||
U32 position = 0;
|
U32 position = 0;
|
||||||
FSE_FUNCTION_TYPE tableSymbol[FSE_MAX_TABLESIZE];
|
FSE_FUNCTION_TYPE tableSymbol[FSE_MAX_TABLESIZE] = {0}; /* should not be necessary, but analyzer complain without it, and performance loss is negligible with it */
|
||||||
U32 highThreshold = tableSize-1;
|
U32 highThreshold = tableSize-1;
|
||||||
unsigned symbol;
|
unsigned symbol;
|
||||||
unsigned i;
|
unsigned i;
|
||||||
|
|
||||||
/* safety checks */
|
|
||||||
if (((size_t)CTable) & 3) return (size_t)-FSE_ERROR_GENERIC; /* Must be allocated of 4 bytes boundaries */
|
|
||||||
|
|
||||||
/* header */
|
/* header */
|
||||||
tableU16[-2] = (U16) tableLog;
|
tableU16[-2] = (U16) tableLog;
|
||||||
tableU16[-1] = (U16) maxSymbolValue;
|
tableU16[-1] = (U16) maxSymbolValue;
|
||||||
@ -1501,10 +1500,10 @@ size_t FSE_FUNCTION_NAME(FSE_buildCTable, FSE_FUNCTION_EXTENSION)
|
|||||||
for (i=0; i<tableSize; i++)
|
for (i=0; i<tableSize; i++)
|
||||||
{
|
{
|
||||||
FSE_FUNCTION_TYPE s = tableSymbol[i];
|
FSE_FUNCTION_TYPE s = tableSymbol[i];
|
||||||
tableU16[cumul[s]++] = (U16) (tableSize+i); // Table U16 : sorted by symbol order; gives next state value
|
tableU16[cumul[s]++] = (U16) (tableSize+i); /* Table U16 : sorted by symbol order; gives next state value */
|
||||||
}
|
}
|
||||||
|
|
||||||
// Build Symbol Transformation Table
|
/* Build Symbol Transformation Table */
|
||||||
{
|
{
|
||||||
unsigned s;
|
unsigned s;
|
||||||
unsigned total = 0;
|
unsigned total = 0;
|
||||||
@ -1536,22 +1535,22 @@ size_t FSE_FUNCTION_NAME(FSE_buildCTable, FSE_FUNCTION_EXTENSION)
|
|||||||
|
|
||||||
#define FSE_DECODE_TYPE FSE_TYPE_NAME(FSE_decode_t, FSE_FUNCTION_EXTENSION)
|
#define FSE_DECODE_TYPE FSE_TYPE_NAME(FSE_decode_t, FSE_FUNCTION_EXTENSION)
|
||||||
|
|
||||||
void* FSE_FUNCTION_NAME(FSE_createDTable, FSE_FUNCTION_EXTENSION) (unsigned tableLog)
|
FSE_DTable* FSE_FUNCTION_NAME(FSE_createDTable, FSE_FUNCTION_EXTENSION) (unsigned tableLog)
|
||||||
{
|
{
|
||||||
if (tableLog > FSE_TABLELOG_ABSOLUTE_MAX) tableLog = FSE_TABLELOG_ABSOLUTE_MAX;
|
if (tableLog > FSE_TABLELOG_ABSOLUTE_MAX) tableLog = FSE_TABLELOG_ABSOLUTE_MAX;
|
||||||
return malloc( ((size_t)1<<tableLog) * sizeof (FSE_DECODE_TYPE) );
|
return (FSE_DTable*)malloc( FSE_DTABLE_SIZE_U32(tableLog) * sizeof (U32) );
|
||||||
}
|
}
|
||||||
|
|
||||||
void FSE_FUNCTION_NAME(FSE_freeDTable, FSE_FUNCTION_EXTENSION) (void* DTable)
|
void FSE_FUNCTION_NAME(FSE_freeDTable, FSE_FUNCTION_EXTENSION) (FSE_DTable* dt)
|
||||||
{
|
{
|
||||||
free(DTable);
|
free(dt);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
size_t FSE_FUNCTION_NAME(FSE_buildDTable, FSE_FUNCTION_EXTENSION)
|
size_t FSE_FUNCTION_NAME(FSE_buildDTable, FSE_FUNCTION_EXTENSION)
|
||||||
(void* DTable, const short* const normalizedCounter, unsigned maxSymbolValue, unsigned tableLog)
|
(FSE_DTable* dt, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog)
|
||||||
{
|
{
|
||||||
U32* const base32 = (U32*)DTable;
|
U32* const base32 = (U32*)dt;
|
||||||
FSE_DECODE_TYPE* const tableDecode = (FSE_DECODE_TYPE*) (base32+1);
|
FSE_DECODE_TYPE* const tableDecode = (FSE_DECODE_TYPE*) (base32+1);
|
||||||
const U32 tableSize = 1 << tableLog;
|
const U32 tableSize = 1 << tableLog;
|
||||||
const U32 tableMask = tableSize-1;
|
const U32 tableMask = tableSize-1;
|
||||||
@ -1559,7 +1558,7 @@ size_t FSE_FUNCTION_NAME(FSE_buildDTable, FSE_FUNCTION_EXTENSION)
|
|||||||
U16 symbolNext[FSE_MAX_SYMBOL_VALUE+1];
|
U16 symbolNext[FSE_MAX_SYMBOL_VALUE+1];
|
||||||
U32 position = 0;
|
U32 position = 0;
|
||||||
U32 highThreshold = tableSize-1;
|
U32 highThreshold = tableSize-1;
|
||||||
const S16 largeLimit= 1 << (tableLog-1);
|
const S16 largeLimit= (S16)(1 << (tableLog-1));
|
||||||
U32 noLarge = 1;
|
U32 noLarge = 1;
|
||||||
U32 s;
|
U32 s;
|
||||||
|
|
||||||
@ -1602,7 +1601,7 @@ size_t FSE_FUNCTION_NAME(FSE_buildDTable, FSE_FUNCTION_EXTENSION)
|
|||||||
U32 i;
|
U32 i;
|
||||||
for (i=0; i<tableSize; i++)
|
for (i=0; i<tableSize; i++)
|
||||||
{
|
{
|
||||||
FSE_FUNCTION_TYPE symbol = tableDecode[i].symbol;
|
FSE_FUNCTION_TYPE symbol = (FSE_FUNCTION_TYPE)(tableDecode[i].symbol);
|
||||||
U16 nextState = symbolNext[symbol]++;
|
U16 nextState = symbolNext[symbol]++;
|
||||||
tableDecode[i].nbBits = (BYTE) (tableLog - FSE_highbit32 ((U32)nextState) );
|
tableDecode[i].nbBits = (BYTE) (tableLog - FSE_highbit32 ((U32)nextState) );
|
||||||
tableDecode[i].newState = (U16) ( (nextState << tableDecode[i].nbBits) - tableSize);
|
tableDecode[i].newState = (U16) ( (nextState << tableDecode[i].nbBits) - tableSize);
|
||||||
|
@ -42,7 +42,7 @@ extern "C" {
|
|||||||
/******************************************
|
/******************************************
|
||||||
* Includes
|
* Includes
|
||||||
******************************************/
|
******************************************/
|
||||||
#include <stddef.h> // size_t, ptrdiff_t
|
#include <stddef.h> /* size_t, ptrdiff_t */
|
||||||
|
|
||||||
|
|
||||||
/******************************************
|
/******************************************
|
||||||
@ -50,7 +50,7 @@ extern "C" {
|
|||||||
******************************************/
|
******************************************/
|
||||||
size_t FSE_compress(void* dst, size_t maxDstSize,
|
size_t FSE_compress(void* dst, size_t maxDstSize,
|
||||||
const void* src, size_t srcSize);
|
const void* src, size_t srcSize);
|
||||||
size_t FSE_decompress(void* dst, size_t maxDstSize,
|
size_t FSE_decompress(void* dst, size_t maxDstSize,
|
||||||
const void* cSrc, size_t cSrcSize);
|
const void* cSrc, size_t cSrcSize);
|
||||||
/*
|
/*
|
||||||
FSE_compress():
|
FSE_compress():
|
||||||
@ -58,29 +58,19 @@ FSE_compress():
|
|||||||
'dst' buffer must be already allocated, and sized to handle worst case situations.
|
'dst' buffer must be already allocated, and sized to handle worst case situations.
|
||||||
Worst case size evaluation is provided by FSE_compressBound().
|
Worst case size evaluation is provided by FSE_compressBound().
|
||||||
return : size of compressed data
|
return : size of compressed data
|
||||||
Special values : if result == 0, data is uncompressible => Nothing is stored within cSrc !!
|
Special values : if return == 0, srcData is not compressible => Nothing is stored within cSrc !!!
|
||||||
if result == 1, data is one constant element x srcSize times. Use RLE compression.
|
if return == 1, srcData is a single byte symbol * srcSize times. Use RLE compression.
|
||||||
if FSE_isError(result), it's an error code.
|
if FSE_isError(return), it's an error code.
|
||||||
|
|
||||||
FSE_decompress():
|
FSE_decompress():
|
||||||
Decompress FSE data from buffer 'cSrc', of size 'cSrcSize',
|
Decompress FSE data from buffer 'cSrc', of size 'cSrcSize',
|
||||||
into already allocated destination buffer 'dst', of size 'maxDstSize'.
|
into already allocated destination buffer 'dst', of size 'maxDstSize'.
|
||||||
** Important ** : This function doesn't decompress uncompressed nor RLE data !
|
|
||||||
return : size of regenerated data (<= maxDstSize)
|
return : size of regenerated data (<= maxDstSize)
|
||||||
or an error code, which can be tested using FSE_isError()
|
or an error code, which can be tested using FSE_isError()
|
||||||
*/
|
|
||||||
|
|
||||||
|
** Important ** : FSE_decompress() doesn't decompress non-compressible nor RLE data !!!
|
||||||
size_t FSE_decompressRLE(void* dst, size_t originalSize,
|
Why ? : making this distinction requires a header.
|
||||||
const void* cSrc, size_t cSrcSize);
|
FSE library doesn't manage headers, which are intentionally left to the user layer.
|
||||||
/*
|
|
||||||
FSE_decompressRLE():
|
|
||||||
Decompress specific RLE corner case (equivalent to memset()).
|
|
||||||
cSrcSize must be == 1. originalSize must be exact.
|
|
||||||
return : size of regenerated data (==originalSize)
|
|
||||||
or an error code, which can be tested using FSE_isError()
|
|
||||||
|
|
||||||
Note : there is no function provided for uncompressed data, as it's just a simple memcpy()
|
|
||||||
*/
|
*/
|
||||||
|
|
||||||
|
|
||||||
@ -102,51 +92,103 @@ FSE_compress2():
|
|||||||
Same as FSE_compress(), but allows the selection of 'maxSymbolValue' and 'tableLog'
|
Same as FSE_compress(), but allows the selection of 'maxSymbolValue' and 'tableLog'
|
||||||
Both parameters can be defined as '0' to mean : use default value
|
Both parameters can be defined as '0' to mean : use default value
|
||||||
return : size of compressed data
|
return : size of compressed data
|
||||||
or -1 if there is an error
|
Special values : if return == 0, srcData is not compressible => Nothing is stored within cSrc !!!
|
||||||
|
if return == 1, srcData is a single byte symbol * srcSize times. Use RLE compression.
|
||||||
|
if FSE_isError(return), it's an error code.
|
||||||
*/
|
*/
|
||||||
size_t FSE_compress2 (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog);
|
size_t FSE_compress2 (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog);
|
||||||
|
|
||||||
|
|
||||||
/******************************************
|
/******************************************
|
||||||
FSE detailed API
|
* FSE detailed API
|
||||||
******************************************/
|
******************************************/
|
||||||
/*
|
/*
|
||||||
int FSE_compress(char* dest, const char* source, int inputSize) does the following:
|
FSE_compress() does the following:
|
||||||
1. count symbol occurrence from source[] into table count[]
|
1. count symbol occurrence from source[] into table count[]
|
||||||
2. normalize counters so that sum(count[]) == Power_of_2 (2^tableLog)
|
2. normalize counters so that sum(count[]) == Power_of_2 (2^tableLog)
|
||||||
3. save normalized counters to memory buffer using writeHeader()
|
3. save normalized counters to memory buffer using writeHeader()
|
||||||
4. build encoding table 'CTable' from normalized counters
|
4. build encoding table 'CTable' from normalized counters
|
||||||
5. encode the data stream using encoding table
|
5. encode the data stream using encoding table 'CTable'
|
||||||
|
|
||||||
int FSE_decompress(char* dest, int originalSize, const char* compressed) performs:
|
FSE_decompress() does the following:
|
||||||
1. read normalized counters with readHeader()
|
1. read normalized counters with readHeader()
|
||||||
2. build decoding table 'DTable' from normalized counters
|
2. build decoding table 'DTable' from normalized counters
|
||||||
3. decode the data stream using decoding table
|
3. decode the data stream using decoding table 'DTable'
|
||||||
|
|
||||||
The following API allows triggering specific sub-functions.
|
The following API allows to trigger specific sub-functions for advanced tasks.
|
||||||
|
For example, it's possible to compress several blocks using the same 'CTable',
|
||||||
|
or to save and provide normalized distribution using one's own method.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
/* *** COMPRESSION *** */
|
/* *** COMPRESSION *** */
|
||||||
|
|
||||||
size_t FSE_count(unsigned* count, const unsigned char* src, size_t srcSize, unsigned* maxSymbolValuePtr);
|
/*
|
||||||
|
FSE_count():
|
||||||
unsigned FSE_optimalTableLog(unsigned tableLog, size_t srcSize, unsigned maxSymbolValue);
|
Provides the precise count of each symbol within a table 'count'
|
||||||
size_t FSE_normalizeCount(short* normalizedCounter, unsigned tableLog, const unsigned* count, size_t total, unsigned maxSymbolValue);
|
'count' is a table of unsigned int, of minimum size (maxSymbolValuePtr[0]+1).
|
||||||
|
maxSymbolValuePtr[0] will be updated if detected smaller than initially expected
|
||||||
size_t FSE_headerBound(unsigned maxSymbolValue, unsigned tableLog);
|
return : the count of the most frequent symbol (which is not identified)
|
||||||
size_t FSE_writeHeader (void* headerBuffer, size_t headerBufferSize, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog);
|
if return == srcSize, there is only one symbol.
|
||||||
|
if FSE_isError(return), it's an error code. */
|
||||||
void* FSE_createCTable (unsigned tableLog, unsigned maxSymbolValue);
|
size_t FSE_count(unsigned* count, unsigned* maxSymbolValuePtr, const unsigned char* src, size_t srcSize);
|
||||||
void FSE_freeCTable (void* CTable);
|
|
||||||
size_t FSE_buildCTable(void* CTable, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog);
|
|
||||||
|
|
||||||
size_t FSE_compress_usingCTable (void* dst, size_t dstSize, const void* src, size_t srcSize, const void* CTable);
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
|
FSE_optimalTableLog():
|
||||||
|
dynamically downsize 'tableLog' when conditions are met.
|
||||||
|
It saves CPU time, by using smaller tables, while preserving or even improving compression ratio.
|
||||||
|
return : recommended tableLog (necessarily <= initial 'tableLog') */
|
||||||
|
unsigned FSE_optimalTableLog(unsigned tableLog, size_t srcSize, unsigned maxSymbolValue);
|
||||||
|
|
||||||
|
/*
|
||||||
|
FSE_normalizeCount():
|
||||||
|
normalize counters so that sum(count[]) == Power_of_2 (2^tableLog)
|
||||||
|
'normalizedCounter' is a table of short, of minimum size (maxSymbolValue+1).
|
||||||
|
return : tableLog,
|
||||||
|
or an errorCode, which can be tested using FSE_isError() */
|
||||||
|
size_t FSE_normalizeCount(short* normalizedCounter, unsigned tableLog, const unsigned* count, size_t srcSize, unsigned maxSymbolValue);
|
||||||
|
|
||||||
|
/*
|
||||||
|
FSE_NCountWriteBound():
|
||||||
|
Provides the maximum possible size of an FSE normalized table, given 'maxSymbolValue' and 'tableLog'
|
||||||
|
Typically useful for allocation purpose. */
|
||||||
|
size_t FSE_NCountWriteBound(unsigned maxSymbolValue, unsigned tableLog);
|
||||||
|
|
||||||
|
/*
|
||||||
|
FSE_writeNCount():
|
||||||
|
Compactly save 'normalizedCounter' into 'buffer'.
|
||||||
|
return : size of the compressed table
|
||||||
|
or an errorCode, which can be tested using FSE_isError() */
|
||||||
|
size_t FSE_writeNCount (void* buffer, size_t bufferSize, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog);
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
Constructor and Destructor of type FSE_CTable
|
||||||
|
Not that its size depends on parameters 'tableLog' and 'maxSymbolValue' */
|
||||||
|
typedef unsigned FSE_CTable; /* don't allocate that. It's just a way to be more restrictive than void */
|
||||||
|
FSE_CTable* FSE_createCTable (unsigned tableLog, unsigned maxSymbolValue);
|
||||||
|
void FSE_freeCTable (FSE_CTable* ct);
|
||||||
|
|
||||||
|
/*
|
||||||
|
FSE_buildCTable():
|
||||||
|
Builds 'ct', which must be already allocated, using FSE_createCTable()
|
||||||
|
return : 0
|
||||||
|
or an errorCode, which can be tested using FSE_isError() */
|
||||||
|
size_t FSE_buildCTable(FSE_CTable* ct, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog);
|
||||||
|
|
||||||
|
/*
|
||||||
|
FSE_compress_usingCTable():
|
||||||
|
Compress 'src' using 'ct' into 'dst' which must be already allocated
|
||||||
|
return : size of compressed data
|
||||||
|
or an errorCode, which can be tested using FSE_isError() */
|
||||||
|
size_t FSE_compress_usingCTable (void* dst, size_t dstSize, const void* src, size_t srcSize, const FSE_CTable* ct);
|
||||||
|
|
||||||
|
/*
|
||||||
|
Tutorial :
|
||||||
|
----------
|
||||||
The first step is to count all symbols. FSE_count() provides one quick way to do this job.
|
The first step is to count all symbols. FSE_count() provides one quick way to do this job.
|
||||||
Result will be saved into 'count', a table of unsigned int, which must be already allocated, and have '*maxSymbolValuePtr+1' cells.
|
Result will be saved into 'count', a table of unsigned int, which must be already allocated, and have 'maxSymbolValuePtr[0]+1' cells.
|
||||||
'source' is a table of char of size 'sourceSize'. All values within 'src' MUST be <= *maxSymbolValuePtr
|
'src' is a table of bytes of size 'srcSize'. All values within 'src' MUST be <= maxSymbolValuePtr[0]
|
||||||
*maxSymbolValuePtr will be updated, with its real value (necessarily <= original value)
|
maxSymbolValuePtr[0] will be updated, with its real value (necessarily <= original value)
|
||||||
FSE_count() will return the number of occurrence of the most frequent symbol.
|
FSE_count() will return the number of occurrence of the most frequent symbol.
|
||||||
If there is an error, the function will return an ErrorCode (which can be tested using FSE_isError()).
|
If there is an error, the function will return an ErrorCode (which can be tested using FSE_isError()).
|
||||||
|
|
||||||
@ -170,211 +212,83 @@ For guaranteed success, buffer size must be at least FSE_headerBound().
|
|||||||
The result of the function is the number of bytes written into 'header'.
|
The result of the function is the number of bytes written into 'header'.
|
||||||
If there is an error, the function will return an ErrorCode (which can be tested using FSE_isError()) (for example, buffer size too small).
|
If there is an error, the function will return an ErrorCode (which can be tested using FSE_isError()) (for example, buffer size too small).
|
||||||
|
|
||||||
'normalizedCounter' can then be used to create the compression tables 'CTable'.
|
'normalizedCounter' can then be used to create the compression table 'CTable'.
|
||||||
The space required by 'CTable' must be already allocated. Its size is provided by FSE_sizeof_CTable().
|
The space required by 'CTable' must be already allocated. Its size is provided by FSE_sizeof_CTable().
|
||||||
'CTable' must be aligned of 4 bytes boundaries.
|
'CTable' must be aligned of 4 bytes boundaries.
|
||||||
You can then use FSE_buildCTable() to fill 'CTable'.
|
You can then use FSE_buildCTable() to fill 'CTable'.
|
||||||
In both cases, if there is an error, the function will return an ErrorCode (which can be tested using FSE_isError()).
|
In both cases, if there is an error, the function will return an ErrorCode (which can be tested using FSE_isError()).
|
||||||
|
|
||||||
'CTable' can then be used to compress 'source', with FSE_compress_usingCTable().
|
'CTable' can then be used to compress 'src', with FSE_compress_usingCTable().
|
||||||
Similar to FSE_count(), the convention is that 'source' is assumed to be a table of char of size 'sourceSize'
|
Similar to FSE_count(), the convention is that 'src' is assumed to be a table of char of size 'srcSize'
|
||||||
The function returns the size of compressed data (without header), or -1 if failed.
|
The function returns the size of compressed data (without header).
|
||||||
|
If there is an error, the function will return an ErrorCode (which can be tested using FSE_isError()).
|
||||||
*/
|
*/
|
||||||
|
|
||||||
|
|
||||||
/* *** DECOMPRESSION *** */
|
/* *** DECOMPRESSION *** */
|
||||||
|
|
||||||
size_t FSE_readHeader (short* normalizedCounter, unsigned* maxSymbolValuePtr, unsigned* tableLogPtr, const void* headerBuffer, size_t hbSize);
|
/*
|
||||||
|
FSE_readNCount():
|
||||||
void* FSE_createDTable(unsigned tableLog);
|
Read compactly saved 'normalizedCounter' from 'rBuffer'.
|
||||||
void FSE_freeDTable(void* DTable);
|
return : size read from 'rBuffer'
|
||||||
size_t FSE_buildDTable (void* DTable, const short* const normalizedCounter, unsigned maxSymbolValue, unsigned tableLog);
|
or an errorCode, which can be tested using FSE_isError()
|
||||||
|
maxSymbolValuePtr[0] and tableLogPtr[0] will also be updated with their respective values */
|
||||||
size_t FSE_decompress_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const void* DTable, size_t fastMode);
|
size_t FSE_readNCount (short* normalizedCounter, unsigned* maxSymbolValuePtr, unsigned* tableLogPtr, const void* rBuffer, size_t rBuffSize);
|
||||||
|
|
||||||
/*
|
/*
|
||||||
If the block is RLE compressed, or uncompressed, use the relevant specific functions.
|
Constructor and Destructor of type FSE_DTable
|
||||||
|
Note that its size depends on parameters 'tableLog' */
|
||||||
|
typedef unsigned FSE_DTable; /* don't allocate that. It's just a way to be more restrictive than void */
|
||||||
|
FSE_DTable* FSE_createDTable(unsigned tableLog);
|
||||||
|
void FSE_freeDTable(FSE_DTable* dt);
|
||||||
|
|
||||||
|
/*
|
||||||
|
FSE_buildDTable():
|
||||||
|
Builds 'dt', which must be already allocated, using FSE_createDTable()
|
||||||
|
return : 1 if 'dt' is compatible with fast mode, 0 otherwise,
|
||||||
|
or an errorCode, which can be tested using FSE_isError() */
|
||||||
|
size_t FSE_buildDTable (FSE_DTable* dt, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog);
|
||||||
|
|
||||||
|
/*
|
||||||
|
FSE_decompress_usingDTable():
|
||||||
|
Decompress compressed source 'cSrc' of size 'cSrcSize'
|
||||||
|
using 'dt' into 'dst' which must be already allocated.
|
||||||
|
Use fastMode==1 only if authorized by result of FSE_buildDTable().
|
||||||
|
return : size of regenerated data (necessarily <= maxDstSize)
|
||||||
|
or an errorCode, which can be tested using FSE_isError() */
|
||||||
|
size_t FSE_decompress_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const FSE_DTable* dt, size_t fastMode);
|
||||||
|
|
||||||
|
/*
|
||||||
|
Tutorial :
|
||||||
|
----------
|
||||||
|
(Note : these functions only decompress FSE-compressed blocks.
|
||||||
|
If block is uncompressed, use memcpy() instead
|
||||||
|
If block is a single repeated byte, use memset() instead )
|
||||||
|
|
||||||
The first step is to obtain the normalized frequencies of symbols.
|
The first step is to obtain the normalized frequencies of symbols.
|
||||||
This can be performed by reading a header with FSE_readHeader().
|
This can be performed by reading a header with FSE_readHeader().
|
||||||
'normalizedCounter' must be already allocated, and have at least '*maxSymbolValuePtr+1' cells of short.
|
'normalizedCounter' must be already allocated, and have at least 'maxSymbolValuePtr[0]+1' cells of short.
|
||||||
In practice, that means it's necessary to know 'maxSymbolValue' beforehand,
|
In practice, that means it's necessary to know 'maxSymbolValue' beforehand,
|
||||||
or size the table to handle worst case situations (typically 256).
|
or size the table to handle worst case situations (typically 256).
|
||||||
FSE_readHeader will provide 'tableLog' and 'maxSymbolValue' stored into the header.
|
FSE_readHeader will provide 'tableLog' and 'maxSymbolValue' stored into the header.
|
||||||
The result of FSE_readHeader() is the number of bytes read from 'header'.
|
The result of FSE_readHeader() is the number of bytes read from 'header'.
|
||||||
The following values have special meaning :
|
Note that 'headerSize' must be at least 4 bytes, even if useful information is less than that.
|
||||||
return 2 : there is only a single symbol value. The value is provided into the second byte of header.
|
|
||||||
return 1 : data is uncompressed
|
|
||||||
If there is an error, the function will return an error code, which can be tested using FSE_isError().
|
If there is an error, the function will return an error code, which can be tested using FSE_isError().
|
||||||
|
|
||||||
The next step is to create the decompression tables 'DTable' from 'normalizedCounter'.
|
The next step is to create the decompression tables 'FSE_DTable' from 'normalizedCounter'.
|
||||||
This is performed by the function FSE_buildDTable().
|
This is performed by the function FSE_buildDTable().
|
||||||
The space required by 'DTable' must be already allocated and properly aligned.
|
The space required by 'FSE_DTable' must be already allocated using FSE_createDTable().
|
||||||
One can create a DTable using FSE_createDTable().
|
The function will return 1 if FSE_DTable is compatible with fastMode, 0 otherwise.
|
||||||
The function will return 1 if DTable is compatible with fastMode, 0 otherwise.
|
|
||||||
If there is an error, the function will return an error code, which can be tested using FSE_isError().
|
If there is an error, the function will return an error code, which can be tested using FSE_isError().
|
||||||
|
|
||||||
'DTable' can then be used to decompress 'compressed', with FSE_decompress_usingDTable().
|
'FSE_DTable' can then be used to decompress 'cSrc', with FSE_decompress_usingDTable().
|
||||||
Only trigger fastMode if it was authorized by result of FSE_buildDTable(), otherwise decompression will fail.
|
Only trigger fastMode if it was authorized by the result of FSE_buildDTable(), otherwise decompression will fail.
|
||||||
cSrcSize must be correct, otherwise decompression will fail.
|
cSrcSize must be correct, otherwise decompression will fail.
|
||||||
FSE_decompress_usingDTable() result will tell how many bytes were regenerated.
|
FSE_decompress_usingDTable() result will tell how many bytes were regenerated.
|
||||||
If there is an error, the function will return an error code, which can be tested using FSE_isError().
|
If there is an error, the function will return an error code, which can be tested using FSE_isError().
|
||||||
*/
|
*/
|
||||||
|
|
||||||
|
|
||||||
/******************************************
|
|
||||||
* FSE streaming compression API
|
|
||||||
******************************************/
|
|
||||||
typedef struct
|
|
||||||
{
|
|
||||||
size_t bitContainer;
|
|
||||||
int bitPos;
|
|
||||||
char* startPtr;
|
|
||||||
char* ptr;
|
|
||||||
} FSE_CStream_t;
|
|
||||||
|
|
||||||
typedef struct
|
|
||||||
{
|
|
||||||
ptrdiff_t value;
|
|
||||||
const void* stateTable;
|
|
||||||
const void* symbolTT;
|
|
||||||
unsigned stateLog;
|
|
||||||
} FSE_CState_t;
|
|
||||||
|
|
||||||
void FSE_initCStream(FSE_CStream_t* bitC, void* dstBuffer);
|
|
||||||
void FSE_initCState(FSE_CState_t* CStatePtr, const void* CTable);
|
|
||||||
|
|
||||||
void FSE_encodeByte(FSE_CStream_t* bitC, FSE_CState_t* CStatePtr, unsigned char symbol);
|
|
||||||
void FSE_addBits(FSE_CStream_t* bitC, size_t value, unsigned nbBits);
|
|
||||||
void FSE_flushBits(FSE_CStream_t* bitC);
|
|
||||||
|
|
||||||
void FSE_flushCState(FSE_CStream_t* bitC, const FSE_CState_t* CStatePtr);
|
|
||||||
size_t FSE_closeCStream(FSE_CStream_t* bitC);
|
|
||||||
|
|
||||||
/*
|
|
||||||
These functions are inner components of FSE_compress_usingCTable().
|
|
||||||
They allow creation of custom streams, mixing multiple tables and bit sources.
|
|
||||||
|
|
||||||
A key property to keep in mind is that encoding and decoding are done **in reverse direction**.
|
|
||||||
So the first symbol you will encode is the last you will decode, like a lifo stack.
|
|
||||||
|
|
||||||
You will need a few variables to track your CStream. They are :
|
|
||||||
|
|
||||||
void* CTable; // Provided by FSE_buildCTable()
|
|
||||||
FSE_CStream_t bitC; // bitStream tracking structure
|
|
||||||
FSE_CState_t state; // State tracking structure
|
|
||||||
|
|
||||||
|
|
||||||
The first thing to do is to init the bitStream, and the state.
|
|
||||||
FSE_initCStream(&bitC, dstBuffer);
|
|
||||||
FSE_initState(&state, CTable);
|
|
||||||
|
|
||||||
You can then encode your input data, byte after byte.
|
|
||||||
FSE_encodeByte() outputs a maximum of 'tableLog' bits at a time.
|
|
||||||
Remember decoding will be done in reverse direction.
|
|
||||||
FSE_encodeByte(&bitStream, &state, symbol);
|
|
||||||
|
|
||||||
At any time, you can add any bit sequence.
|
|
||||||
Note : maximum allowed nbBits is 25, for compatibility with 32-bits decoders
|
|
||||||
FSE_addBits(&bitStream, bitField, nbBits);
|
|
||||||
|
|
||||||
The above methods don't commit data to memory, they just store it into local register, for speed.
|
|
||||||
Local register size is 64-bits on 64-bits systems, 32-bits on 32-bits systems (size_t).
|
|
||||||
Writing data to memory is a manual operation, performed by the flushBits function.
|
|
||||||
FSE_flushBits(&bitStream);
|
|
||||||
|
|
||||||
Your last FSE encoding operation shall be to flush your last state value(s).
|
|
||||||
FSE_flushState(&bitStream, &state);
|
|
||||||
|
|
||||||
You must then close the bitStream if you opened it with FSE_initCStream().
|
|
||||||
It's possible to embed some user-info into the header, as an optionalId [0-31].
|
|
||||||
The function returns the size in bytes of CStream.
|
|
||||||
If there is an error, it returns an errorCode (which can be tested using FSE_isError()).
|
|
||||||
size_t size = FSE_closeCStream(&bitStream, optionalId);
|
|
||||||
*/
|
|
||||||
|
|
||||||
|
|
||||||
/******************************************
|
|
||||||
* FSE streaming decompression API
|
|
||||||
******************************************/
|
|
||||||
//typedef unsigned int bitD_t;
|
|
||||||
typedef size_t bitD_t;
|
|
||||||
|
|
||||||
typedef struct
|
|
||||||
{
|
|
||||||
bitD_t bitContainer;
|
|
||||||
unsigned bitsConsumed;
|
|
||||||
const char* ptr;
|
|
||||||
const char* start;
|
|
||||||
} FSE_DStream_t;
|
|
||||||
|
|
||||||
typedef struct
|
|
||||||
{
|
|
||||||
bitD_t state;
|
|
||||||
const void* table;
|
|
||||||
} FSE_DState_t;
|
|
||||||
|
|
||||||
|
|
||||||
size_t FSE_initDStream(FSE_DStream_t* bitD, const void* srcBuffer, size_t srcSize);
|
|
||||||
void FSE_initDState(FSE_DState_t* DStatePtr, FSE_DStream_t* bitD, const void* DTable);
|
|
||||||
|
|
||||||
unsigned char FSE_decodeSymbol(FSE_DState_t* DStatePtr, FSE_DStream_t* bitD);
|
|
||||||
bitD_t FSE_readBits(FSE_DStream_t* bitD, unsigned nbBits);
|
|
||||||
unsigned int FSE_reloadDStream(FSE_DStream_t* bitD);
|
|
||||||
|
|
||||||
unsigned FSE_endOfDStream(const FSE_DStream_t* bitD);
|
|
||||||
unsigned FSE_endOfDState(const FSE_DState_t* DStatePtr);
|
|
||||||
|
|
||||||
/*
|
|
||||||
Let's now decompose FSE_decompress_usingDTable() into its unitary elements.
|
|
||||||
You will decode FSE-encoded symbols from the bitStream,
|
|
||||||
and also any other bitFields you put in, **in reverse order**.
|
|
||||||
|
|
||||||
You will need a few variables to track your bitStream. They are :
|
|
||||||
|
|
||||||
FSE_DStream_t DStream; // Stream context
|
|
||||||
FSE_DState_t DState; // State context. Multiple ones are possible
|
|
||||||
const void* DTable; // Decoding table, provided by FSE_buildDTable()
|
|
||||||
U32 tableLog; // Provided by FSE_readHeader()
|
|
||||||
|
|
||||||
The first thing to do is to init the bitStream.
|
|
||||||
errorCode = FSE_initDStream(&DStream, &optionalId, srcBuffer, srcSize);
|
|
||||||
|
|
||||||
You should then retrieve your initial state(s) (multiple ones are possible) :
|
|
||||||
errorCode = FSE_initDState(&DState, &DStream, DTable, tableLog);
|
|
||||||
|
|
||||||
You can then decode your data, symbol after symbol.
|
|
||||||
For information the maximum number of bits read by FSE_decodeSymbol() is 'tableLog'.
|
|
||||||
Keep in mind that symbols are decoded in reverse order, like a lifo stack (last in, first out).
|
|
||||||
unsigned char symbol = FSE_decodeSymbol(&DState, &DStream);
|
|
||||||
|
|
||||||
You can retrieve any bitfield you eventually stored into the bitStream (in reverse order)
|
|
||||||
Note : maximum allowed nbBits is 25
|
|
||||||
unsigned int bitField = FSE_readBits(&DStream, nbBits);
|
|
||||||
|
|
||||||
All above operations only read from local register (which size is controlled by bitD_t==32 bits).
|
|
||||||
Reading data from memory is manually performed by the reload method.
|
|
||||||
endSignal = FSE_reloadDStream(&DStream);
|
|
||||||
|
|
||||||
FSE_reloadDStream() result tells if there is still some more data to read from DStream.
|
|
||||||
0 : there is still some data left into the DStream.
|
|
||||||
1 Dstream reached end of buffer, but is not yet fully extracted. It will not load data from memory any more.
|
|
||||||
2 Dstream reached its exact end, corresponding in general to decompression completed.
|
|
||||||
3 Dstream went too far. Decompression result is corrupted.
|
|
||||||
|
|
||||||
When reaching end of buffer(1), progress slowly if you decode multiple symbols per loop,
|
|
||||||
to properly detect the exact end of stream.
|
|
||||||
After each decoded symbol, check if DStream is fully consumed using this simple test :
|
|
||||||
FSE_reloadDStream(&DStream) >= 2
|
|
||||||
|
|
||||||
When it's done, verify decompression is fully completed, by checking both DStream and the relevant states.
|
|
||||||
Checking if DStream has reached its end is performed by :
|
|
||||||
FSE_endOfDStream(&DStream);
|
|
||||||
Check also the states. There might be some entropy left there, still able to decode some high probability symbol.
|
|
||||||
FSE_endOfDState(&DState);
|
|
||||||
*/
|
|
||||||
|
|
||||||
|
|
||||||
#if defined (__cplusplus)
|
#if defined (__cplusplus)
|
||||||
}
|
}
|
||||||
#endif
|
#endif
|
||||||
|
@ -40,24 +40,19 @@ extern "C" {
|
|||||||
|
|
||||||
|
|
||||||
/******************************************
|
/******************************************
|
||||||
* Tool functions
|
* FSE API compatible with DLL
|
||||||
******************************************/
|
******************************************/
|
||||||
#define FSE_MAX_HEADERSIZE 512
|
#include "fse.h"
|
||||||
#define FSE_COMPRESSBOUND(size) (size + (size>>7) + FSE_MAX_HEADERSIZE) /* Macro can be useful for static allocation */
|
|
||||||
|
|
||||||
|
|
||||||
/******************************************
|
/******************************************
|
||||||
* Static allocation
|
* Static allocation
|
||||||
******************************************/
|
******************************************/
|
||||||
/* You can statically allocate a CTable as a table of U32 using below macro */
|
#define FSE_MAX_HEADERSIZE 512
|
||||||
|
#define FSE_COMPRESSBOUND(size) (size + (size>>7) + FSE_MAX_HEADERSIZE) /* Macro can be useful for static allocation */
|
||||||
|
/* You can statically allocate a CTable as a table of unsigned using below macro */
|
||||||
#define FSE_CTABLE_SIZE_U32(maxTableLog, maxSymbolValue) (1 + (1<<(maxTableLog-1)) + ((maxSymbolValue+1)*2))
|
#define FSE_CTABLE_SIZE_U32(maxTableLog, maxSymbolValue) (1 + (1<<(maxTableLog-1)) + ((maxSymbolValue+1)*2))
|
||||||
#define FSE_DTABLE_SIZE_U32(maxTableLog) ((1<<maxTableLog)+1)
|
#define FSE_DTABLE_SIZE_U32(maxTableLog) (1 + (1<<maxTableLog))
|
||||||
|
|
||||||
|
|
||||||
/******************************************
|
|
||||||
* FSE supported API for DLL
|
|
||||||
******************************************/
|
|
||||||
#include "fse.h"
|
|
||||||
|
|
||||||
|
|
||||||
/******************************************
|
/******************************************
|
||||||
@ -65,7 +60,7 @@ extern "C" {
|
|||||||
******************************************/
|
******************************************/
|
||||||
#define FSE_LIST_ERRORS(ITEM) \
|
#define FSE_LIST_ERRORS(ITEM) \
|
||||||
ITEM(FSE_OK_NoError) ITEM(FSE_ERROR_GENERIC) \
|
ITEM(FSE_OK_NoError) ITEM(FSE_ERROR_GENERIC) \
|
||||||
ITEM(FSE_ERROR_tableLog_tooLarge) ITEM(FSE_ERROR_maxSymbolValue_tooLarge) \
|
ITEM(FSE_ERROR_tableLog_tooLarge) ITEM(FSE_ERROR_maxSymbolValue_tooLarge) ITEM(FSE_ERROR_maxSymbolValue_tooSmall) \
|
||||||
ITEM(FSE_ERROR_dstSize_tooSmall) ITEM(FSE_ERROR_srcSize_wrong)\
|
ITEM(FSE_ERROR_dstSize_tooSmall) ITEM(FSE_ERROR_srcSize_wrong)\
|
||||||
ITEM(FSE_ERROR_corruptionDetected) \
|
ITEM(FSE_ERROR_corruptionDetected) \
|
||||||
ITEM(FSE_ERROR_maxCode)
|
ITEM(FSE_ERROR_maxCode)
|
||||||
@ -77,26 +72,182 @@ typedef enum { FSE_LIST_ERRORS(FSE_GENERATE_ENUM) } FSE_errorCodes; /* enum is
|
|||||||
/******************************************
|
/******************************************
|
||||||
* FSE advanced API
|
* FSE advanced API
|
||||||
******************************************/
|
******************************************/
|
||||||
size_t FSE_countFast(unsigned* count, const unsigned char* src, size_t srcSize, unsigned* maxSymbolValuePtr);
|
size_t FSE_countFast(unsigned* count, unsigned* maxSymbolValuePtr, const unsigned char* src, size_t srcSize);
|
||||||
/* same as FSE_count(), but won't check if input really respect that all values within src are <= *maxSymbolValuePtr */
|
/* same as FSE_count(), but blindly trust that all values within src are <= maxSymbolValuePtr[0] */
|
||||||
|
|
||||||
size_t FSE_buildCTable_raw (void* CTable, unsigned nbBits);
|
size_t FSE_buildCTable_raw (FSE_CTable* ct, unsigned nbBits);
|
||||||
/* create a fake CTable, designed to not compress an input where each element uses nbBits */
|
/* build a fake FSE_CTable, designed to not compress an input, where each symbol uses nbBits */
|
||||||
|
|
||||||
size_t FSE_buildCTable_rle (void* CTable, unsigned char symbolValue);
|
size_t FSE_buildCTable_rle (FSE_CTable* ct, unsigned char symbolValue);
|
||||||
/* create a fake CTable, designed to compress a single identical value */
|
/* build a fake FSE_CTable, designed to compress always the same symbolValue */
|
||||||
|
|
||||||
size_t FSE_buildDTable_raw (void* DTable, unsigned nbBits);
|
size_t FSE_buildDTable_raw (FSE_DTable* dt, unsigned nbBits);
|
||||||
/* create a fake DTable, designed to read an uncompressed bitstream where each element uses nbBits */
|
/* build a fake FSE_DTable, designed to read an uncompressed bitstream where each symbol uses nbBits */
|
||||||
|
|
||||||
size_t FSE_buildDTable_rle (void* DTable, unsigned char symbolValue);
|
size_t FSE_buildDTable_rle (FSE_DTable* dt, unsigned char symbolValue);
|
||||||
/* create a fake DTable, designed to always generate the same symbolValue */
|
/* build a fake FSE_DTable, designed to always generate the same symbolValue */
|
||||||
|
|
||||||
|
|
||||||
/******************************************
|
/******************************************
|
||||||
* FSE streaming API
|
* FSE symbol compression API
|
||||||
******************************************/
|
******************************************/
|
||||||
bitD_t FSE_readBitsFast(FSE_DStream_t* bitD, unsigned nbBits);
|
/*
|
||||||
|
This API consists of small unitary functions, which highly benefit from being inlined.
|
||||||
|
You will want to enable link-time-optimization to ensure these functions are properly inlined in your binary.
|
||||||
|
Visual seems to do it automatically.
|
||||||
|
For gcc or clang, you'll need to add -flto flag at compilation and linking stages.
|
||||||
|
*/
|
||||||
|
|
||||||
|
typedef struct
|
||||||
|
{
|
||||||
|
size_t bitContainer;
|
||||||
|
int bitPos;
|
||||||
|
char* startPtr;
|
||||||
|
char* ptr;
|
||||||
|
} FSE_CStream_t;
|
||||||
|
|
||||||
|
typedef struct
|
||||||
|
{
|
||||||
|
ptrdiff_t value;
|
||||||
|
const void* stateTable;
|
||||||
|
const void* symbolTT;
|
||||||
|
unsigned stateLog;
|
||||||
|
} FSE_CState_t;
|
||||||
|
|
||||||
|
void FSE_initCStream(FSE_CStream_t* bitC, void* dstBuffer);
|
||||||
|
void FSE_initCState(FSE_CState_t* CStatePtr, const FSE_CTable* ct);
|
||||||
|
|
||||||
|
void FSE_encodeSymbol(FSE_CStream_t* bitC, FSE_CState_t* CStatePtr, unsigned char symbol);
|
||||||
|
void FSE_addBits(FSE_CStream_t* bitC, size_t value, unsigned nbBits);
|
||||||
|
void FSE_flushBits(FSE_CStream_t* bitC);
|
||||||
|
|
||||||
|
void FSE_flushCState(FSE_CStream_t* bitC, const FSE_CState_t* CStatePtr);
|
||||||
|
size_t FSE_closeCStream(FSE_CStream_t* bitC);
|
||||||
|
|
||||||
|
/*
|
||||||
|
These functions are inner components of FSE_compress_usingCTable().
|
||||||
|
They allow the creation of custom streams, mixing multiple tables and bit sources.
|
||||||
|
|
||||||
|
A key property to keep in mind is that encoding and decoding are done **in reverse direction**.
|
||||||
|
So the first symbol you will encode is the last you will decode, like a LIFO stack.
|
||||||
|
|
||||||
|
You will need a few variables to track your CStream. They are :
|
||||||
|
|
||||||
|
FSE_CTable ct; // Provided by FSE_buildCTable()
|
||||||
|
FSE_CStream_t bitC; // bitStream tracking structure
|
||||||
|
FSE_CState_t state; // State tracking structure (can have several)
|
||||||
|
|
||||||
|
|
||||||
|
The first thing to do is to init bitStream and state.
|
||||||
|
FSE_initCStream(&bitC, dstBuffer);
|
||||||
|
FSE_initCState(&state, ct);
|
||||||
|
|
||||||
|
You can then encode your input data, byte after byte.
|
||||||
|
FSE_encodeByte() outputs a maximum of 'tableLog' bits at a time.
|
||||||
|
Remember decoding will be done in reverse direction.
|
||||||
|
FSE_encodeByte(&bitStream, &state, symbol);
|
||||||
|
|
||||||
|
At any time, you can also add any bit sequence.
|
||||||
|
Note : maximum allowed nbBits is 25, for compatibility with 32-bits decoders
|
||||||
|
FSE_addBits(&bitStream, bitField, nbBits);
|
||||||
|
|
||||||
|
The above methods don't commit data to memory, they just store it into local register, for speed.
|
||||||
|
Local register size is 64-bits on 64-bits systems, 32-bits on 32-bits systems (size_t).
|
||||||
|
Writing data to memory is a manual operation, performed by the flushBits function.
|
||||||
|
FSE_flushBits(&bitStream);
|
||||||
|
|
||||||
|
Your last FSE encoding operation shall be to flush your last state value(s).
|
||||||
|
FSE_flushState(&bitStream, &state);
|
||||||
|
|
||||||
|
Finally, you must then close the bitStream.
|
||||||
|
The function returns the size in bytes of CStream.
|
||||||
|
If there is an error, it returns an errorCode (which can be tested using FSE_isError()).
|
||||||
|
size_t size = FSE_closeCStream(&bitStream);
|
||||||
|
*/
|
||||||
|
|
||||||
|
|
||||||
|
/******************************************
|
||||||
|
* FSE symbol decompression API
|
||||||
|
******************************************/
|
||||||
|
typedef struct
|
||||||
|
{
|
||||||
|
size_t bitContainer;
|
||||||
|
unsigned bitsConsumed;
|
||||||
|
const char* ptr;
|
||||||
|
const char* start;
|
||||||
|
} FSE_DStream_t;
|
||||||
|
|
||||||
|
typedef struct
|
||||||
|
{
|
||||||
|
size_t state;
|
||||||
|
const void* table; /* precise table may vary, depending on U16 */
|
||||||
|
} FSE_DState_t;
|
||||||
|
|
||||||
|
|
||||||
|
size_t FSE_initDStream(FSE_DStream_t* bitD, const void* srcBuffer, size_t srcSize);
|
||||||
|
void FSE_initDState(FSE_DState_t* DStatePtr, FSE_DStream_t* bitD, const FSE_DTable* dt);
|
||||||
|
|
||||||
|
unsigned char FSE_decodeSymbol(FSE_DState_t* DStatePtr, FSE_DStream_t* bitD);
|
||||||
|
size_t FSE_readBits(FSE_DStream_t* bitD, unsigned nbBits);
|
||||||
|
unsigned int FSE_reloadDStream(FSE_DStream_t* bitD);
|
||||||
|
|
||||||
|
unsigned FSE_endOfDStream(const FSE_DStream_t* bitD);
|
||||||
|
unsigned FSE_endOfDState(const FSE_DState_t* DStatePtr);
|
||||||
|
|
||||||
|
/*
|
||||||
|
Let's now decompose FSE_decompress_usingDTable() into its unitary components.
|
||||||
|
You will decode FSE-encoded symbols from the bitStream,
|
||||||
|
and also any other bitFields you put in, **in reverse order**.
|
||||||
|
|
||||||
|
You will need a few variables to track your bitStream. They are :
|
||||||
|
|
||||||
|
FSE_DStream_t DStream; // Stream context
|
||||||
|
FSE_DState_t DState; // State context. Multiple ones are possible
|
||||||
|
FSE_DTable dt; // Decoding table, provided by FSE_buildDTable()
|
||||||
|
U32 tableLog; // Provided by FSE_readHeader()
|
||||||
|
|
||||||
|
The first thing to do is to init the bitStream.
|
||||||
|
errorCode = FSE_initDStream(&DStream, &optionalId, srcBuffer, srcSize);
|
||||||
|
|
||||||
|
You should then retrieve your initial state(s) :
|
||||||
|
errorCode = FSE_initDState(&DState, &DStream, dt, tableLog);
|
||||||
|
|
||||||
|
You can then decode your data, symbol after symbol.
|
||||||
|
For information the maximum number of bits read by FSE_decodeSymbol() is 'tableLog'.
|
||||||
|
Keep in mind that symbols are decoded in reverse order, like a LIFO stack (last in, first out).
|
||||||
|
unsigned char symbol = FSE_decodeSymbol(&DState, &DStream);
|
||||||
|
|
||||||
|
You can retrieve any bitfield you eventually stored into the bitStream (in reverse order)
|
||||||
|
Note : maximum allowed nbBits is 25
|
||||||
|
unsigned int bitField = FSE_readBits(&DStream, nbBits);
|
||||||
|
|
||||||
|
All above operations only read from local register (which size is controlled by bitD_t==32 bits).
|
||||||
|
Refueling the register from memory is manually performed by the reload method.
|
||||||
|
endSignal = FSE_reloadDStream(&DStream);
|
||||||
|
|
||||||
|
FSE_reloadDStream() result tells if there is still some more data to read from DStream.
|
||||||
|
0 : there is still some data left into the DStream.
|
||||||
|
1 : Dstream reached end of buffer, but is not yet fully extracted. It will not load data from memory any more.
|
||||||
|
2 : Dstream reached its exact end, corresponding in general to decompression completed.
|
||||||
|
3 : Dstream went too far. Decompression result is corrupted.
|
||||||
|
|
||||||
|
When reaching end of buffer(1), progress slowly, notably if you decode multiple symbols per loop,
|
||||||
|
to properly detect the exact end of stream.
|
||||||
|
After each decoded symbol, check if DStream is fully consumed using this simple test :
|
||||||
|
FSE_reloadDStream(&DStream) >= 2
|
||||||
|
|
||||||
|
When it's done, verify decompression is fully completed, by checking both DStream and the relevant states.
|
||||||
|
Checking if DStream has reached its end is performed by :
|
||||||
|
FSE_endOfDStream(&DStream);
|
||||||
|
Check also the states. There might be some entropy left there, able to decode some high probability (>50%) symbol.
|
||||||
|
FSE_endOfDState(&DState);
|
||||||
|
*/
|
||||||
|
|
||||||
|
|
||||||
|
/******************************************
|
||||||
|
* FSE unsafe symbol API
|
||||||
|
******************************************/
|
||||||
|
size_t FSE_readBitsFast(FSE_DStream_t* bitD, unsigned nbBits);
|
||||||
/* faster, but works only if nbBits >= 1 (otherwise, result will be corrupted) */
|
/* faster, but works only if nbBits >= 1 (otherwise, result will be corrupted) */
|
||||||
|
|
||||||
unsigned char FSE_decodeSymbolFast(FSE_DState_t* DStatePtr, FSE_DStream_t* bitD);
|
unsigned char FSE_decodeSymbolFast(FSE_DState_t* DStatePtr, FSE_DStream_t* bitD);
|
||||||
|
@ -146,7 +146,7 @@ static const U32 g_maxDistance = 4 * BLOCKSIZE;
|
|||||||
static const U32 g_maxLimit = 1 GB;
|
static const U32 g_maxLimit = 1 GB;
|
||||||
static const U32 g_searchStrength = 8;
|
static const U32 g_searchStrength = 8;
|
||||||
|
|
||||||
#define WORKPLACESIZE (BLOCKSIZE*11/4)
|
#define WORKPLACESIZE (BLOCKSIZE*3)
|
||||||
#define MINMATCH 4
|
#define MINMATCH 4
|
||||||
#define MLbits 7
|
#define MLbits 7
|
||||||
#define LLbits 6
|
#define LLbits 6
|
||||||
@ -178,11 +178,13 @@ static unsigned ZSTD_isLittleEndian(void)
|
|||||||
return one.c[0];
|
return one.c[0];
|
||||||
}
|
}
|
||||||
|
|
||||||
static U16 ZSTD_read16(const void* p) { return *(U16*)p; }
|
static U16 ZSTD_read16(const void* p) { U16 r; memcpy(&r, p, sizeof(r)); return r; }
|
||||||
|
|
||||||
static U32 ZSTD_read32(const void* p) { return *(U32*)p; }
|
static U32 ZSTD_read32(const void* p) { U32 r; memcpy(&r, p, sizeof(r)); return r; }
|
||||||
|
|
||||||
static size_t ZSTD_read_ARCH(const void* p) { return *(size_t*)p; }
|
static U64 ZSTD_read64(const void* p) { U64 r; memcpy(&r, p, sizeof(r)); return r; }
|
||||||
|
|
||||||
|
static size_t ZSTD_read_ARCH(const void* p) { size_t r; memcpy(&r, p, sizeof(r)); return r; }
|
||||||
|
|
||||||
static void ZSTD_copy4(void* dst, const void* src) { memcpy(dst, src, 4); }
|
static void ZSTD_copy4(void* dst, const void* src) { memcpy(dst, src, 4); }
|
||||||
|
|
||||||
@ -290,6 +292,8 @@ typedef struct {
|
|||||||
void* buffer;
|
void* buffer;
|
||||||
U32* offsetStart;
|
U32* offsetStart;
|
||||||
U32* offset;
|
U32* offset;
|
||||||
|
BYTE* offCodeStart;
|
||||||
|
BYTE* offCode;
|
||||||
BYTE* litStart;
|
BYTE* litStart;
|
||||||
BYTE* lit;
|
BYTE* lit;
|
||||||
BYTE* litLengthStart;
|
BYTE* litLengthStart;
|
||||||
@ -310,7 +314,7 @@ void ZSTD_resetSeqStore(seqStore_t* ssPtr)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
typedef struct
|
typedef struct ZSTD_Cctx_s
|
||||||
{
|
{
|
||||||
const BYTE* base;
|
const BYTE* base;
|
||||||
U32 current;
|
U32 current;
|
||||||
@ -324,28 +328,28 @@ typedef struct
|
|||||||
} cctxi_t;
|
} cctxi_t;
|
||||||
|
|
||||||
|
|
||||||
ZSTD_cctx_t ZSTD_createCCtx(void)
|
ZSTD_Cctx* ZSTD_createCCtx(void)
|
||||||
{
|
{
|
||||||
cctxi_t* ctx = (cctxi_t*) malloc( sizeof(cctxi_t) );
|
ZSTD_Cctx* ctx = (ZSTD_Cctx*) malloc( sizeof(ZSTD_Cctx) );
|
||||||
|
if (ctx==NULL) return NULL;
|
||||||
ctx->seqStore.buffer = malloc(WORKPLACESIZE);
|
ctx->seqStore.buffer = malloc(WORKPLACESIZE);
|
||||||
ctx->seqStore.offsetStart = (U32*) (ctx->seqStore.buffer);
|
ctx->seqStore.offsetStart = (U32*) (ctx->seqStore.buffer);
|
||||||
ctx->seqStore.litStart = (BYTE*) (ctx->seqStore.offsetStart + (BLOCKSIZE>>2));
|
ctx->seqStore.offCodeStart = (BYTE*) (ctx->seqStore.offsetStart + (BLOCKSIZE>>2));
|
||||||
|
ctx->seqStore.litStart = ctx->seqStore.offCodeStart + (BLOCKSIZE>>2);
|
||||||
ctx->seqStore.litLengthStart = ctx->seqStore.litStart + BLOCKSIZE;
|
ctx->seqStore.litLengthStart = ctx->seqStore.litStart + BLOCKSIZE;
|
||||||
ctx->seqStore.matchLengthStart = ctx->seqStore.litLengthStart + (BLOCKSIZE>>2);
|
ctx->seqStore.matchLengthStart = ctx->seqStore.litLengthStart + (BLOCKSIZE>>2);
|
||||||
ctx->seqStore.dumpsStart = ctx->seqStore.matchLengthStart + (BLOCKSIZE>>2);
|
ctx->seqStore.dumpsStart = ctx->seqStore.matchLengthStart + (BLOCKSIZE>>2);
|
||||||
return (ZSTD_cctx_t)ctx;
|
return ctx;
|
||||||
}
|
}
|
||||||
|
|
||||||
void ZSTD_resetCCtx(ZSTD_cctx_t cctx)
|
void ZSTD_resetCCtx(ZSTD_Cctx* ctx)
|
||||||
{
|
{
|
||||||
cctxi_t* ctx = (cctxi_t*)cctx;
|
|
||||||
ctx->base = NULL;
|
ctx->base = NULL;
|
||||||
memset(ctx->hashTable, 0, HASH_TABLESIZE*4);
|
memset(ctx->hashTable, 0, HASH_TABLESIZE*4);
|
||||||
}
|
}
|
||||||
|
|
||||||
size_t ZSTD_freeCCtx(ZSTD_cctx_t cctx)
|
size_t ZSTD_freeCCtx(ZSTD_Cctx* ctx)
|
||||||
{
|
{
|
||||||
cctxi_t* ctx = (cctxi_t*) (cctx);
|
|
||||||
free(ctx->seqStore.buffer);
|
free(ctx->seqStore.buffer);
|
||||||
free(ctx);
|
free(ctx);
|
||||||
return 0;
|
return 0;
|
||||||
@ -457,7 +461,7 @@ static unsigned ZSTD_NbCommonBytes (register size_t val)
|
|||||||
_BitScanReverse( &r, (unsigned long)val );
|
_BitScanReverse( &r, (unsigned long)val );
|
||||||
return (unsigned)(r>>3);
|
return (unsigned)(r>>3);
|
||||||
# elif defined(__GNUC__) && (GCC_VERSION >= 304) && !defined(LZ4_FORCE_SW_BITCOUNT)
|
# elif defined(__GNUC__) && (GCC_VERSION >= 304) && !defined(LZ4_FORCE_SW_BITCOUNT)
|
||||||
return (__builtin_clz(val) >> 3);
|
return (__builtin_clz((U32)val) >> 3);
|
||||||
# else
|
# else
|
||||||
unsigned r;
|
unsigned r;
|
||||||
if (!(val>>16)) { r=2; val>>=8; } else { r=0; val>>=24; }
|
if (!(val>>16)) { r=2; val>>=8; } else { r=0; val>>=24; }
|
||||||
@ -503,7 +507,7 @@ static size_t ZSTD_compressRle (void* dst, size_t maxDstSize, const void* src, s
|
|||||||
/* at this stage : dstSize >= FSE_compressBound(srcSize) > (ZSTD_blockHeaderSize+1) (checked by ZSTD_compressLiterals()) */
|
/* at this stage : dstSize >= FSE_compressBound(srcSize) > (ZSTD_blockHeaderSize+1) (checked by ZSTD_compressLiterals()) */
|
||||||
(void)maxDstSize;
|
(void)maxDstSize;
|
||||||
|
|
||||||
ostart[ZSTD_blockHeaderSize] = *(BYTE*)src;
|
ostart[ZSTD_blockHeaderSize] = *(const BYTE*)src;
|
||||||
|
|
||||||
/* Build header */
|
/* Build header */
|
||||||
ostart[0] = (BYTE)(srcSize>>16);
|
ostart[0] = (BYTE)(srcSize>>16);
|
||||||
@ -535,7 +539,7 @@ static size_t ZSTD_noCompressBlock (void* dst, size_t maxDstSize, const void* sr
|
|||||||
/* return : size of CStream in bits */
|
/* return : size of CStream in bits */
|
||||||
static size_t ZSTD_compressLiterals_usingCTable(void* dst, size_t dstSize,
|
static size_t ZSTD_compressLiterals_usingCTable(void* dst, size_t dstSize,
|
||||||
const void* src, size_t srcSize,
|
const void* src, size_t srcSize,
|
||||||
const void* CTable)
|
const FSE_CTable* CTable)
|
||||||
{
|
{
|
||||||
const BYTE* const istart = (const BYTE*)src;
|
const BYTE* const istart = (const BYTE*)src;
|
||||||
const BYTE* ip = istart;
|
const BYTE* ip = istart;
|
||||||
@ -553,32 +557,32 @@ static size_t ZSTD_compressLiterals_usingCTable(void* dst, size_t dstSize,
|
|||||||
// join to mod 2
|
// join to mod 2
|
||||||
if (srcSize & 1)
|
if (srcSize & 1)
|
||||||
{
|
{
|
||||||
FSE_encodeByte(&bitC, &CState1, *ip++);
|
FSE_encodeSymbol(&bitC, &CState1, *ip++);
|
||||||
FSE_flushBits(&bitC);
|
FSE_flushBits(&bitC);
|
||||||
}
|
}
|
||||||
|
|
||||||
// join to mod 4
|
// join to mod 4
|
||||||
if ((sizeof(size_t)*8 > LitFSELog*4+7 ) && (srcSize & 2)) // test bit 2
|
if ((sizeof(size_t)*8 > LitFSELog*4+7 ) && (srcSize & 2)) // test bit 2
|
||||||
{
|
{
|
||||||
FSE_encodeByte(&bitC, &CState2, *ip++);
|
FSE_encodeSymbol(&bitC, &CState2, *ip++);
|
||||||
FSE_encodeByte(&bitC, &CState1, *ip++);
|
FSE_encodeSymbol(&bitC, &CState1, *ip++);
|
||||||
FSE_flushBits(&bitC);
|
FSE_flushBits(&bitC);
|
||||||
}
|
}
|
||||||
|
|
||||||
// 2 or 4 encoding per loop
|
// 2 or 4 encoding per loop
|
||||||
while (ip<iend)
|
while (ip<iend)
|
||||||
{
|
{
|
||||||
FSE_encodeByte(&bitC, &CState2, *ip++);
|
FSE_encodeSymbol(&bitC, &CState2, *ip++);
|
||||||
|
|
||||||
if (sizeof(size_t)*8 < LitFSELog*2+7 ) // this test must be static
|
if (sizeof(size_t)*8 < LitFSELog*2+7 ) // this test must be static
|
||||||
FSE_flushBits(&bitC);
|
FSE_flushBits(&bitC);
|
||||||
|
|
||||||
FSE_encodeByte(&bitC, &CState1, *ip++);
|
FSE_encodeSymbol(&bitC, &CState1, *ip++);
|
||||||
|
|
||||||
if (sizeof(size_t)*8 > LitFSELog*4+7 ) // this test must be static
|
if (sizeof(size_t)*8 > LitFSELog*4+7 ) // this test must be static
|
||||||
{
|
{
|
||||||
FSE_encodeByte(&bitC, &CState2, *ip++);
|
FSE_encodeSymbol(&bitC, &CState2, *ip++);
|
||||||
FSE_encodeByte(&bitC, &CState1, *ip++);
|
FSE_encodeSymbol(&bitC, &CState1, *ip++);
|
||||||
}
|
}
|
||||||
|
|
||||||
FSE_flushBits(&bitC);
|
FSE_flushBits(&bitC);
|
||||||
@ -618,26 +622,24 @@ static size_t ZSTD_compressLiterals (void* dst, size_t dstSize,
|
|||||||
if (dstSize < FSE_compressBound(srcSize)) return (size_t)-ZSTD_ERROR_maxDstSize_tooSmall;
|
if (dstSize < FSE_compressBound(srcSize)) return (size_t)-ZSTD_ERROR_maxDstSize_tooSmall;
|
||||||
|
|
||||||
/* Scan input and build symbol stats */
|
/* Scan input and build symbol stats */
|
||||||
errorCode = FSE_count (count, ip, srcSize, &maxSymbolValue);
|
errorCode = FSE_count (count, &maxSymbolValue, ip, srcSize);
|
||||||
if (FSE_isError(errorCode)) return (size_t)-ZSTD_ERROR_GENERIC;
|
if (FSE_isError(errorCode)) return (size_t)-ZSTD_ERROR_GENERIC;
|
||||||
if (errorCode == srcSize) return 1;
|
if (errorCode == srcSize) return 1;
|
||||||
//if (errorCode < ((srcSize * 7) >> 10)) return 0;
|
if (errorCode < (srcSize >> 6)) return 0; /* cheap heuristic : probably not compressible enough */
|
||||||
//if (errorCode < (srcSize >> 7)) return 0;
|
|
||||||
if (errorCode < (srcSize >> 6)) return 0; /* heuristic : probably not compressible enough */
|
|
||||||
|
|
||||||
tableLog = FSE_optimalTableLog(tableLog, srcSize, maxSymbolValue);
|
tableLog = FSE_optimalTableLog(tableLog, srcSize, maxSymbolValue);
|
||||||
errorCode = (int)FSE_normalizeCount (norm, tableLog, count, srcSize, maxSymbolValue);
|
errorCode = (int)FSE_normalizeCount (norm, tableLog, count, srcSize, maxSymbolValue);
|
||||||
if (FSE_isError(errorCode)) return (size_t)-ZSTD_ERROR_GENERIC;
|
if (FSE_isError(errorCode)) return (size_t)-ZSTD_ERROR_GENERIC;
|
||||||
|
|
||||||
/* Write table description header */
|
/* Write table description header */
|
||||||
errorCode = FSE_writeHeader (op, FSE_MAX_HEADERSIZE, norm, maxSymbolValue, tableLog);
|
errorCode = FSE_writeNCount (op, FSE_MAX_HEADERSIZE, norm, maxSymbolValue, tableLog);
|
||||||
if (FSE_isError(errorCode)) return (size_t)-ZSTD_ERROR_GENERIC;
|
if (FSE_isError(errorCode)) return (size_t)-ZSTD_ERROR_GENERIC;
|
||||||
op += errorCode;
|
op += errorCode;
|
||||||
|
|
||||||
/* Compress */
|
/* Compress */
|
||||||
errorCode = FSE_buildCTable (&CTable, norm, maxSymbolValue, tableLog);
|
errorCode = FSE_buildCTable (CTable, norm, maxSymbolValue, tableLog);
|
||||||
if (FSE_isError(errorCode)) return (size_t)-ZSTD_ERROR_GENERIC;
|
if (FSE_isError(errorCode)) return (size_t)-ZSTD_ERROR_GENERIC;
|
||||||
errorCode = ZSTD_compressLiterals_usingCTable(op, oend - op, ip, srcSize, &CTable);
|
errorCode = ZSTD_compressLiterals_usingCTable(op, oend - op, ip, srcSize, CTable);
|
||||||
if (ZSTD_isError(errorCode)) return errorCode;
|
if (ZSTD_isError(errorCode)) return errorCode;
|
||||||
op += errorCode;
|
op += errorCode;
|
||||||
|
|
||||||
@ -681,7 +683,7 @@ static size_t ZSTD_compressSequences(BYTE* dst, size_t maxDstSize,
|
|||||||
const BYTE* op_matchLength = seqStorePtr->matchLength;
|
const BYTE* op_matchLength = seqStorePtr->matchLength;
|
||||||
const size_t nbSeq = op_litLength - op_litLength_start;
|
const size_t nbSeq = op_litLength - op_litLength_start;
|
||||||
BYTE* op;
|
BYTE* op;
|
||||||
BYTE offsetBits_start[BLOCKSIZE / 4];
|
BYTE* offsetBits_start = seqStorePtr->offCodeStart;
|
||||||
BYTE* offsetBitsPtr = offsetBits_start;
|
BYTE* offsetBitsPtr = offsetBits_start;
|
||||||
const size_t minGain = ZSTD_minGain(srcSize);
|
const size_t minGain = ZSTD_minGain(srcSize);
|
||||||
const size_t maxCSize = srcSize - minGain;
|
const size_t maxCSize = srcSize - minGain;
|
||||||
@ -689,7 +691,6 @@ static size_t ZSTD_compressSequences(BYTE* dst, size_t maxDstSize,
|
|||||||
const size_t maxLSize = maxCSize > minSeqSize ? maxCSize - minSeqSize : 0;
|
const size_t maxLSize = maxCSize > minSeqSize ? maxCSize - minSeqSize : 0;
|
||||||
BYTE* seqHead;
|
BYTE* seqHead;
|
||||||
|
|
||||||
|
|
||||||
/* init */
|
/* init */
|
||||||
op = dst;
|
op = dst;
|
||||||
|
|
||||||
@ -740,8 +741,8 @@ static size_t ZSTD_compressSequences(BYTE* dst, size_t maxDstSize,
|
|||||||
|
|
||||||
/* Encoding table of Literal Lengths */
|
/* Encoding table of Literal Lengths */
|
||||||
max = MaxLL;
|
max = MaxLL;
|
||||||
mostFrequent = FSE_countFast(count, seqStorePtr->litLengthStart, nbSeq, &max);
|
mostFrequent = FSE_countFast(count, &max, seqStorePtr->litLengthStart, nbSeq);
|
||||||
if (mostFrequent == nbSeq)
|
if ((mostFrequent == nbSeq) && (nbSeq > 2))
|
||||||
{
|
{
|
||||||
*op++ = *(seqStorePtr->litLengthStart);
|
*op++ = *(seqStorePtr->litLengthStart);
|
||||||
FSE_buildCTable_rle(CTable_LitLength, (BYTE)max);
|
FSE_buildCTable_rle(CTable_LitLength, (BYTE)max);
|
||||||
@ -756,7 +757,7 @@ static size_t ZSTD_compressSequences(BYTE* dst, size_t maxDstSize,
|
|||||||
{
|
{
|
||||||
tableLog = FSE_optimalTableLog(LLFSELog, nbSeq, max);
|
tableLog = FSE_optimalTableLog(LLFSELog, nbSeq, max);
|
||||||
FSE_normalizeCount(norm, tableLog, count, nbSeq, max);
|
FSE_normalizeCount(norm, tableLog, count, nbSeq, max);
|
||||||
op += FSE_writeHeader(op, maxDstSize, norm, max, tableLog);
|
op += FSE_writeNCount(op, maxDstSize, norm, max, tableLog);
|
||||||
FSE_buildCTable(CTable_LitLength, norm, max, tableLog);
|
FSE_buildCTable(CTable_LitLength, norm, max, tableLog);
|
||||||
LLtype = bt_compressed;
|
LLtype = bt_compressed;
|
||||||
}
|
}
|
||||||
@ -773,9 +774,9 @@ static size_t ZSTD_compressSequences(BYTE* dst, size_t maxDstSize,
|
|||||||
if (op_offset_start[i]==0) offsetBits_start[i]=0;
|
if (op_offset_start[i]==0) offsetBits_start[i]=0;
|
||||||
}
|
}
|
||||||
offsetBitsPtr += nbSeq;
|
offsetBitsPtr += nbSeq;
|
||||||
mostFrequent = FSE_countFast(count, offsetBits_start, nbSeq, &max);
|
mostFrequent = FSE_countFast(count, &max, offsetBits_start, nbSeq);
|
||||||
}
|
}
|
||||||
if (mostFrequent == nbSeq)
|
if ((mostFrequent == nbSeq) && (nbSeq > 2))
|
||||||
{
|
{
|
||||||
*op++ = *offsetBits_start;
|
*op++ = *offsetBits_start;
|
||||||
FSE_buildCTable_rle(CTable_OffsetBits, (BYTE)max);
|
FSE_buildCTable_rle(CTable_OffsetBits, (BYTE)max);
|
||||||
@ -790,15 +791,15 @@ static size_t ZSTD_compressSequences(BYTE* dst, size_t maxDstSize,
|
|||||||
{
|
{
|
||||||
tableLog = FSE_optimalTableLog(OffFSELog, nbSeq, max);
|
tableLog = FSE_optimalTableLog(OffFSELog, nbSeq, max);
|
||||||
FSE_normalizeCount(norm, tableLog, count, nbSeq, max);
|
FSE_normalizeCount(norm, tableLog, count, nbSeq, max);
|
||||||
op += FSE_writeHeader(op, maxDstSize, norm, max, tableLog);
|
op += FSE_writeNCount(op, maxDstSize, norm, max, tableLog);
|
||||||
FSE_buildCTable(CTable_OffsetBits, norm, max, tableLog);
|
FSE_buildCTable(CTable_OffsetBits, norm, max, tableLog);
|
||||||
Offtype = bt_compressed;
|
Offtype = bt_compressed;
|
||||||
}
|
}
|
||||||
|
|
||||||
/* Encoding Table of MatchLengths */
|
/* Encoding Table of MatchLengths */
|
||||||
max = MaxML;
|
max = MaxML;
|
||||||
mostFrequent = FSE_countFast(count, seqStorePtr->matchLengthStart, nbSeq, &max);
|
mostFrequent = FSE_countFast(count, &max, seqStorePtr->matchLengthStart, nbSeq);
|
||||||
if (mostFrequent == nbSeq)
|
if ((mostFrequent == nbSeq) && (nbSeq > 2))
|
||||||
{
|
{
|
||||||
*op++ = *seqStorePtr->matchLengthStart;
|
*op++ = *seqStorePtr->matchLengthStart;
|
||||||
FSE_buildCTable_rle(CTable_MatchLength, (BYTE)max);
|
FSE_buildCTable_rle(CTable_MatchLength, (BYTE)max);
|
||||||
@ -813,7 +814,7 @@ static size_t ZSTD_compressSequences(BYTE* dst, size_t maxDstSize,
|
|||||||
{
|
{
|
||||||
tableLog = FSE_optimalTableLog(MLFSELog, nbSeq, max);
|
tableLog = FSE_optimalTableLog(MLFSELog, nbSeq, max);
|
||||||
FSE_normalizeCount(norm, tableLog, count, nbSeq, max);
|
FSE_normalizeCount(norm, tableLog, count, nbSeq, max);
|
||||||
op += FSE_writeHeader(op, maxDstSize, norm, max, tableLog);
|
op += FSE_writeNCount(op, maxDstSize, norm, max, tableLog);
|
||||||
FSE_buildCTable(CTable_MatchLength, norm, max, tableLog);
|
FSE_buildCTable(CTable_MatchLength, norm, max, tableLog);
|
||||||
MLtype = bt_compressed;
|
MLtype = bt_compressed;
|
||||||
}
|
}
|
||||||
@ -838,12 +839,12 @@ static size_t ZSTD_compressSequences(BYTE* dst, size_t maxDstSize,
|
|||||||
BYTE offCode = *(--offsetBitsPtr); /* 32b*/ /* 64b*/
|
BYTE offCode = *(--offsetBitsPtr); /* 32b*/ /* 64b*/
|
||||||
U32 nbBits = (offCode-1) * (!!offCode);
|
U32 nbBits = (offCode-1) * (!!offCode);
|
||||||
BYTE litLength = *(--op_litLength); /* (7)*/ /* (7)*/
|
BYTE litLength = *(--op_litLength); /* (7)*/ /* (7)*/
|
||||||
FSE_encodeByte(&blockStream, &stateMatchLength, matchLength); /* 17 */ /* 17 */
|
FSE_encodeSymbol(&blockStream, &stateMatchLength, matchLength); /* 17 */ /* 17 */
|
||||||
if (ZSTD_32bits()) FSE_flushBits(&blockStream); /* 7 */
|
if (ZSTD_32bits()) FSE_flushBits(&blockStream); /* 7 */
|
||||||
FSE_addBits(&blockStream, offset, nbBits); /* 32 */ /* 42 */
|
FSE_addBits(&blockStream, offset, nbBits); /* 32 */ /* 42 */
|
||||||
if (ZSTD_32bits()) FSE_flushBits(&blockStream); /* 7 */
|
if (ZSTD_32bits()) FSE_flushBits(&blockStream); /* 7 */
|
||||||
FSE_encodeByte(&blockStream, &stateOffsetBits, offCode); /* 16 */ /* 51 */
|
FSE_encodeSymbol(&blockStream, &stateOffsetBits, offCode); /* 16 */ /* 51 */
|
||||||
FSE_encodeByte(&blockStream, &stateLitLength, litLength); /* 26 */ /* 61 */
|
FSE_encodeSymbol(&blockStream, &stateLitLength, litLength); /* 26 */ /* 61 */
|
||||||
FSE_flushBits(&blockStream); /* 7 */ /* 7 */
|
FSE_flushBits(&blockStream); /* 7 */ /* 7 */
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -915,7 +916,7 @@ static const U64 prime7bytes = 58295818150454627ULL;
|
|||||||
//static U32 ZSTD_hashPtr(const void* p) { return ( ((*(U64*)p & 0xFFFFFFFFFFFFFF) * prime7bytes) >> (64-HASH_LOG)); }
|
//static U32 ZSTD_hashPtr(const void* p) { return ( ((*(U64*)p & 0xFFFFFFFFFFFFFF) * prime7bytes) >> (64-HASH_LOG)); }
|
||||||
|
|
||||||
//static U32 ZSTD_hashPtr(const void* p) { return ( (*(U64*)p * prime8bytes) >> (64-HASH_LOG)); }
|
//static U32 ZSTD_hashPtr(const void* p) { return ( (*(U64*)p * prime8bytes) >> (64-HASH_LOG)); }
|
||||||
static U32 ZSTD_hashPtr(const void* p) { return ( (*(U64*)p * prime7bytes) >> (56-HASH_LOG)) & HASH_MASK; }
|
static U32 ZSTD_hashPtr(const void* p) { return ( (ZSTD_read64(p) * prime7bytes) >> (56-HASH_LOG)) & HASH_MASK; }
|
||||||
//static U32 ZSTD_hashPtr(const void* p) { return ( (*(U64*)p * prime6bytes) >> (48-HASH_LOG)) & HASH_MASK; }
|
//static U32 ZSTD_hashPtr(const void* p) { return ( (*(U64*)p * prime6bytes) >> (48-HASH_LOG)) & HASH_MASK; }
|
||||||
//static U32 ZSTD_hashPtr(const void* p) { return ( (*(U64*)p * prime5bytes) >> (40-HASH_LOG)) & HASH_MASK; }
|
//static U32 ZSTD_hashPtr(const void* p) { return ( (*(U64*)p * prime5bytes) >> (40-HASH_LOG)) & HASH_MASK; }
|
||||||
//static U32 ZSTD_hashPtr(const void* p) { return ( (*(U32*)p * KNUTH) >> (32-HASH_LOG)); }
|
//static U32 ZSTD_hashPtr(const void* p) { return ( (*(U32*)p * KNUTH) >> (32-HASH_LOG)); }
|
||||||
@ -927,7 +928,6 @@ static const BYTE* ZSTD_updateMatch(U32* table, const BYTE* p, const BYTE* start
|
|||||||
U32 h = ZSTD_hashPtr(p);
|
U32 h = ZSTD_hashPtr(p);
|
||||||
const BYTE* r;
|
const BYTE* r;
|
||||||
r = table[h] + start;
|
r = table[h] + start;
|
||||||
//table[h] = (U32)(p - start);
|
|
||||||
ZSTD_addPtr(table, p, start);
|
ZSTD_addPtr(table, p, start);
|
||||||
return r;
|
return r;
|
||||||
}
|
}
|
||||||
@ -961,7 +961,7 @@ static size_t ZSTD_compressBlock(void* cctx, void* dst, size_t maxDstSize, const
|
|||||||
/* Main Search Loop */
|
/* Main Search Loop */
|
||||||
while (ip < ilimit)
|
while (ip < ilimit)
|
||||||
{
|
{
|
||||||
const BYTE* match = (BYTE*) ZSTD_updateMatch(HashTable, ip, base);
|
const BYTE* match = (const BYTE*) ZSTD_updateMatch(HashTable, ip, base);
|
||||||
|
|
||||||
if (!ZSTD_checkMatch(match,ip)) { ip += ((ip-anchor) >> g_searchStrength) + 1; continue; }
|
if (!ZSTD_checkMatch(match,ip)) { ip += ((ip-anchor) >> g_searchStrength) + 1; continue; }
|
||||||
|
|
||||||
@ -998,7 +998,7 @@ static size_t ZSTD_compressBlock(void* cctx, void* dst, size_t maxDstSize, const
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
size_t ZSTD_compressBegin(ZSTD_cctx_t ctx, void* dst, size_t maxDstSize)
|
size_t ZSTD_compressBegin(ZSTD_Cctx* ctx, void* dst, size_t maxDstSize)
|
||||||
{
|
{
|
||||||
/* Sanity check */
|
/* Sanity check */
|
||||||
if (maxDstSize < ZSTD_frameHeaderSize) return (size_t)-ZSTD_ERROR_maxDstSize_tooSmall;
|
if (maxDstSize < ZSTD_frameHeaderSize) return (size_t)-ZSTD_ERROR_maxDstSize_tooSmall;
|
||||||
@ -1013,13 +1013,12 @@ size_t ZSTD_compressBegin(ZSTD_cctx_t ctx, void* dst, size_t maxDstSize)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/* this should be auto-vectorized by compiler */
|
|
||||||
static void ZSTD_scaleDownCtx(void* cctx, const U32 limit)
|
static void ZSTD_scaleDownCtx(void* cctx, const U32 limit)
|
||||||
{
|
{
|
||||||
cctxi_t* ctx = (cctxi_t*) cctx;
|
cctxi_t* ctx = (cctxi_t*) cctx;
|
||||||
int i;
|
int i;
|
||||||
|
|
||||||
#if defined(__AVX2__) /* <immintrin.h> */
|
#if defined(__AVX2__)
|
||||||
/* AVX2 version */
|
/* AVX2 version */
|
||||||
__m256i* h = ctx->hashTable;
|
__m256i* h = ctx->hashTable;
|
||||||
const __m256i limit8 = _mm256_set1_epi32(limit);
|
const __m256i limit8 = _mm256_set1_epi32(limit);
|
||||||
@ -1031,6 +1030,7 @@ static void ZSTD_scaleDownCtx(void* cctx, const U32 limit)
|
|||||||
_mm256_storeu_si256((__m256i*)(h+i), src);
|
_mm256_storeu_si256((__m256i*)(h+i), src);
|
||||||
}
|
}
|
||||||
#else
|
#else
|
||||||
|
/* this should be auto-vectorized by compiler */
|
||||||
U32* h = ctx->hashTable;
|
U32* h = ctx->hashTable;
|
||||||
for (i=0; i<HASH_TABLESIZE; ++i)
|
for (i=0; i<HASH_TABLESIZE; ++i)
|
||||||
{
|
{
|
||||||
@ -1042,7 +1042,6 @@ static void ZSTD_scaleDownCtx(void* cctx, const U32 limit)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/* this should be auto-vectorized by compiler */
|
|
||||||
static void ZSTD_limitCtx(void* cctx, const U32 limit)
|
static void ZSTD_limitCtx(void* cctx, const U32 limit)
|
||||||
{
|
{
|
||||||
cctxi_t* ctx = (cctxi_t*) cctx;
|
cctxi_t* ctx = (cctxi_t*) cctx;
|
||||||
@ -1057,7 +1056,7 @@ static void ZSTD_limitCtx(void* cctx, const U32 limit)
|
|||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
#if defined(__AVX2__) /* <immintrin.h> */
|
#if defined(__AVX2__)
|
||||||
/* AVX2 version */
|
/* AVX2 version */
|
||||||
{
|
{
|
||||||
__m256i* h = ctx->hashTable;
|
__m256i* h = ctx->hashTable;
|
||||||
@ -1071,6 +1070,7 @@ static void ZSTD_limitCtx(void* cctx, const U32 limit)
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
#else
|
#else
|
||||||
|
/* this should be auto-vectorized by compiler */
|
||||||
{
|
{
|
||||||
U32* h = (U32*)(ctx->hashTable);
|
U32* h = (U32*)(ctx->hashTable);
|
||||||
for (i=0; i<HASH_TABLESIZE; ++i)
|
for (i=0; i<HASH_TABLESIZE; ++i)
|
||||||
@ -1082,7 +1082,7 @@ static void ZSTD_limitCtx(void* cctx, const U32 limit)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
size_t ZSTD_compressContinue(ZSTD_cctx_t cctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize)
|
size_t ZSTD_compressContinue(ZSTD_Cctx* cctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize)
|
||||||
{
|
{
|
||||||
cctxi_t* ctx = (cctxi_t*) cctx;
|
cctxi_t* ctx = (cctxi_t*) cctx;
|
||||||
const BYTE* const istart = (const BYTE* const)src;
|
const BYTE* const istart = (const BYTE* const)src;
|
||||||
@ -1092,6 +1092,7 @@ size_t ZSTD_compressContinue(ZSTD_cctx_t cctx, void* dst, size_t maxDstSize, con
|
|||||||
const U32 updateRate = 2 * BLOCKSIZE;
|
const U32 updateRate = 2 * BLOCKSIZE;
|
||||||
|
|
||||||
/* Init */
|
/* Init */
|
||||||
|
if (maxDstSize < ZSTD_compressBound(srcSize) - 4 /* frame header size*/) return (size_t)-ZSTD_ERROR_maxDstSize_tooSmall;
|
||||||
if (ctx->base==NULL)
|
if (ctx->base==NULL)
|
||||||
ctx->base = (const BYTE*)src, ctx->current=0, ctx->nextUpdate = g_maxDistance;
|
ctx->base = (const BYTE*)src, ctx->current=0, ctx->nextUpdate = g_maxDistance;
|
||||||
if (src != ctx->base + ctx->current) /* not contiguous */
|
if (src != ctx->base + ctx->current) /* not contiguous */
|
||||||
@ -1122,7 +1123,6 @@ size_t ZSTD_compressContinue(ZSTD_cctx_t cctx, void* dst, size_t maxDstSize, con
|
|||||||
}
|
}
|
||||||
|
|
||||||
/* compress */
|
/* compress */
|
||||||
if (maxDstSize < ZSTD_blockHeaderSize) return (size_t)-ZSTD_ERROR_maxDstSize_tooSmall;
|
|
||||||
cSize = ZSTD_compressBlock(ctx, op+ZSTD_blockHeaderSize, maxDstSize-ZSTD_blockHeaderSize, ip, blockSize);
|
cSize = ZSTD_compressBlock(ctx, op+ZSTD_blockHeaderSize, maxDstSize-ZSTD_blockHeaderSize, ip, blockSize);
|
||||||
if (cSize == 0)
|
if (cSize == 0)
|
||||||
{
|
{
|
||||||
@ -1148,7 +1148,7 @@ size_t ZSTD_compressContinue(ZSTD_cctx_t cctx, void* dst, size_t maxDstSize, con
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
size_t ZSTD_compressEnd(ZSTD_cctx_t ctx, void* dst, size_t maxDstSize)
|
size_t ZSTD_compressEnd(ZSTD_Cctx* ctx, void* dst, size_t maxDstSize)
|
||||||
{
|
{
|
||||||
BYTE* op = (BYTE*)dst;
|
BYTE* op = (BYTE*)dst;
|
||||||
|
|
||||||
@ -1165,9 +1165,10 @@ size_t ZSTD_compressEnd(ZSTD_cctx_t ctx, void* dst, size_t maxDstSize)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
static size_t ZSTD_compressCCtx(void* ctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize)
|
static size_t ZSTD_compressCCtx(ZSTD_Cctx* ctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize)
|
||||||
{
|
{
|
||||||
BYTE* const ostart = (BYTE* const)dst;
|
BYTE* const ostart = (BYTE* const)dst;
|
||||||
|
BYTE* const oend = ostart + maxDstSize;
|
||||||
BYTE* op = ostart;
|
BYTE* op = ostart;
|
||||||
|
|
||||||
/* Header */
|
/* Header */
|
||||||
@ -1180,7 +1181,7 @@ static size_t ZSTD_compressCCtx(void* ctx, void* dst, size_t maxDstSize, const v
|
|||||||
|
|
||||||
/* Compression */
|
/* Compression */
|
||||||
{
|
{
|
||||||
size_t cSize = ZSTD_compressContinue(ctx, op, maxDstSize, src, srcSize);
|
size_t cSize = ZSTD_compressContinue(ctx, op, oend-op, src, srcSize);
|
||||||
if (ZSTD_isError(cSize)) return cSize;
|
if (ZSTD_isError(cSize)) return cSize;
|
||||||
op += cSize;
|
op += cSize;
|
||||||
maxDstSize -= cSize;
|
maxDstSize -= cSize;
|
||||||
@ -1188,7 +1189,7 @@ static size_t ZSTD_compressCCtx(void* ctx, void* dst, size_t maxDstSize, const v
|
|||||||
|
|
||||||
/* Close frame */
|
/* Close frame */
|
||||||
{
|
{
|
||||||
size_t endSize = ZSTD_compressEnd(ctx, op, maxDstSize);
|
size_t endSize = ZSTD_compressEnd(ctx, op, oend-op);
|
||||||
if(ZSTD_isError(endSize)) return endSize;
|
if(ZSTD_isError(endSize)) return endSize;
|
||||||
op += endSize;
|
op += endSize;
|
||||||
}
|
}
|
||||||
@ -1199,7 +1200,7 @@ static size_t ZSTD_compressCCtx(void* ctx, void* dst, size_t maxDstSize, const v
|
|||||||
|
|
||||||
size_t ZSTD_compress(void* dst, size_t maxDstSize, const void* src, size_t srcSize)
|
size_t ZSTD_compress(void* dst, size_t maxDstSize, const void* src, size_t srcSize)
|
||||||
{
|
{
|
||||||
void* ctx;
|
ZSTD_Cctx* ctx;
|
||||||
size_t r;
|
size_t r;
|
||||||
|
|
||||||
ctx = ZSTD_createCCtx();
|
ctx = ZSTD_createCCtx();
|
||||||
@ -1209,6 +1210,7 @@ size_t ZSTD_compress(void* dst, size_t maxDstSize, const void* src, size_t srcSi
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
/**************************************************************
|
/**************************************************************
|
||||||
* Decompression code
|
* Decompression code
|
||||||
**************************************************************/
|
**************************************************************/
|
||||||
@ -1245,7 +1247,7 @@ static size_t ZSTD_copyUncompressedBlock(void* dst, size_t maxDstSize, const voi
|
|||||||
FORCE_INLINE size_t ZSTD_decompressLiterals_usingDTable_generic(
|
FORCE_INLINE size_t ZSTD_decompressLiterals_usingDTable_generic(
|
||||||
void* const dst, size_t maxDstSize,
|
void* const dst, size_t maxDstSize,
|
||||||
const void* src, size_t srcSize,
|
const void* src, size_t srcSize,
|
||||||
const void* DTable, U32 fast)
|
const FSE_DTable* DTable, U32 fast)
|
||||||
{
|
{
|
||||||
BYTE* op = (BYTE*) dst;
|
BYTE* op = (BYTE*) dst;
|
||||||
BYTE* const olimit = op;
|
BYTE* const olimit = op;
|
||||||
@ -1305,7 +1307,7 @@ FORCE_INLINE size_t ZSTD_decompressLiterals_usingDTable_generic(
|
|||||||
static size_t ZSTD_decompressLiterals_usingDTable(
|
static size_t ZSTD_decompressLiterals_usingDTable(
|
||||||
void* const dst, size_t maxDstSize,
|
void* const dst, size_t maxDstSize,
|
||||||
const void* src, size_t srcSize,
|
const void* src, size_t srcSize,
|
||||||
const void* DTable, U32 fast)
|
const FSE_DTable* DTable, U32 fast)
|
||||||
{
|
{
|
||||||
if (fast) return ZSTD_decompressLiterals_usingDTable_generic(dst, maxDstSize, src, srcSize, DTable, 1);
|
if (fast) return ZSTD_decompressLiterals_usingDTable_generic(dst, maxDstSize, src, srcSize, DTable, 1);
|
||||||
return ZSTD_decompressLiterals_usingDTable_generic(dst, maxDstSize, src, srcSize, DTable, 0);
|
return ZSTD_decompressLiterals_usingDTable_generic(dst, maxDstSize, src, srcSize, DTable, 0);
|
||||||
@ -1317,7 +1319,7 @@ static size_t ZSTD_decompressLiterals(void* ctx, void* dst, size_t maxDstSize,
|
|||||||
/* assumed : blockType == blockCompressed */
|
/* assumed : blockType == blockCompressed */
|
||||||
const BYTE* ip = (const BYTE*)src;
|
const BYTE* ip = (const BYTE*)src;
|
||||||
short norm[256];
|
short norm[256];
|
||||||
void* DTable = ctx;
|
FSE_DTable* DTable = (FSE_DTable*)ctx;
|
||||||
U32 maxSymbolValue = 255;
|
U32 maxSymbolValue = 255;
|
||||||
U32 tableLog;
|
U32 tableLog;
|
||||||
U32 fastMode;
|
U32 fastMode;
|
||||||
@ -1325,7 +1327,7 @@ static size_t ZSTD_decompressLiterals(void* ctx, void* dst, size_t maxDstSize,
|
|||||||
|
|
||||||
if (srcSize < 2) return (size_t)-ZSTD_ERROR_wrongLBlockSize; /* too small input size */
|
if (srcSize < 2) return (size_t)-ZSTD_ERROR_wrongLBlockSize; /* too small input size */
|
||||||
|
|
||||||
errorCode = FSE_readHeader (norm, &maxSymbolValue, &tableLog, ip, srcSize);
|
errorCode = FSE_readNCount (norm, &maxSymbolValue, &tableLog, ip, srcSize);
|
||||||
if (FSE_isError(errorCode)) return (size_t)-ZSTD_ERROR_GENERIC;
|
if (FSE_isError(errorCode)) return (size_t)-ZSTD_ERROR_GENERIC;
|
||||||
ip += errorCode;
|
ip += errorCode;
|
||||||
srcSize -= errorCode;
|
srcSize -= errorCode;
|
||||||
@ -1367,9 +1369,9 @@ size_t ZSTD_decodeLiteralsBlock(void* ctx,
|
|||||||
}
|
}
|
||||||
case bt_compressed:
|
case bt_compressed:
|
||||||
{
|
{
|
||||||
size_t cSize = ZSTD_decompressLiterals(ctx, dst, maxDstSize, ip, litcSize);
|
size_t litSize = ZSTD_decompressLiterals(ctx, dst, maxDstSize, ip, litcSize);
|
||||||
if (ZSTD_isError(cSize)) return cSize;
|
if (ZSTD_isError(litSize)) return litSize;
|
||||||
*litPtr = oend - cSize;
|
*litPtr = oend - litSize;
|
||||||
ip += litcSize;
|
ip += litcSize;
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
@ -1382,7 +1384,7 @@ size_t ZSTD_decodeLiteralsBlock(void* ctx,
|
|||||||
|
|
||||||
|
|
||||||
size_t ZSTD_decodeSeqHeaders(size_t* lastLLPtr, const BYTE** dumpsPtr,
|
size_t ZSTD_decodeSeqHeaders(size_t* lastLLPtr, const BYTE** dumpsPtr,
|
||||||
void* DTableLL, void* DTableML, void* DTableOffb,
|
FSE_DTable* DTableLL, FSE_DTable* DTableML, FSE_DTable* DTableOffb,
|
||||||
const void* src, size_t srcSize)
|
const void* src, size_t srcSize)
|
||||||
{
|
{
|
||||||
const BYTE* const istart = (const BYTE* const)src;
|
const BYTE* const istart = (const BYTE* const)src;
|
||||||
@ -1429,7 +1431,7 @@ size_t ZSTD_decodeSeqHeaders(size_t* lastLLPtr, const BYTE** dumpsPtr,
|
|||||||
FSE_buildDTable_raw(DTableLL, LLbits); break;
|
FSE_buildDTable_raw(DTableLL, LLbits); break;
|
||||||
default :
|
default :
|
||||||
max = MaxLL;
|
max = MaxLL;
|
||||||
headerSize = FSE_readHeader(norm, &max, &LLlog, ip, iend-ip);
|
headerSize = FSE_readNCount(norm, &max, &LLlog, ip, iend-ip);
|
||||||
if (FSE_isError(headerSize)) return (size_t)-ZSTD_ERROR_GENERIC;
|
if (FSE_isError(headerSize)) return (size_t)-ZSTD_ERROR_GENERIC;
|
||||||
ip += headerSize;
|
ip += headerSize;
|
||||||
FSE_buildDTable(DTableLL, norm, max, LLlog);
|
FSE_buildDTable(DTableLL, norm, max, LLlog);
|
||||||
@ -1446,7 +1448,7 @@ size_t ZSTD_decodeSeqHeaders(size_t* lastLLPtr, const BYTE** dumpsPtr,
|
|||||||
FSE_buildDTable_raw(DTableOffb, Offbits); break;
|
FSE_buildDTable_raw(DTableOffb, Offbits); break;
|
||||||
default :
|
default :
|
||||||
max = MaxOff;
|
max = MaxOff;
|
||||||
headerSize = FSE_readHeader(norm, &max, &Offlog, ip, iend-ip);
|
headerSize = FSE_readNCount(norm, &max, &Offlog, ip, iend-ip);
|
||||||
if (FSE_isError(headerSize)) return (size_t)-ZSTD_ERROR_GENERIC;
|
if (FSE_isError(headerSize)) return (size_t)-ZSTD_ERROR_GENERIC;
|
||||||
ip += headerSize;
|
ip += headerSize;
|
||||||
FSE_buildDTable(DTableOffb, norm, max, Offlog);
|
FSE_buildDTable(DTableOffb, norm, max, Offlog);
|
||||||
@ -1463,7 +1465,7 @@ size_t ZSTD_decodeSeqHeaders(size_t* lastLLPtr, const BYTE** dumpsPtr,
|
|||||||
FSE_buildDTable_raw(DTableML, MLbits); break;
|
FSE_buildDTable_raw(DTableML, MLbits); break;
|
||||||
default :
|
default :
|
||||||
max = MaxML;
|
max = MaxML;
|
||||||
headerSize = FSE_readHeader(norm, &max, &MLlog, ip, iend-ip);
|
headerSize = FSE_readNCount(norm, &max, &MLlog, ip, iend-ip);
|
||||||
if (FSE_isError(headerSize)) return (size_t)-ZSTD_ERROR_GENERIC;
|
if (FSE_isError(headerSize)) return (size_t)-ZSTD_ERROR_GENERIC;
|
||||||
ip += headerSize;
|
ip += headerSize;
|
||||||
FSE_buildDTable(DTableML, norm, max, MLlog);
|
FSE_buildDTable(DTableML, norm, max, MLlog);
|
||||||
@ -1491,9 +1493,9 @@ FORCE_INLINE size_t ZSTD_decompressBlock(void* ctx, void* dst, size_t maxDstSize
|
|||||||
const BYTE* litEnd;
|
const BYTE* litEnd;
|
||||||
const size_t dec32table[] = {4, 1, 2, 1, 4, 4, 4, 4}; /* added */
|
const size_t dec32table[] = {4, 1, 2, 1, 4, 4, 4, 4}; /* added */
|
||||||
const size_t dec64table[] = {8, 8, 8, 7, 8, 9,10,11}; /* substracted */
|
const size_t dec64table[] = {8, 8, 8, 7, 8, 9,10,11}; /* substracted */
|
||||||
void* DTableML = ctx;
|
FSE_DTable* DTableML = (FSE_DTable*)ctx;
|
||||||
void* DTableLL = ((U32*)ctx) + FSE_DTABLE_SIZE_U32(MLFSELog);
|
FSE_DTable* DTableLL = DTableML + FSE_DTABLE_SIZE_U32(MLFSELog);
|
||||||
void* DTableOffb = ((U32*)DTableLL) + FSE_DTABLE_SIZE_U32(LLFSELog);
|
FSE_DTable* DTableOffb = DTableLL + FSE_DTABLE_SIZE_U32(LLFSELog);
|
||||||
|
|
||||||
/* blockType == blockCompressed, srcSize is trusted */
|
/* blockType == blockCompressed, srcSize is trusted */
|
||||||
|
|
||||||
@ -1508,7 +1510,7 @@ FORCE_INLINE size_t ZSTD_decompressBlock(void* ctx, void* dst, size_t maxDstSize
|
|||||||
ip, iend-ip);
|
ip, iend-ip);
|
||||||
if (ZSTD_isError(errorCode)) return errorCode;
|
if (ZSTD_isError(errorCode)) return errorCode;
|
||||||
/* end pos */
|
/* end pos */
|
||||||
if ((litPtr>=ostart) && (litPtr<=oend))
|
if ((litPtr>=ostart) && (litPtr<=oend)) /* decoded literals are into dst buffer */
|
||||||
litEnd = oend - lastLLSize;
|
litEnd = oend - lastLLSize;
|
||||||
else
|
else
|
||||||
litEnd = ip - lastLLSize;
|
litEnd = ip - lastLLSize;
|
||||||
@ -1519,7 +1521,6 @@ FORCE_INLINE size_t ZSTD_decompressBlock(void* ctx, void* dst, size_t maxDstSize
|
|||||||
FSE_DStream_t DStream;
|
FSE_DStream_t DStream;
|
||||||
FSE_DState_t stateLL, stateOffb, stateML;
|
FSE_DState_t stateLL, stateOffb, stateML;
|
||||||
size_t prevOffset = 0, offset = 0;
|
size_t prevOffset = 0, offset = 0;
|
||||||
size_t qutt=0;
|
|
||||||
|
|
||||||
FSE_initDStream(&DStream, ip, iend-ip);
|
FSE_initDStream(&DStream, ip, iend-ip);
|
||||||
FSE_initDState(&stateLL, &DStream, DTableLL);
|
FSE_initDState(&stateLL, &DStream, DTableLL);
|
||||||
@ -1545,7 +1546,6 @@ _another_round:
|
|||||||
if (add < 255) litLength += add;
|
if (add < 255) litLength += add;
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
//litLength = (*(U32*)dumps) & 0xFFFFFF;
|
|
||||||
litLength = ZSTD_readLE32(dumps) & 0xFFFFFF;
|
litLength = ZSTD_readLE32(dumps) & 0xFFFFFF;
|
||||||
dumps += 3;
|
dumps += 3;
|
||||||
}
|
}
|
||||||
@ -1578,7 +1578,7 @@ _another_round:
|
|||||||
if (add < 255) matchLength += add;
|
if (add < 255) matchLength += add;
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
matchLength = ZSTD_readLE32(dumps) & 0xFFFFFF;
|
matchLength = ZSTD_readLE32(dumps) & 0xFFFFFF; /* no pb : dumps is always followed by seq tables > 1 byte */
|
||||||
dumps += 3;
|
dumps += 3;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -1587,8 +1587,10 @@ _another_round:
|
|||||||
/* copy Match */
|
/* copy Match */
|
||||||
{
|
{
|
||||||
BYTE* const endMatch = op + matchLength;
|
BYTE* const endMatch = op + matchLength;
|
||||||
|
size_t qutt=0;
|
||||||
U64 saved[2];
|
U64 saved[2];
|
||||||
|
|
||||||
|
/* save beginning of literal sequence, in case of write overlap */
|
||||||
if ((size_t)(litPtr - endMatch) < 12)
|
if ((size_t)(litPtr - endMatch) < 12)
|
||||||
{
|
{
|
||||||
qutt = endMatch + 12 - litPtr;
|
qutt = endMatch + 12 - litPtr;
|
||||||
@ -1624,7 +1626,7 @@ _another_round:
|
|||||||
op = endMatch;
|
op = endMatch;
|
||||||
|
|
||||||
if ((size_t)(litPtr - endMatch) < 12)
|
if ((size_t)(litPtr - endMatch) < 12)
|
||||||
memcpy((void*)litPtr, saved, qutt);
|
memcpy(endMatch + (litPtr - endMatch), saved, qutt); /* required as litPtr is const ptr */
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1712,7 +1714,7 @@ size_t ZSTD_decompress(void* dst, size_t maxDstSize, const void* src, size_t src
|
|||||||
* Streaming Decompression API
|
* Streaming Decompression API
|
||||||
*******************************/
|
*******************************/
|
||||||
|
|
||||||
typedef struct
|
typedef struct ZSTD_Dctx_s
|
||||||
{
|
{
|
||||||
U32 ctx[FSE_DTABLE_SIZE_U32(LLFSELog) + FSE_DTABLE_SIZE_U32(OffFSELog) + FSE_DTABLE_SIZE_U32(MLFSELog)];
|
U32 ctx[FSE_DTABLE_SIZE_U32(LLFSELog) + FSE_DTABLE_SIZE_U32(OffFSELog) + FSE_DTABLE_SIZE_U32(MLFSELog)];
|
||||||
size_t expected;
|
size_t expected;
|
||||||
@ -1721,27 +1723,28 @@ typedef struct
|
|||||||
} dctx_t;
|
} dctx_t;
|
||||||
|
|
||||||
|
|
||||||
ZSTD_dctx_t ZSTD_createDCtx(void)
|
ZSTD_Dctx* ZSTD_createDCtx(void)
|
||||||
{
|
{
|
||||||
dctx_t* dctx = (dctx_t*)malloc(sizeof(dctx_t));
|
ZSTD_Dctx* dctx = (ZSTD_Dctx*)malloc(sizeof(ZSTD_Dctx));
|
||||||
|
if (dctx==NULL) return NULL;
|
||||||
dctx->expected = ZSTD_frameHeaderSize;
|
dctx->expected = ZSTD_frameHeaderSize;
|
||||||
dctx->phase = 0;
|
dctx->phase = 0;
|
||||||
return (ZSTD_dctx_t)dctx;
|
return dctx;
|
||||||
}
|
}
|
||||||
|
|
||||||
size_t ZSTD_freeDCtx(ZSTD_dctx_t dctx)
|
size_t ZSTD_freeDCtx(ZSTD_Dctx* dctx)
|
||||||
{
|
{
|
||||||
free(dctx);
|
free(dctx);
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
size_t ZSTD_nextSrcSizeToDecompress(ZSTD_dctx_t dctx)
|
size_t ZSTD_nextSrcSizeToDecompress(ZSTD_Dctx* dctx)
|
||||||
{
|
{
|
||||||
return ((dctx_t*)dctx)->expected;
|
return ((dctx_t*)dctx)->expected;
|
||||||
}
|
}
|
||||||
|
|
||||||
size_t ZSTD_decompressContinue(ZSTD_dctx_t dctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize)
|
size_t ZSTD_decompressContinue(ZSTD_Dctx* dctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize)
|
||||||
{
|
{
|
||||||
dctx_t* ctx = (dctx_t*)dctx;
|
dctx_t* ctx = (dctx_t*)dctx;
|
||||||
|
|
||||||
|
@ -48,7 +48,7 @@ extern "C" {
|
|||||||
#define ZSTD_VERSION_MAJOR 0 /* for breaking interface changes */
|
#define ZSTD_VERSION_MAJOR 0 /* for breaking interface changes */
|
||||||
#define ZSTD_VERSION_MINOR 0 /* for new (non-breaking) interface capabilities */
|
#define ZSTD_VERSION_MINOR 0 /* for new (non-breaking) interface capabilities */
|
||||||
#define ZSTD_VERSION_RELEASE 2 /* for tweaks, bug-fixes, or development */
|
#define ZSTD_VERSION_RELEASE 2 /* for tweaks, bug-fixes, or development */
|
||||||
#define ZSTD_VERSION_NUMBER (ZSTD_VERSION_MAJOR *100*100 + ZSTD_VERSION_MINOR *100 + ZSTD_VERSION_RELEASE)
|
#define ZSTD_VERSION_NUMBER (ZSTD_VERSION_MAJOR *100*100 + ZSTD_VERSION_MINOR *100 + ZSTD_VERSION_RELEASE)
|
||||||
unsigned ZSTD_versionNumber (void);
|
unsigned ZSTD_versionNumber (void);
|
||||||
|
|
||||||
|
|
||||||
@ -70,7 +70,7 @@ ZSTD_compress() :
|
|||||||
or an error code if it fails (which can be tested using ZSTD_isError())
|
or an error code if it fails (which can be tested using ZSTD_isError())
|
||||||
|
|
||||||
ZSTD_decompress() :
|
ZSTD_decompress() :
|
||||||
compressedSize : is obviously the source size
|
compressedSize : is the exact source size
|
||||||
maxOriginalSize : is the size of the 'dst' buffer, which must be already allocated.
|
maxOriginalSize : is the size of the 'dst' buffer, which must be already allocated.
|
||||||
It must be equal or larger than originalSize, otherwise decompression will fail.
|
It must be equal or larger than originalSize, otherwise decompression will fail.
|
||||||
return : the number of bytes decompressed into destination buffer (originalSize)
|
return : the number of bytes decompressed into destination buffer (originalSize)
|
||||||
@ -81,7 +81,7 @@ ZSTD_decompress() :
|
|||||||
/**************************************
|
/**************************************
|
||||||
* Tool functions
|
* Tool functions
|
||||||
**************************************/
|
**************************************/
|
||||||
size_t ZSTD_compressBound(size_t srcSize); /* maximum compressed size */
|
size_t ZSTD_compressBound(size_t srcSize); /* maximum compressed size (worst case scenario) */
|
||||||
|
|
||||||
/* Error Management */
|
/* Error Management */
|
||||||
unsigned ZSTD_isError(size_t code); /* tells if a return value is an error code */
|
unsigned ZSTD_isError(size_t code); /* tells if a return value is an error code */
|
||||||
|
@ -45,20 +45,21 @@ extern "C" {
|
|||||||
/**************************************
|
/**************************************
|
||||||
* Streaming functions
|
* Streaming functions
|
||||||
**************************************/
|
**************************************/
|
||||||
typedef void* ZSTD_cctx_t;
|
typedef struct ZSTD_Cctx_s ZSTD_Cctx;
|
||||||
ZSTD_cctx_t ZSTD_createCCtx(void);
|
ZSTD_Cctx* ZSTD_createCCtx(void);
|
||||||
size_t ZSTD_freeCCtx(ZSTD_cctx_t cctx);
|
size_t ZSTD_freeCCtx(ZSTD_Cctx* cctx);
|
||||||
|
|
||||||
size_t ZSTD_compressBegin(ZSTD_cctx_t cctx, void* dst, size_t maxDstSize);
|
size_t ZSTD_compressBegin(ZSTD_Cctx* cctx, void* dst, size_t maxDstSize);
|
||||||
size_t ZSTD_compressContinue(ZSTD_cctx_t cctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize);
|
size_t ZSTD_compressContinue(ZSTD_Cctx* cctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize);
|
||||||
size_t ZSTD_compressEnd(ZSTD_cctx_t cctx, void* dst, size_t maxDstSize);
|
size_t ZSTD_compressEnd(ZSTD_Cctx* cctx, void* dst, size_t maxDstSize);
|
||||||
|
|
||||||
typedef void* ZSTD_dctx_t;
|
|
||||||
ZSTD_dctx_t ZSTD_createDCtx(void);
|
|
||||||
size_t ZSTD_freeDCtx(ZSTD_dctx_t dctx);
|
|
||||||
|
|
||||||
size_t ZSTD_nextSrcSizeToDecompress(ZSTD_dctx_t dctx);
|
typedef struct ZSTD_Dctx_s ZSTD_Dctx;
|
||||||
size_t ZSTD_decompressContinue(ZSTD_dctx_t dctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize);
|
ZSTD_Dctx* ZSTD_createDCtx(void);
|
||||||
|
size_t ZSTD_freeDCtx(ZSTD_Dctx* dctx);
|
||||||
|
|
||||||
|
size_t ZSTD_nextSrcSizeToDecompress(ZSTD_Dctx* dctx);
|
||||||
|
size_t ZSTD_decompressContinue(ZSTD_Dctx* dctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize);
|
||||||
/*
|
/*
|
||||||
Use above functions alternatively.
|
Use above functions alternatively.
|
||||||
ZSTD_nextSrcSizeToDecompress() tells how much bytes to provide as input to ZSTD_decompressContinue().
|
ZSTD_nextSrcSizeToDecompress() tells how much bytes to provide as input to ZSTD_decompressContinue().
|
||||||
|
Loading…
Reference in New Issue
Block a user