diff --git a/programs/main.cpp b/programs/main.cpp index b91bd732f216fc57b3dfe8c29529556114087843..fad2d35f3bd6448e3f9414c3628d048286e662bc 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -2,10 +2,15 @@ #include #include +#ifdef __linux__ +#include +#endif + #include #include #include #include +#include #include /// pair #if !defined(ARCADIA_BUILD) @@ -57,6 +62,7 @@ int mainEntryClickHouseStatus(int argc, char ** argv); int mainEntryClickHouseRestart(int argc, char ** argv); #endif +#define ARRAY_SIZE(a) (sizeof(a)/sizeof((a)[0])) namespace { @@ -150,28 +156,29 @@ enum class InstructionFail AVX512 = 8 }; -const char * instructionFailToString(InstructionFail fail) +std::pair instructionFailToString(InstructionFail fail) { switch (fail) { +#define ret(x) return std::make_pair(x, ARRAY_SIZE(x) - 1) case InstructionFail::NONE: - return "NONE"; + ret("NONE"); case InstructionFail::SSE3: - return "SSE3"; + ret("SSE3"); case InstructionFail::SSSE3: - return "SSSE3"; + ret("SSSE3"); case InstructionFail::SSE4_1: - return "SSE4.1"; + ret("SSE4.1"); case InstructionFail::SSE4_2: - return "SSE4.2"; + ret("SSE4.2"); case InstructionFail::POPCNT: - return "POPCNT"; + ret("POPCNT"); case InstructionFail::AVX: - return "AVX"; + ret("AVX"); case InstructionFail::AVX2: - return "AVX2"; + ret("AVX2"); case InstructionFail::AVX512: - return "AVX512"; + ret("AVX512"); } __builtin_unreachable(); } @@ -238,7 +245,7 @@ void checkRequiredInstructionsImpl(volatile InstructionFail & fail) } /// This function is safe to use in static initializers. -void writeError(const char * data, size_t size) +void writeErrorLen(const char * data, size_t size) { while (size != 0) { @@ -254,6 +261,12 @@ void writeError(const char * data, size_t size) } } } +/// Macros to avoid using strlen(), since it may fail if SSE is not supported. +#define writeError(data) do \ + { \ + static_assert(__builtin_constant_p(data)); \ + writeErrorLen(data, ARRAY_SIZE(data) - 1); \ + } while (false) /// Check SSE and others instructions availability. Calls exit on fail. /// This function must be called as early as possible, even before main, because static initializers may use unavailable instructions. @@ -272,8 +285,7 @@ void checkRequiredInstructions() /// Typical implementation of strlen is using SSE4.2 or AVX2. /// But this is not the case because it's compiler builtin and is executed at compile time. - const char * msg = "Can not set signal handler\n"; - writeError(msg, strlen(msg)); + writeError("Can not set signal handler\n"); _Exit(1); } @@ -281,12 +293,9 @@ void checkRequiredInstructions() if (sigsetjmp(jmpbuf, 1)) { - const char * msg1 = "Instruction check fail. The CPU does not support "; - writeError(msg1, strlen(msg1)); - const char * msg2 = instructionFailToString(fail); - writeError(msg2, strlen(msg2)); - const char * msg3 = " instruction set.\n"; - writeError(msg3, strlen(msg3)); + writeError("Instruction check fail. The CPU does not support "); + std::apply(writeErrorLen, instructionFailToString(fail)); + writeError(" instruction set.\n"); _Exit(1); } @@ -294,13 +303,60 @@ void checkRequiredInstructions() if (sigaction(signal, &sa_old, nullptr)) { - const char * msg = "Can not set signal handler\n"; - writeError(msg, strlen(msg)); + writeError("Can not set signal handler\n"); _Exit(1); } } -struct Checker { Checker() { checkRequiredInstructions(); } } checker; +#ifdef __linux__ +/// clickhouse uses jemalloc as a production allocator +/// and jemalloc relies on working MADV_DONTNEED, +/// which doesn't work under qemu +/// +/// but do this only under for linux, since only it return zeroed pages after MADV_DONTNEED +/// (and jemalloc assumes this too, see contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in) +void checkRequiredMadviseFlags() +{ + size_t size = 1 << 16; + void * addr = mmap(nullptr, size, PROT_READ|PROT_WRITE, MAP_PRIVATE|MAP_ANONYMOUS, -1, 0); + if (addr == MAP_FAILED) + { + writeError("Can not mmap pages for MADV_DONTNEED check\n"); + _Exit(1); + } + memset(addr, 'A', size); + + if (!madvise(addr, size, MADV_DONTNEED)) + { + /// Suboptimal, but should be simple. + for (size_t i = 0; i < size; ++i) + { + if (reinterpret_cast(addr)[i] != 0) + { + writeError("MADV_DONTNEED does not zeroed page. jemalloc will be broken\n"); + _Exit(1); + } + } + } + + if (munmap(addr, size)) + { + writeError("Can not munmap pages for MADV_DONTNEED check\n"); + _Exit(1); + } +} +#endif + +struct Checker +{ + Checker() + { + checkRequiredInstructions(); +#ifdef __linux__ + checkRequiredMadviseFlags(); +#endif + } +} checker; } diff --git a/src/AggregateFunctions/QuantileTDigest.h b/src/AggregateFunctions/QuantileTDigest.h index b90979c02b99d3400a5331df9d582403a3f8c10d..02d43ede66db6b2623ceb403130bf5c760fb2145 100644 --- a/src/AggregateFunctions/QuantileTDigest.h +++ b/src/AggregateFunctions/QuantileTDigest.h @@ -36,7 +36,7 @@ namespace ErrorCodes * uses asin, which slows down the algorithm a bit. */ template -class QuantileTDigest +class TDigest { using Value = Float32; using Count = Float32; @@ -86,20 +86,12 @@ class QuantileTDigest /// The memory will be allocated to several elements at once, so that the state occupies 64 bytes. static constexpr size_t bytes_in_arena = 128 - sizeof(PODArray) - sizeof(Count) - sizeof(UInt32); - using Summary = PODArrayWithStackMemory; + using Centroids = PODArrayWithStackMemory; - Summary summary; + Centroids centroids; Count count = 0; UInt32 unmerged = 0; - /** Linear interpolation at the point x on the line (x1, y1)..(x2, y2) - */ - static Value interpolate(Value x, Value x1, Value y1, Value x2, Value y2) - { - double k = (x - x1) / (x2 - x1); - return y1 + k * (y2 - y1); - } - struct RadixSortTraits { using Element = Centroid; @@ -122,13 +114,14 @@ class QuantileTDigest */ void addCentroid(const Centroid & c) { - summary.push_back(c); + centroids.push_back(c); count += c.count; ++unmerged; if (unmerged >= params.max_unmerged) compress(); } +public: /** Performs compression of accumulated centroids * When merging, the invariant is retained to the maximum size of each * centroid that does not exceed `4 q (1 - q) \ delta N`. @@ -137,16 +130,16 @@ class QuantileTDigest { if (unmerged > 0) { - RadixSort::executeLSD(summary.data(), summary.size()); + RadixSort::executeLSD(centroids.data(), centroids.size()); - if (summary.size() > 3) + if (centroids.size() > 3) { /// A pair of consecutive bars of the histogram. - auto l = summary.begin(); + auto l = centroids.begin(); auto r = std::next(l); Count sum = 0; - while (r != summary.end()) + while (r != centroids.end()) { // we use quantile which gives us the smallest error @@ -188,14 +181,13 @@ class QuantileTDigest } /// At the end of the loop, all values to the right of l were "eaten". - summary.resize(l - summary.begin() + 1); + centroids.resize(l - centroids.begin() + 1); } unmerged = 0; } } -public: /** Adds to the digest a change in `x` with a weight of `cnt` (default 1) */ void add(T x, UInt64 cnt = 1) @@ -203,17 +195,17 @@ public: addCentroid(Centroid(Value(x), Count(cnt))); } - void merge(const QuantileTDigest & other) + void merge(const TDigest & other) { - for (const auto & c : other.summary) + for (const auto & c : other.centroids) addCentroid(c); } void serialize(WriteBuffer & buf) { compress(); - writeVarUInt(summary.size(), buf); - buf.write(reinterpret_cast(summary.data()), summary.size() * sizeof(summary[0])); + writeVarUInt(centroids.size(), buf); + buf.write(reinterpret_cast(centroids.data()), centroids.size() * sizeof(centroids[0])); } void deserialize(ReadBuffer & buf) @@ -222,36 +214,113 @@ public: readVarUInt(size, buf); if (size > params.max_unmerged) - throw Exception("Too large t-digest summary size", ErrorCodes::TOO_LARGE_ARRAY_SIZE); + throw Exception("Too large t-digest centroids size", ErrorCodes::TOO_LARGE_ARRAY_SIZE); - summary.resize(size); - buf.read(reinterpret_cast(summary.data()), size * sizeof(summary[0])); + centroids.resize(size); + buf.read(reinterpret_cast(centroids.data()), size * sizeof(centroids[0])); count = 0; - for (const auto & c : summary) + for (const auto & c : centroids) count += c.count; } + Count getCount() + { + return count; + } + + const Centroids & getCentroids() const + { + return centroids; + } + + void reset() + { + centroids.resize(0); + count = 0; + unmerged = 0; + } +}; + +template +class QuantileTDigest +{ + using Value = Float32; + using Count = Float32; + + /** We store two t-digests. When an amount of elements in sub_tdigest become more than merge_threshold + * we merge sub_tdigest in main_tdigest and reset sub_tdigest. This method is needed to decrease an amount of + * centroids in t-digest (experiments show that after merge_threshold the size of t-digest significantly grows, + * but merging two big t-digest decreases it). + */ + TDigest main_tdigest; + TDigest sub_tdigest; + size_t merge_threshold = 1e7; + + /** Linear interpolation at the point x on the line (x1, y1)..(x2, y2) + */ + static Value interpolate(Value x, Value x1, Value y1, Value x2, Value y2) + { + double k = (x - x1) / (x2 - x1); + return y1 + k * (y2 - y1); + } + + void mergeTDigests() + { + main_tdigest.merge(sub_tdigest); + sub_tdigest.reset(); + } + +public: + void add(T x, UInt64 cnt = 1) + { + if (sub_tdigest.getCount() >= merge_threshold) + mergeTDigests(); + sub_tdigest.add(x, cnt); + } + + void merge(const QuantileTDigest & other) + { + mergeTDigests(); + main_tdigest.merge(other.main_tdigest); + main_tdigest.merge(other.sub_tdigest); + } + + void serialize(WriteBuffer & buf) + { + mergeTDigests(); + main_tdigest.serialize(buf); + } + + void deserialize(ReadBuffer & buf) + { + sub_tdigest.reset(); + main_tdigest.deserialize(buf); + } + /** Calculates the quantile q [0, 1] based on the digest. * For an empty digest returns NaN. */ template ResultType getImpl(Float64 level) { - if (summary.empty()) + mergeTDigests(); + + auto & centroids = main_tdigest.getCentroids(); + if (centroids.empty()) return std::is_floating_point_v ? NAN : 0; - compress(); + main_tdigest.compress(); - if (summary.size() == 1) - return summary.front().mean; + if (centroids.size() == 1) + return centroids.front().mean; - Float64 x = level * count; + Float64 x = level * main_tdigest.getCount(); Float64 prev_x = 0; Count sum = 0; - Value prev_mean = summary.front().mean; + Value prev_mean = centroids.front().mean; - for (const auto & c : summary) + for (const auto & c : centroids) { Float64 current_x = sum + c.count * 0.5; @@ -263,7 +332,7 @@ public: prev_x = current_x; } - return summary.back().mean; + return centroids.back().mean; } /** Get multiple quantiles (`size` parts). @@ -274,29 +343,32 @@ public: template void getManyImpl(const Float64 * levels, const size_t * levels_permutation, size_t size, ResultType * result) { - if (summary.empty()) + mergeTDigests(); + + auto & centroids = main_tdigest.getCentroids(); + if (centroids.empty()) { for (size_t result_num = 0; result_num < size; ++result_num) result[result_num] = std::is_floating_point_v ? NAN : 0; return; } - compress(); + main_tdigest.compress(); - if (summary.size() == 1) + if (centroids.size() == 1) { for (size_t result_num = 0; result_num < size; ++result_num) - result[result_num] = summary.front().mean; + result[result_num] = centroids.front().mean; return; } - Float64 x = levels[levels_permutation[0]] * count; + Float64 x = levels[levels_permutation[0]] * main_tdigest.getCount(); Float64 prev_x = 0; Count sum = 0; - Value prev_mean = summary.front().mean; + Value prev_mean = centroids.front().mean; size_t result_num = 0; - for (const auto & c : summary) + for (const auto & c : centroids) { Float64 current_x = sum + c.count * 0.5; @@ -308,7 +380,7 @@ public: if (result_num >= size) return; - x = levels[levels_permutation[result_num]] * count; + x = levels[levels_permutation[result_num]] * main_tdigest.getCount(); } sum += c.count; @@ -316,7 +388,7 @@ public: prev_x = current_x; } - auto rest_of_results = summary.back().mean; + auto rest_of_results = centroids.back().mean; for (; result_num < size; ++result_num) result[levels_permutation[result_num]] = rest_of_results; } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6eb853828c504c2a212296c77ab6099e1d2d5a7c..3ecb79c3fce9b9abc6183e93407cf163e5565991 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -469,6 +469,7 @@ class IColumn; M(Bool, output_format_enable_streaming, false, "Enable streaming in output formats that support it.", 0) \ M(Bool, output_format_write_statistics, true, "Write statistics about read rows, bytes, time elapsed in suitable output formats.", 0) \ M(Bool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \ + M(Bool, output_format_pretty_row_numbers, false, "Add row numbers before each row for pretty output format", 0) \ #define LIST_OF_SETTINGS(M) \ COMMON_SETTINGS(M) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 368b760044de4a8148674f2cc25b5521e1bb240c..065b14f86b7d26484b51949d8272def0961ecaa6 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -107,6 +107,7 @@ static FormatSettings getOutputFormatSetting(const Settings & settings, const Co format_settings.pretty.charset = settings.output_format_pretty_grid_charset.toString() == "ASCII" ? FormatSettings::Pretty::Charset::ASCII : FormatSettings::Pretty::Charset::UTF8; + format_settings.pretty.output_format_pretty_row_numbers = settings.output_format_pretty_row_numbers; format_settings.template_settings.resultset_format = settings.format_template_resultset; format_settings.template_settings.row_format = settings.format_template_row; format_settings.template_settings.row_between_delimiter = settings.format_template_rows_between_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index cd5cab8cf5aa518482c3b8274334a4c629261d9a..a97bd9bf6c62d5004bda2403853eaa0e477a3621 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -45,6 +45,8 @@ struct FormatSettings UInt64 max_value_width = 10000; bool color = true; + bool output_format_pretty_row_numbers = false; + enum class Charset { UTF8, diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h index 40498353195131dadaee71a4af503bfd78b52920..a5c88cad1d6f25bf13897066dc6dfe22c2a275ed 100644 --- a/src/Functions/FunctionsJSON.h +++ b/src/Functions/FunctionsJSON.h @@ -279,7 +279,7 @@ public: String getName() const override { return Name::name; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } - bool useDefaultImplementationForConstants() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 9a9c150fad40045ee329a702fd6be2c0d89a8a1f..b07cf83eb85e848288d03444e1c1a661fb9beb5d 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -33,6 +33,10 @@ void PrettyBlockOutputFormat::calculateWidths( WidthsPerColumn & widths, Widths & max_padded_widths, Widths & name_widths) { size_t num_rows = std::min(chunk.getNumRows(), format_settings.pretty.max_rows); + + /// len(num_rows) + len(". ") + row_number_width = std::floor(std::log10(num_rows)) + 3; + size_t num_columns = chunk.getNumColumns(); const auto & columns = chunk.getColumns(); @@ -196,9 +200,20 @@ void PrettyBlockOutputFormat::write(const Chunk & chunk, PortKind port_kind) std::string middle_values_separator_s = middle_values_separator.str(); std::string bottom_separator_s = bottom_separator.str(); + if (format_settings.pretty.output_format_pretty_row_numbers) + { + /// Write left blank + writeString(String(row_number_width, ' '), out); + } /// Output the block writeString(top_separator_s, out); + if (format_settings.pretty.output_format_pretty_row_numbers) + { + /// Write left blank + writeString(String(row_number_width, ' '), out); + } + /// Names writeCString(grid_symbols.bold_bar, out); writeCString(" ", out); @@ -238,12 +253,35 @@ void PrettyBlockOutputFormat::write(const Chunk & chunk, PortKind port_kind) writeCString(grid_symbols.bold_bar, out); writeCString("\n", out); + if (format_settings.pretty.output_format_pretty_row_numbers) + { + /// Write left blank + writeString(String(row_number_width, ' '), out); + } writeString(middle_names_separator_s, out); for (size_t i = 0; i < num_rows && total_rows + i < max_rows; ++i) { if (i != 0) + { + if (format_settings.pretty.output_format_pretty_row_numbers) + { + /// Write left blank + writeString(String(row_number_width, ' '), out); + } writeString(middle_values_separator_s, out); + } + + if (format_settings.pretty.output_format_pretty_row_numbers) + { + // Write row number; + auto row_num_string = std::to_string(i + 1) + ". "; + for (size_t j = 0; j < row_number_width - row_num_string.size(); ++j) + { + writeCString(" ", out); + } + writeString(row_num_string, out); + } writeCString(grid_symbols.bar, out); @@ -262,6 +300,11 @@ void PrettyBlockOutputFormat::write(const Chunk & chunk, PortKind port_kind) writeCString("\n", out); } + if (format_settings.pretty.output_format_pretty_row_numbers) + { + /// Write left blank + writeString(String(row_number_width, ' '), out); + } writeString(bottom_separator_s, out); total_rows += num_rows; diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h index 59628cf322c2f79050e2cdb1e56c2ea61aee2dac..de79fe5ee2ab2f9fce65b8c3bb43638a05380fd8 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h @@ -33,6 +33,8 @@ protected: size_t terminal_width = 0; bool suffix_written = false; + size_t row_number_width = 7; // "10000. " + const FormatSettings format_settings; using Widths = PODArray; diff --git a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp index 254bb9381f2c64bb7986bcdb033049b4eebacbac..e832dd833888c2cc9f013b1e0441287e741939aa 100644 --- a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp @@ -69,6 +69,12 @@ void PrettyCompactBlockOutputFormat::writeHeader( const Widths & max_widths, const Widths & name_widths) { + if (format_settings.pretty.output_format_pretty_row_numbers) + { + /// Write left blank + writeString(String(row_number_width, ' '), out); + } + const GridSymbols & grid_symbols = format_settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8 ? utf8_grid_symbols : ascii_grid_symbols; @@ -117,6 +123,12 @@ void PrettyCompactBlockOutputFormat::writeHeader( void PrettyCompactBlockOutputFormat::writeBottom(const Widths & max_widths) { + if (format_settings.pretty.output_format_pretty_row_numbers) + { + /// Write left blank + writeString(String(row_number_width, ' '), out); + } + const GridSymbols & grid_symbols = format_settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8 ? utf8_grid_symbols : ascii_grid_symbols; @@ -144,6 +156,17 @@ void PrettyCompactBlockOutputFormat::writeRow( const WidthsPerColumn & widths, const Widths & max_widths) { + if (format_settings.pretty.output_format_pretty_row_numbers) + { + // Write row number; + auto row_num_string = std::to_string(row_num + 1) + ". "; + for (size_t i = 0; i < row_number_width - row_num_string.size(); ++i) + { + writeCString(" ", out); + } + writeString(row_num_string, out); + } + const GridSymbols & grid_symbols = format_settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8 ? utf8_grid_symbols : ascii_grid_symbols; diff --git a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp index 6b42ea57e1f2d38eecbfb36883a690021d6fa7d7..f1a5cbac8e0372aa313dceb66c44ba16b981b446 100644 --- a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp @@ -29,6 +29,10 @@ void PrettySpaceBlockOutputFormat::write(const Chunk & chunk, PortKind port_kind Widths name_widths; calculateWidths(header, chunk, widths, max_widths, name_widths); + if (format_settings.pretty.output_format_pretty_row_numbers) + { + writeString(String(row_number_width, ' '), out); + } /// Names for (size_t i = 0; i < num_columns; ++i) { @@ -66,6 +70,16 @@ void PrettySpaceBlockOutputFormat::write(const Chunk & chunk, PortKind port_kind for (size_t row = 0; row < num_rows && total_rows + row < max_rows; ++row) { + if (format_settings.pretty.output_format_pretty_row_numbers) + { + // Write row number; + auto row_num_string = std::to_string(row + 1) + ". "; + for (size_t i = 0; i < row_number_width - row_num_string.size(); ++i) + { + writeCString(" ", out); + } + writeString(row_num_string, out); + } for (size_t column = 0; column < num_columns; ++column) { if (column != 0) diff --git a/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.reference b/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.reference index 03ed07cf1a48ee9b8325152638b3ba4684e512ab..bcc7aebeae865e5b14926ca75e68f8c025d7e445 100644 --- a/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.reference +++ b/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.reference @@ -2,4 +2,4 @@ Instruction check fail. The CPU does not support SSSE3 instruction set. Instruction check fail. The CPU does not support SSE4.1 instruction set. Instruction check fail. The CPU does not support SSE4.2 instruction set. Instruction check fail. The CPU does not support POPCNT instruction set. -1 +MADV_DONTNEED does not zeroed page. jemalloc will be broken diff --git a/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.reference b/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.reference new file mode 100644 index 0000000000000000000000000000000000000000..9010c371fdff182552f4d52a64217c90d5d0e311 --- /dev/null +++ b/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.reference @@ -0,0 +1,212 @@ +┏━━━━━━━━┓ +┃ number ┃ +┡━━━━━━━━┩ +│ 0 │ +├────────┤ +│ 1 │ +├────────┤ +│ 2 │ +├────────┤ +│ 3 │ +├────────┤ +│ 4 │ +├────────┤ +│ 5 │ +├────────┤ +│ 6 │ +├────────┤ +│ 7 │ +├────────┤ +│ 8 │ +├────────┤ +│ 9 │ +└────────┘ +┌─number─┐ +│ 0 │ +│ 1 │ +│ 2 │ +│ 3 │ +│ 4 │ +│ 5 │ +│ 6 │ +│ 7 │ +│ 8 │ +│ 9 │ +└────────┘ +┌─number─┐ +│ 0 │ +│ 1 │ +│ 2 │ +│ 3 │ +│ 4 │ +│ 5 │ +│ 6 │ +│ 7 │ +│ 8 │ +│ 9 │ +└────────┘ +┏━━━━━━━━┓ +┃ number ┃ +┡━━━━━━━━┩ +│ 0 │ +├────────┤ +│ 1 │ +├────────┤ +│ 2 │ +├────────┤ +│ 3 │ +├────────┤ +│ 4 │ +├────────┤ +│ 5 │ +├────────┤ +│ 6 │ +├────────┤ +│ 7 │ +├────────┤ +│ 8 │ +├────────┤ +│ 9 │ +└────────┘ +┌─number─┐ +│ 0 │ +│ 1 │ +│ 2 │ +│ 3 │ +│ 4 │ +│ 5 │ +│ 6 │ +│ 7 │ +│ 8 │ +│ 9 │ +└────────┘ + number + + 0 + 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + number + + 0 + 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + ┏━━━━━━━━┓ + ┃ number ┃ + ┡━━━━━━━━┩ + 1. │ 0 │ + ├────────┤ + 2. │ 1 │ + ├────────┤ + 3. │ 2 │ + ├────────┤ + 4. │ 3 │ + ├────────┤ + 5. │ 4 │ + ├────────┤ + 6. │ 5 │ + ├────────┤ + 7. │ 6 │ + ├────────┤ + 8. │ 7 │ + ├────────┤ + 9. │ 8 │ + ├────────┤ +10. │ 9 │ + └────────┘ + ┌─number─┐ + 1. │ 0 │ + 2. │ 1 │ + 3. │ 2 │ + 4. │ 3 │ + 5. │ 4 │ + 6. │ 5 │ + 7. │ 6 │ + 8. │ 7 │ + 9. │ 8 │ +10. │ 9 │ + └────────┘ + ┌─number─┐ + 1. │ 0 │ + 2. │ 1 │ + 3. │ 2 │ + 4. │ 3 │ + 5. │ 4 │ + 6. │ 5 │ + 7. │ 6 │ + 8. │ 7 │ + 9. │ 8 │ +10. │ 9 │ + └────────┘ + ┏━━━━━━━━┓ + ┃ number ┃ + ┡━━━━━━━━┩ + 1. │ 0 │ + ├────────┤ + 2. │ 1 │ + ├────────┤ + 3. │ 2 │ + ├────────┤ + 4. │ 3 │ + ├────────┤ + 5. │ 4 │ + ├────────┤ + 6. │ 5 │ + ├────────┤ + 7. │ 6 │ + ├────────┤ + 8. │ 7 │ + ├────────┤ + 9. │ 8 │ + ├────────┤ +10. │ 9 │ + └────────┘ + ┌─number─┐ + 1. │ 0 │ + 2. │ 1 │ + 3. │ 2 │ + 4. │ 3 │ + 5. │ 4 │ + 6. │ 5 │ + 7. │ 6 │ + 8. │ 7 │ + 9. │ 8 │ +10. │ 9 │ + └────────┘ + number + + 1. 0 + 2. 1 + 3. 2 + 4. 3 + 5. 4 + 6. 5 + 7. 6 + 8. 7 + 9. 8 +10. 9 + number + + 1. 0 + 2. 1 + 3. 2 + 4. 3 + 5. 4 + 6. 5 + 7. 6 + 8. 7 + 9. 8 +10. 9 diff --git a/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.sql b/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.sql new file mode 100644 index 0000000000000000000000000000000000000000..3c0eeaf0e536d27feef26a189f2d1856947a9a43 --- /dev/null +++ b/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.sql @@ -0,0 +1,15 @@ +SELECT * FROM numbers(10) FORMAT Pretty; +SELECT * FROM numbers(10) FORMAT PrettyCompact; +SELECT * FROM numbers(10) FORMAT PrettyCompactMonoBlock; +SELECT * FROM numbers(10) FORMAT PrettyNoEscapes; +SELECT * FROM numbers(10) FORMAT PrettyCompactNoEscapes; +SELECT * FROM numbers(10) FORMAT PrettySpaceNoEscapes; +SELECT * FROM numbers(10) FORMAT PrettySpace; +SET output_format_pretty_row_numbers=1; +SELECT * FROM numbers(10) FORMAT Pretty; +SELECT * FROM numbers(10) FORMAT PrettyCompact; +SELECT * FROM numbers(10) FORMAT PrettyCompactMonoBlock; +SELECT * FROM numbers(10) FORMAT PrettyNoEscapes; +SELECT * FROM numbers(10) FORMAT PrettyCompactNoEscapes; +SELECT * FROM numbers(10) FORMAT PrettySpaceNoEscapes; +SELECT * FROM numbers(10) FORMAT PrettySpace;