This commit is contained in:
Roman Peshkurov 2015-07-10 13:51:23 +03:00
commit da606a7ecc
40 changed files with 1171 additions and 883 deletions

View File

@ -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>
struct NegateImpl
{
@ -319,6 +345,12 @@ template <typename T> using Else = T;
/// Used to indicate undefined operation
struct InvalidType;
template <>
struct DataTypeFromFieldType<NumberTraits::Error>
{
using Type = InvalidType;
};
template <typename DataType> struct IsIntegral { static constexpr auto value = false; };
template <> struct IsIntegral<DataTypeUInt8> { 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<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
* Integral + Date -> Date
* Date - Date -> Int32
* Date - Integral -> Date
* least(Date, Date) -> Date
* greatest(Date, Date) -> Date
* All other operations are not defined and return InvalidType, operations on
* distinct date types are also undefined (e.g. DataTypeDate - DataTypeDateTime) */
template <template <typename, typename> class Operation, typename LeftDataType, typename RightDataType>
@ -387,7 +421,13 @@ struct DateBinaryOperationTraits
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
template <typename LeftDataType, typename RightDataType, typename ColumnType,
typename std::enable_if<IsDateOrDateTime<LeftDataType>::value || IsDateOrDateTime<RightDataType>::value>::type * = nullptr>
template <typename LeftDataType, typename RightDataType, typename ColumnType>
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()))
return false;
using ResultDataType = typename DateBinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
return executeRightTypeDispatch<LeftDataType, RightDataType, ResultDataType>(
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
template <typename LeftDataType, typename RightDataType, typename ResultDataType, typename ColumnType,
typename std::enable_if<std::is_same<ResultDataType, InvalidType>::value>::type * = nullptr>
bool executeRightTypeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result,
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
@ -527,7 +559,7 @@ private:
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)
{
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>;
block.getByPosition(result).column = col_res;
@ -538,7 +570,7 @@ private:
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>;
block.getByPosition(result).column = col_res;
@ -550,14 +582,14 @@ private:
return true;
}
return false;
throw Exception("Logical error: unexpected type of column", ErrorCodes::LOGICAL_ERROR);
}
/// ColumnConst overload
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)
{
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>;
block.getByPosition(result).column = col_res;
@ -568,7 +600,7 @@ private:
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;
BinaryOperationImpl<T0, T1, Op<T0, T1>, ResultType>::constant_constant(col_left->getData(), col_right->getData(), res);
@ -582,26 +614,12 @@ private:
return false;
}
template <typename LeftDataType,
typename std::enable_if<IsDateOrDateTime<LeftDataType>::value>::type * = nullptr>
template <typename LeftDataType>
bool executeLeftType(Block & block, const ColumnNumbers & arguments, const size_t result)
{
if (!typeid_cast<const LeftDataType *>(block.getByPosition(arguments[0]).type.get()))
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;
if ( executeLeftTypeImpl<LeftDataType, ColumnVector<T0>>(block, arguments, result)
@ -614,7 +632,7 @@ private:
template <typename LeftDataType, typename ColumnType>
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)
|| executeRightType<LeftDataType, DataTypeDateTime>(block, arguments, result, col_left)
@ -718,7 +736,7 @@ private:
template <typename T0>
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;
@ -731,7 +749,7 @@ private:
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;
@ -815,6 +833,8 @@ struct NameBitXor { static constexpr auto name = "bitXor"; };
struct NameBitNot { static constexpr auto name = "bitNot"; };
struct NameBitShiftLeft { static constexpr auto name = "bitShiftLeft"; };
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<MinusImpl, NameMinus> FunctionMinus;
@ -831,7 +851,8 @@ typedef FunctionBinaryArithmetic<BitXorImpl, NameBitXor> FunctionBitXor;
typedef FunctionUnaryArithmetic<BitNotImpl, NameBitNot> FunctionBitNot;
typedef FunctionBinaryArithmetic<BitShiftLeftImpl, NameBitShiftLeft> FunctionBitShiftLeft;
typedef FunctionBinaryArithmetic<BitShiftRightImpl, NameBitShiftRight> FunctionBitShiftRight;
typedef FunctionBinaryArithmetic<LeastImpl, NameLeast> FunctionLeast;
typedef FunctionBinaryArithmetic<GreatestImpl, NameGreatest> FunctionGreatest;
/// Оптимизации для целочисленного деления на константу.

View File

@ -69,7 +69,7 @@ struct ConvertImpl<DataTypeDate, DataTypeDateTime, Name>
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
{
typedef DataTypeDate::FieldType FromFieldType;
auto & 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))
{
@ -108,7 +108,7 @@ struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name>
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
{
auto & 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))
{
@ -158,7 +158,7 @@ struct ConvertImpl32Or64ToDate
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
{
auto & 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))
@ -248,7 +248,7 @@ namespace details { namespace {
* объекту from_date_lut. Эта функция возвращает timestamp представлящий те же дату и время
* в часовом поясе соответствующем объекту to_date_lut.
*/
time_t convertTimestamp(time_t source_timestamp, DateLUTImpl & from_date_lut, DateLUTImpl & 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;
@ -271,7 +271,7 @@ struct DateTimeToStringConverter
static void vector_vector(const PODArray<FromFieldType> & vec_from, const ColumnString::Chars_t & data,
const ColumnString::Offsets_t & offsets, ColumnString & vec_to)
{
auto & local_date_lut = DateLUT::instance();
const auto & local_date_lut = DateLUT::instance();
ColumnString::Chars_t & data_to = vec_to.getChars();
ColumnString::Offsets_t & offsets_to = vec_to.getOffsets();
@ -287,7 +287,7 @@ struct DateTimeToStringConverter
{
ColumnString::Offset_t cur_offset = offsets[i];
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
auto & remote_date_lut = DateLUT::instance(time_zone);
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);
@ -302,8 +302,8 @@ struct DateTimeToStringConverter
static void vector_constant(const PODArray<FromFieldType> & vec_from, const std::string & data,
ColumnString & vec_to)
{
auto & local_date_lut = DateLUT::instance();
auto & remote_date_lut = DateLUT::instance(data);
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();
@ -323,11 +323,30 @@ struct DateTimeToStringConverter
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)
{
auto & local_date_lut = DateLUT::instance();
const auto & local_date_lut = DateLUT::instance();
ColumnString::Chars_t & data_to = vec_to.getChars();
ColumnString::Offsets_t & offsets_to = vec_to.getOffsets();
@ -343,7 +362,7 @@ struct DateTimeToStringConverter
{
ColumnString::Offset_t cur_offset = offsets[i];
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
auto & remote_date_lut = DateLUT::instance(time_zone);
const auto & remote_date_lut = DateLUT::instance(time_zone);
auto ti = convertTimestamp(from, remote_date_lut, local_date_lut);
formatImpl<DataTypeDateTime>(ti, write_buffer);
@ -357,8 +376,8 @@ struct DateTimeToStringConverter
static void constant_constant(FromFieldType from, const std::string & data, std::string & to)
{
auto & local_date_lut = DateLUT::instance();
auto & remote_date_lut = DateLUT::instance(data);
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);
@ -366,6 +385,14 @@ struct DateTimeToStringConverter
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());
}
};
}}
@ -392,12 +419,12 @@ struct ConvertImpl<DataTypeDateTime, DataTypeString, Name>
auto & vec_from = sources->getData();
auto & vec_to = *col_to;
Op::vector_constant(vec_from, "", vec_to);
Op::vector_constant(vec_from, vec_to);
}
else if (const_source)
{
std::string res;
Op::constant_constant(const_source->getData(), "", res);
Op::constant_constant(const_source->getData(), res);
block.getByPosition(result).column = new ColumnConstString(const_source->size(), res);
}
else
@ -533,7 +560,7 @@ struct StringToTimestampConverter
static void vector_vector(const ColumnString::Chars_t & vec_from, const ColumnString::Chars_t & data,
const ColumnString::Offsets_t & offsets, PODArray<ToFieldType> & vec_to)
{
auto & local_date_lut = DateLUT::instance();
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;
@ -546,7 +573,7 @@ struct StringToTimestampConverter
ColumnString::Offset_t cur_offset = offsets[i];
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
auto & remote_date_lut = DateLUT::instance(time_zone);
const auto & remote_date_lut = DateLUT::instance(time_zone);
auto ti = convertTimestamp(x, local_date_lut, remote_date_lut);
@ -562,8 +589,8 @@ struct StringToTimestampConverter
static void vector_constant(const ColumnString::Chars_t & vec_from, const std::string & data,
PODArray<ToFieldType> & vec_to)
{
auto & local_date_lut = DateLUT::instance();
auto & remote_date_lut = DateLUT::instance(data);
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;
@ -581,10 +608,26 @@ struct StringToTimestampConverter
}
}
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)
{
auto & local_date_lut = DateLUT::instance();
const auto & local_date_lut = DateLUT::instance();
ReadBufferFromString read_buffer(from);
DataTypeDateTime::FieldType x = 0;
@ -596,7 +639,7 @@ struct StringToTimestampConverter
{
ColumnString::Offset_t cur_offset = offsets[i];
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
auto & remote_date_lut = DateLUT::instance(time_zone);
const auto & remote_date_lut = DateLUT::instance(time_zone);
auto ti = convertTimestamp(x, local_date_lut, remote_date_lut);
@ -607,8 +650,8 @@ struct StringToTimestampConverter
static void constant_constant(const std::string & from, const std::string & data, ToFieldType & to)
{
auto & local_date_lut = DateLUT::instance();
auto & remote_date_lut = DateLUT::instance(data);
const auto & local_date_lut = DateLUT::instance();
const auto & remote_date_lut = DateLUT::instance(data);
ReadBufferFromString read_buffer(from);
DataTypeDateTime::FieldType x = 0;
@ -616,6 +659,14 @@ struct StringToTimestampConverter
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;
}
};
}}
@ -646,12 +697,12 @@ struct ConvertImpl<DataTypeString, DataTypeInt32, NameToUnixTimestamp>
size_t size = sources->size();
vec_to.resize(size);
Op::vector_constant(vec_from, "", vec_to);
Op::vector_constant(vec_from, vec_to);
}
else if (const_source)
{
ToFieldType res;
Op::constant_constant(const_source->getData(), "", res);
Op::constant_constant(const_source->getData(), res);
block.getByPosition(result).column = new ColumnConst<ToFieldType>(const_source->size(), res);
}
else

View File

@ -51,32 +51,32 @@ namespace DB
struct ToYearImpl
{
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(t); }
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(DayNum_t(d)); }
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, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(DayNum_t(d)); }
};
struct ToMonthImpl
{
static inline UInt8 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toMonth(t); }
static inline UInt8 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toMonth(DayNum_t(d)); }
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, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toMonth(DayNum_t(d)); }
};
struct ToDayOfMonthImpl
{
static inline UInt8 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfMonth(t); }
static inline UInt8 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfMonth(DayNum_t(d)); }
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, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfMonth(DayNum_t(d)); }
};
struct ToDayOfWeekImpl
{
static inline UInt8 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfWeek(t); }
static inline UInt8 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfWeek(DayNum_t(d)); }
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, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfWeek(DayNum_t(d)); }
};
struct ToHourImpl
{
static inline UInt8 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toHourInaccurate(t); }
static inline UInt8 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
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, 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);
}
@ -84,8 +84,8 @@ struct ToHourImpl
struct ToMinuteImpl
{
static inline UInt8 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toMinuteInaccurate(t); }
static inline UInt8 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
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, 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);
}
@ -93,8 +93,8 @@ struct ToMinuteImpl
struct ToSecondImpl
{
static inline UInt8 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toSecondInaccurate(t); }
static inline UInt8 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
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, 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);
}
@ -102,49 +102,49 @@ struct ToSecondImpl
struct ToMondayImpl
{
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfWeek(remote_date_lut.toDayNum(t)); }
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfWeek(DayNum_t(d)); }
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, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfWeek(DayNum_t(d)); }
};
struct ToStartOfMonthImpl
{
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfMonth(remote_date_lut.toDayNum(t)); }
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfMonth(DayNum_t(d)); }
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, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfMonth(DayNum_t(d)); }
};
struct ToStartOfQuarterImpl
{
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfQuarter(remote_date_lut.toDayNum(t)); }
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfQuarter(DayNum_t(d)); }
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, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfQuarter(DayNum_t(d)); }
};
struct ToStartOfYearImpl
{
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfYear(remote_date_lut.toDayNum(t)); }
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfYear(DayNum_t(d)); }
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, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfYear(DayNum_t(d)); }
};
struct ToTimeImpl
{
/// При переводе во время, дату будем приравнивать к 1970-01-02.
static inline UInt32 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
time_t remote_t = remote_date_lut.toTimeInaccurate(t) + 86400;
time_t remote_ts = remote_date_lut.toTimeInaccurate(t) + 86400;
if (&remote_date_lut == &local_date_lut)
return remote_t;
return remote_ts;
else
{
const auto & values = remote_date_lut.getValues(remote_t);
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_t),
remote_date_lut.toMinuteInaccurate(remote_t),
remote_date_lut.toSecondInaccurate(remote_t));
remote_date_lut.toHourInaccurate(remote_ts),
remote_date_lut.toMinuteInaccurate(remote_ts),
remote_date_lut.toSecondInaccurate(remote_ts));
}
}
static inline UInt32 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_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 toTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
@ -152,8 +152,8 @@ struct ToTimeImpl
struct ToStartOfMinuteImpl
{
static inline UInt32 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toStartOfMinuteInaccurate(t); }
static inline UInt32 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
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, 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);
}
@ -161,8 +161,8 @@ struct ToStartOfMinuteImpl
struct ToStartOfFiveMinuteImpl
{
static inline UInt32 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toStartOfFiveMinuteInaccurate(t); }
static inline UInt32 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
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, 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);
}
@ -170,8 +170,8 @@ struct ToStartOfFiveMinuteImpl
struct ToStartOfHourImpl
{
static inline UInt32 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toStartOfHourInaccurate(t); }
static inline UInt32 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
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, 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);
}
@ -179,33 +179,33 @@ struct ToStartOfHourImpl
struct ToRelativeYearNumImpl
{
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(t); }
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(DayNum_t(d)); }
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, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(DayNum_t(d)); }
};
struct ToRelativeMonthNumImpl
{
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeMonthNum(t); }
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeMonthNum(DayNum_t(d)); }
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, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeMonthNum(DayNum_t(d)); }
};
struct ToRelativeWeekNumImpl
{
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeWeekNum(t); }
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeWeekNum(DayNum_t(d)); }
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, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeWeekNum(DayNum_t(d)); }
};
struct ToRelativeDayNumImpl
{
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toDayNum(t); }
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return static_cast<DayNum_t>(d); }
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, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return static_cast<DayNum_t>(d); }
};
struct ToRelativeHourNumImpl
{
static inline UInt32 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeHourNum(t); }
static inline UInt32 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
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, 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);
}
@ -213,8 +213,8 @@ struct ToRelativeHourNumImpl
struct ToRelativeMinuteNumImpl
{
static inline UInt32 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeMinuteNum(t); }
static inline UInt32 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
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, 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);
}
@ -222,8 +222,8 @@ struct ToRelativeMinuteNumImpl
struct ToRelativeSecondNumImpl
{
static inline UInt32 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return t; }
static inline UInt32 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return t; }
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);
}
@ -235,14 +235,14 @@ 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)
{
auto & local_date_lut = DateLUT::instance();
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);
auto & remote_date_lut = DateLUT::instance(time_zone);
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;
}
@ -251,23 +251,30 @@ struct Transformer
static void vector_constant(const PODArray<FromType> & vec_from, const std::string & data,
PODArray<ToType> & vec_to)
{
auto & local_date_lut = DateLUT::instance();
auto & remote_date_lut = DateLUT::instance(data);
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)
{
auto & local_date_lut = DateLUT::instance();
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);
auto & remote_date_lut = DateLUT::instance(time_zone);
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;
}
@ -275,10 +282,16 @@ struct Transformer
static void constant_constant(const FromType & from, const std::string & data, ToType & to)
{
auto & local_date_lut = DateLUT::instance();
auto & remote_date_lut = DateLUT::instance(data);
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>
@ -304,12 +317,12 @@ struct DateTimeTransformImpl
size_t size = vec_from.size();
vec_to.resize(size);
Op::vector_constant(vec_from, "", vec_to);
Op::vector_constant(vec_from, vec_to);
}
else if (const_source)
{
ToType res;
Op::constant_constant(const_source->getData(), "", res);
Op::constant_constant(const_source->getData(), res);
block.getByPosition(result).column = new ColumnConst<ToType>(const_source->size(), res);
}
else

View File

@ -725,7 +725,11 @@ public:
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);
}
@ -745,6 +749,7 @@ public:
replicated_column.name = name;
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);
++prerequisite_index;

View File

@ -375,7 +375,7 @@ inline void writeDateTimeText(time_t datetime, WriteBuffer & buf)
return;
}
auto & date_lut = DateLUT::instance();
const auto & date_lut = DateLUT::instance();
const auto & values = date_lut.getValues(datetime);
s[0] += values.year / 1000;

View File

@ -678,9 +678,15 @@ public:
std::string getModePrefix() const;
bool supportsSampling() const { return !!sampling_expression; }
bool supportsFinal() const { return !sign_column.empty(); }
bool supportsPrewhere() const { return true; }
bool supportsFinal() const
{
return mode == Mode::Collapsing
|| mode == Mode::Summing
|| mode == Mode::Aggregating;
}
UInt64 getMaxDataPartIndex();
std::string getTableName() const override

View File

@ -1007,7 +1007,7 @@ private:
text.resize(embedded_stack_trace_pos);
std::cerr << "Received exception from server:" << std::endl
<< "Code: " << e.code() << ". " << text;
<< "Code: " << e.code() << ". " << text << std::endl;
}

View File

@ -21,6 +21,8 @@ void registerFunctionsArithmetic(FunctionFactory & factory)
factory.registerFunction<FunctionBitNot>();
factory.registerFunction<FunctionBitShiftLeft>();
factory.registerFunction<FunctionBitShiftRight>();
factory.registerFunction<FunctionLeast>();
factory.registerFunction<FunctionGreatest>();
}
}

View File

@ -210,7 +210,7 @@ void QueryLog::flush()
{
LOG_TRACE(log, "Flushing query log");
auto & date_lut = DateLUT::instance();
const auto & date_lut = DateLUT::instance();
Block block = createBlock();

View File

@ -95,7 +95,7 @@ QueryParseResult QueryParser::parse(std::istream & s)
if (result.date_first > result.date_last)
throw Exception("First date is bigger than last date.", ErrorCodes::FIRST_DATE_IS_BIGGER_THAN_LAST_DATE);
auto & 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.cut_date_last = false;

View File

@ -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)
{
auto & date_lut = DateLUT::instance();
const auto & date_lut = DateLUT::instance();
/// Имя директории для куска иммет вид: YYYYMMDD_YYYYMMDD_N_N_L.
String res;
@ -153,7 +153,7 @@ void ActiveDataPartSet::parsePartName(const String & file_name, Part & part, con
const Poco::RegularExpression::MatchVec & matches = *matches_p;
auto & 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.right_date = date_lut.YYYYMMDDToDayNum(parse<UInt32>(file_name.substr(matches[2].offset, matches[2].length)));

View File

@ -46,7 +46,7 @@ bool MergeTreeDataMerger::selectPartsToMerge(MergeTreeData::DataPartsVector & pa
{
MergeTreeData::DataParts data_parts = data.getDataParts();
auto & date_lut = DateLUT::instance();
const auto & date_lut = DateLUT::instance();
size_t min_max = -1U;
size_t min_min = -1U;

View File

@ -8,6 +8,8 @@
#include <DB/DataStreams/AddingConstColumnBlockInputStream.h>
#include <DB/DataStreams/CreatingSetsBlockInputStream.h>
#include <DB/DataStreams/NullBlockInputStream.h>
#include <DB/DataStreams/SummingSortedBlockInputStream.h>
#include <DB/DataStreams/AggregatingSortedBlockInputStream.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/Common/VirtualColumnUtils.h>
@ -292,7 +294,10 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
/// Добавим столбцы, нужные для вычисления первичного ключа и знака.
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.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());
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)
use_uncompressed_cache = false;
ExpressionActionsPtr sign_filter_expression;
String sign_filter_column;
createPositiveSignCondition(sign_filter_expression, sign_filter_column);
BlockInputStreams to_collapse;
BlockInputStreams to_merge;
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");
}
to_collapse.push_back(new ExpressionBlockInputStream(source_stream, data.getPrimaryExpression()));
to_merge.push_back(new ExpressionBlockInputStream(source_stream, data.getPrimaryExpression()));
}
BlockInputStreams res;
if (to_collapse.size() == 1)
res.push_back(new FilterBlockInputStream(new ExpressionBlockInputStream(to_collapse[0], sign_filter_expression), sign_filter_column));
else if (to_collapse.size() > 1)
res.push_back(new CollapsingFinalBlockInputStream(to_collapse, data.getSortDescription(), data.sign_column));
if (to_merge.size() == 1)
{
if (!data.sign_column.empty())
{
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;
}

View File

@ -12,7 +12,7 @@ BlocksWithDateIntervals MergeTreeDataWriter::splitBlockIntoParts(const Block & b
{
data.check(block, true);
auto & date_lut = DateLUT::instance();
const auto & date_lut = DateLUT::instance();
size_t rows = block.rows();
size_t columns = block.columns();
@ -77,7 +77,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithDa
UInt16 min_date = block_with_dates.min_date;
UInt16 max_date = block_with_dates.max_date;
auto & date_lut = DateLUT::instance();
const auto & date_lut = DateLUT::instance();
size_t part_size = (block.rows() + data.index_granularity - 1) / data.index_granularity;

View File

@ -2214,7 +2214,7 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params,
static String getFakePartNameForDrop(const String & month_name, UInt64 left, UInt64 right)
{
/// Диапазон дат - весь месяц.
auto & lut = DateLUT::instance();
const auto & lut = DateLUT::instance();
time_t start_time = lut.YYYYMMDDToDate(parse<UInt32>(month_name + "01"));
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);

File diff suppressed because one or more lines are too long

View File

@ -1,43 +1,43 @@
99
2014-06-01 9
2014-06-02 9
2014-06-03 9
2014-06-04 9
2014-06-05 9
2014-06-06 9
2014-06-07 9
2014-06-08 9
2014-06-09 9
2014-06-10 9
154
2014-06-01 9
2014-06-02 9
2014-06-03 9
2014-06-04 9
2014-06-05 9
2014-06-06 14
2014-06-07 14
2014-06-08 14
2014-06-09 14
2014-06-10 14
2014-06-11 9
2014-06-12 9
2014-06-13 9
2014-06-14 9
2014-06-15 9
154
2014-06-01 9
2014-06-02 9
2014-06-03 9
2014-06-04 9
2014-06-05 9
2014-06-06 14
2014-06-07 14
2014-06-08 14
2014-06-09 14
2014-06-10 14
2014-06-11 9
2014-06-12 9
2014-06-13 9
2014-06-14 9
2014-06-15 9
100
2014-06-01 10
2014-06-02 10
2014-06-03 10
2014-06-04 10
2014-06-05 10
2014-06-06 10
2014-06-07 10
2014-06-08 10
2014-06-09 10
2014-06-10 10
155
2014-06-01 10
2014-06-02 10
2014-06-03 10
2014-06-04 10
2014-06-05 10
2014-06-06 15
2014-06-07 15
2014-06-08 15
2014-06-09 15
2014-06-10 15
2014-06-11 10
2014-06-12 10
2014-06-13 10
2014-06-14 10
2014-06-15 10
155
2014-06-01 10
2014-06-02 10
2014-06-03 10
2014-06-04 10
2014-06-05 10
2014-06-06 15
2014-06-07 15
2014-06-08 15
2014-06-09 15
2014-06-10 15
2014-06-11 10
2014-06-12 10
2014-06-13 10
2014-06-14 10
2014-06-15 10

View File

@ -1,43 +1,43 @@
99
2014-06-01 9
2014-06-02 9
2014-06-03 9
2014-06-04 9
2014-06-05 9
2014-06-06 9
2014-06-07 9
2014-06-08 9
2014-06-09 9
2014-06-10 9
154
2014-06-01 9
2014-06-02 9
2014-06-03 9
2014-06-04 9
2014-06-05 9
2014-06-06 14
2014-06-07 14
2014-06-08 14
2014-06-09 14
2014-06-10 14
2014-06-11 9
2014-06-12 9
2014-06-13 9
2014-06-14 9
2014-06-15 9
154
2014-06-01 9
2014-06-02 9
2014-06-03 9
2014-06-04 9
2014-06-05 9
2014-06-06 14
2014-06-07 14
2014-06-08 14
2014-06-09 14
2014-06-10 14
2014-06-11 9
2014-06-12 9
2014-06-13 9
2014-06-14 9
2014-06-15 9
100
2014-06-01 10
2014-06-02 10
2014-06-03 10
2014-06-04 10
2014-06-05 10
2014-06-06 10
2014-06-07 10
2014-06-08 10
2014-06-09 10
2014-06-10 10
155
2014-06-01 10
2014-06-02 10
2014-06-03 10
2014-06-04 10
2014-06-05 10
2014-06-06 15
2014-06-07 15
2014-06-08 15
2014-06-09 15
2014-06-10 15
2014-06-11 10
2014-06-12 10
2014-06-13 10
2014-06-14 10
2014-06-15 10
155
2014-06-01 10
2014-06-02 10
2014-06-03 10
2014-06-04 10
2014-06-05 10
2014-06-06 15
2014-06-07 15
2014-06-08 15
2014-06-09 15
2014-06-10 15
2014-06-11 10
2014-06-12 10
2014-06-13 10
2014-06-14 10
2014-06-15 10

View File

@ -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

View File

@ -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;

View File

@ -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

View 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();

View File

@ -1,11 +1,8 @@
#include <Yandex/DateLUT.h>
#include <Poco/Exception.h>
#include <unicode/timezone.h>
#include <unicode/unistr.h>
std::string DateLUT::default_time_zone;
DateLUT::DateLUT()
{
using namespace icu;
@ -16,6 +13,7 @@ DateLUT::DateLUT()
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());
@ -79,37 +77,33 @@ DateLUT::DateLUT()
throw Poco::Exception("Could not find any time zone information.");
date_lut_impl_list = std::make_unique<DateLUTImplList>(group_id);
}
DateLUTImpl & DateLUT::instance(const std::string & time_zone)
{
auto & date_lut = Singleton<DateLUT>::instance();
return date_lut.get(time_zone);
}
DateLUTImpl & DateLUT::get(const std::string & time_zone)
{
const std::string & actual_time_zone = time_zone.empty() ? default_time_zone : time_zone;
auto it = time_zone_to_group.find(actual_time_zone);
/// Инициализация указателя на реализацию для часового пояса по-умолчанию.
auto it = time_zone_to_group.find(default_time_zone);
if (it == time_zone_to_group.end())
throw Poco::Exception("Invalid time zone " + actual_time_zone);
throw Poco::Exception("Failed to get default time zone information.");
default_group_id = it->second;
const auto & 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_acquire);
tmp = wrapper.load(std::memory_order_relaxed);
if (tmp == nullptr)
{
tmp = new DateLUTImpl(actual_time_zone);
tmp = new DateLUTImpl(time_zone);
wrapper.store(tmp, std::memory_order_release);
}
}
return *tmp;
}

View File

@ -51,10 +51,12 @@ GDateTimePtr createGDateTime(const GTimeZonePtr & p_tz, const GDateTimePtr & p_d
if (dt == nullptr)
throw Poco::Exception("Null pointer.");
GDateTime * local_dt = g_date_time_new(p_tz.get(),
g_date_time_get_year(dt),
g_date_time_get_month(dt),
g_date_time_get_day_of_month(dt),
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));
@ -77,11 +79,12 @@ GDateTimePtr toNextDay(const GTimeZonePtr & p_tz, const GDateTimePtr & p_dt)
GDateTimePtr p_next_dt = GDateTimePtr(dt);
GDateTime * next_dt = p_next_dt.get();
dt = g_date_time_new(p_tz.get(),
g_date_time_get_year(next_dt),
g_date_time_get_month(next_dt),
g_date_time_get_day_of_month(next_dt),
0, 0, 0);
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.");
@ -110,12 +113,16 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone)
start_of_day = g_date_time_to_unix(dt);
Values & values = lut[i];
gint year;
gint month;
gint day;
g_date_time_get_ymd(dt, &year, &month, &day);
values.year = g_date_time_get_year(dt);
values.month = g_date_time_get_month(dt);
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.day_of_month = g_date_time_get_day_of_month(dt);
values.date = start_of_day;
/// Переходим на следующий день.

View File

@ -33,7 +33,7 @@ static time_t orderedIdentifierToDate(unsigned value)
void loop(time_t begin, time_t end, int step)
{
auto & date_lut = DateLUT::instance();
const auto & date_lut = DateLUT::instance();
for (time_t t = begin; t < end; t += step)
std::cout << toString(t)

View File

@ -35,7 +35,7 @@ static time_t orderedIdentifierToDate(unsigned value)
void loop(time_t begin, time_t end, int step)
{
auto & date_lut = DateLUT::instance();
const auto & date_lut = DateLUT::instance();
for (time_t t = begin; t < end; t += step)
{

View File

@ -7,7 +7,7 @@ int main(int argc, char ** argv)
/** В DateLUT был глюк - для времён из дня 1970-01-01, возвращался номер часа больше 23. */
static const time_t TIME = 66130;
auto & date_lut = DateLUT::instance();
const auto & date_lut = DateLUT::instance();
std::cerr << date_lut.toHourInaccurate(TIME) << std::endl;
std::cerr << date_lut.toDayNum(TIME) << std::endl;

View File

@ -29,7 +29,7 @@ private:
void init(time_t time)
{
auto & date_lut = DateLUT::instance();
const auto & date_lut = DateLUT::instance();
const auto & values = date_lut.getValues(time);
m_year = values.year;

View File

@ -43,7 +43,7 @@ private:
return;
}
auto & date_lut = DateLUT::instance();
const auto & date_lut = DateLUT::instance();
const auto & values = date_lut.getValues(time);
m_year = values.year;

View File

@ -145,7 +145,7 @@ private:
time_t getDateTimeImpl() const
{
auto & date_lut = DateLUT::instance();
const auto & date_lut = DateLUT::instance();
if (m_length == 10)
{
@ -173,7 +173,7 @@ private:
time_t getDateImpl() const
{
auto & date_lut = DateLUT::instance();
const auto & date_lut = DateLUT::instance();
if (m_length == 10 || m_length == 19)
{
@ -216,7 +216,7 @@ private:
return getDateImpl();
else
{
auto & date_lut = DateLUT::instance();
const auto & date_lut = DateLUT::instance();
return date_lut.toDate(getIntImpl());
}
}

View File

@ -1 +1 @@
https://github.com/Cyan4973/zstd/tree/765207c54934d478488c236749b01c7d6fc63d70/
https://github.com/Cyan4973/zstd/tree/1eca5f52994434d3b0427c9014403cf01495f54a/

View File

@ -241,14 +241,11 @@ typedef struct
int deltaFindState;
U16 maxState;
BYTE minBitsOut;
/* one byte padding */
/* one byte padding ; total 8 bytes */
} FSE_symbolCompressionTransform;
typedef struct
{
U32 fakeTable[FSE_CTABLE_SIZE_U32(FSE_MAX_TABLELOG, FSE_MAX_SYMBOL_VALUE)]; /* compatible with FSE_compressU16() */
} CTable_max_t;
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)];
/****************************************************************
* Internal functions
@ -299,13 +296,15 @@ static short FSE_abs(short a)
/****************************************************************
* 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;
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,
unsigned safeWrite)
{
@ -341,9 +340,9 @@ static size_t FSE_writeHeader_generic (void* header, size_t headerBufferSize,
while (charnum >= start+24)
{
start+=24;
bitStream += 0xFFFF<<bitCount;
bitStream += 0xFFFFU << bitCount;
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+=2;
bitStream>>=16;
@ -370,7 +369,7 @@ static size_t FSE_writeHeader_generic (void* header, size_t headerBufferSize,
short count = normalizedCounter[charnum++];
const short max = (short)((2*threshold-1)-remaining);
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 */
if (count>=threshold) count += max; /* [0..max[ [max..threshold[ (...) [threshold+max 2*threshold[ */
bitStream += count << bitCount;
@ -400,24 +399,26 @@ static size_t FSE_writeHeader_generic (void* header, size_t headerBufferSize,
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_MIN_TABLELOG) return (size_t)-FSE_ERROR_GENERIC; /* Unsupported */
if (headerBufferSize < FSE_headerBound(maxSymbolValue, tableLog))
return FSE_writeHeader_generic(header, headerBufferSize, normalizedCounter, maxSymbolValue, tableLog, 0);
if (headerBufferSize < FSE_NCountWriteBound(maxSymbolValue, tableLog))
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 BYTE* const istart = (const BYTE*) headerBuffer;
const BYTE* const iend = istart + hbSize;
const BYTE* ip = istart;
int nbBits;
int remaining;
@ -427,6 +428,7 @@ size_t FSE_readHeader (short* normalizedCounter, unsigned* maxSVPtr, unsigned* t
unsigned charnum = 0;
int previous0 = 0;
if (hbSize < 4) return (size_t)-FSE_ERROR_srcSize_wrong;
bitStream = FSE_readLE32(ip);
nbBits = (bitStream & 0xF) + FSE_MIN_TABLELOG; /* extract tableLog */
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;
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;
ip += bitCount>>3;
bitCount &= 7;
@ -488,16 +490,27 @@ size_t FSE_readHeader (short* normalizedCounter, unsigned* maxSVPtr, unsigned* t
threshold >>= 1;
}
ip += bitCount>>3;
bitCount &= 7;
bitStream = FSE_readLE32(ip) >> bitCount;
{
const BYTE* itarget = ip + (bitCount>>3);
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;
*maxSVPtr = charnum-1;
ip += bitCount>0;
if ((size_t)(ip-istart) >= hbSize) return (size_t)-FSE_ERROR_srcSize_wrong; /* arguably a bit late , tbd */
ip += (bitCount+7)>>3;
if ((size_t)(ip-istart) > hbSize) return (size_t)-FSE_ERROR_srcSize_wrong;
return ip-istart;
}
@ -506,7 +519,7 @@ size_t FSE_readHeader (short* normalizedCounter, unsigned* maxSVPtr, unsigned* t
* FSE Compression Code
****************************************************************/
/*
CTable is a variable size structure which contains :
FSE_CTable[0] is a variable size structure which contains :
U16 tableLog;
U16 maxSymbolValue;
U16 nextStateNumber[1 << tableLog]; // This size is variable
@ -523,17 +536,17 @@ size_t FSE_sizeof_CTable (unsigned maxSymbolValue, unsigned tableLog)
return size;
}
void* FSE_createCTable (unsigned maxSymbolValue, unsigned tableLog)
FSE_CTable* FSE_createCTable (unsigned maxSymbolValue, unsigned tableLog)
{
size_t size;
if (tableLog > FSE_TABLELOG_ABSOLUTE_MAX) tableLog = FSE_TABLELOG_ABSOLUTE_MAX;
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;
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(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_MAX_TABLELOG) tableLog = FSE_MAX_TABLELOG;
return tableLog;
@ -671,7 +684,7 @@ static size_t FSE_normalizeM2(short* norm, U32 tableLog, const unsigned* count,
U32 maxV = 0, maxC =0;
for (s=0; s<=maxSymbolValue; s++)
if (count[s] > maxC) maxV=s, maxC=count[s];
norm[maxV] += ToDistribute;
norm[maxV] += (short)ToDistribute;
return 0;
}
@ -690,7 +703,7 @@ static size_t FSE_normalizeM2(short* norm, U32 tableLog, const unsigned* count,
U32 weight = sEnd - sStart;
if (weight < 1)
return (size_t)-FSE_ERROR_GENERIC;
norm[s] = weight;
norm[s] = (short)weight;
tmpTotal = end;
}
}
@ -779,19 +792,18 @@ size_t FSE_normalizeCount (short* normalizedCounter, unsigned tableLog,
}
/* fake CTable, for raw (uncompressed) input */
size_t FSE_buildCTable_raw (void* CTable, unsigned nbBits)
/* fake FSE_CTable, for raw (uncompressed) input */
size_t FSE_buildCTable_raw (FSE_CTable* ct, unsigned nbBits)
{
const unsigned tableSize = 1 << nbBits;
const unsigned tableMask = tableSize - 1;
const unsigned maxSymbolValue = tableMask;
U16* tableU16 = ( (U16*) CTable) + 2;
FSE_symbolCompressionTransform* symbolTT = (FSE_symbolCompressionTransform*) ((((U32*)CTable)+1) + (tableSize>>1));
U16* tableU16 = ( (U16*) ct) + 2;
FSE_symbolCompressionTransform* symbolTT = (FSE_symbolCompressionTransform*) ((((U32*)ct)+1) + (tableSize>>1));
unsigned s;
/* Sanity checks */
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 */
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 */
size_t FSE_buildCTable_rle (void* CTable, BYTE symbolValue)
/* fake FSE_CTable, for rle (100% always same symbol) input */
size_t FSE_buildCTable_rle (FSE_CTable* ct, BYTE symbolValue)
{
const unsigned tableSize = 1;
U16* tableU16 = ( (U16*) CTable) + 2;
FSE_symbolCompressionTransform* symbolTT = (FSE_symbolCompressionTransform*) ((U32*)CTable + 2);
/* safety checks */
if (((size_t)CTable) & 3) return (size_t)-FSE_ERROR_GENERIC; /* Must be 4 bytes aligned */
U16* tableU16 = ( (U16*) ct) + 2;
FSE_symbolCompressionTransform* symbolTT = (FSE_symbolCompressionTransform*) ((U32*)ct + 2);
/* header */
tableU16[-2] = (U16) 0;
@ -850,12 +859,12 @@ void FSE_initCStream(FSE_CStream_t* bitC, void* start)
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->stateTable = ((const U16*) CTable) + 2;
statePtr->symbolTT = (const U32*)CTable + 1 + (tableLog ? (1<<(tableLog-1)) : 1);
statePtr->stateTable = ((const U16*) ct) + 2;
statePtr->symbolTT = (const FSE_symbolCompressionTransform*)((const U32*)ct + 1 + (tableLog ? (1<<(tableLog-1)) : 1));
statePtr->stateLog = tableLog;
}
@ -866,14 +875,14 @@ void FSE_addBits(FSE_CStream_t* bitC, size_t value, unsigned 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 U16* const stateTable = (const U16*) statePtr->stateTable;
int nbBitsOut = symbolTT[symbol].minBitsOut;
nbBitsOut -= (int)((symbolTT[symbol].maxState - statePtr->value) >> 31);
const FSE_symbolCompressionTransform symbolTT = ((const FSE_symbolCompressionTransform*)(statePtr->symbolTT))[symbol];
const U16* const stateTable = (const U16*)(statePtr->stateTable);
int nbBitsOut = symbolTT.minBitsOut;
nbBitsOut -= (int)((symbolTT.maxState - statePtr->value) >> 31);
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)
@ -908,7 +917,7 @@ size_t FSE_closeCStream(FSE_CStream_t* bitC)
size_t FSE_compress_usingCTable (void* dst, size_t dstSize,
const void* src, size_t srcSize,
const void* CTable)
const FSE_CTable* ct)
{
const BYTE* const istart = (const BYTE*) src;
const BYTE* ip;
@ -921,7 +930,7 @@ size_t FSE_compress_usingCTable (void* dst, size_t dstSize,
/* init */
(void)dstSize; /* objective : ensure it fits into dstBuffer (Todo) */
FSE_initCStream(&bitC, dst);
FSE_initCState(&CState1, CTable);
FSE_initCState(&CState1, ct);
CState2 = CState1;
ip=iend;
@ -929,32 +938,32 @@ size_t FSE_compress_usingCTable (void* dst, size_t dstSize,
/* join to even */
if (srcSize & 1)
{
FSE_encodeByte(&bitC, &CState1, *--ip);
FSE_encodeSymbol(&bitC, &CState1, *--ip);
FSE_flushBits(&bitC);
}
/* join to mod 4 */
if ((sizeof(size_t)*8 > FSE_MAX_TABLELOG*4+7 ) && (srcSize & 2)) /* test bit 2 */
{
FSE_encodeByte(&bitC, &CState2, *--ip);
FSE_encodeByte(&bitC, &CState1, *--ip);
FSE_encodeSymbol(&bitC, &CState2, *--ip);
FSE_encodeSymbol(&bitC, &CState1, *--ip);
FSE_flushBits(&bitC);
}
/* 2 or 4 encoding per loop */
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 */
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 */
{
FSE_encodeByte(&bitC, &CState2, *--ip);
FSE_encodeByte(&bitC, &CState1, *--ip);
FSE_encodeSymbol(&bitC, &CState2, *--ip);
FSE_encodeSymbol(&bitC, &CState1, *--ip);
}
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];
S16 norm[FSE_MAX_SYMBOL_VALUE+1];
CTable_max_t CTable;
CTable_max_t ct;
size_t errorCode;
/* 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;
/* 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 (errorCode == srcSize) return 1;
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;
/* 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;
op += errorCode;
/* Compress */
errorCode = FSE_buildCTable (&CTable, norm, maxSymbolValue, tableLog);
errorCode = FSE_buildCTable (ct, norm, maxSymbolValue, tableLog);
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 */
if ( (size_t)(op-ostart) >= srcSize-1 )
@ -1033,24 +1042,12 @@ typedef struct
BYTE nbBits;
} 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 (void* DTable, BYTE symbolValue)
size_t FSE_buildDTable_rle (FSE_DTable* dt, BYTE symbolValue)
{
U32* const base32 = (U32*)DTable;
U32* const base32 = (U32*)dt;
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;
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);
const unsigned tableSize = 1 << nbBits;
const unsigned tableMask = tableSize - 1;
@ -1072,7 +1069,6 @@ size_t FSE_buildDTable_raw (void* DTable, unsigned nbBits)
/* Sanity checks */
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 */
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 >= sizeof(bitD_t))
if (srcSize >= sizeof(size_t))
{
U32 contain32;
bitD->start = (char*)srcBuffer;
bitD->ptr = (char*)srcBuffer + srcSize - sizeof(bitD_t);
bitD->start = (const char*)srcBuffer;
bitD->ptr = (const char*)srcBuffer + srcSize - sizeof(size_t);
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 */
bitD->bitsConsumed = 8 - FSE_highbit32(contain32);
}
else
{
U32 contain32;
bitD->start = (char*)srcBuffer;
bitD->start = (const char*)srcBuffer;
bitD->ptr = bitD->start;
bitD->bitContainer = *(BYTE*)(bitD->start);
bitD->bitContainer = *(const BYTE*)(bitD->start);
switch(srcSize)
{
case 7: bitD->bitContainer += (bitD_t)(((BYTE*)(bitD->start))[6]) << (sizeof(bitD_t)*8 - 16);
case 6: bitD->bitContainer += (bitD_t)(((BYTE*)(bitD->start))[5]) << (sizeof(bitD_t)*8 - 24);
case 5: bitD->bitContainer += (bitD_t)(((BYTE*)(bitD->start))[4]) << (sizeof(bitD_t)*8 - 32);
case 4: bitD->bitContainer += (bitD_t)(((BYTE*)(bitD->start))[3]) << 24;
case 3: bitD->bitContainer += (bitD_t)(((BYTE*)(bitD->start))[2]) << 16;
case 2: bitD->bitContainer += (bitD_t)(((BYTE*)(bitD->start))[1]) << 8;
case 7: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[6]) << (sizeof(size_t)*8 - 16);
case 6: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[5]) << (sizeof(size_t)*8 - 24);
case 5: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[4]) << (sizeof(size_t)*8 - 32);
case 4: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[3]) << 24;
case 3: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[2]) << 16;
case 2: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[1]) << 8;
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 */
bitD->bitsConsumed = 8 - FSE_highbit32(contain32);
bitD->bitsConsumed += (U32)(sizeof(bitD_t) - srcSize)*8;
bitD->bitsConsumed += (U32)(sizeof(size_t) - srcSize)*8;
}
return srcSize;
@ -1135,27 +1131,28 @@ size_t FSE_initDStream(FSE_DStream_t* bitD, const void* srcBuffer, size_t srcSiz
/* FSE_readBits
* Read next n bits from the bitContainer.
* Use the fast variant *only* if n > 0.
* Note : for this function to work properly on 32-bits, don't read more than maxNbBits==25
* 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.
*/
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;
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;
return value;
}
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->bitsConsumed &= 7;
@ -1164,8 +1161,8 @@ unsigned FSE_reloadDStream(FSE_DStream_t* bitD)
}
if (bitD->ptr == bitD->start)
{
if (bitD->bitsConsumed < sizeof(bitD_t)*8) return 1;
if (bitD->bitsConsumed == sizeof(bitD_t)*8) return 2;
if (bitD->bitsConsumed < sizeof(size_t)*8) return 1;
if (bitD->bitsConsumed == sizeof(size_t)*8) return 2;
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]);
FSE_reloadDStream(bitD);
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 U32 nbBits = DInfo.nbBits;
BYTE symbol = DInfo.symbol;
bitD_t lowBits = FSE_readBits(bitD, nbBits);
size_t lowBits = FSE_readBits(bitD, nbBits);
DStatePtr->state = DInfo.newState + lowBits;
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 U32 nbBits = DInfo.nbBits;
BYTE symbol = DInfo.symbol;
bitD_t lowBits = FSE_readBitsFast(bitD, nbBits);
size_t lowBits = FSE_readBitsFast(bitD, nbBits);
DStatePtr->state = DInfo.newState + lowBits;
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)
{
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(
void* dst, size_t maxDstSize,
const void* cSrc, size_t cSrcSize,
const void* DTable, unsigned fast)
const FSE_DTable* dt, unsigned fast)
{
BYTE* const ostart = (BYTE*) dst;
BYTE* op = ostart;
@ -1235,15 +1232,16 @@ FORCE_INLINE size_t FSE_decompress_usingDTable_generic(
BYTE* const olimit = omax-3;
FSE_DStream_t bitD;
FSE_DState_t state1, state2;
FSE_DState_t state1;
FSE_DState_t state2;
size_t errorCode;
/* Init */
errorCode = FSE_initDStream(&bitD, cSrc, cSrcSize); /* replaced last arg by maxCompressed Size */
if (FSE_isError(errorCode)) return errorCode;
FSE_initDState(&state1, &bitD, DTable);
FSE_initDState(&state2, &bitD, DTable);
FSE_initDState(&state1, &bitD, dt);
FSE_initDState(&state2, &bitD, dt);
/* 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);
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);
*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(&state2, &bitD) : FSE_decodeSymbol(&state2, &bitD);
@ -1264,14 +1262,15 @@ FORCE_INLINE size_t FSE_decompress_usingDTable_generic(
}
/* tail */
/* note : FSE_reloadDStream(&bitD) >= 1; Ends at exactly 2 */
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;
*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;
*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,
const void* cSrc, size_t cSrcSize,
const void* DTable, size_t fastMode)
const FSE_DTable* dt, size_t fastMode)
{
/* select fast mode (static) */
if (fastMode) return FSE_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, DTable, 1);
return FSE_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, DTable, 0);
if (fastMode) return FSE_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, dt, 1);
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* ip = istart;
short counting[FSE_MAX_SYMBOL_VALUE+1];
FSE_decode_t DTable[FSE_DTABLE_SIZE_U32(FSE_MAX_TABLELOG)];
unsigned maxSymbolValue = FSE_MAX_SYMBOL_VALUE;
DTable_max_t dt; /* Static analyzer seems unable to understand this table will be properly initialized later */
unsigned tableLog;
unsigned maxSymbolValue = FSE_MAX_SYMBOL_VALUE;
size_t errorCode, fastMode;
if (cSrcSize<2) return (size_t)-FSE_ERROR_srcSize_wrong; /* too small input size */
/* 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 (errorCode >= cSrcSize) return (size_t)-FSE_ERROR_srcSize_wrong; /* too small input size */
ip += errorCode;
cSrcSize -= errorCode;
fastMode = FSE_buildDTable (DTable, counting, maxSymbolValue, tableLog);
fastMode = FSE_buildDTable (dt, counting, maxSymbolValue, tableLog);
if (FSE_isError(fastMode)) return fastMode;
/* 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
designed to be included
for type-specific functions (template equivalent in C)
Objective is to write such functions only once, for better maintenance
for type-specific functions (template emulation in C)
Objective is to write these functions only once, for improved maintenance
*/
/* safety checks */
@ -1348,7 +1347,8 @@ size_t FSE_decompress(void* dst, size_t maxDstSize, const void* cSrc, size_t cSr
/* 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* 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--;
*maxSymbolValuePtr = maxSymbolValue;
return (int)max;
return (size_t)max;
}
/* 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))
{
*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; }
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 tableMask = tableSize - 1;
U16* tableU16 = ( (U16*) CTable) + 2;
FSE_symbolCompressionTransform* symbolTT = (FSE_symbolCompressionTransform*) (((U32*)CTable) + 1 + (tableLog ? tableSize>>1 : 1) );
U16* tableU16 = ( (U16*) ct) + 2;
FSE_symbolCompressionTransform* symbolTT = (FSE_symbolCompressionTransform*) (((U32*)ct) + 1 + (tableLog ? tableSize>>1 : 1) );
const unsigned step = FSE_tableStep(tableSize);
unsigned cumul[FSE_MAX_SYMBOL_VALUE+2];
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;
unsigned symbol;
unsigned i;
/* safety checks */
if (((size_t)CTable) & 3) return (size_t)-FSE_ERROR_GENERIC; /* Must be allocated of 4 bytes boundaries */
/* header */
tableU16[-2] = (U16) tableLog;
tableU16[-1] = (U16) maxSymbolValue;
@ -1501,10 +1500,10 @@ size_t FSE_FUNCTION_NAME(FSE_buildCTable, FSE_FUNCTION_EXTENSION)
for (i=0; i<tableSize; 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 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)
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;
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)
(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);
const U32 tableSize = 1 << tableLog;
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];
U32 position = 0;
U32 highThreshold = tableSize-1;
const S16 largeLimit= 1 << (tableLog-1);
const S16 largeLimit= (S16)(1 << (tableLog-1));
U32 noLarge = 1;
U32 s;
@ -1602,7 +1601,7 @@ size_t FSE_FUNCTION_NAME(FSE_buildDTable, FSE_FUNCTION_EXTENSION)
U32 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]++;
tableDecode[i].nbBits = (BYTE) (tableLog - FSE_highbit32 ((U32)nextState) );
tableDecode[i].newState = (U16) ( (nextState << tableDecode[i].nbBits) - tableSize);

View File

@ -42,7 +42,7 @@ extern "C" {
/******************************************
* 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,
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);
/*
FSE_compress():
@ -58,29 +58,19 @@ FSE_compress():
'dst' buffer must be already allocated, and sized to handle worst case situations.
Worst case size evaluation is provided by FSE_compressBound().
return : size of compressed data
Special values : if result == 0, data is uncompressible => Nothing is stored within cSrc !!
if result == 1, data is one constant element x srcSize times. Use RLE compression.
if FSE_isError(result), it's an error code.
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.
FSE_decompress():
Decompress FSE data from buffer 'cSrc', of size 'cSrcSize',
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)
or an error code, which can be tested using FSE_isError()
*/
size_t FSE_decompressRLE(void* dst, size_t originalSize,
const void* cSrc, size_t cSrcSize);
/*
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()
** Important ** : FSE_decompress() doesn't decompress non-compressible nor RLE data !!!
Why ? : making this distinction requires a header.
FSE library doesn't manage headers, which are intentionally left to the user layer.
*/
@ -102,51 +92,103 @@ FSE_compress2():
Same as FSE_compress(), but allows the selection of 'maxSymbolValue' and 'tableLog'
Both parameters can be defined as '0' to mean : use default value
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);
/******************************************
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[]
2. normalize counters so that sum(count[]) == Power_of_2 (2^tableLog)
3. save normalized counters to memory buffer using writeHeader()
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()
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 *** */
size_t FSE_count(unsigned* count, const unsigned char* src, size_t srcSize, unsigned* maxSymbolValuePtr);
unsigned FSE_optimalTableLog(unsigned tableLog, size_t srcSize, unsigned maxSymbolValue);
size_t FSE_normalizeCount(short* normalizedCounter, unsigned tableLog, const unsigned* count, size_t total, unsigned maxSymbolValue);
size_t FSE_headerBound(unsigned maxSymbolValue, unsigned tableLog);
size_t FSE_writeHeader (void* headerBuffer, size_t headerBufferSize, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog);
void* FSE_createCTable (unsigned tableLog, unsigned maxSymbolValue);
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_count():
Provides the precise count of each symbol within a table 'count'
'count' is a table of unsigned int, of minimum size (maxSymbolValuePtr[0]+1).
maxSymbolValuePtr[0] will be updated if detected smaller than initially expected
return : the count of the most frequent symbol (which is not identified)
if return == srcSize, there is only one symbol.
if FSE_isError(return), it's an error code. */
size_t FSE_count(unsigned* count, unsigned* maxSymbolValuePtr, const unsigned char* src, size_t srcSize);
/*
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.
Result will be saved into 'count', a table of unsigned int, which must be already allocated, and have '*maxSymbolValuePtr+1' cells.
'source' is a table of char of size 'sourceSize'. All values within 'src' MUST be <= *maxSymbolValuePtr
*maxSymbolValuePtr will be updated, with its real value (necessarily <= original value)
Result will be saved into 'count', a table of unsigned int, which must be already allocated, and have 'maxSymbolValuePtr[0]+1' cells.
'src' is a table of bytes of size 'srcSize'. All values within 'src' MUST be <= maxSymbolValuePtr[0]
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.
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'.
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().
'CTable' must be aligned of 4 bytes boundaries.
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()).
'CTable' can then be used to compress 'source', with FSE_compress_usingCTable().
Similar to FSE_count(), the convention is that 'source' is assumed to be a table of char of size 'sourceSize'
The function returns the size of compressed data (without header), or -1 if failed.
'CTable' can then be used to compress 'src', with FSE_compress_usingCTable().
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).
If there is an error, the function will return an ErrorCode (which can be tested using FSE_isError()).
*/
/* *** DECOMPRESSION *** */
size_t FSE_readHeader (short* normalizedCounter, unsigned* maxSymbolValuePtr, unsigned* tableLogPtr, const void* headerBuffer, size_t hbSize);
void* FSE_createDTable(unsigned tableLog);
void FSE_freeDTable(void* DTable);
size_t FSE_buildDTable (void* DTable, const short* const normalizedCounter, unsigned maxSymbolValue, unsigned tableLog);
size_t FSE_decompress_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const void* DTable, size_t fastMode);
/*
FSE_readNCount():
Read compactly saved 'normalizedCounter' from 'rBuffer'.
return : size read from 'rBuffer'
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_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.
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,
or size the table to handle worst case situations (typically 256).
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 following values have special meaning :
return 2 : there is only a single symbol value. The value is provided into the second byte of header.
return 1 : data is uncompressed
Note that 'headerSize' must be at least 4 bytes, even if useful information is less than that.
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().
The space required by 'DTable' must be already allocated and properly aligned.
One can create a DTable using FSE_createDTable().
The function will return 1 if DTable is compatible with fastMode, 0 otherwise.
The space required by 'FSE_DTable' must be already allocated using FSE_createDTable().
The function will return 1 if FSE_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().
'DTable' can then be used to decompress 'compressed', with FSE_decompress_usingDTable().
Only trigger fastMode if it was authorized by result of FSE_buildDTable(), otherwise decompression will fail.
'FSE_DTable' can then be used to decompress 'cSrc', with FSE_decompress_usingDTable().
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.
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().
*/
/******************************************
* 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)
}
#endif

View File

@ -40,24 +40,19 @@ extern "C" {
/******************************************
* Tool functions
* FSE API compatible with DLL
******************************************/
#define FSE_MAX_HEADERSIZE 512
#define FSE_COMPRESSBOUND(size) (size + (size>>7) + FSE_MAX_HEADERSIZE) /* Macro can be useful for static allocation */
#include "fse.h"
/******************************************
* 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_DTABLE_SIZE_U32(maxTableLog) ((1<<maxTableLog)+1)
/******************************************
* FSE supported API for DLL
******************************************/
#include "fse.h"
#define FSE_DTABLE_SIZE_U32(maxTableLog) (1 + (1<<maxTableLog))
/******************************************
@ -65,7 +60,7 @@ extern "C" {
******************************************/
#define FSE_LIST_ERRORS(ITEM) \
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_corruptionDetected) \
ITEM(FSE_ERROR_maxCode)
@ -77,26 +72,182 @@ typedef enum { FSE_LIST_ERRORS(FSE_GENERATE_ENUM) } FSE_errorCodes; /* enum is
/******************************************
* FSE advanced API
******************************************/
size_t FSE_countFast(unsigned* count, const unsigned char* src, size_t srcSize, unsigned* maxSymbolValuePtr);
/* same as FSE_count(), but won't check if input really respect that all values within src are <= *maxSymbolValuePtr */
size_t FSE_countFast(unsigned* count, unsigned* maxSymbolValuePtr, const unsigned char* src, size_t srcSize);
/* same as FSE_count(), but blindly trust that all values within src are <= maxSymbolValuePtr[0] */
size_t FSE_buildCTable_raw (void* CTable, unsigned nbBits);
/* create a fake CTable, designed to not compress an input where each element uses nbBits */
size_t FSE_buildCTable_raw (FSE_CTable* ct, unsigned 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);
/* create a fake CTable, designed to compress a single identical value */
size_t FSE_buildCTable_rle (FSE_CTable* ct, unsigned char symbolValue);
/* build a fake FSE_CTable, designed to compress always the same symbolValue */
size_t FSE_buildDTable_raw (void* DTable, unsigned nbBits);
/* create a fake DTable, designed to read an uncompressed bitstream where each element uses nbBits */
size_t FSE_buildDTable_raw (FSE_DTable* dt, unsigned 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);
/* create a fake DTable, designed to always generate the same symbolValue */
size_t FSE_buildDTable_rle (FSE_DTable* dt, unsigned char 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) */
unsigned char FSE_decodeSymbolFast(FSE_DState_t* DStatePtr, FSE_DStream_t* bitD);

View File

@ -146,7 +146,7 @@ static const U32 g_maxDistance = 4 * BLOCKSIZE;
static const U32 g_maxLimit = 1 GB;
static const U32 g_searchStrength = 8;
#define WORKPLACESIZE (BLOCKSIZE*11/4)
#define WORKPLACESIZE (BLOCKSIZE*3)
#define MINMATCH 4
#define MLbits 7
#define LLbits 6
@ -178,11 +178,13 @@ static unsigned ZSTD_isLittleEndian(void)
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); }
@ -290,6 +292,8 @@ typedef struct {
void* buffer;
U32* offsetStart;
U32* offset;
BYTE* offCodeStart;
BYTE* offCode;
BYTE* litStart;
BYTE* lit;
BYTE* litLengthStart;
@ -310,7 +314,7 @@ void ZSTD_resetSeqStore(seqStore_t* ssPtr)
}
typedef struct
typedef struct ZSTD_Cctx_s
{
const BYTE* base;
U32 current;
@ -324,28 +328,28 @@ typedef struct
} 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.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.matchLengthStart = ctx->seqStore.litLengthStart + (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;
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);
return 0;
@ -457,7 +461,7 @@ static unsigned ZSTD_NbCommonBytes (register size_t val)
_BitScanReverse( &r, (unsigned long)val );
return (unsigned)(r>>3);
# elif defined(__GNUC__) && (GCC_VERSION >= 304) && !defined(LZ4_FORCE_SW_BITCOUNT)
return (__builtin_clz(val) >> 3);
return (__builtin_clz((U32)val) >> 3);
# else
unsigned r;
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()) */
(void)maxDstSize;
ostart[ZSTD_blockHeaderSize] = *(BYTE*)src;
ostart[ZSTD_blockHeaderSize] = *(const BYTE*)src;
/* Build header */
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 */
static size_t ZSTD_compressLiterals_usingCTable(void* dst, size_t dstSize,
const void* src, size_t srcSize,
const void* CTable)
const FSE_CTable* CTable)
{
const BYTE* const istart = (const BYTE*)src;
const BYTE* ip = istart;
@ -553,32 +557,32 @@ static size_t ZSTD_compressLiterals_usingCTable(void* dst, size_t dstSize,
// join to mod 2
if (srcSize & 1)
{
FSE_encodeByte(&bitC, &CState1, *ip++);
FSE_encodeSymbol(&bitC, &CState1, *ip++);
FSE_flushBits(&bitC);
}
// join to mod 4
if ((sizeof(size_t)*8 > LitFSELog*4+7 ) && (srcSize & 2)) // test bit 2
{
FSE_encodeByte(&bitC, &CState2, *ip++);
FSE_encodeByte(&bitC, &CState1, *ip++);
FSE_encodeSymbol(&bitC, &CState2, *ip++);
FSE_encodeSymbol(&bitC, &CState1, *ip++);
FSE_flushBits(&bitC);
}
// 2 or 4 encoding per loop
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
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
{
FSE_encodeByte(&bitC, &CState2, *ip++);
FSE_encodeByte(&bitC, &CState1, *ip++);
FSE_encodeSymbol(&bitC, &CState2, *ip++);
FSE_encodeSymbol(&bitC, &CState1, *ip++);
}
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;
/* 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 (errorCode == srcSize) return 1;
//if (errorCode < ((srcSize * 7) >> 10)) return 0;
//if (errorCode < (srcSize >> 7)) return 0;
if (errorCode < (srcSize >> 6)) return 0; /* heuristic : probably not compressible enough */
if (errorCode < (srcSize >> 6)) return 0; /* cheap heuristic : probably not compressible enough */
tableLog = FSE_optimalTableLog(tableLog, srcSize, maxSymbolValue);
errorCode = (int)FSE_normalizeCount (norm, tableLog, count, srcSize, maxSymbolValue);
if (FSE_isError(errorCode)) return (size_t)-ZSTD_ERROR_GENERIC;
/* 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;
op += errorCode;
/* 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;
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;
op += errorCode;
@ -681,7 +683,7 @@ static size_t ZSTD_compressSequences(BYTE* dst, size_t maxDstSize,
const BYTE* op_matchLength = seqStorePtr->matchLength;
const size_t nbSeq = op_litLength - op_litLength_start;
BYTE* op;
BYTE offsetBits_start[BLOCKSIZE / 4];
BYTE* offsetBits_start = seqStorePtr->offCodeStart;
BYTE* offsetBitsPtr = offsetBits_start;
const size_t minGain = ZSTD_minGain(srcSize);
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;
BYTE* seqHead;
/* init */
op = dst;
@ -740,8 +741,8 @@ static size_t ZSTD_compressSequences(BYTE* dst, size_t maxDstSize,
/* Encoding table of Literal Lengths */
max = MaxLL;
mostFrequent = FSE_countFast(count, seqStorePtr->litLengthStart, nbSeq, &max);
if (mostFrequent == nbSeq)
mostFrequent = FSE_countFast(count, &max, seqStorePtr->litLengthStart, nbSeq);
if ((mostFrequent == nbSeq) && (nbSeq > 2))
{
*op++ = *(seqStorePtr->litLengthStart);
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);
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);
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;
}
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;
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);
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);
Offtype = bt_compressed;
}
/* Encoding Table of MatchLengths */
max = MaxML;
mostFrequent = FSE_countFast(count, seqStorePtr->matchLengthStart, nbSeq, &max);
if (mostFrequent == nbSeq)
mostFrequent = FSE_countFast(count, &max, seqStorePtr->matchLengthStart, nbSeq);
if ((mostFrequent == nbSeq) && (nbSeq > 2))
{
*op++ = *seqStorePtr->matchLengthStart;
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);
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);
MLtype = bt_compressed;
}
@ -838,12 +839,12 @@ static size_t ZSTD_compressSequences(BYTE* dst, size_t maxDstSize,
BYTE offCode = *(--offsetBitsPtr); /* 32b*/ /* 64b*/
U32 nbBits = (offCode-1) * (!!offCode);
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 */
FSE_addBits(&blockStream, offset, nbBits); /* 32 */ /* 42 */
if (ZSTD_32bits()) FSE_flushBits(&blockStream); /* 7 */
FSE_encodeByte(&blockStream, &stateOffsetBits, offCode); /* 16 */ /* 51 */
FSE_encodeByte(&blockStream, &stateLitLength, litLength); /* 26 */ /* 61 */
FSE_encodeSymbol(&blockStream, &stateOffsetBits, offCode); /* 16 */ /* 51 */
FSE_encodeSymbol(&blockStream, &stateLitLength, litLength); /* 26 */ /* 61 */
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 * 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 * prime5bytes) >> (40-HASH_LOG)) & HASH_MASK; }
//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);
const BYTE* r;
r = table[h] + start;
//table[h] = (U32)(p - start);
ZSTD_addPtr(table, p, start);
return r;
}
@ -961,7 +961,7 @@ static size_t ZSTD_compressBlock(void* cctx, void* dst, size_t maxDstSize, const
/* Main Search Loop */
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; }
@ -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 */
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)
{
cctxi_t* ctx = (cctxi_t*) cctx;
int i;
#if defined(__AVX2__) /* <immintrin.h> */
#if defined(__AVX2__)
/* AVX2 version */
__m256i* h = ctx->hashTable;
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);
}
#else
/* this should be auto-vectorized by compiler */
U32* h = ctx->hashTable;
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)
{
cctxi_t* ctx = (cctxi_t*) cctx;
@ -1057,7 +1056,7 @@ static void ZSTD_limitCtx(void* cctx, const U32 limit)
return;
}
#if defined(__AVX2__) /* <immintrin.h> */
#if defined(__AVX2__)
/* AVX2 version */
{
__m256i* h = ctx->hashTable;
@ -1071,6 +1070,7 @@ static void ZSTD_limitCtx(void* cctx, const U32 limit)
}
}
#else
/* this should be auto-vectorized by compiler */
{
U32* h = (U32*)(ctx->hashTable);
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;
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;
/* Init */
if (maxDstSize < ZSTD_compressBound(srcSize) - 4 /* frame header size*/) return (size_t)-ZSTD_ERROR_maxDstSize_tooSmall;
if (ctx->base==NULL)
ctx->base = (const BYTE*)src, ctx->current=0, ctx->nextUpdate = g_maxDistance;
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 */
if (maxDstSize < ZSTD_blockHeaderSize) return (size_t)-ZSTD_ERROR_maxDstSize_tooSmall;
cSize = ZSTD_compressBlock(ctx, op+ZSTD_blockHeaderSize, maxDstSize-ZSTD_blockHeaderSize, ip, blockSize);
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;
@ -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 oend = ostart + maxDstSize;
BYTE* op = ostart;
/* Header */
@ -1180,7 +1181,7 @@ static size_t ZSTD_compressCCtx(void* ctx, void* dst, size_t maxDstSize, const v
/* 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;
op += cSize;
maxDstSize -= cSize;
@ -1188,7 +1189,7 @@ static size_t ZSTD_compressCCtx(void* ctx, void* dst, size_t maxDstSize, const v
/* 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;
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)
{
void* ctx;
ZSTD_Cctx* ctx;
size_t r;
ctx = ZSTD_createCCtx();
@ -1209,6 +1210,7 @@ size_t ZSTD_compress(void* dst, size_t maxDstSize, const void* src, size_t srcSi
}
/**************************************************************
* 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(
void* const dst, size_t maxDstSize,
const void* src, size_t srcSize,
const void* DTable, U32 fast)
const FSE_DTable* DTable, U32 fast)
{
BYTE* op = (BYTE*) dst;
BYTE* const olimit = op;
@ -1305,7 +1307,7 @@ FORCE_INLINE size_t ZSTD_decompressLiterals_usingDTable_generic(
static size_t ZSTD_decompressLiterals_usingDTable(
void* const dst, size_t maxDstSize,
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);
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 */
const BYTE* ip = (const BYTE*)src;
short norm[256];
void* DTable = ctx;
FSE_DTable* DTable = (FSE_DTable*)ctx;
U32 maxSymbolValue = 255;
U32 tableLog;
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 */
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;
ip += errorCode;
srcSize -= errorCode;
@ -1367,9 +1369,9 @@ size_t ZSTD_decodeLiteralsBlock(void* ctx,
}
case bt_compressed:
{
size_t cSize = ZSTD_decompressLiterals(ctx, dst, maxDstSize, ip, litcSize);
if (ZSTD_isError(cSize)) return cSize;
*litPtr = oend - cSize;
size_t litSize = ZSTD_decompressLiterals(ctx, dst, maxDstSize, ip, litcSize);
if (ZSTD_isError(litSize)) return litSize;
*litPtr = oend - litSize;
ip += litcSize;
break;
}
@ -1382,7 +1384,7 @@ size_t ZSTD_decodeLiteralsBlock(void* ctx,
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 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;
default :
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;
ip += headerSize;
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;
default :
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;
ip += headerSize;
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;
default :
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;
ip += headerSize;
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 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 */
void* DTableML = ctx;
void* DTableLL = ((U32*)ctx) + FSE_DTABLE_SIZE_U32(MLFSELog);
void* DTableOffb = ((U32*)DTableLL) + FSE_DTABLE_SIZE_U32(LLFSELog);
FSE_DTable* DTableML = (FSE_DTable*)ctx;
FSE_DTable* DTableLL = DTableML + FSE_DTABLE_SIZE_U32(MLFSELog);
FSE_DTable* DTableOffb = DTableLL + FSE_DTABLE_SIZE_U32(LLFSELog);
/* 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);
if (ZSTD_isError(errorCode)) return errorCode;
/* end pos */
if ((litPtr>=ostart) && (litPtr<=oend))
if ((litPtr>=ostart) && (litPtr<=oend)) /* decoded literals are into dst buffer */
litEnd = oend - lastLLSize;
else
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_DState_t stateLL, stateOffb, stateML;
size_t prevOffset = 0, offset = 0;
size_t qutt=0;
FSE_initDStream(&DStream, ip, iend-ip);
FSE_initDState(&stateLL, &DStream, DTableLL);
@ -1545,7 +1546,6 @@ _another_round:
if (add < 255) litLength += add;
else
{
//litLength = (*(U32*)dumps) & 0xFFFFFF;
litLength = ZSTD_readLE32(dumps) & 0xFFFFFF;
dumps += 3;
}
@ -1578,7 +1578,7 @@ _another_round:
if (add < 255) matchLength += add;
else
{
matchLength = ZSTD_readLE32(dumps) & 0xFFFFFF;
matchLength = ZSTD_readLE32(dumps) & 0xFFFFFF; /* no pb : dumps is always followed by seq tables > 1 byte */
dumps += 3;
}
}
@ -1587,8 +1587,10 @@ _another_round:
/* copy Match */
{
BYTE* const endMatch = op + matchLength;
size_t qutt=0;
U64 saved[2];
/* save beginning of literal sequence, in case of write overlap */
if ((size_t)(litPtr - endMatch) < 12)
{
qutt = endMatch + 12 - litPtr;
@ -1624,7 +1626,7 @@ _another_round:
op = endMatch;
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
*******************************/
typedef struct
typedef struct ZSTD_Dctx_s
{
U32 ctx[FSE_DTABLE_SIZE_U32(LLFSELog) + FSE_DTABLE_SIZE_U32(OffFSELog) + FSE_DTABLE_SIZE_U32(MLFSELog)];
size_t expected;
@ -1721,27 +1723,28 @@ typedef struct
} 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->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);
return 0;
}
size_t ZSTD_nextSrcSizeToDecompress(ZSTD_dctx_t dctx)
size_t ZSTD_nextSrcSizeToDecompress(ZSTD_Dctx* dctx)
{
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;

View File

@ -48,7 +48,7 @@ extern "C" {
#define ZSTD_VERSION_MAJOR 0 /* for breaking interface changes */
#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_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);
@ -70,7 +70,7 @@ ZSTD_compress() :
or an error code if it fails (which can be tested using ZSTD_isError())
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.
It must be equal or larger than originalSize, otherwise decompression will fail.
return : the number of bytes decompressed into destination buffer (originalSize)
@ -81,7 +81,7 @@ ZSTD_decompress() :
/**************************************
* 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 */
unsigned ZSTD_isError(size_t code); /* tells if a return value is an error code */

View File

@ -45,20 +45,21 @@ extern "C" {
/**************************************
* Streaming functions
**************************************/
typedef void* ZSTD_cctx_t;
ZSTD_cctx_t ZSTD_createCCtx(void);
size_t ZSTD_freeCCtx(ZSTD_cctx_t cctx);
typedef struct ZSTD_Cctx_s ZSTD_Cctx;
ZSTD_Cctx* ZSTD_createCCtx(void);
size_t ZSTD_freeCCtx(ZSTD_Cctx* cctx);
size_t ZSTD_compressBegin(ZSTD_cctx_t 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_compressEnd(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* cctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize);
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);
size_t ZSTD_decompressContinue(ZSTD_dctx_t dctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize);
typedef struct ZSTD_Dctx_s ZSTD_Dctx;
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.
ZSTD_nextSrcSizeToDecompress() tells how much bytes to provide as input to ZSTD_decompressContinue().