Remove log debug

This commit is contained in:
divanik 2024-03-18 16:03:12 +00:00
parent c5d35bb187
commit 265d91747f
5 changed files with 35 additions and 64 deletions

View File

@ -29,6 +29,15 @@ extern const int TOO_MANY_ROWS;
namespace
{
template <iota_supported_types T>
inline void iota_with_step_optimized(T * begin, size_t count, T first_value, T step)
{
if (step == 1)
iota(begin, count, first_value);
else
iota_with_step(begin, count, first_value, step);
}
class NumbersSource : public ISource
{
public:
@ -66,14 +75,8 @@ protected:
UInt64 * pos = vec.data(); /// This also accelerates the code.
UInt64 * current_end = &vec[real_block_size];
if (step == 1)
{
iota(pos, static_cast<size_t>(current_end - pos), curr);
}
else
{
iota_with_step(pos, static_cast<size_t>(current_end - pos), curr, step);
}
iota_with_step_optimized(pos, static_cast<size_t>(current_end - pos), curr, step);
next += chunk_step;
@ -101,8 +104,6 @@ using RangesWithStep = std::vector<RangeWithStep>;
std::optional<RangeWithStep> stepped_range_from_range(const Range & r, UInt64 step, UInt64 remainder)
{
// LOG_DEBUG(&Poco::Logger::get("Stepped from range"),
// "stepped from range");
if ((r.right.get<UInt64>() == 0) && (!r.right_included))
return std::nullopt;
UInt64 begin = (r.left.get<UInt64>() / step) * step;
@ -155,7 +156,7 @@ public:
using RangesStatePtr = std::shared_ptr<RangesState>;
[[maybe_unused]] NumbersRangedSource(
NumbersRangedSource(
const RangesWithStep & ranges_,
RangesStatePtr & ranges_state_,
UInt64 base_block_size_,
@ -167,17 +168,6 @@ public:
, base_block_size(base_block_size_)
, step(step_)
{
// for (const auto& range_with_step : ranges_) {
// // LOG_DEBUG(&Poco::Logger::get("Ranges With Step"),
// // "Ranges: {} {} {} {} {}",
// // range_with_step.range.left.get<UInt64>(),
// // range_with_step.range.right.get<UInt64>(),
// // range_with_step.range.left_included,
// // range_with_step.range.right_included,
// // range_with_step.step);
// // LOG_DEBUG(&Poco::Logger::get("Ranges With Step"),
// // "Step: {}", step);
// }
}
String getName() const override { return "NumbersRange"; }
@ -242,8 +232,6 @@ protected:
RangesPos start, end;
auto block_size = findRanges(start, end, base_block_size);
// LOG_DEBUG(&Poco::Logger::get("Found range"), "Evth: {} {} {} {} {} {}", start.offset_in_ranges, static_cast<UInt64>(start.offset_in_range), end.offset_in_ranges, static_cast<UInt64>(end.offset_in_range), base_block_size, block_size);
if (!block_size)
return {};
@ -259,11 +247,6 @@ protected:
while (block_size - provided != 0)
{
UInt64 need = block_size - provided;
// LOG_DEBUG(&Poco::Logger::get("Indices:"),
// "Indices: {} {}, provided: {}",
// ranges.size(),
// cursor.offset_in_ranges,
// provided);
auto & range = ranges[cursor.offset_in_ranges];
UInt128 can_provide = cursor.offset_in_ranges == end.offset_in_ranges
@ -286,14 +269,7 @@ protected:
auto start_value_64 = static_cast<UInt64>(start_value);
auto end_value_64 = static_cast<UInt64>(end_value);
auto size = (end_value_64 - start_value_64) / this->step;
if (step == 1)
{
iota(pos, static_cast<size_t>(size), start_value_64);
}
else
{
iota_with_step(pos, static_cast<size_t>(size), start_value_64, step);
}
iota_with_step_optimized(pos, static_cast<size_t>(size), start_value_64, step);
pos += size;
}
};
@ -302,14 +278,7 @@ protected:
{
UInt64 start_value = range.left + cursor.offset_in_range * step;
/// end_value will never overflow
if (step == 1)
{
iota(pos, static_cast<size_t>(need), start_value);
}
else
{
iota_with_step(pos, static_cast<size_t>(need), start_value, step);
}
iota_with_step_optimized(pos, static_cast<size_t>(need), start_value, step);
pos += need;
provided += need;
cursor.offset_in_range += need;
@ -467,9 +436,6 @@ Pipe ReadFromSystemNumbersStep::makePipe()
Pipe pipe;
Ranges ranges;
// LOG_DEBUG(&Poco::Logger::get("parameters"), "Parameters: {} {} {} {}", numbers_storage.step, numbers_storage.offset, numbers_storage.limit.has_value(), numbers_storage.limit.has_value() ? numbers_storage.limit.value() : UInt64{0});
if (numbers_storage.limit.has_value() && (numbers_storage.limit.value() == 0))
{
pipe.addSource(std::make_shared<NullSource>(NumbersSource::createHeader(numbers_storage.column_name)));

View File

@ -40,9 +40,7 @@ void StorageSystemNumbers::read(
size_t max_block_size,
size_t num_streams)
{
// LOG_DEBUG(&Poco::Logger::get("Reading from SystemNumbers"), "Limit : {}", limit.value());
query_plan.addStep(std::make_unique<ReadFromSystemNumbersStep>(
column_names, query_info, storage_snapshot, context, shared_from_this(), max_block_size, num_streams));
}
}

View File

@ -122,6 +122,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b
attachNoDescription<StorageSystemNumbers>(context, system_database, "numbers", "Generates all natural numbers, starting from 0 (to 2^64 - 1, and then again) in sorted order.", false, "number");
attachNoDescription<StorageSystemNumbers>(context, system_database, "numbers_mt", "Multithreaded version of `system.numbers`. Numbers order is not guaranteed.", true, "number");
attachNoDescription<StorageSystemNumbers>(context, system_database, "generate_series", "Generates arithmetic progression of natural numbers in sorted order in a given segment with a given step", false, "generate_series");
attachNoDescription<StorageSystemNumbers>(context, system_database, "generateSeries", "Generates arithmetic progression of natural numbers in sorted order in a given segment with a given step", false, "generate_series");
attachNoDescription<StorageSystemZeros>(context, system_database, "zeros", "Produces unlimited number of non-materialized zeros.", false);
attachNoDescription<StorageSystemZeros>(context, system_database, "zeros_mt", "Multithreaded version of system.zeros.", true);
attach<StorageSystemDatabases>(context, system_database, "databases", "Lists all databases of the current server.");

View File

@ -24,14 +24,14 @@ namespace ErrorCodes
namespace
{
/* numbers(limit), numbers_mt(limit)
* - the same as SELECT number FROM system.numbers LIMIT limit.
* Used for testing purposes, as a simple example of table function.
*/
constexpr std::array<const char *, 2> names = {"generate_series", "generateSeries"};
template <size_t alias_num>
class TableFunctionGenerateSeries : public ITableFunction
{
public:
static constexpr auto name = "generate_series";
static_assert(alias_num < names.size());
static constexpr auto name = names[alias_num];
std::string getName() const override { return name; }
bool hasStaticStructure() const override { return true; }
@ -49,13 +49,15 @@ private:
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
};
ColumnsDescription TableFunctionGenerateSeries::getActualTableStructure(ContextPtr /*context*/, bool /*is_insert_query*/) const
template <size_t alias_num>
ColumnsDescription TableFunctionGenerateSeries<alias_num>::getActualTableStructure(ContextPtr /*context*/, bool /*is_insert_query*/) const
{
/// NOTE: https://bugs.llvm.org/show_bug.cgi?id=47418
return ColumnsDescription{{{"generate_series", std::make_shared<DataTypeUInt64>()}}};
}
StoragePtr TableFunctionGenerateSeries::executeImpl(
template <size_t alias_num>
StoragePtr TableFunctionGenerateSeries<alias_num>::executeImpl(
const ASTPtr & ast_function,
ContextPtr context,
const std::string & table_name,
@ -86,7 +88,8 @@ StoragePtr TableFunctionGenerateSeries::executeImpl(
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' requires 'limit' or 'offset, limit'.", getName());
}
UInt64 TableFunctionGenerateSeries::evaluateArgument(ContextPtr context, ASTPtr & argument) const
template <size_t alias_num>
UInt64 TableFunctionGenerateSeries<alias_num>::evaluateArgument(ContextPtr context, ASTPtr & argument) const
{
const auto & [field, type] = evaluateConstantExpression(argument, context);
@ -108,8 +111,8 @@ UInt64 TableFunctionGenerateSeries::evaluateArgument(ContextPtr context, ASTPtr
void registerTableFunctionGenerateSeries(TableFunctionFactory & factory)
{
factory.registerFunction<TableFunctionGenerateSeries>({.documentation = {}, .allow_readonly = true});
// factory.registerFunction<TableFunctionGenerateSeries>({.documentation = {}, .allow_readonly = true});
factory.registerFunction<TableFunctionGenerateSeries<0>>({.documentation = {}, .allow_readonly = true});
factory.registerFunction<TableFunctionGenerateSeries<1>>({.documentation = {}, .allow_readonly = true});
}
}

View File

@ -1,3 +1,4 @@
#include <optional>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/Context.h>
#include <Interpreters/convertFieldToType.h>
@ -8,6 +9,7 @@
#include <TableFunctions/TableFunctionFactory.h>
#include <Common/FieldVisitorToString.h>
#include <Common/typeid_cast.h>
#include "base/types.h"
#include "registerTableFunctions.h"
@ -68,15 +70,16 @@ StoragePtr TableFunctionNumbers<multithreaded>::executeImpl(
{
auto arguments = function->arguments->children;
if (arguments.size() != 1 && arguments.size() != 2)
if ((arguments.empty()) || (arguments.size() >= 4))
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' requires 'length' or 'offset, length'.", getName());
UInt64 offset = arguments.size() == 2 ? evaluateArgument(context, arguments[0]) : 0;
UInt64 length = arguments.size() == 2 ? evaluateArgument(context, arguments[1]) : evaluateArgument(context, arguments[0]);
UInt64 offset = arguments.size() >= 2 ? evaluateArgument(context, arguments[0]) : 0;
UInt64 length = arguments.size() >= 2 ? evaluateArgument(context, arguments[1]) : evaluateArgument(context, arguments[0]);
UInt64 step = arguments.size() == 3 ? evaluateArgument(context, arguments[2]) : 1;
auto res = std::make_shared<StorageSystemNumbers>(
StorageID(getDatabaseName(), table_name), multithreaded, std::string{"number"}, length, offset);
StorageID(getDatabaseName(), table_name), multithreaded, std::string{"number"}, length, offset, step);
res->startup();
return res;
}