提交 7f264fdf 编写于 作者: A alesapin

Merge branch 'master' into fix_codebrowser

......@@ -2,10 +2,15 @@
#include <setjmp.h>
#include <unistd.h>
#ifdef __linux__
#include <sys/mman.h>
#endif
#include <new>
#include <iostream>
#include <vector>
#include <string>
#include <tuple>
#include <utility> /// 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<const char *, size_t> 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<unsigned char *>(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;
}
......
......@@ -36,7 +36,7 @@ namespace ErrorCodes
* uses asin, which slows down the algorithm a bit.
*/
template <typename T>
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<Centroid>) - sizeof(Count) - sizeof(UInt32);
using Summary = PODArrayWithStackMemory<Centroid, bytes_in_arena>;
using Centroids = PODArrayWithStackMemory<Centroid, bytes_in_arena>;
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<RadixSortTraits>::executeLSD(summary.data(), summary.size());
RadixSort<RadixSortTraits>::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<const char *>(summary.data()), summary.size() * sizeof(summary[0]));
writeVarUInt(centroids.size(), buf);
buf.write(reinterpret_cast<const char *>(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<char *>(summary.data()), size * sizeof(summary[0]));
centroids.resize(size);
buf.read(reinterpret_cast<char *>(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 <typename T>
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<T> main_tdigest;
TDigest<T> 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 <typename ResultType>
ResultType getImpl(Float64 level)
{
if (summary.empty())
mergeTDigests();
auto & centroids = main_tdigest.getCentroids();
if (centroids.empty())
return std::is_floating_point_v<ResultType> ? 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 <typename ResultType>
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<ResultType> ? 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;
}
......
......@@ -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) \
......
......@@ -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;
......
......@@ -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,
......
......@@ -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
{
......
......@@ -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;
......
......@@ -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<size_t>;
......
......@@ -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;
......
......@@ -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)
......
......@@ -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
┏━━━━━━━━┓
┃ 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
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;
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册