This commit is contained in:
Maria Khristenko 2024-09-18 23:12:01 -07:00 committed by GitHub
commit 7c7b8a72fb
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
39 changed files with 1695 additions and 825 deletions

View File

@ -340,7 +340,9 @@ try
processConfig();
adjustSettings();
initTTYBuffer(toProgressOption(config().getString("progress", "default")));
initTTYBuffer(toProgressOption(config().getString("progress", "default")),
toProgressOption(config().getString("progress-table", "default")));
initKeystrokeInterceptor();
ASTAlterCommand::setFormatAlterCommandsWithParentheses(true);
{

View File

@ -513,7 +513,9 @@ try
SCOPE_EXIT({ cleanup(); });
initTTYBuffer(toProgressOption(getClientConfiguration().getString("progress", "default")));
initTTYBuffer(toProgressOption(getClientConfiguration().getString("progress", "default")),
toProgressOption(config().getString("progress-table", "default")));
initKeystrokeInterceptor();
ASTAlterCommand::setFormatAlterCommandsWithParentheses(true);
/// try to load user defined executable functions, throw on error and die

View File

@ -172,7 +172,8 @@ void ClientApplicationBase::init(int argc, char ** argv)
("stage", po::value<std::string>()->default_value("complete"), "Request query processing up to specified stage: complete,fetch_columns,with_mergeable_state,with_mergeable_state_after_aggregation,with_mergeable_state_after_aggregation_and_limit")
("progress", po::value<ProgressOption>()->implicit_value(ProgressOption::TTY, "tty")->default_value(ProgressOption::DEFAULT, "default"), "Print progress of queries execution - to TTY: tty|on|1|true|yes; to STDERR non-interactive mode: err; OFF: off|0|false|no; DEFAULT - interactive to TTY, non-interactive is off")
("progress-table", po::value<ProgressOption>()->implicit_value(ProgressOption::TTY, "tty")->default_value(ProgressOption::DEFAULT, "default"), "Print progress table with main metrics of queries execution - to TTY: tty|on|1|true|yes; to STDERR non-interactive mode: err; OFF: off|0|false|no; DEFAULT - interactive to TTY, non-interactive is off")
("enable-progress-table-toggle", po::value<bool>()->default_value(true), "Enable toggling of the progress table by pressing the control key (Space). This is applicable in interactive mode.")
("disable_suggestion,A", "Disable loading suggestion data. Note that suggestion data is loaded asynchronously through a second connection to ClickHouse server. Also it is reasonable to disable suggestion if you want to paste a query with TAB characters. Shorthand option -A is for those who get used to mysql client.")
("wait_for_suggestions_to_load", "Load suggestion data synchonously.")
("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)")
@ -317,6 +318,26 @@ void ClientApplicationBase::init(int argc, char ** argv)
break;
}
}
if (options.count("progress-table"))
{
switch (options["progress-table"].as<ProgressOption>())
{
case DEFAULT:
config().setString("progress-table", "default");
break;
case OFF:
config().setString("progress-table", "off");
break;
case TTY:
config().setString("progress-table", "tty");
break;
case ERR:
config().setString("progress-table", "err");
break;
}
}
if (options.count("enable-progress-table-toggle"))
getClientConfiguration().setBool("enable-progress-table-toggle", options["enable-progress-table-toggle"].as<bool>());
if (options.count("echo"))
getClientConfiguration().setBool("echo", true);
if (options.count("disable_suggestion"))

View File

@ -15,6 +15,7 @@
#include <Common/Exception.h>
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <Common/typeid_cast.h>
#include <Common/KeystrokeInterceptor.h>
#include <Common/TerminalSize.h>
#include <Common/StringUtils.h>
#include <Common/filesystemHelpers.h>
@ -274,6 +275,7 @@ ClientBase::ClientBase(
: std_in(in_fd_)
, std_out(out_fd_)
, progress_indication(output_stream_, in_fd_, err_fd_)
, progress_table(output_stream_, in_fd_, err_fd_)
, in_fd(in_fd_)
, out_fd(out_fd_)
, err_fd(err_fd_)
@ -423,6 +425,8 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query)
/// If results are written INTO OUTFILE, we can avoid clearing progress to avoid flicker.
if (need_render_progress && tty_buf && (!select_into_file || select_into_file_and_stdout))
progress_indication.clearProgressOutput(*tty_buf);
if (need_render_progress_table && tty_buf && (!select_into_file || select_into_file_and_stdout))
progress_table.clearTableOutput(*tty_buf);
try
{
@ -438,13 +442,20 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query)
/// Received data block is immediately displayed to the user.
output_format->flush();
/// Restore progress bar after data block.
/// Restore progress bar and progress table after data block.
if (need_render_progress && tty_buf)
{
if (select_into_file && !select_into_file_and_stdout)
error_stream << "\r";
progress_indication.writeProgress(*tty_buf);
}
if (need_render_progress_table && tty_buf)
{
if (!need_render_progress && select_into_file && !select_into_file_and_stdout)
error_stream << "\r";
bool toggle_enabled = getClientConfiguration().getBool("enable-progress-table-toggle", true);
progress_table.writeTable(*tty_buf, show_progress_table.load(), toggle_enabled);
}
}
@ -453,6 +464,8 @@ void ClientBase::onLogData(Block & block)
initLogsOutputStream();
if (need_render_progress && tty_buf)
progress_indication.clearProgressOutput(*tty_buf);
if (need_render_progress_table && tty_buf)
progress_table.clearTableOutput(*tty_buf);
logs_out_stream->writeLogs(block);
logs_out_stream->flush();
}
@ -782,16 +795,23 @@ void ClientBase::setDefaultFormatsAndCompressionFromConfiguration()
}
}
void ClientBase::initTTYBuffer(ProgressOption progress)
void ClientBase::initTTYBuffer(ProgressOption progress_option, ProgressOption progress_table_option)
{
if (tty_buf)
return;
if (progress == ProgressOption::OFF || (!is_interactive && progress == ProgressOption::DEFAULT))
{
need_render_progress = false;
return;
}
if (progress_option == ProgressOption::OFF || (!is_interactive && progress_option == ProgressOption::DEFAULT))
need_render_progress = false;
if (progress_table_option == ProgressOption::OFF || (!is_interactive && progress_table_option == ProgressOption::DEFAULT))
need_render_progress_table = false;
if (!need_render_progress && !need_render_progress_table)
return;
/// If need_render_progress and need_render_progress_table are enabled,
/// use ProgressOption that was set for the progress bar for progress table as well.
ProgressOption progress = progress_option ? progress_option : progress_table_option;
static constexpr auto tty_file_name = "/dev/tty";
@ -837,7 +857,20 @@ void ClientBase::initTTYBuffer(ProgressOption progress)
tty_buf = std::make_unique<WriteBufferFromFileDescriptor>(STDERR_FILENO, buf_size);
}
else
{
need_render_progress = false;
need_render_progress_table = false;
}
}
void ClientBase::initKeystrokeInterceptor()
{
if (is_interactive && need_render_progress_table && getClientConfiguration().getBool("enable-progress-table-toggle", true))
{
keystroke_interceptor = std::make_unique<KeystrokeInterceptor>(in_fd);
keystroke_interceptor->registerCallback(' ', [this]() { show_progress_table = !show_progress_table; });
}
}
void ClientBase::updateSuggest(const ASTPtr & ast)
@ -1101,6 +1134,9 @@ void ClientBase::receiveResult(ASTPtr parsed_query, Int32 signals_before_stop, b
std::exception_ptr local_format_error;
if (keystroke_interceptor)
keystroke_interceptor->startIntercept();
while (true)
{
Stopwatch receive_watch(CLOCK_MONOTONIC_COARSE);
@ -1155,8 +1191,17 @@ void ClientBase::receiveResult(ASTPtr parsed_query, Int32 signals_before_stop, b
local_format_error = std::current_exception();
connection->sendCancel();
}
catch (...)
{
if (keystroke_interceptor)
keystroke_interceptor->stopIntercept();
throw;
}
}
if (keystroke_interceptor)
keystroke_interceptor->stopIntercept();
if (local_format_error)
std::rethrow_exception(local_format_error);
@ -1252,6 +1297,8 @@ void ClientBase::onEndOfStream()
{
if (need_render_progress && tty_buf)
progress_indication.clearProgressOutput(*tty_buf);
if (need_render_progress_table && tty_buf)
progress_table.clearTableOutput(*tty_buf);
if (output_format)
{
@ -1330,9 +1377,15 @@ void ClientBase::onProfileEvents(Block & block)
thread_times[host_name].peak_memory_usage = value;
}
progress_indication.updateThreadEventData(thread_times);
progress_table.updateTable(block);
if (need_render_progress && tty_buf)
progress_indication.writeProgress(*tty_buf);
if (need_render_progress_table && tty_buf)
{
bool toggle_enabled = getClientConfiguration().getBool("enable-progress-table-toggle", true);
progress_table.writeTable(*tty_buf, show_progress_table.load(), toggle_enabled);
}
if (profile_events.print)
{
@ -1343,6 +1396,8 @@ void ClientBase::onProfileEvents(Block & block)
initLogsOutputStream();
if (need_render_progress && tty_buf)
progress_indication.clearProgressOutput(*tty_buf);
if (need_render_progress_table && tty_buf)
progress_table.clearTableOutput(*tty_buf);
logs_out_stream->writeProfileEvents(block);
logs_out_stream->flush();
@ -1824,6 +1879,8 @@ void ClientBase::cancelQuery()
connection->sendCancel();
if (need_render_progress && tty_buf)
progress_indication.clearProgressOutput(*tty_buf);
if (need_render_progress_table && tty_buf)
progress_table.clearTableOutput(*tty_buf);
if (is_interactive)
output_stream << "Cancelling query." << std::endl;
@ -1890,6 +1947,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
processed_rows = 0;
written_first_block = false;
progress_indication.resetProgress();
progress_table.resetTable();
profile_events.watch.restart();
{
@ -2016,6 +2074,8 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
initLogsOutputStream();
if (need_render_progress && tty_buf)
progress_indication.clearProgressOutput(*tty_buf);
if (need_render_progress_table && tty_buf)
progress_table.clearTableOutput(*tty_buf);
logs_out_stream->writeProfileEvents(profile_events.last_block);
logs_out_stream->flush();
@ -2029,6 +2089,8 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
output_stream << processed_rows << " row" << (processed_rows == 1 ? "" : "s") << " in set. ";
output_stream << "Elapsed: " << progress_indication.elapsedSeconds() << " sec. ";
progress_indication.writeFinalProgress();
if (need_render_progress_table && show_progress_table)
progress_table.writeFinalTable();
output_stream << std::endl << std::endl;
}
else

View File

@ -6,6 +6,7 @@
#include <Common/DNSResolver.h>
#include <Common/InterruptListener.h>
#include <Common/ProgressIndication.h>
#include <Common/ProgressTable.h>
#include <Common/ShellCommand.h>
#include <Common/Stopwatch.h>
#include <Core/ExternalTable.h>
@ -68,6 +69,7 @@ ProgressOption toProgressOption(std::string progress);
std::istream& operator>> (std::istream & in, ProgressOption & progress);
class InternalTextLogs;
class KeystrokeInterceptor;
class WriteBufferFromFileDescriptor;
/**
@ -245,7 +247,8 @@ protected:
void setDefaultFormatsAndCompressionFromConfiguration();
void initTTYBuffer(ProgressOption progress);
void initTTYBuffer(ProgressOption progress_option, ProgressOption progress_table_option);
void initKeystrokeInterceptor();
/// Should be one of the first, to be destroyed the last,
/// since other members can use them.
@ -255,6 +258,8 @@ protected:
/// Client context is a context used only by the client to parse queries, process query parameters and to connect to clickhouse-server.
ContextMutablePtr client_context;
std::unique_ptr<KeystrokeInterceptor> keystroke_interceptor;
bool is_interactive = false; /// Use either interactive line editing interface or batch mode.
bool delayed_interactive = false;
@ -332,7 +337,10 @@ protected:
String server_display_name;
ProgressIndication progress_indication;
ProgressTable progress_table;
bool need_render_progress = true;
bool need_render_progress_table = true;
std::atomic_bool show_progress_table = false;
bool need_render_profile_events = true;
bool written_first_block = false;
size_t processed_rows = 0; /// How many rows have been read or written.

View File

@ -0,0 +1,94 @@
#include <chrono>
#include <memory>
#include <Common/KeystrokeInterceptor.h>
#include <termios.h>
#include <unistd.h>
#include <base/defines.h>
namespace DB
{
KeystrokeInterceptor::KeystrokeInterceptor(int fd_) : fd(fd_)
{
}
KeystrokeInterceptor::~KeystrokeInterceptor()
{
stopIntercept();
}
void KeystrokeInterceptor::registerCallback(char key, KeystrokeInterceptor::Callback cb)
{
callbacks.emplace(key, cb);
}
void KeystrokeInterceptor::startIntercept()
{
std::lock_guard<std::mutex> lock(mutex);
if (intercept_thread && intercept_thread->joinable())
return;
chassert(!orig_termios);
stop_requested = false;
/// Save terminal state.
orig_termios = std::make_unique<struct termios>();
tcgetattr(fd, orig_termios.get());
/// Enable raw terminal mode.
struct termios raw = *orig_termios;
raw.c_lflag &= ~(ECHO | ICANON);
raw.c_cc[VMIN] = 0;
raw.c_cc[VTIME] = 1;
tcsetattr(fd, TCSAFLUSH, &raw);
intercept_thread = std::make_unique<std::thread>(&KeystrokeInterceptor::run, this, callbacks);
}
void KeystrokeInterceptor::stopIntercept()
{
stop_requested = true;
std::lock_guard<std::mutex> lock(mutex);
if (intercept_thread && intercept_thread->joinable())
{
intercept_thread->join();
intercept_thread.reset();
}
/// Reset original terminal mode.
if (orig_termios)
{
tcsetattr(fd, TCSAFLUSH, orig_termios.get());
orig_termios.reset();
}
}
void KeystrokeInterceptor::run(KeystrokeInterceptor::CallbackMap map)
{
while (!stop_requested)
{
runImpl(map);
std::this_thread::sleep_for(std::chrono::milliseconds(200));
}
}
void KeystrokeInterceptor::runImpl(const DB::KeystrokeInterceptor::CallbackMap & map) const
{
char ch;
if (read(fd, &ch, 1) > 0)
{
auto it = map.find(ch);
if (it != map.end())
{
auto fn = it->second;
fn();
}
}
}
}

View File

@ -0,0 +1,42 @@
#pragma once
#include <functional>
#include <memory>
#include <mutex>
#include <thread>
#include <unordered_map>
struct termios;
namespace DB
{
class KeystrokeInterceptor
{
using Callback = std::function<void()>;
using CallbackMap = std::unordered_map<char, Callback>;
public:
explicit KeystrokeInterceptor(int fd_);
~KeystrokeInterceptor();
void registerCallback(char key, Callback cb);
void startIntercept();
void stopIntercept();
private:
void run(CallbackMap);
void runImpl(const CallbackMap &) const;
const int fd;
std::mutex mutex;
CallbackMap callbacks;
std::unique_ptr<std::thread> intercept_thread;
std::unique_ptr<struct termios> orig_termios;
std::atomic_bool stop_requested = false;
};
}

File diff suppressed because it is too large Load Diff

View File

@ -149,6 +149,15 @@ namespace ProfileEvents
static const Event num_counters;
};
enum class ValueType : uint8_t
{
Number,
Bytes,
Milliseconds,
Microseconds,
Nanoseconds,
};
/// Increment a counter for event. Thread-safe.
void increment(Event event, Count amount = 1);
@ -165,6 +174,9 @@ namespace ProfileEvents
/// Get description of event by identifier. Returns statically allocated string.
const char * getDocumentation(Event event);
/// Get value type of event by identifier. Returns enum value.
ValueType getValueType(Event event);
/// Get index just after last event identifier.
Event end();

View File

@ -0,0 +1,474 @@
#include "ProgressTable.h"
#include "Common/AllocatorWithMemoryTracking.h"
#include "Common/ProfileEvents.h"
#include "base/defines.h"
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <Core/Block.h>
#include <IO/Operators.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <Interpreters/ProfileEventsExt.h>
#include <base/terminalColors.h>
#include <Common/TerminalSize.h>
#include <Common/formatReadable.h>
#include <format>
#include <numeric>
#include <unordered_map>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace
{
constexpr UInt64 THREAD_GROUP_ID = 0;
constexpr std::string_view CLEAR_TO_END_OF_LINE = "\033[K";
constexpr std::string_view CLEAR_TO_END_OF_SCREEN = "\033[0J";
constexpr std::string_view RESET_COLOR = "\033[0m";
constexpr std::string_view HIDE_CURSOR = "\033[?25l";
constexpr std::string_view SHOW_CURSOR = "\033[?25h";
std::string moveUpNLines(size_t N)
{
return std::format("\033[{}A", N);
}
std::string formatReadableValue(ProfileEvents::ValueType value_type, double value)
{
switch (value_type)
{
case ProfileEvents::ValueType::Number:
return formatReadableQuantity(value, /*precision*/ std::floor(value) == value && fabs(value) < 1000 ? 0 : 2);
case ProfileEvents::ValueType::Bytes:
return formatReadableSizeWithDecimalSuffix(value);
case ProfileEvents::ValueType::Nanoseconds:
return formatReadableTime(value);
case ProfileEvents::ValueType::Microseconds:
return formatReadableTime(value * 1e3);
case ProfileEvents::ValueType::Milliseconds:
return formatReadableTime(value * 1e6);
}
}
const std::unordered_map<std::string_view, ProfileEvents::Event> & getEventNameToEvent()
{
/// TODO: MemoryTracker::USAGE_EVENT_NAME and PEAK_USAGE_EVENT_NAME
static std::unordered_map<std::string_view, ProfileEvents::Event> event_name_to_event;
if (!event_name_to_event.empty())
return event_name_to_event;
for (ProfileEvents::Event event = ProfileEvents::Event(0); event < ProfileEvents::end(); ++event)
{
event_name_to_event.emplace(ProfileEvents::getName(event), event);
}
return event_name_to_event;
}
std::string_view setColorForProgress(double progress, double max_progress)
{
constexpr std::array<std::string_view, 5> colors = {
"\033[38;5;236m", /// Dark Grey
"\033[38;5;250m", /// Light Grey
"\033[38;5;34m", /// Green
"\033[38;5;226m", /// Yellow
"\033[1;33m", /// Bold
};
constexpr std::array<double, 4> fractions = {
0.05,
0.20,
0.80,
0.95,
};
if (max_progress == 0)
return colors.front();
auto fraction = progress / max_progress;
auto dist = std::upper_bound(fractions.begin(), fractions.end(), fraction) - fractions.begin();
return colors[dist];
}
std::string_view setColorForBytesBasedMetricsProgress(double progress)
{
constexpr std::array<std::string_view, 7> colors = {
"\033[38;5;236m", /// Dark Grey
"\033[38;5;250m", /// Light Grey
"\033[38;5;34m", /// Green
"\033[38;5;226m", /// Yellow
"\033[38;5;208m", /// Orange
"\033[1;33m", /// Bold
"\033[38;5;160m", /// Red: corresponds to >= 1T/s. Not realistic, unless there is a bug.
};
/// Bytes.
constexpr std::array<uint64_t, 6> thresholds = {
1LL << 20,
100LL << 20,
1'000LL << 20,
10'000LL << 20,
100'000LL << 20,
1'000'000LL << 20,
};
auto dist = std::upper_bound(thresholds.begin(), thresholds.end(), progress) - thresholds.begin();
return colors[dist];
}
std::string_view setColorForTimeBasedMetricsProgress(ProfileEvents::ValueType value_type, double progress)
{
/// Time units in a second.
auto units = [](ProfileEvents::ValueType t) -> double
{
switch (t)
{
case ProfileEvents::ValueType::Milliseconds:
return 1e3;
case ProfileEvents::ValueType::Microseconds:
return 1e6;
case ProfileEvents::ValueType::Nanoseconds:
return 1e9;
default:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong value type, expecting time units");
}
}(value_type);
constexpr std::array<std::string_view, 5> colors = {
"\033[38;5;236m", /// Dark Grey
"\033[38;5;250m", /// Light Grey
"\033[38;5;34m", /// Green
"\033[38;5;226m", /// Yellow
"\033[1;33m" /// Bold
};
const std::array<double, 4> thresholds = {0.001 * units, 0.01 * units, 0.1 * units, 1.0 * units};
auto dist = std::upper_bound(thresholds.begin(), thresholds.end(), progress) - thresholds.begin();
return colors[dist];
}
std::string_view setColorForStaleMetrics()
{
return "\033[38;5;236m"; /// Dark Grey
}
std::string_view setColorForDocumentation()
{
return "\033[38;5;236m"; /// Dark Grey
}
template <typename Out>
void writeWithWidth(Out & out, std::string_view s, size_t width)
{
if (s.size() >= width)
out << s << " ";
else
out << s << std::string(width - s.size(), ' ');
}
template <typename Out>
void writeWithWidthStrict(Out & out, std::string_view s, size_t width)
{
chassert(width != 0);
if (s.size() > width)
out << s.substr(0, width - 1) << "";
else
out << s;
}
}
void ProgressTable::writeTable(WriteBufferFromFileDescriptor & message, bool show_table, bool toggle_enabled)
{
std::lock_guard lock{mutex};
if (!show_table)
{
if (written_first_block)
message << CLEAR_TO_END_OF_SCREEN;
if (toggle_enabled)
{
message << HIDE_CURSOR;
message << "\n";
message << "Press the space key to toggle the display of the progress table.";
message << moveUpNLines(1);
message.next();
}
return;
}
const auto & event_name_to_event = getEventNameToEvent();
size_t terminal_width = getTerminalWidth(in_fd, err_fd);
if (terminal_width < column_event_name_width + COLUMN_VALUE_WIDTH + COLUMN_PROGRESS_WIDTH)
return;
if (metrics.empty())
return;
message << HIDE_CURSOR;
message << "\n";
writeWithWidth(message, COLUMN_EVENT_NAME, column_event_name_width);
writeWithWidth(message, COLUMN_VALUE, COLUMN_VALUE_WIDTH);
writeWithWidth(message, COLUMN_PROGRESS, COLUMN_PROGRESS_WIDTH);
writeWithWidth(message, COLUMN_DOCUMENTATION_NAME, COLUMN_DOCUMENTATION_WIDTH);
message << CLEAR_TO_END_OF_LINE;
double elapsed_sec = watch.elapsedSeconds();
for (auto & [name, per_host_info] : metrics)
{
message << "\n";
if (per_host_info.isStale(elapsed_sec))
message << setColorForStaleMetrics();
writeWithWidth(message, name, column_event_name_width);
auto value = per_host_info.getSummaryValue();
auto value_type = getValueType(event_name_to_event.at(name));
writeWithWidth(message, formatReadableValue(value_type, value), COLUMN_VALUE_WIDTH);
/// Get the maximum progress before it is updated in getSummaryProgress.
auto max_progress = per_host_info.getMaxProgress();
auto progress = per_host_info.getSummaryProgress(elapsed_sec);
switch (value_type)
{
case ProfileEvents::ValueType::Number:
message << setColorForProgress(progress, max_progress);
break;
case ProfileEvents::ValueType::Bytes:
message << setColorForBytesBasedMetricsProgress(progress);
break;
case ProfileEvents::ValueType::Milliseconds:
[[fallthrough]];
case ProfileEvents::ValueType::Microseconds:
[[fallthrough]];
case ProfileEvents::ValueType::Nanoseconds:
message << setColorForTimeBasedMetricsProgress(value_type, progress);
break;
}
writeWithWidth(message, formatReadableValue(value_type, progress) + "/s", COLUMN_PROGRESS_WIDTH);
message << setColorForDocumentation();
const auto * doc = getDocumentation(event_name_to_event.at(name));
writeWithWidthStrict(message, doc, COLUMN_DOCUMENTATION_WIDTH);
message << RESET_COLOR;
message << CLEAR_TO_END_OF_LINE;
}
message << moveUpNLines(tableSize());
message.next();
}
void ProgressTable::writeFinalTable()
{
std::lock_guard lock{mutex};
const auto & event_name_to_event = getEventNameToEvent();
size_t terminal_width = getTerminalWidth(in_fd, err_fd);
if (terminal_width < column_event_name_width + COLUMN_VALUE_WIDTH)
return;
if (metrics.empty())
return;
output_stream << "\n";
writeWithWidth(output_stream, COLUMN_EVENT_NAME, column_event_name_width);
writeWithWidth(output_stream, COLUMN_VALUE, COLUMN_VALUE_WIDTH);
for (auto & [name, per_host_info] : metrics)
{
output_stream << "\n";
writeWithWidth(output_stream, name, column_event_name_width);
auto value = per_host_info.getSummaryValue();
auto value_type = getValueType(event_name_to_event.at(name));
writeWithWidth(output_stream, formatReadableValue(value_type, value), COLUMN_VALUE_WIDTH);
}
}
void ProgressTable::updateTable(const Block & block)
{
const auto & array_thread_id = typeid_cast<const ColumnUInt64 &>(*block.getByName("thread_id").column).getData();
const auto & names = typeid_cast<const ColumnString &>(*block.getByName("name").column);
const auto & host_names = typeid_cast<const ColumnString &>(*block.getByName("host_name").column);
const auto & array_values = typeid_cast<const ColumnInt64 &>(*block.getByName("value").column).getData();
const auto & array_type = typeid_cast<const ColumnInt8 &>(*block.getByName("type").column).getData();
const double time_now = watch.elapsedSeconds();
size_t max_event_name_width = COLUMN_EVENT_NAME.size();
std::lock_guard lock{mutex};
const auto & event_name_to_event = getEventNameToEvent();
for (size_t row_num = 0, rows = block.rows(); row_num < rows; ++row_num)
{
auto thread_id = array_thread_id[row_num];
/// In ProfileEvents packets thread id 0 specifies common profiling information
/// for all threads executing current query on specific host. So instead of summing per thread
/// consumption it's enough to look for data with thread id 0.
if (thread_id != THREAD_GROUP_ID)
continue;
auto value = array_values[row_num];
auto name = names.getDataAt(row_num).toString();
auto host_name = host_names.getDataAt(row_num).toString();
auto type = static_cast<ProfileEvents::Type>(array_type[row_num]);
/// Got unexpected event name.
if (!event_name_to_event.contains(name))
continue;
/// Store non-zero values.
if (value == 0)
continue;
auto it = metrics.find(name);
/// If the table has already been written, then do not add new metrics to avoid jitter.
if (it == metrics.end() && written_first_block)
continue;
if (!written_first_block)
it = metrics.try_emplace(name).first;
it->second.updateHostValue(host_name, type, value, time_now);
max_event_name_width = std::max(max_event_name_width, name.size());
}
if (!written_first_block)
column_event_name_width = max_event_name_width + 1;
written_first_block = true;
}
void ProgressTable::clearTableOutput(WriteBufferFromFileDescriptor & message)
{
message << CLEAR_TO_END_OF_SCREEN;
message << SHOW_CURSOR;
message.next();
}
void ProgressTable::resetTable()
{
std::lock_guard lock{mutex};
watch.restart();
metrics.clear();
written_first_block = false;
}
size_t ProgressTable::tableSize() const
{
/// Number of lines + header.
return metrics.empty() ? 0 : metrics.size() + 1;
}
ProgressTable::MetricInfo::MetricInfo(ProfileEvents::Type t) : type(t)
{
}
void ProgressTable::MetricInfo::updateValue(Int64 new_value, double new_time)
{
/// If the value has not been updated for a long time
/// reset the time in snapshots to a second ago.
if (new_time - new_snapshot.time >= 0.5 || new_snapshot.time == 0)
{
prev_shapshot = {new_snapshot.value, new_time - 1.0};
cur_shapshot = {new_snapshot.value, new_time - 1.0};
}
switch (type)
{
case ProfileEvents::Type::INCREMENT:
new_snapshot.value = new_snapshot.value + new_value;
break;
case ProfileEvents::Type::GAUGE:
new_snapshot.value = new_value;
break;
}
new_snapshot.time = new_time;
if (new_snapshot.time - cur_shapshot.time >= 0.5)
prev_shapshot = std::exchange(cur_shapshot, new_snapshot);
update_time = new_time;
}
bool ProgressTable::MetricInfo::isStale(double now) const
{
return update_time != 0 && now - update_time >= 5.0;
}
double ProgressTable::MetricInfo::calculateProgress(double time_now) const
{
/// If the value has not been updated for a long time, the progress is 0.
if (time_now - new_snapshot.time >= 0.5)
return 0;
return (cur_shapshot.value - prev_shapshot.value) / (cur_shapshot.time - prev_shapshot.time);
}
double ProgressTable::MetricInfo::getValue() const
{
return new_snapshot.value;
}
void ProgressTable::MetricInfoPerHost::updateHostValue(const HostName & host, ProfileEvents::Type type, Int64 new_value, double new_time)
{
auto it = host_to_metric.find(host);
if (it == host_to_metric.end())
it = host_to_metric.emplace(host, type).first;
it->second.updateValue(new_value, new_time);
}
double ProgressTable::MetricInfoPerHost::getSummaryValue()
{
return std::accumulate(
host_to_metric.cbegin(),
host_to_metric.cend(),
0.0,
[](double acc, const auto & host_data)
{
const MetricInfo & info = host_data.second;
return acc + info.getValue();
});
}
double ProgressTable::MetricInfoPerHost::getSummaryProgress(double time_now)
{
auto progress = std::accumulate(
host_to_metric.cbegin(),
host_to_metric.cend(),
0.0,
[time_now](double acc, const auto & host_data)
{
const MetricInfo & info = host_data.second;
return acc + info.calculateProgress(time_now);
});
max_progress = std::max(max_progress, progress);
return progress;
}
double ProgressTable::MetricInfoPerHost::getMaxProgress() const
{
return max_progress;
}
bool ProgressTable::MetricInfoPerHost::isStale(double now) const
{
return std::all_of(host_to_metric.cbegin(), host_to_metric.cend(), [&now](const auto & p) { return p.second.isStale(now); });
}
}

119
src/Common/ProgressTable.h Normal file
View File

@ -0,0 +1,119 @@
#pragma once
#include <Interpreters/ProfileEventsExt.h>
#include <base/types.h>
#include <Common/ProfileEvents.h>
#include <Common/Stopwatch.h>
#include <map>
#include <mutex>
#include <ostream>
#include <string_view>
#include <unistd.h>
#include <unordered_map>
namespace DB
{
class WriteBufferFromFileDescriptor;
class Block;
class ProgressTable
{
public:
explicit ProgressTable(std::ostream & output_stream_, int in_fd_ = STDIN_FILENO, int err_fd_ = STDERR_FILENO)
: output_stream(output_stream_), in_fd(in_fd_), err_fd(err_fd_)
{}
/// Write progress table with metrics.
void writeTable(WriteBufferFromFileDescriptor & message, bool show_table, bool toggle_enabled);
void clearTableOutput(WriteBufferFromFileDescriptor & message);
void writeFinalTable();
/// Update the metric values. They can be updated from:
/// onProfileEvents in clickhouse-client;
void updateTable(const Block & block);
/// Reset progress table values.
void resetTable();
private:
class MetricInfo
{
public:
explicit MetricInfo(ProfileEvents::Type t);
void updateValue(Int64 new_value, double new_time);
double calculateProgress(double time_now) const;
double getValue() const;
bool isStale(double now) const;
private:
const ProfileEvents::Type type;
struct Snapshot
{
Int64 value = 0;
double time = 0;
};
/// The previous and current snapshots are used to calculateProgress.
/// They contain outdated by about a second information.
/// The new snapshot is used to updateValue and getValue.
/// If you use a new snapshot to calculate progress, then the time difference between
/// the previous update will be very small, so progress will jitter.
Snapshot prev_shapshot;
Snapshot cur_shapshot;
Snapshot new_snapshot;
double update_time = 0.0;
};
class MetricInfoPerHost
{
public:
using HostName = String;
void updateHostValue(const HostName & host, ProfileEvents::Type type, Int64 new_value, double new_time);
double getSummaryValue();
double getSummaryProgress(double time_now);
double getMaxProgress() const;
bool isStale(double now) const;
private:
std::unordered_map<HostName, MetricInfo> host_to_metric;
double max_progress = 0;
};
size_t tableSize() const;
using MetricName = String;
/// The server periodically sends Block with profile events.
/// This information is stored here.
std::map<MetricName, MetricInfoPerHost> metrics;
/// It is possible concurrent access to the metrics.
std::mutex mutex;
/// Track query execution time on client.
Stopwatch watch;
bool written_first_block = false;
size_t column_event_name_width = 20;
static constexpr std::string_view COLUMN_EVENT_NAME = "Event name";
static constexpr std::string_view COLUMN_VALUE = "Value";
static constexpr std::string_view COLUMN_PROGRESS = "Progress";
static constexpr std::string_view COLUMN_DOCUMENTATION_NAME = "Documentation";
static constexpr size_t COLUMN_VALUE_WIDTH = 20;
static constexpr size_t COLUMN_PROGRESS_WIDTH = 20;
static constexpr size_t COLUMN_DOCUMENTATION_WIDTH = 100;
std::ostream & output_stream;
int in_fd;
int err_fd;
};
}

View File

@ -82,3 +82,16 @@ std::string formatReadableQuantity(double value, int precision)
formatReadableQuantity(value, out, precision);
return out.str();
}
void formatReadableTime(double ns, DB::WriteBuffer & out, int precision)
{
const char * units[] = {" ns", " us", " ms", " s"};
formatReadable(ns, out, precision, units, sizeof(units) / sizeof(units[0]), 1000);
}
std::string formatReadableTime(double ns, int precision)
{
DB::WriteBufferFromOwnString out;
formatReadableTime(ns, out, precision);
return out.str();
}

View File

@ -23,6 +23,9 @@ std::string formatReadableSizeWithDecimalSuffix(double value, int precision = 2)
void formatReadableQuantity(double value, DB::WriteBuffer & out, int precision = 2);
std::string formatReadableQuantity(double value, int precision = 2);
/// Prints the passed time in nanoseconds as 123.45 ms.
void formatReadableTime(double ns, DB::WriteBuffer & out, int precision = 2);
std::string formatReadableTime(double ns, int precision = 2);
/// Wrapper around value. If used with fmt library (e.g. for log messages),
/// value is automatically formatted as size with binary suffix.

View File

@ -141,7 +141,8 @@ def call_with_retry(query: str, timeout: int = 30, retry_count: int = 5) -> None
def make_query_command(query: str) -> str:
return (
f'clickhouse client -q "{query}" --max_untracked_memory=1Gi '
"--memory_profiler_step=1Gi --max_memory_usage_for_user=0 --max_memory_usage_in_client=1000000000"
"--memory_profiler_step=1Gi --max_memory_usage_for_user=0 --max_memory_usage_in_client=1000000000 "
"--enable-progress-table-toggle=0"
)

View File

@ -20,7 +20,7 @@ expect_after {
-i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --enable-progress-table-toggle=0 --history_file=$history_file"
expect ":) "
send -- "DROP TABLE IF EXISTS test_01179\r"

View File

@ -20,7 +20,7 @@ expect_after {
-i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --enable-progress-table-toggle=0 --history_file=$history_file"
expect ":) "
# Make a query with syntax error

View File

@ -25,7 +25,7 @@ expect_after {
# useful debugging configuration
# exp_internal 1
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --enable-progress-table-toggle=0 --history_file=$history_file"
expect ":) "
send -- "SELECT 1\r"
@ -67,7 +67,7 @@ expect ":) "
send -- ""
expect eof
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --highlight 0 --multiline --history_file=$history_file"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --enable-progress-table-toggle=0 --highlight 0 --multiline --history_file=$history_file"
expect ":) "
send -- "SELECT 1;\r"

View File

@ -21,7 +21,7 @@ expect_after {
-i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --enable-progress-table-toggle=0 --history_file=$history_file"
expect ":) "
send -- "SELECT 1\r"

View File

@ -20,7 +20,7 @@ expect_after {
-i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --enable-progress-table-toggle=0 --history_file=$history_file"
expect ":) "
# Make a query
@ -33,7 +33,7 @@ exec kill -9 [exp_pid]
close
# Run client one more time and press "up" to see the last recorded query
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --enable-progress-table-toggle=0 --history_file=$history_file"
expect ":) "
send -- "\[A"
expect "for the history"

View File

@ -20,7 +20,7 @@ expect_after {
-i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --enable-progress-table-toggle=0 --history_file=$history_file"
expect ":) "
# Make a query

View File

@ -24,7 +24,7 @@ expect_after {
-i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion -m --history_file=$history_file --highlight 0"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --enable-progress-table-toggle=0 -m --history_file=$history_file --highlight 0"
expect "\n:) "
send -- "DROP TABLE IF EXISTS t01565;\r"

View File

@ -20,7 +20,7 @@ expect_after {
-i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --enable-progress-table-toggle=0 --history_file=$history_file"
expect ":) "
# regression for heap-buffer-overflow issue (under ASAN)

View File

@ -26,7 +26,7 @@ expect_after {
set Debug_type 0
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --enable-progress-table-toggle=0 --history_file=$history_file"
expect ":) "
# Check debug type
@ -44,7 +44,7 @@ expect eof
if { $Debug_type > 0} {
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --enable-progress-table-toggle=0 --history_file=$history_file"
expect "Warnings:"
expect " * Server was built in debug mode. It will work slowly."
expect ":) "
@ -58,7 +58,7 @@ send -- "q\r"
expect eof
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_for_all_queries=123 --history_file=$history_file"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --enable-progress-table-toggle=0 --max_memory_usage_for_all_queries=123 --history_file=$history_file"
expect "Warnings:"
expect " * Obsolete setting"
expect ":) "

View File

@ -21,7 +21,7 @@ expect_after {
-i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --enable-progress-table-toggle=0 --history_file=$history_file"
expect ":) "
send -- "DROP TABLE IF EXISTS test_02047\r"

View File

@ -20,7 +20,7 @@ expect_after {
-i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_LOCAL --disable_suggestion"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_LOCAL --disable_suggestion --enable-progress-table-toggle=0"
expect ":) "
send -- "drop table if exists t\r"

View File

@ -20,7 +20,7 @@ expect_after {
-i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --enable-progress-table-toggle=0 --history_file=$history_file"
expect ":) "
# Send a command

View File

@ -22,7 +22,7 @@ expect_after {
}
system "echo \"drop table if exists t; create table t(i String) engine=Memory; insert into t select 'test string'\" > $CLICKHOUSE_TMP/file_02112"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT --disable_suggestion --interactive --queries-file $CLICKHOUSE_TMP/file_02112"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT --disable_suggestion --enable-progress-table-toggle=0 --interactive --queries-file $CLICKHOUSE_TMP/file_02112"
expect ":) "
send -- "select i from t format TSV\r"

View File

@ -20,7 +20,7 @@ expect_after {
-i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_LOCAL --disable_suggestion --interactive --query 'create table t(i Int32) engine=Memory; insert into t select 1'"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_LOCAL --disable_suggestion --enable-progress-table-toggle=0 --interactive --query 'create table t(i Int32) engine=Memory; insert into t select 1'"
expect ":) "
send -- "select * from t format TSV\r"

View File

@ -21,7 +21,7 @@ expect_after {
}
system "echo \"drop table if exists t; create table t(i String) engine=Memory; insert into t select 'test string'\" > $CLICKHOUSE_TMP/file_02112"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_LOCAL --disable_suggestion --interactive --queries-file $CLICKHOUSE_TMP/file_02112"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_LOCAL --disable_suggestion --enable-progress-table-toggle=0 --interactive --queries-file $CLICKHOUSE_TMP/file_02112"
expect ":) "
send -- "select \* from t format TSV\r"

View File

@ -21,7 +21,7 @@ expect_after {
-i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --enable-progress-table-toggle=0 --history_file=$history_file"
# (?n) - Do not match new lines
expect -re "(?n)ClickHouse client version \[\\d\]{2}\.\[\\d\]{1,2}\.\[\\d\]{1,2}\.\[\\d\]{1,}.*\r"

View File

@ -24,7 +24,7 @@ expect_after {
# useful debugging configuration
# exp_internal 1
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --highlight 0 --history_file=$history_file"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --enable-progress-table-toggle=0 --highlight 0 --history_file=$history_file"
expect ":) "
# Make a query

View File

@ -22,7 +22,7 @@ expect_after {
}
spawn bash -c "echo 'select 1;\nselect 2;\nselect 3' > queries_02352"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT --disable_suggestion"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT --disable_suggestion --enable-progress-table-toggle=0"
expect ":) "
send -- "\\i queries_02352\r"

View File

@ -21,7 +21,7 @@ expect_after {
-i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --enable-progress-table-toggle=0 --history_file=$history_file"
expect ":) "
# -----------------------------------------

View File

@ -20,7 +20,7 @@ expect_after {
-i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --enable-progress-table-toggle=0 --history_file=$history_file"
expect ":) "
# Send a command

View File

@ -22,7 +22,7 @@ expect_after {
-i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --enable-progress-table-toggle=0 --history_file=$history_file"
expect ":) "
send -- "DROP USER IF EXISTS 02907_suggestions_readonly_user\r"
@ -51,7 +51,7 @@ set timeout 60
send -- "exit\r"
expect eof
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --enable-progress-table-toggle=0 --history_file=$history_file"
expect ":) "
send -- "DROP USER 02907_suggestions_readonly_user\r"

View File

@ -21,7 +21,7 @@ expect_after {
-i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --enable-progress-table-toggle=0 --history_file=$history_file"
expect ":) "
# send Ctrl-R (octal code of R is 022, see ascii(7))

View File

@ -21,7 +21,7 @@ expect_after {
-i $any_spawn_id timeout { exit 1 }
}
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file"
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --enable-progress-table-toggle=0 --history_file=$history_file"
expect ":) "
send -- "SELECT \x99\r"

View File

@ -16,7 +16,10 @@ class client(object):
def __init__(self, command=None, name="", log=None):
self.client = uexpect.spawn(["/bin/bash", "--noediting"])
if command is None:
command = os.environ.get("CLICKHOUSE_BINARY", "clickhouse") + " client"
options = "--enable-progress-table-toggle=0"
command = (
os.environ.get("CLICKHOUSE_BINARY", "clickhouse") + " client " + options
)
self.client.command = command
self.client.eol("\r")
self.client.logger(log, prefix=name)

View File

@ -68,6 +68,7 @@ EXTERN_TYPES_EXCLUDES=(
ProfileEvents::Timer
ProfileEvents::Type
ProfileEvents::TypeEnum
ProfileEvents::ValueType
ProfileEvents::dumpToMapColumn
ProfileEvents::getProfileEvents
ProfileEvents::ThreadIdToCountersSnapshot
@ -308,6 +309,7 @@ std_cerr_cout_excludes=(
_fuzzer
# OK
src/Common/ProgressIndication.cpp
src/Common/ProgressTable.cpp
# only under #ifdef DBMS_HASH_MAP_DEBUG_RESIZES, that is used only in tests
src/Common/HashTable/HashTable.h
# SensitiveDataMasker::printStats()