Merge upstream/master into ncb/support-hints-for-db-engines

This commit is contained in:
Bharat Nallan Chakravarthy 2024-01-03 08:44:11 -08:00
commit d63765fb19
111 changed files with 836 additions and 716 deletions

View File

@ -196,7 +196,7 @@ These settings should be defined in the disk configuration section.
- `max_elements` - a limit for a number of cache files. Default: `10000000`.
- `load_metadata_threads` - number of threads being used to load cache metadata on starting time. Default: `1`.
- `load_metadata_threads` - number of threads being used to load cache metadata on starting time. Default: `16`.
File Cache **query/profile settings**:

View File

@ -728,12 +728,7 @@ void LocalServer::processConfig()
/// We load temporary database first, because projections need it.
DatabaseCatalog::instance().initializeAndLoadTemporaryDatabase();
/** Init dummy default DB
* NOTE: We force using isolated default database to avoid conflicts with default database from server environment
* Otherwise, metadata of temporary File(format, EXPLICIT_PATH) tables will pollute metadata/ directory;
* if such tables will not be dropped, clickhouse-server will not be able to load them due to security reasons.
*/
std::string default_database = config().getString("default_database", "_local");
std::string default_database = config().getString("default_database", "default");
DatabaseCatalog::instance().attachDatabase(default_database, createClickHouseLocalDatabaseOverlay(default_database, global_context));
global_context->setCurrentDatabase(default_database);

View File

@ -5,6 +5,7 @@
#include <Core/ColumnsWithTypeAndName.h>
#include <Columns/IColumn.h>
namespace DB
{
namespace ErrorCodes
@ -16,7 +17,7 @@ class IFunctionBase;
using FunctionBasePtr = std::shared_ptr<const IFunctionBase>;
/** A column containing a lambda expression.
* Behaves like a constant-column. Contains an expression, but not input or output data.
* Contains an expression and captured columns, but not input arguments.
*/
class ColumnFunction final : public COWHelper<IColumn, ColumnFunction>
{
@ -207,8 +208,6 @@ private:
bool is_function_compiled;
void appendArgument(const ColumnWithTypeAndName & column);
void addOffsetsForReplication(const IColumn::Offsets & offsets);
};
const ColumnFunction * checkAndGetShortCircuitArgument(const ColumnPtr & column);

View File

@ -88,7 +88,7 @@ public:
{
/// A more understandable error message.
if (e.code() == DB::ErrorCodes::CANNOT_READ_ALL_DATA || e.code() == DB::ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF)
throw DB::ParsingException(e.code(), "File {} is empty. You must fill it manually with appropriate value.", path);
throw DB::Exception(e.code(), "File {} is empty. You must fill it manually with appropriate value.", path);
else
throw;
}

View File

@ -616,48 +616,4 @@ ExecutionStatus ExecutionStatus::fromText(const std::string & data)
return status;
}
ParsingException::ParsingException() = default;
ParsingException::ParsingException(const std::string & msg, int code)
: Exception(msg, code)
{
}
/// We use additional field formatted_message_ to make this method const.
std::string ParsingException::displayText() const
{
try
{
formatted_message = message();
bool need_newline = false;
if (!file_name.empty())
{
formatted_message += fmt::format(": (in file/uri {})", file_name);
need_newline = true;
}
if (line_number != -1)
{
formatted_message += fmt::format(": (at row {})", line_number);
need_newline = true;
}
if (need_newline)
formatted_message += "\n";
}
catch (...) {} // NOLINT(bugprone-empty-catch)
if (!formatted_message.empty())
{
std::string result = name();
result.append(": ");
result.append(formatted_message);
return result;
}
else
{
return Exception::displayText();
}
}
}

View File

@ -235,43 +235,6 @@ private:
const char * className() const noexcept override { return "DB::ErrnoException"; }
};
/// Special class of exceptions, used mostly in ParallelParsingInputFormat for
/// more convenient calculation of problem line number.
class ParsingException : public Exception
{
ParsingException(const std::string & msg, int code);
public:
ParsingException();
// Format message with fmt::format, like the logging functions.
template <typename... Args>
ParsingException(int code, FormatStringHelper<Args...> fmt, Args &&... args) : Exception(fmt::format(fmt.fmt_str, std::forward<Args>(args)...), code)
{
message_format_string = fmt.message_format_string;
}
std::string displayText() const override;
ssize_t getLineNumber() const { return line_number; }
void setLineNumber(int line_number_) { line_number = line_number_;}
String getFileName() const { return file_name; }
void setFileName(const String & file_name_) { file_name = file_name_; }
Exception * clone() const override { return new ParsingException(*this); }
void rethrow() const override { throw *this; } // NOLINT
private:
ssize_t line_number{-1};
String file_name;
mutable std::string formatted_message;
const char * name() const noexcept override { return "DB::ParsingException"; }
const char * className() const noexcept override { return "DB::ParsingException"; }
};
using Exceptions = std::vector<std::exception_ptr>;
/** Try to write an exception to the log (and forget about it).

View File

@ -3,10 +3,16 @@
#include <Common/JSONBuilder.h>
#include <Core/InterpolateDescription.h>
#include <Interpreters/convertFieldToType.h>
#include <Core/SettingsEnums.h>
#include <Common/IntervalKind.h>
#include <Parsers/ASTOrderByElement.h>
#include <Parsers/ASTInterpolateElement.h>
#include <Interpreters/Aliases.h>
#include <Interpreters/ActionsDAG.h>
namespace DB
{
InterpolateDescription::InterpolateDescription(ActionsDAGPtr actions_, const Aliases & aliases)
: actions(actions_)
{
@ -28,5 +34,4 @@ namespace DB
result_columns_order.push_back(name);
}
}
}

View File

@ -2,20 +2,18 @@
#include <unordered_map>
#include <memory>
#include <cstddef>
#include <string>
#include <Core/Field.h>
#include <Core/SettingsEnums.h>
#include <Common/IntervalKind.h>
#include <Parsers/ASTOrderByElement.h>
#include <Parsers/ASTInterpolateElement.h>
#include <Functions/FunctionsMiscellaneous.h>
#include <Interpreters/Aliases.h>
#include <Core/NamesAndTypes.h>
#include <Parsers/IAST_fwd.h>
namespace DB
{
class ActionsDAG;
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
using Aliases = std::unordered_map<String, ASTPtr>;
/// Interpolate description
struct InterpolateDescription
{

View File

@ -157,7 +157,7 @@ class IColumn;
M(Bool, allow_suspicious_fixed_string_types, false, "In CREATE TABLE statement allows creating columns of type FixedString(n) with n > 256. FixedString with length >= 256 is suspicious and most likely indicates misusage", 0) \
M(Bool, allow_suspicious_indices, false, "Reject primary/secondary indexes and sorting keys with identical expressions", 0) \
M(Bool, allow_suspicious_ttl_expressions, false, "Reject TTL expressions that don't depend on any of table's columns. It indicates a user error most of the time.", 0) \
M(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \
M(Bool, compile_expressions, true, "Compile some scalar functions and operators to native code.", 0) \
M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \
M(Bool, compile_aggregate_expressions, true, "Compile aggregate functions to native code.", 0) \
M(UInt64, min_count_to_compile_aggregate_expression, 3, "The number of identical aggregate expressions before they are JIT-compiled", 0) \

View File

@ -390,7 +390,7 @@ void SerializationArray::deserializeBinaryBulkWithMultipleStreams(
/// Check consistency between offsets and elements subcolumns.
/// But if elements column is empty - it's ok for columns of Nested types that was added by ALTER.
if (!nested_column->empty() && nested_column->size() != last_offset)
throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read all array values: read just {} of {}",
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read all array values: read just {} of {}",
toString(nested_column->size()), toString(last_offset));
column = std::move(mutable_column);
@ -445,7 +445,7 @@ static void deserializeTextImpl(IColumn & column, ReadBuffer & istr, Reader && r
if (*istr.position() == ',')
++istr.position();
else
throw ParsingException(ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT,
throw Exception(ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT,
"Cannot read array from text, expected comma or end of array, found '{}'",
*istr.position());
}

View File

@ -359,7 +359,7 @@ ReturnType SerializationNullable::deserializeTextEscapedAndRawImpl(IColumn & col
nested_column.popBack(1);
if (null_representation.find('\t') != std::string::npos || null_representation.find('\n') != std::string::npos)
throw DB::ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "TSV custom null representation "
throw DB::Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "TSV custom null representation "
"containing '\\t' or '\\n' may not work correctly for large input.");
WriteBufferFromOwnString parsed_value;
@ -367,7 +367,7 @@ ReturnType SerializationNullable::deserializeTextEscapedAndRawImpl(IColumn & col
nested_serialization->serializeTextEscaped(nested_column, nested_column.size() - 1, parsed_value, settings);
else
nested_serialization->serializeTextRaw(nested_column, nested_column.size() - 1, parsed_value, settings);
throw DB::ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while parsing \"{}{}\" as Nullable"
throw DB::Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while parsing \"{}{}\" as Nullable"
" at position {}: got \"{}\", which was deserialized as \"{}\". "
"It seems that input data is ill-formatted.",
std::string(pos, buf.buffer().end()),
@ -452,7 +452,7 @@ ReturnType SerializationNullable::deserializeTextQuotedImpl(IColumn & column, Re
/// It can happen only if there is an unquoted string instead of a number.
/// We also should delete incorrectly deserialized value from nested column.
nested_column.popBack(1);
throw DB::ParsingException(
throw DB::Exception(
ErrorCodes::CANNOT_READ_ALL_DATA,
"Error while parsing Nullable: got an unquoted string {} instead of a number",
String(buf.position(), std::min(10ul, buf.available())));
@ -589,12 +589,12 @@ ReturnType SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadB
if (null_representation.find(settings.csv.delimiter) != std::string::npos || null_representation.find('\r') != std::string::npos
|| null_representation.find('\n') != std::string::npos)
throw DB::ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "CSV custom null representation containing "
throw DB::Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "CSV custom null representation containing "
"format_csv_delimiter, '\\r' or '\\n' may not work correctly for large input.");
WriteBufferFromOwnString parsed_value;
nested_serialization->serializeTextCSV(nested_column, nested_column.size() - 1, parsed_value, settings);
throw DB::ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while parsing \"{}{}\" as Nullable"
throw DB::Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while parsing \"{}{}\" as Nullable"
" at position {}: got \"{}\", which was deserialized as \"{}\". "
"It seems that input data is ill-formatted.",
std::string(pos, buf.buffer().end()),

View File

@ -1055,7 +1055,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
for (auto & [_, intermediate, to] : replicated_tables_to_rename)
rename_table(intermediate, to);
LOG_DEBUG(log, "Renames completed succesessfully");
LOG_DEBUG(log, "Renames completed successfully");
for (const auto & id : dropped_tables)
DatabaseCatalog::instance().waitTableFinallyDropped(id);

View File

@ -43,7 +43,7 @@ namespace JSONUtils
{
const auto current_object_size = memory.size() + static_cast<size_t>(pos - in.position());
if (min_bytes != 0 && current_object_size > 10 * min_bytes)
throw ParsingException(ErrorCodes::INCORRECT_DATA,
throw Exception(ErrorCodes::INCORRECT_DATA,
"Size of JSON object at position {} is extremely large. Expected not greater than {} bytes, but current is {} bytes per row. "
"Increase the value setting 'min_chunk_bytes_for_parallel_parsing' or check your data manually, "
"most likely JSON is malformed", in.count(), min_bytes, current_object_size);

View File

@ -120,7 +120,7 @@ Block NativeReader::read()
if (istr.eof())
{
if (use_index)
throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Input doesn't contain all data for index.");
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Input doesn't contain all data for index.");
return res;
}

View File

@ -47,7 +47,6 @@
#include <Common/Exception.h>
#include <Core/AccurateComparison.h>
#include <Functions/IFunctionAdaptors.h>
#include <Functions/FunctionsMiscellaneous.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/DateTimeTransforms.h>
#include <Functions/toFixedString.h>

View File

@ -159,7 +159,6 @@ private:
class FunctionCapture : public IFunctionBase
{
public:
using Capture = ExecutableFunctionCapture::Capture;
using CapturePtr = ExecutableFunctionCapture::CapturePtr;
FunctionCapture(
@ -201,10 +200,10 @@ public:
FunctionCaptureOverloadResolver(
ExpressionActionsPtr expression_actions_,
const Names & captured_names_,
const NamesAndTypesList & lambda_arguments_,
const DataTypePtr & function_return_type_,
const String & expression_return_name_)
const Names & captured_names,
const NamesAndTypesList & lambda_arguments,
const DataTypePtr & function_return_type,
const String & expression_return_name)
: expression_actions(std::move(expression_actions_))
{
/// Check that expression does not contain unusual actions that will break columns structure.
@ -219,9 +218,9 @@ public:
arguments_map[arg.name] = arg.type;
DataTypes captured_types;
captured_types.reserve(captured_names_.size());
captured_types.reserve(captured_names.size());
for (const auto & captured_name : captured_names_)
for (const auto & captured_name : captured_names)
{
auto it = arguments_map.find(captured_name);
if (it == arguments_map.end())
@ -232,21 +231,21 @@ public:
}
DataTypes argument_types;
argument_types.reserve(lambda_arguments_.size());
for (const auto & lambda_argument : lambda_arguments_)
argument_types.reserve(lambda_arguments.size());
for (const auto & lambda_argument : lambda_arguments)
argument_types.push_back(lambda_argument.type);
return_type = std::make_shared<DataTypeFunction>(argument_types, function_return_type_);
return_type = std::make_shared<DataTypeFunction>(argument_types, function_return_type);
name = "Capture[" + toString(captured_types) + "](" + toString(argument_types) + ") -> "
+ function_return_type_->getName();
+ function_return_type->getName();
capture = std::make_shared<Capture>(Capture{
.captured_names = captured_names_,
.captured_names = captured_names,
.captured_types = std::move(captured_types),
.lambda_arguments = lambda_arguments_,
.return_name = expression_return_name_,
.return_type = function_return_type_,
.lambda_arguments = lambda_arguments,
.return_name = expression_return_name,
.return_type = function_return_type,
});
}

View File

@ -74,6 +74,8 @@ public:
size_t getNumberOfArguments() const override { return 0; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
bool useDefaultImplementationForConstants() const override { return true; }
/// Called if at least one function argument is a lambda expression.
/// For argument-lambda expressions, it defines the types of arguments of these expressions.
void getLambdaArgumentTypes(DataTypes & arguments) const override
@ -370,10 +372,10 @@ public:
/// Put all the necessary columns multiplied by the sizes of arrays into the columns.
auto replicated_column_function_ptr = IColumn::mutate(column_function->replicate(column_first_array->getOffsets()));
auto * replicated_column_function = typeid_cast<ColumnFunction *>(replicated_column_function_ptr.get());
replicated_column_function->appendArguments(arrays);
auto & replicated_column_function = typeid_cast<ColumnFunction &>(*replicated_column_function_ptr);
replicated_column_function.appendArguments(arrays);
auto lambda_result = replicated_column_function->reduce();
auto lambda_result = replicated_column_function.reduce();
/// Convert LowCardinality(T) -> T and Const(LowCardinality(T)) -> Const(T),
/// because we removed LowCardinality from return type of lambda expression.

View File

@ -20,33 +20,6 @@ PeekableReadBuffer::PeekableReadBuffer(ReadBuffer & sub_buf_, size_t start_size_
checkStateCorrect();
}
void PeekableReadBuffer::reset()
{
checkStateCorrect();
}
void PeekableReadBuffer::setSubBuffer(ReadBuffer & sub_buf_)
{
sub_buf = &sub_buf_;
resetImpl();
}
void PeekableReadBuffer::resetImpl()
{
peeked_size = 0;
checkpoint = std::nullopt;
checkpoint_in_own_memory = false;
use_stack_memory = true;
if (!currentlyReadFromOwnMemory())
sub_buf->position() = pos;
Buffer & sub_working = sub_buf->buffer();
BufferBase::set(sub_working.begin(), sub_working.size(), sub_buf->offset());
checkStateCorrect();
}
bool PeekableReadBuffer::peekNext()
{
checkStateCorrect();

View File

@ -74,12 +74,6 @@ public:
/// This data will be lost after destruction of peekable buffer.
bool hasUnreadData() const;
// for streaming reading (like in Kafka) we need to restore initial state of the buffer
// without recreating the buffer.
void reset();
void setSubBuffer(ReadBuffer & sub_buf_);
const ReadBuffer & getSubBuffer() const { return *sub_buf; }
private:

View File

@ -89,7 +89,7 @@ void NO_INLINE throwAtAssertionFailed(const char * s, ReadBuffer & buf)
else
out << " before: " << quote << String(buf.position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf.buffer().end() - buf.position()));
throw ParsingException(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "Cannot parse input: expected {}", out.str());
throw Exception(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "Cannot parse input: expected {}", out.str());
}
@ -562,7 +562,7 @@ static ReturnType readAnyQuotedStringInto(Vector & s, ReadBuffer & buf)
if (buf.eof() || *buf.position() != quote)
{
if constexpr (throw_exception)
throw ParsingException(ErrorCodes::CANNOT_PARSE_QUOTED_STRING,
throw Exception(ErrorCodes::CANNOT_PARSE_QUOTED_STRING,
"Cannot parse quoted string: expected opening quote '{}', got '{}'",
std::string{quote}, buf.eof() ? "EOF" : std::string{*buf.position()});
else
@ -608,7 +608,7 @@ static ReturnType readAnyQuotedStringInto(Vector & s, ReadBuffer & buf)
}
if constexpr (throw_exception)
throw ParsingException(ErrorCodes::CANNOT_PARSE_QUOTED_STRING, "Cannot parse quoted string: expected closing quote");
throw Exception(ErrorCodes::CANNOT_PARSE_QUOTED_STRING, "Cannot parse quoted string: expected closing quote");
else
return ReturnType(false);
}
@ -958,7 +958,7 @@ ReturnType readJSONStringInto(Vector & s, ReadBuffer & buf)
auto error = [](FormatStringHelper<> message [[maybe_unused]], int code [[maybe_unused]])
{
if constexpr (throw_exception)
throw ParsingException(code, std::move(message));
throw Exception(code, std::move(message));
return ReturnType(false);
};
@ -1009,7 +1009,7 @@ ReturnType readJSONObjectOrArrayPossiblyInvalid(Vector & s, ReadBuffer & buf)
auto error = [](FormatStringHelper<> message [[maybe_unused]], int code [[maybe_unused]])
{
if constexpr (throw_exception)
throw ParsingException(code, std::move(message));
throw Exception(code, std::move(message));
return ReturnType(false);
};
@ -1185,7 +1185,7 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D
else
{
if constexpr (throw_exception)
throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime");
throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime");
else
return false;
}
@ -1212,7 +1212,7 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D
s_pos[size] = 0;
if constexpr (throw_exception)
throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime {}", s);
throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime {}", s);
else
return false;
}
@ -1235,7 +1235,7 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D
s_pos[size] = 0;
if constexpr (throw_exception)
throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse time component of DateTime {}", s);
throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse time component of DateTime {}", s);
else
return false;
}
@ -1266,7 +1266,7 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D
if (too_short && negative_multiplier != -1)
{
if constexpr (throw_exception)
throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime");
throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime");
else
return false;
}

View File

@ -296,7 +296,7 @@ inline void readBoolTextWord(bool & x, ReadBuffer & buf, bool support_upper_case
[[fallthrough]];
}
default:
throw ParsingException(ErrorCodes::CANNOT_PARSE_BOOL, "Unexpected Bool value");
throw Exception(ErrorCodes::CANNOT_PARSE_BOOL, "Unexpected Bool value");
}
}
@ -340,7 +340,7 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf)
if (has_sign)
{
if constexpr (throw_exception)
throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER,
throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER,
"Cannot parse number with multiple sign (+/-) characters");
else
return ReturnType(false);
@ -357,7 +357,7 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf)
if (has_sign)
{
if constexpr (throw_exception)
throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER,
throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER,
"Cannot parse number with multiple sign (+/-) characters");
else
return ReturnType(false);
@ -368,7 +368,7 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf)
else
{
if constexpr (throw_exception)
throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Unsigned type must not contain '-' symbol");
throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Unsigned type must not contain '-' symbol");
else
return ReturnType(false);
}
@ -430,7 +430,7 @@ end:
if (has_sign && !has_number)
{
if constexpr (throw_exception)
throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER,
throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER,
"Cannot parse number with a sign character but without any numeric character");
else
return ReturnType(false);
@ -837,7 +837,7 @@ inline ReturnType readUUIDTextImpl(UUID & uuid, ReadBuffer & buf)
if constexpr (throw_exception)
{
throw ParsingException(ErrorCodes::CANNOT_PARSE_UUID, "Cannot parse uuid {}", s);
throw Exception(ErrorCodes::CANNOT_PARSE_UUID, "Cannot parse uuid {}", s);
}
else
{
@ -855,7 +855,7 @@ inline ReturnType readUUIDTextImpl(UUID & uuid, ReadBuffer & buf)
if constexpr (throw_exception)
{
throw ParsingException(ErrorCodes::CANNOT_PARSE_UUID, "Cannot parse uuid {}", s);
throw Exception(ErrorCodes::CANNOT_PARSE_UUID, "Cannot parse uuid {}", s);
}
else
{
@ -881,7 +881,7 @@ inline ReturnType readIPv4TextImpl(IPv4 & ip, ReadBuffer & buf)
return ReturnType(true);
if constexpr (std::is_same_v<ReturnType, void>)
throw ParsingException(ErrorCodes::CANNOT_PARSE_IPV4, "Cannot parse IPv4 {}", std::string_view(buf.position(), buf.available()));
throw Exception(ErrorCodes::CANNOT_PARSE_IPV4, "Cannot parse IPv4 {}", std::string_view(buf.position(), buf.available()));
else
return ReturnType(false);
}
@ -903,7 +903,7 @@ inline ReturnType readIPv6TextImpl(IPv6 & ip, ReadBuffer & buf)
return ReturnType(true);
if constexpr (std::is_same_v<ReturnType, void>)
throw ParsingException(ErrorCodes::CANNOT_PARSE_IPV6, "Cannot parse IPv6 {}", std::string_view(buf.position(), buf.available()));
throw Exception(ErrorCodes::CANNOT_PARSE_IPV6, "Cannot parse IPv6 {}", std::string_view(buf.position(), buf.available()));
else
return ReturnType(false);
}
@ -944,7 +944,7 @@ inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, cons
if (!buf.eof() && !isNumericASCII(*buf.position()))
{
if constexpr (throw_exception)
throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse datetime");
throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse datetime");
else
return false;
}
@ -1017,7 +1017,7 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re
{
readDateTimeTextImpl<ReturnType, true>(whole, buf, date_lut);
}
catch (const DB::ParsingException &)
catch (const DB::Exception &)
{
if (buf.eof() || *buf.position() != '.')
throw;
@ -1125,7 +1125,7 @@ inline void readDateTimeText(LocalDateTime & datetime, ReadBuffer & buf)
if (10 != size)
{
s[size] = 0;
throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime {}", s);
throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime {}", s);
}
datetime.year((s[0] - '0') * 1000 + (s[1] - '0') * 100 + (s[2] - '0') * 10 + (s[3] - '0'));
@ -1141,7 +1141,7 @@ inline void readDateTimeText(LocalDateTime & datetime, ReadBuffer & buf)
if (8 != size)
{
s[size] = 0;
throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse time component of DateTime {}", s);
throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse time component of DateTime {}", s);
}
datetime.hour((s[0] - '0') * 10 + (s[1] - '0'));
@ -1174,7 +1174,7 @@ inline ReturnType readTimeTextImpl(time_t & time, ReadBuffer & buf)
s[size] = 0;
if constexpr (throw_exception)
throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime {}", s);
throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime {}", s);
else
return false;
}
@ -1482,7 +1482,7 @@ void readQuoted(std::vector<T> & x, ReadBuffer & buf)
if (*buf.position() == ',')
++buf.position();
else
throw ParsingException(ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT, "Cannot read array from text");
throw Exception(ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT, "Cannot read array from text");
}
first = false;
@ -1505,7 +1505,7 @@ void readDoubleQuoted(std::vector<T> & x, ReadBuffer & buf)
if (*buf.position() == ',')
++buf.position();
else
throw ParsingException(ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT, "Cannot read array from text");
throw Exception(ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT, "Cannot read array from text");
}
first = false;

View File

@ -95,7 +95,7 @@ ReturnType parseDateTimeBestEffortImpl(
FmtArgs && ...fmt_args [[maybe_unused]])
{
if constexpr (std::is_same_v<ReturnType, void>)
throw ParsingException(error_code, std::move(fmt_string), std::forward<FmtArgs>(fmt_args)...);
throw Exception(error_code, std::move(fmt_string), std::forward<FmtArgs>(fmt_args)...);
else
return false;
};

View File

@ -121,7 +121,7 @@ inline bool readDigits(ReadBuffer & buf, T & x, uint32_t & digits, int32_t & exp
if (!tryReadIntText(addition_exp, buf))
{
if constexpr (_throw_on_error)
throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot parse exponent while reading decimal");
throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot parse exponent while reading decimal");
else
return false;
}
@ -134,7 +134,7 @@ inline bool readDigits(ReadBuffer & buf, T & x, uint32_t & digits, int32_t & exp
if (digits_only)
{
if constexpr (_throw_on_error)
throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Unexpected symbol while reading decimal");
throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Unexpected symbol while reading decimal");
return false;
}
stop = true;

View File

@ -160,7 +160,7 @@ ReturnType readFloatTextPreciseImpl(T & x, ReadBuffer & buf)
if (unlikely(res.ec != std::errc()))
{
if constexpr (throw_exception)
throw ParsingException(
throw Exception(
ErrorCodes::CANNOT_PARSE_NUMBER,
"Cannot read floating point value here: {}",
String(initial_position, buf.buffer().end() - initial_position));
@ -253,7 +253,7 @@ ReturnType readFloatTextPreciseImpl(T & x, ReadBuffer & buf)
if (unlikely(res.ec != std::errc() || res.ptr - tmp_buf != num_copied_chars))
{
if constexpr (throw_exception)
throw ParsingException(
throw Exception(
ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value here: {}", String(tmp_buf, num_copied_chars));
else
return ReturnType(false);
@ -342,7 +342,7 @@ ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in)
if (in.eof())
{
if constexpr (throw_exception)
throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value");
throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value");
else
return false;
}
@ -400,7 +400,7 @@ ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in)
if (in.eof())
{
if constexpr (throw_exception)
throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value: nothing after exponent");
throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value: nothing after exponent");
else
return false;
}
@ -438,7 +438,7 @@ ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in)
if (in.eof())
{
if constexpr (throw_exception)
throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value: no digits read");
throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value: no digits read");
else
return false;
}
@ -449,14 +449,14 @@ ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in)
if (in.eof())
{
if constexpr (throw_exception)
throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value: nothing after plus sign");
throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value: nothing after plus sign");
else
return false;
}
else if (negative)
{
if constexpr (throw_exception)
throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value: plus after minus sign");
throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value: plus after minus sign");
else
return false;
}

View File

@ -115,6 +115,7 @@ public:
explicit ActionsDAG(const ColumnsWithTypeAndName & inputs_);
const Nodes & getNodes() const { return nodes; }
static Nodes detachNodes(ActionsDAG && dag) { return std::move(dag.nodes); }
const NodeRawConstPtrs & getOutputs() const { return outputs; }
/** Output nodes can contain any column returned from DAG.
* You may manually change it if needed.

View File

@ -664,26 +664,26 @@ void Aggregator::compileAggregateFunctionsIfNeeded()
for (size_t i = 0; i < aggregate_functions.size(); ++i)
{
const auto * function = aggregate_functions[i];
bool function_is_compilable = function->isCompilable();
if (!function_is_compilable)
continue;
size_t offset_of_aggregate_function = offsets_of_aggregate_states[i];
if (function->isCompilable())
AggregateFunctionWithOffset function_to_compile
{
AggregateFunctionWithOffset function_to_compile
{
.function = function,
.aggregate_data_offset = offset_of_aggregate_function
};
.function = function,
.aggregate_data_offset = offset_of_aggregate_function
};
functions_to_compile.emplace_back(std::move(function_to_compile));
functions_to_compile.emplace_back(std::move(function_to_compile));
functions_description += function->getDescription();
functions_description += ' ';
functions_description += function->getDescription();
functions_description += ' ';
functions_description += std::to_string(offset_of_aggregate_function);
functions_description += ' ';
}
functions_description += std::to_string(offset_of_aggregate_function);
functions_description += ' ';
is_aggregate_function_compiled[i] = function->isCompilable();
is_aggregate_function_compiled[i] = true;
}
if (functions_to_compile.empty())
@ -1685,14 +1685,13 @@ bool Aggregator::executeOnBlock(Columns columns,
/// For the case when there are no keys (all aggregate into one row).
if (result.type == AggregatedDataVariants::Type::without_key)
{
/// TODO: Enable compilation after investigation
// #if USE_EMBEDDED_COMPILER
// if (compiled_aggregate_functions_holder)
// {
// executeWithoutKeyImpl<true>(result.without_key, row_begin, row_end, aggregate_functions_instructions.data(), result.aggregates_pool);
// }
// else
// #endif
#if USE_EMBEDDED_COMPILER
if (compiled_aggregate_functions_holder && !hasSparseArguments(aggregate_functions_instructions.data()))
{
executeWithoutKeyImpl<true>(result.without_key, row_begin, row_end, aggregate_functions_instructions.data(), result.aggregates_pool);
}
else
#endif
{
executeWithoutKeyImpl<false>(result.without_key, row_begin, row_end, aggregate_functions_instructions.data(), result.aggregates_pool);
}

View File

@ -767,7 +767,6 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing(
};
StreamingFormatExecutor executor(header, format, std::move(on_error), std::move(adding_defaults_transform));
std::unique_ptr<ReadBuffer> last_buffer;
auto chunk_info = std::make_shared<AsyncInsertInfo>();
auto query_for_logging = serializeQuery(*key.query, insert_context->getSettingsRef().log_queries_cut_to_length);
@ -783,11 +782,6 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing(
auto buffer = std::make_unique<ReadBufferFromString>(*bytes);
size_t num_bytes = bytes->size();
size_t num_rows = executor.execute(*buffer);
/// Keep buffer, because it still can be used
/// in destructor, while resetting buffer at next iteration.
last_buffer = std::move(buffer);
total_rows += num_rows;
chunk_info->offsets.push_back(total_rows);
chunk_info->tokens.push_back(entry->async_dedup_token);
@ -796,8 +790,6 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing(
current_exception.clear();
}
format->addBuffer(std::move(last_buffer));
Chunk chunk(executor.getResultColumns(), total_rows);
chunk.setChunkInfo(std::move(chunk_info));
return chunk;

View File

@ -8,7 +8,7 @@ static constexpr int FILECACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE = 32 * 1024 * 1024;
static constexpr int FILECACHE_DEFAULT_FILE_SEGMENT_ALIGNMENT = 4 * 1024 * 1024; /// 4Mi
static constexpr int FILECACHE_DEFAULT_BACKGROUND_DOWNLOAD_THREADS = 5;
static constexpr int FILECACHE_DEFAULT_BACKGROUND_DOWNLOAD_QUEUE_SIZE_LIMIT = 5000;
static constexpr int FILECACHE_DEFAULT_LOAD_METADATA_THREADS = 1;
static constexpr int FILECACHE_DEFAULT_LOAD_METADATA_THREADS = 16;
static constexpr int FILECACHE_DEFAULT_MAX_ELEMENTS = 10000000;
static constexpr int FILECACHE_DEFAULT_HITS_THRESHOLD = 0;
static constexpr size_t FILECACHE_BYPASS_THRESHOLD = 256 * 1024 * 1024;

View File

@ -67,7 +67,8 @@ static void compileFunction(llvm::Module & module, const IFunctionBase & functio
{
const auto & function_argument_types = function.getArgumentTypes();
llvm::IRBuilder<> b(module.getContext());
auto & context = module.getContext();
llvm::IRBuilder<> b(context);
auto * size_type = b.getIntNTy(sizeof(size_t) * 8);
auto * data_type = llvm::StructType::get(b.getInt8PtrTy(), b.getInt8PtrTy());
auto * func_type = llvm::FunctionType::get(b.getVoidTy(), { size_type, data_type->getPointerTo() }, /*isVarArg=*/false);
@ -75,6 +76,8 @@ static void compileFunction(llvm::Module & module, const IFunctionBase & functio
/// Create function in module
auto * func = llvm::Function::Create(func_type, llvm::Function::ExternalLinkage, function.getName(), module);
func->setAttributes(llvm::AttributeList::get(context, {{2, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)}}));
auto * args = func->args().begin();
llvm::Value * rows_count_arg = args++;
llvm::Value * columns_arg = args++;
@ -196,6 +199,9 @@ static void compileCreateAggregateStatesFunctions(llvm::Module & module, const s
auto * create_aggregate_states_function_type = llvm::FunctionType::get(b.getVoidTy(), { aggregate_data_places_type }, false);
auto * create_aggregate_states_function = llvm::Function::Create(create_aggregate_states_function_type, llvm::Function::ExternalLinkage, name, module);
create_aggregate_states_function->setAttributes(
llvm::AttributeList::get(context, {{1, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)}}));
auto * arguments = create_aggregate_states_function->args().begin();
llvm::Value * aggregate_data_place_arg = arguments++;
@ -241,6 +247,11 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module,
auto * add_into_aggregate_states_func_declaration = llvm::FunctionType::get(b.getVoidTy(), { size_type, size_type, column_type->getPointerTo(), places_type }, false);
auto * add_into_aggregate_states_func = llvm::Function::Create(add_into_aggregate_states_func_declaration, llvm::Function::ExternalLinkage, name, module);
add_into_aggregate_states_func->setAttributes(llvm::AttributeList::get(
context,
{{3, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)},
{4, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)}}));
auto * arguments = add_into_aggregate_states_func->args().begin();
llvm::Value * row_start_arg = arguments++;
llvm::Value * row_end_arg = arguments++;
@ -296,7 +307,7 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module,
llvm::Value * aggregation_place = nullptr;
if (places_argument_type == AddIntoAggregateStatesPlacesArgumentType::MultiplePlaces)
aggregation_place = b.CreateLoad(b.getInt8Ty()->getPointerTo(), b.CreateGEP(b.getInt8Ty()->getPointerTo(), places_arg, counter_phi));
aggregation_place = b.CreateLoad(b.getInt8Ty()->getPointerTo(), b.CreateInBoundsGEP(b.getInt8Ty()->getPointerTo(), places_arg, counter_phi));
else
aggregation_place = places_arg;
@ -313,7 +324,7 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module,
auto & column = columns[previous_columns_size + column_argument_index];
const auto & argument_type = arguments_types[column_argument_index];
auto * column_data_element = b.CreateLoad(column.data_element_type, b.CreateGEP(column.data_element_type, column.data_ptr, counter_phi));
auto * column_data_element = b.CreateLoad(column.data_element_type, b.CreateInBoundsGEP(column.data_element_type, column.data_ptr, counter_phi));
if (!argument_type->isNullable())
{
@ -321,7 +332,7 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module,
continue;
}
auto * column_null_data_with_offset = b.CreateGEP(b.getInt8Ty(), column.null_data_ptr, counter_phi);
auto * column_null_data_with_offset = b.CreateInBoundsGEP(b.getInt8Ty(), column.null_data_ptr, counter_phi);
auto * is_null = b.CreateICmpNE(b.CreateLoad(b.getInt8Ty(), column_null_data_with_offset), b.getInt8(0));
auto * nullable_unitialized = llvm::Constant::getNullValue(toNullableType(b, column.data_element_type));
auto * first_insert = b.CreateInsertValue(nullable_unitialized, column_data_element, {0});
@ -354,7 +365,8 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module,
static void compileMergeAggregatesStates(llvm::Module & module, const std::vector<AggregateFunctionWithOffset> & functions, const std::string & name)
{
llvm::IRBuilder<> b(module.getContext());
auto & context = module.getContext();
llvm::IRBuilder<> b(context);
auto * aggregate_data_place_type = b.getInt8Ty()->getPointerTo();
auto * aggregate_data_places_type = aggregate_data_place_type->getPointerTo();
@ -365,6 +377,11 @@ static void compileMergeAggregatesStates(llvm::Module & module, const std::vecto
auto * merge_aggregates_states_func
= llvm::Function::Create(merge_aggregates_states_func_declaration, llvm::Function::ExternalLinkage, name, module);
merge_aggregates_states_func->setAttributes(llvm::AttributeList::get(
context,
{{1, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)},
{2, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)}}));
auto * arguments = merge_aggregates_states_func->args().begin();
llvm::Value * aggregate_data_places_dst_arg = arguments++;
llvm::Value * aggregate_data_places_src_arg = arguments++;
@ -426,6 +443,11 @@ static void compileInsertAggregatesIntoResultColumns(llvm::Module & module, cons
auto * insert_aggregates_into_result_func_declaration = llvm::FunctionType::get(b.getVoidTy(), { size_type, size_type, column_type->getPointerTo(), aggregate_data_places_type }, false);
auto * insert_aggregates_into_result_func = llvm::Function::Create(insert_aggregates_into_result_func_declaration, llvm::Function::ExternalLinkage, name, module);
insert_aggregates_into_result_func->setAttributes(llvm::AttributeList::get(
context,
{{3, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)},
{4, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)}}));
auto * arguments = insert_aggregates_into_result_func->args().begin();
llvm::Value * row_start_arg = arguments++;
llvm::Value * row_end_arg = arguments++;
@ -460,7 +482,7 @@ static void compileInsertAggregatesIntoResultColumns(llvm::Module & module, cons
auto * counter_phi = b.CreatePHI(row_start_arg->getType(), 2);
counter_phi->addIncoming(row_start_arg, entry);
auto * aggregate_data_place = b.CreateLoad(b.getInt8Ty()->getPointerTo(), b.CreateGEP(b.getInt8Ty()->getPointerTo(), aggregate_data_places_arg, counter_phi));
auto * aggregate_data_place = b.CreateLoad(b.getInt8Ty()->getPointerTo(), b.CreateInBoundsGEP(b.getInt8Ty()->getPointerTo(), aggregate_data_places_arg, counter_phi));
for (size_t i = 0; i < functions.size(); ++i)
{
@ -470,11 +492,11 @@ static void compileInsertAggregatesIntoResultColumns(llvm::Module & module, cons
const auto * aggregate_function_ptr = functions[i].function;
auto * final_value = aggregate_function_ptr->compileGetResult(b, aggregation_place_with_offset);
auto * result_column_data_element = b.CreateGEP(columns[i].data_element_type, columns[i].data_ptr, counter_phi);
auto * result_column_data_element = b.CreateInBoundsGEP(columns[i].data_element_type, columns[i].data_ptr, counter_phi);
if (columns[i].null_data_ptr)
{
b.CreateStore(b.CreateExtractValue(final_value, {0}), result_column_data_element);
auto * result_column_is_null_element = b.CreateGEP(b.getInt8Ty(), columns[i].null_data_ptr, counter_phi);
auto * result_column_is_null_element = b.CreateInBoundsGEP(b.getInt8Ty(), columns[i].null_data_ptr, counter_phi);
b.CreateStore(b.CreateSelect(b.CreateExtractValue(final_value, {1}), b.getInt8(1), b.getInt8(0)), result_column_is_null_element);
}
else

View File

@ -8,16 +8,13 @@
#include <Analyzer/LambdaNode.h>
#include <Analyzer/SortNode.h>
#include <Analyzer/WindowNode.h>
#include <Analyzer/UnionNode.h>
#include <Analyzer/QueryNode.h>
#include <Analyzer/ConstantValue.h>
#include <DataTypes/FieldToDataType.h>
#include <DataTypes/DataTypeSet.h>
#include <Common/FieldVisitorToString.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Columns/ColumnSet.h>
#include <Columns/ColumnConst.h>
@ -33,6 +30,7 @@
#include <Planner/TableExpressionData.h>
#include <Planner/Utils.h>
namespace DB
{

View File

@ -34,15 +34,13 @@ MutableColumns StreamingFormatExecutor::getResultColumns()
size_t StreamingFormatExecutor::execute(ReadBuffer & buffer)
{
auto & initial_buf = format->getReadBuffer();
format->setReadBuffer(buffer);
size_t rows = execute();
/// Format destructor can touch read buffer (for example when we use PeekableReadBuffer),
/// but we cannot control lifetime of provided read buffer. To avoid heap use after free
/// we can set initial read buffer back, because initial read buffer was created before
/// format, so it will be destructed after it.
format->setReadBuffer(initial_buf);
return rows;
/// we call format->resetReadBuffer() method that resets all buffers inside format.
SCOPE_EXIT(format->resetReadBuffer());
return execute();
}
size_t StreamingFormatExecutor::execute()

View File

@ -1,6 +1,7 @@
#include <Processors/Formats/IInputFormat.h>
#include <IO/ReadBuffer.h>
#include <IO/WithFileName.h>
#include <Common/Exception.h>
namespace DB
{
@ -11,6 +12,21 @@ IInputFormat::IInputFormat(Block header, ReadBuffer * in_)
column_mapping = std::make_shared<ColumnMapping>();
}
Chunk IInputFormat::generate()
{
try
{
return read();
}
catch (Exception & e)
{
auto file_name = getFileNameFromReadBuffer(getReadBuffer());
if (!file_name.empty())
e.addMessage(fmt::format("(in file/uri {})", file_name));
throw;
}
}
void IInputFormat::resetParser()
{
chassert(in);
@ -24,7 +40,6 @@ void IInputFormat::resetParser()
void IInputFormat::setReadBuffer(ReadBuffer & in_)
{
chassert(in); // not supported by random-access formats
in = &in_;
}

View File

@ -27,6 +27,11 @@ public:
/// ReadBuffer can be nullptr for random-access formats.
IInputFormat(Block header, ReadBuffer * in_);
Chunk generate() override;
/// All data reading from the read buffer must be performed by this method.
virtual Chunk read() = 0;
/** In some usecase (hello Kafka) we need to read a lot of tiny streams in exactly the same format.
* The recreating of parser for each small stream takes too long, so we introduce a method
* resetParser() which allow to reset the state of parser to continue reading of
@ -36,7 +41,7 @@ public:
virtual void resetParser();
virtual void setReadBuffer(ReadBuffer & in_);
ReadBuffer & getReadBuffer() const { chassert(in); return *in; }
virtual void resetReadBuffer() { in = nullptr; }
virtual const BlockMissingValues & getMissingValues() const
{
@ -49,8 +54,9 @@ public:
/// Must be called from ParallelParsingInputFormat before readPrefix
void setColumnMapping(ColumnMappingPtr column_mapping_) { column_mapping = column_mapping_; }
size_t getCurrentUnitNumber() const { return current_unit_number; }
void setCurrentUnitNumber(size_t current_unit_number_) { current_unit_number = current_unit_number_; }
/// Set the number of rows that was already read in
/// parallel parsing before creating this parser.
virtual void setRowsReadBefore(size_t /*rows*/) {}
void addBuffer(std::unique_ptr<ReadBuffer> buffer) { owned_buffers.emplace_back(std::move(buffer)); }
@ -61,6 +67,8 @@ public:
void needOnlyCount() { need_only_count = true; }
protected:
ReadBuffer & getReadBuffer() const { chassert(in); return *in; }
virtual Chunk getChunkForCount(size_t rows);
ColumnMappingPtr column_mapping{};
@ -70,9 +78,6 @@ protected:
bool need_only_count = false;
private:
/// Number of currently parsed chunk (if parallel parsing is enabled)
size_t current_unit_number = 0;
std::vector<std::unique_ptr<ReadBuffer>> owned_buffers;
};

View File

@ -83,7 +83,7 @@ void IRowInputFormat::logError()
errors_logger->logError(InputFormatErrorsLogger::ErrorEntry{now_time, total_rows, diagnostic, raw_data});
}
Chunk IRowInputFormat::generate()
Chunk IRowInputFormat::read()
{
if (total_rows == 0)
{
@ -93,10 +93,6 @@ Chunk IRowInputFormat::generate()
}
catch (Exception & e)
{
auto file_name = getFileNameFromReadBuffer(getReadBuffer());
if (!file_name.empty())
e.addMessage(fmt::format("(in file/uri {})", file_name));
e.addMessage("(while reading header)");
throw;
}
@ -132,8 +128,6 @@ Chunk IRowInputFormat::generate()
{
try
{
++total_rows;
info.read_columns.clear();
continue_reading = readRow(columns, info);
@ -148,6 +142,8 @@ Chunk IRowInputFormat::generate()
}
}
++total_rows;
/// Some formats may read row AND say the read is finished.
/// For such a case, get the number or rows from first column.
if (!columns.empty())
@ -162,6 +158,8 @@ Chunk IRowInputFormat::generate()
}
catch (Exception & e)
{
++total_rows;
/// Logic for possible skipping of errors.
if (!isParseError(e.code()))
@ -204,27 +202,6 @@ Chunk IRowInputFormat::generate()
}
}
}
catch (ParsingException & e)
{
String verbose_diagnostic;
try
{
verbose_diagnostic = getDiagnosticInfo();
}
catch (const Exception & exception)
{
verbose_diagnostic = "Cannot get verbose diagnostic: " + exception.message();
}
catch (...) // NOLINT(bugprone-empty-catch)
{
/// Error while trying to obtain verbose diagnostic. Ok to ignore.
}
e.setFileName(getFileNameFromReadBuffer(getReadBuffer()));
e.setLineNumber(static_cast<int>(total_rows));
e.addMessage(verbose_diagnostic);
throw;
}
catch (Exception & e)
{
if (!isParseError(e.code()))
@ -244,10 +221,6 @@ Chunk IRowInputFormat::generate()
/// Error while trying to obtain verbose diagnostic. Ok to ignore.
}
auto file_name = getFileNameFromReadBuffer(getReadBuffer());
if (!file_name.empty())
e.addMessage(fmt::format("(in file/uri {})", file_name));
e.addMessage(fmt::format("(at row {})\n", total_rows));
e.addMessage(verbose_diagnostic);
throw;

View File

@ -42,7 +42,7 @@ public:
IRowInputFormat(Block header, ReadBuffer & in_, Params params_);
Chunk generate() override;
Chunk read() override;
void resetParser() override;
@ -79,10 +79,12 @@ protected:
const BlockMissingValues & getMissingValues() const override { return block_missing_values; }
size_t getTotalRows() const { return total_rows; }
size_t getRowNum() const { return total_rows; }
size_t getApproxBytesReadForChunk() const override { return approx_bytes_read_for_chunk; }
void setRowsReadBefore(size_t rows) override { total_rows = rows; }
Serializations serializations;
private:

View File

@ -28,7 +28,7 @@ ArrowBlockInputFormat::ArrowBlockInputFormat(ReadBuffer & in_, const Block & hea
{
}
Chunk ArrowBlockInputFormat::generate()
Chunk ArrowBlockInputFormat::read()
{
Chunk res;
block_missing_values.clear();
@ -64,7 +64,7 @@ Chunk ArrowBlockInputFormat::generate()
{
auto rows = file_reader->RecordBatchCountRows(record_batch_current++);
if (!rows.ok())
throw ParsingException(
throw Exception(
ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading batch of Arrow data: {}", rows.status().ToString());
return getChunkForCount(*rows);
}
@ -73,12 +73,12 @@ Chunk ArrowBlockInputFormat::generate()
}
if (!batch_result.ok())
throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA,
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA,
"Error while reading batch of Arrow data: {}", batch_result.status().ToString());
auto table_result = arrow::Table::FromRecordBatches({*batch_result});
if (!table_result.ok())
throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA,
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA,
"Error while reading batch of Arrow data: {}", table_result.status().ToString());
++record_batch_current;
@ -213,7 +213,7 @@ std::optional<size_t> ArrowSchemaReader::readNumberOrRows()
auto rows = file_reader->CountRows();
if (!rows.ok())
throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading batch of Arrow data: {}", rows.status().ToString());
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading batch of Arrow data: {}", rows.status().ToString());
return *rows;
}

View File

@ -30,7 +30,7 @@ public:
size_t getApproxBytesReadForChunk() const override { return approx_bytes_read_for_chunk; }
private:
Chunk generate() override;
Chunk read() override;
void onCancel() override
{

View File

@ -7,7 +7,6 @@
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/WriteBufferFromString.h>
#include <IO/copyData.h>
#include <IO/PeekableReadBuffer.h>
#include <arrow/buffer.h>
#include <arrow/util/future.h>
#include <arrow/io/memory.h>

View File

@ -186,7 +186,7 @@ static AvroDeserializer::DeserializeFn createDecimalDeserializeFn(const avro::No
tmp = decoder.decodeBytes();
if (tmp.size() > field_type_size || tmp.empty())
throw ParsingException(
throw Exception(
ErrorCodes::CANNOT_PARSE_UUID,
"Cannot parse type {}, expected non-empty binary data with size equal to or less than {}, got {}",
target_type->getName(),
@ -274,7 +274,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(const avro
{
decoder.decodeString(tmp);
if (tmp.length() != 36)
throw ParsingException(ErrorCodes::CANNOT_PARSE_UUID, "Cannot parse uuid {}", tmp);
throw Exception(ErrorCodes::CANNOT_PARSE_UUID, "Cannot parse uuid {}", tmp);
const UUID uuid = parseUUID({reinterpret_cast<const UInt8 *>(tmp.data()), tmp.length()});
assert_cast<DataTypeUUID::ColumnType &>(column).insertValue(uuid);
@ -530,7 +530,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(const avro
{
decoder.decodeFixed(fixed_size, tmp);
if (tmp.size() != 36)
throw ParsingException(ErrorCodes::CANNOT_PARSE_UUID, "Cannot parse UUID from type Fixed, because it's size ({}) is not equal to the size of UUID (36)", fixed_size);
throw Exception(ErrorCodes::CANNOT_PARSE_UUID, "Cannot parse UUID from type Fixed, because it's size ({}) is not equal to the size of UUID (36)", fixed_size);
const UUID uuid = parseUUID({reinterpret_cast<const UInt8 *>(tmp.data()), tmp.size()});
assert_cast<DataTypeUUID::ColumnType &>(column).insertValue(uuid);

View File

@ -1031,17 +1031,17 @@ fileSegmentationEngineBSONEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t
readBinaryLittleEndian(document_size, in);
if (document_size < sizeof(document_size))
throw ParsingException(ErrorCodes::INCORRECT_DATA, "Size of BSON document is invalid");
throw Exception(ErrorCodes::INCORRECT_DATA, "Size of BSON document is invalid");
if (min_bytes != 0 && document_size > 10 * min_bytes)
throw ParsingException(
throw Exception(
ErrorCodes::INCORRECT_DATA,
"Size of BSON document is extremely large. Expected not greater than {} bytes, but current is {} bytes per row. Increase "
"the value setting 'min_chunk_bytes_for_parallel_parsing' or check your data manually, most likely BSON is malformed",
min_bytes, document_size);
if (document_size < sizeof(document_size))
throw ParsingException(ErrorCodes::INCORRECT_DATA, "Size of BSON document is invalid");
throw Exception(ErrorCodes::INCORRECT_DATA, "Size of BSON document is invalid");
size_t old_size = memory.size();
memory.resize(old_size + document_size);

View File

@ -57,9 +57,6 @@ public:
void resetParser() override;
private:
void readPrefix() override { }
void readSuffix() override { }
bool readRow(MutableColumns & columns, RowReadExtension & ext) override;
bool allowSyncAfterError() const override { return true; }
void syncAfterError() override;

View File

@ -106,13 +106,14 @@ void CSVRowInputFormat::syncAfterError()
void CSVRowInputFormat::setReadBuffer(ReadBuffer & in_)
{
buf->setSubBuffer(in_);
buf = std::make_unique<PeekableReadBuffer>(in_);
RowInputFormatWithNamesAndTypes::setReadBuffer(*buf);
}
void CSVRowInputFormat::resetParser()
void CSVRowInputFormat::resetReadBuffer()
{
RowInputFormatWithNamesAndTypes::resetParser();
buf->reset();
buf.reset();
RowInputFormatWithNamesAndTypes::resetReadBuffer();
}
void CSVFormatReader::skipRow()

View File

@ -27,7 +27,7 @@ public:
String getName() const override { return "CSVRowInputFormat"; }
void setReadBuffer(ReadBuffer & in_) override;
void resetParser() override;
void resetReadBuffer() override;
protected:
CSVRowInputFormat(const Block & header_, std::shared_ptr<PeekableReadBuffer> in_, const Params & params_,

View File

@ -91,7 +91,14 @@ void CustomSeparatedRowInputFormat::syncAfterError()
void CustomSeparatedRowInputFormat::setReadBuffer(ReadBuffer & in_)
{
buf->setSubBuffer(in_);
buf = std::make_unique<PeekableReadBuffer>(in_);
RowInputFormatWithNamesAndTypes::setReadBuffer(*buf);
}
void CustomSeparatedRowInputFormat::resetReadBuffer()
{
buf.reset();
RowInputFormatWithNamesAndTypes::resetReadBuffer();
}
CustomSeparatedFormatReader::CustomSeparatedFormatReader(
@ -100,12 +107,6 @@ CustomSeparatedFormatReader::CustomSeparatedFormatReader(
{
}
void CustomSeparatedRowInputFormat::resetParser()
{
RowInputFormatWithNamesAndTypes::resetParser();
buf->reset();
}
void CustomSeparatedFormatReader::skipPrefixBeforeHeader()
{
skipSpaces();

View File

@ -18,9 +18,9 @@ public:
const Params & params_,
bool with_names_, bool with_types_, bool ignore_spaces_, const FormatSettings & format_settings_);
void resetParser() override;
String getName() const override { return "CustomSeparatedRowInputFormat"; }
void setReadBuffer(ReadBuffer & in_) override;
void resetReadBuffer() override;
private:
CustomSeparatedRowInputFormat(

View File

@ -888,7 +888,7 @@ void DWARFBlockInputFormat::parseRanges(
}
}
Chunk DWARFBlockInputFormat::generate()
Chunk DWARFBlockInputFormat::read()
{
initializeIfNeeded();

View File

@ -30,7 +30,7 @@ public:
size_t getApproxBytesReadForChunk() const override { return approx_bytes_read_for_chunk; }
protected:
Chunk generate() override;
Chunk read() override;
void onCancel() override
{

View File

@ -27,12 +27,18 @@ JSONAsRowInputFormat::JSONAsRowInputFormat(const Block & header_, std::unique_pt
header_.columns());
}
void JSONAsRowInputFormat::resetParser()
void JSONAsRowInputFormat::setReadBuffer(ReadBuffer & in_)
{
JSONEachRowRowInputFormat::resetParser();
buf->reset();
buf = std::make_unique<PeekableReadBuffer>(in_);
JSONEachRowRowInputFormat::setReadBuffer(*buf);
}
void JSONAsRowInputFormat::resetReadBuffer()
{
buf.reset();
JSONEachRowRowInputFormat::resetReadBuffer();
}
bool JSONAsRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &)
{
@ -68,12 +74,6 @@ bool JSONAsRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &)
return !buf->eof();
}
void JSONAsRowInputFormat::setReadBuffer(ReadBuffer & in_)
{
buf->setSubBuffer(in_);
}
JSONAsStringRowInputFormat::JSONAsStringRowInputFormat(
const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings_)
: JSONAsRowInputFormat(header_, in_, params_, format_settings_)

View File

@ -18,8 +18,8 @@ class JSONAsRowInputFormat : public JSONEachRowRowInputFormat
public:
JSONAsRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings);
void resetParser() override;
void setReadBuffer(ReadBuffer & in_) override;
void resetReadBuffer() override;
private:
JSONAsRowInputFormat(const Block & header_, std::unique_ptr<PeekableReadBuffer> buf_, Params params_, const FormatSettings & format_settings);

View File

@ -109,7 +109,7 @@ void JSONColumnsBlockInputFormatBase::setReadBuffer(ReadBuffer & in_)
IInputFormat::setReadBuffer(in_);
}
Chunk JSONColumnsBlockInputFormatBase::generate()
Chunk JSONColumnsBlockInputFormatBase::read()
{
MutableColumns columns = getPort().getHeader().cloneEmptyColumns();
block_missing_values.clear();

View File

@ -56,7 +56,7 @@ public:
size_t getApproxBytesReadForChunk() const override { return approx_bytes_read_for_chunk; }
protected:
Chunk generate() override;
Chunk read() override;
size_t readColumn(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, const String & column_name);

View File

@ -142,7 +142,7 @@ inline bool JSONEachRowRowInputFormat::advanceToNextKey(size_t key_index)
skipWhitespaceIfAny(*in);
if (in->eof())
throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected end of stream while parsing JSONEachRow format");
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected end of stream while parsing JSONEachRow format");
else if (*in->position() == '}')
{
++in->position();
@ -205,7 +205,7 @@ bool JSONEachRowRowInputFormat::readRow(MutableColumns & columns, RowReadExtensi
return false;
skipWhitespaceIfAny(*in);
bool is_first_row = getCurrentUnitNumber() == 0 && getTotalRows() == 1;
bool is_first_row = getRowNum() == 0;
if (checkEndOfData(is_first_row))
return false;
@ -308,7 +308,7 @@ size_t JSONEachRowRowInputFormat::countRows(size_t max_block_size)
return 0;
size_t num_rows = 0;
bool is_first_row = getCurrentUnitNumber() == 0 && getTotalRows() == 0;
bool is_first_row = getRowNum() == 0;
skipWhitespaceIfAny(*in);
while (num_rows < max_block_size && !checkEndOfData(is_first_row))
{

View File

@ -60,13 +60,14 @@ void JSONRowInputFormat::readSuffix()
void JSONRowInputFormat::setReadBuffer(DB::ReadBuffer & in_)
{
peekable_buf->setSubBuffer(in_);
peekable_buf = std::make_unique<PeekableReadBuffer>(in_);
JSONEachRowRowInputFormat::setReadBuffer(*peekable_buf);
}
void JSONRowInputFormat::resetParser()
void JSONRowInputFormat::resetReadBuffer()
{
JSONEachRowRowInputFormat::resetParser();
peekable_buf->reset();
peekable_buf.reset();
JSONEachRowRowInputFormat::resetReadBuffer();
}
JSONRowSchemaReader::JSONRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_)

View File

@ -24,7 +24,7 @@ public:
String getName() const override { return "JSONRowInputFormat"; }
void setReadBuffer(ReadBuffer & in_) override;
void resetParser() override;
void resetReadBuffer() override;
private:
JSONRowInputFormat(

View File

@ -56,10 +56,21 @@ MsgPackRowInputFormat::MsgPackRowInputFormat(const Block & header_, std::unique_
void MsgPackRowInputFormat::resetParser()
{
IRowInputFormat::resetParser();
buf->reset();
visitor.reset();
}
void MsgPackRowInputFormat::setReadBuffer(ReadBuffer & in_)
{
buf = std::make_unique<PeekableReadBuffer>(in_);
IRowInputFormat::setReadBuffer(*buf);
}
void MsgPackRowInputFormat::resetReadBuffer()
{
buf.reset();
IRowInputFormat::resetReadBuffer();
}
void MsgPackVisitor::set_info(IColumn & column, DataTypePtr type, UInt8 & read) // NOLINT
{
while (!info_stack.empty())
@ -543,11 +554,6 @@ bool MsgPackRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &
return true;
}
void MsgPackRowInputFormat::setReadBuffer(ReadBuffer & in_)
{
buf->setSubBuffer(in_);
}
MsgPackSchemaReader::MsgPackSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_)
: IRowSchemaReader(buf, format_settings_), buf(in_), number_of_columns(format_settings_.msgpack.number_of_columns)
{

View File

@ -69,6 +69,7 @@ public:
String getName() const override { return "MagPackRowInputFormat"; }
void resetParser() override;
void setReadBuffer(ReadBuffer & in_) override;
void resetReadBuffer() override;
private:
MsgPackRowInputFormat(const Block & header_, std::unique_ptr<PeekableReadBuffer> buf_, Params params_, const FormatSettings & settings);

View File

@ -35,7 +35,7 @@ public:
reader->resetParser();
}
Chunk generate() override
Chunk read() override
{
block_missing_values.clear();
size_t block_start = getDataOffsetMaybeCompressed(*in);

View File

@ -905,7 +905,7 @@ bool NativeORCBlockInputFormat::prepareStripeReader()
return true;
}
Chunk NativeORCBlockInputFormat::generate()
Chunk NativeORCBlockInputFormat::read()
{
block_missing_values.clear();

View File

@ -62,7 +62,7 @@ public:
size_t getApproxBytesReadForChunk() const override { return approx_bytes_read_for_chunk; }
protected:
Chunk generate() override;
Chunk read() override;
void onCancel() override { is_stopped = 1; }

View File

@ -27,7 +27,7 @@ ORCBlockInputFormat::ORCBlockInputFormat(ReadBuffer & in_, Block header_, const
{
}
Chunk ORCBlockInputFormat::generate()
Chunk ORCBlockInputFormat::read()
{
block_missing_values.clear();
@ -48,7 +48,7 @@ Chunk ORCBlockInputFormat::generate()
auto batch_result = file_reader->ReadStripe(stripe_current, include_indices);
if (!batch_result.ok())
throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Failed to create batch reader: {}", batch_result.status().ToString());
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Failed to create batch reader: {}", batch_result.status().ToString());
auto batch = batch_result.ValueOrDie();
if (!batch)
@ -56,7 +56,7 @@ Chunk ORCBlockInputFormat::generate()
auto table_result = arrow::Table::FromRecordBatches({batch});
if (!table_result.ok())
throw ParsingException(
throw Exception(
ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading batch of ORC data: {}", table_result.status().ToString());
/// We should extract the number of rows directly from the stripe, because in case when

View File

@ -32,7 +32,7 @@ public:
size_t getApproxBytesReadForChunk() const override { return approx_bytes_read_for_chunk; }
protected:
Chunk generate() override;
Chunk read() override;
void onCancel() override
{

View File

@ -23,7 +23,7 @@ OneInputFormat::OneInputFormat(const Block & header, ReadBuffer & in_) : IInputF
header.getByPosition(0).type->getName());
}
Chunk OneInputFormat::generate()
Chunk OneInputFormat::read()
{
if (done)
return {};

View File

@ -14,7 +14,7 @@ public:
String getName() const override { return "One"; }
protected:
Chunk generate() override;
Chunk read() override;
private:
bool done = false;

View File

@ -61,7 +61,7 @@ void ParallelParsingInputFormat::segmentatorThreadFunction(ThreadGroupPtr thread
}
catch (...)
{
onBackgroundException(successfully_read_rows_count);
onBackgroundException();
}
}
@ -90,7 +90,7 @@ void ParallelParsingInputFormat::parserThreadFunction(ThreadGroupPtr thread_grou
ReadBuffer read_buffer(unit.segment.data(), unit.segment.size(), 0);
InputFormatPtr input_format = internal_parser_creator(read_buffer);
input_format->setCurrentUnitNumber(current_ticket_number);
input_format->setRowsReadBefore(unit.offset);
input_format->setErrorsLogger(errors_logger);
InternalParser parser(input_format);
@ -132,28 +132,16 @@ void ParallelParsingInputFormat::parserThreadFunction(ThreadGroupPtr thread_grou
}
catch (...)
{
onBackgroundException(unit.offset);
onBackgroundException();
}
}
void ParallelParsingInputFormat::onBackgroundException(size_t offset)
void ParallelParsingInputFormat::onBackgroundException()
{
std::lock_guard lock(mutex);
if (!background_exception)
{
background_exception = std::current_exception();
if (ParsingException * e = exception_cast<ParsingException *>(background_exception))
{
/// NOTE: it is not that safe to use line number hack here (may exceed INT_MAX)
if (e->getLineNumber() != -1)
e->setLineNumber(static_cast<int>(e->getLineNumber() + offset));
auto file_name = getFileNameFromReadBuffer(getReadBuffer());
if (!file_name.empty())
e->setFileName(file_name);
}
}
if (is_server)
tryLogCurrentException(__PRETTY_FUNCTION__);
@ -164,7 +152,7 @@ void ParallelParsingInputFormat::onBackgroundException(size_t offset)
segmentator_condvar.notify_all();
}
Chunk ParallelParsingInputFormat::generate()
Chunk ParallelParsingInputFormat::read()
{
/// Delayed launching of segmentator thread
if (unlikely(!parsing_started.exchange(true)))

View File

@ -135,7 +135,7 @@ public:
private:
Chunk generate() override final;
Chunk read() override final;
void onCancel() override final
{
@ -333,7 +333,7 @@ private:
/// threads. This function is used by segmentator and parsed threads.
/// readImpl() is called from the main thread, so the exception handling
/// is different.
void onBackgroundException(size_t offset);
void onBackgroundException();
};
}

View File

@ -570,7 +570,7 @@ void ParquetBlockInputFormat::decodeOneChunk(size_t row_group_batch_idx, std::un
// We may be able to schedule more work now, but can't call scheduleMoreWorkIfNeeded() right
// here because we're running on the same thread pool, so it'll deadlock if thread limit is
// reached. Wake up generate() instead.
// reached. Wake up read() instead.
condvar.notify_all();
};
@ -579,7 +579,7 @@ void ParquetBlockInputFormat::decodeOneChunk(size_t row_group_batch_idx, std::un
auto batch = row_group_batch.record_batch_reader->Next();
if (!batch.ok())
throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading Parquet data: {}", batch.status().ToString());
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading Parquet data: {}", batch.status().ToString());
if (!*batch)
{
@ -637,7 +637,7 @@ void ParquetBlockInputFormat::scheduleMoreWorkIfNeeded(std::optional<size_t> row
}
}
Chunk ParquetBlockInputFormat::generate()
Chunk ParquetBlockInputFormat::read()
{
initializeIfNeeded();

View File

@ -65,7 +65,7 @@ public:
size_t getApproxBytesReadForChunk() const override { return previous_approx_bytes_read_for_chunk; }
private:
Chunk generate() override;
Chunk read() override;
void onCancel() override
{
@ -142,7 +142,7 @@ private:
// reading its data (using RAM). Row group becomes inactive when we finish reading and
// delivering all its blocks and free the RAM. Size of the window is max_decoding_threads.
//
// Decoded blocks are placed in `pending_chunks` queue, then picked up by generate().
// Decoded blocks are placed in `pending_chunks` queue, then picked up by read().
// If row group decoding runs too far ahead of delivery (by `max_pending_chunks_per_row_group`
// chunks), we pause the stream for the row group, to avoid using too much memory when decoded
// chunks are much bigger than the compressed data.
@ -150,7 +150,7 @@ private:
// Also:
// * If preserve_order = true, we deliver chunks strictly in order of increasing row group.
// Decoding may still proceed in later row groups.
// * If max_decoding_threads <= 1, we run all tasks inline in generate(), without thread pool.
// * If max_decoding_threads <= 1, we run all tasks inline in read(), without thread pool.
// Potential improvements:
// * Plan all read ranges ahead of time, for the whole file, and do prefetching for them
@ -189,7 +189,7 @@ private:
Status status = Status::NotStarted;
// Window of chunks that were decoded but not returned from generate():
// Window of chunks that were decoded but not returned from read():
//
// (delivered) next_chunk_idx
// v v v
@ -215,7 +215,7 @@ private:
std::unique_ptr<ArrowColumnToCHColumn> arrow_column_to_ch_column;
};
// Chunk ready to be delivered by generate().
// Chunk ready to be delivered by read().
struct PendingChunk
{
Chunk chunk;
@ -265,7 +265,7 @@ private:
// Done NotStarted
std::mutex mutex;
// Wakes up the generate() call, if any.
// Wakes up the read() call, if any.
std::condition_variable condvar;
std::vector<RowGroupBatchState> row_group_batches;

View File

@ -140,7 +140,7 @@ ParquetMetadataInputFormat::ParquetMetadataInputFormat(ReadBuffer & in_, Block h
checkHeader(getPort().getHeader());
}
Chunk ParquetMetadataInputFormat::generate()
Chunk ParquetMetadataInputFormat::read()
{
Chunk res;
if (done)

View File

@ -63,7 +63,7 @@ public:
void resetParser() override;
private:
Chunk generate() override;
Chunk read() override;
void onCancel() override
{

View File

@ -61,7 +61,7 @@ bool ProtobufListInputFormat::readRow(MutableColumns & columns, RowReadExtension
size_t ProtobufListInputFormat::countRows(size_t max_block_size)
{
if (getTotalRows() == 0)
if (getRowNum() == 0)
reader->startMessage(true);
if (reader->eof())

View File

@ -84,10 +84,16 @@ RegexpRowInputFormat::RegexpRowInputFormat(
{
}
void RegexpRowInputFormat::resetParser()
void RegexpRowInputFormat::setReadBuffer(ReadBuffer & in_)
{
IRowInputFormat::resetParser();
buf->reset();
buf = std::make_unique<PeekableReadBuffer>(in_);
IRowInputFormat::setReadBuffer(*buf);
}
void RegexpRowInputFormat::resetReadBuffer()
{
buf.reset();
IRowInputFormat::resetReadBuffer();
}
bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns)
@ -128,11 +134,6 @@ bool RegexpRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &
return true;
}
void RegexpRowInputFormat::setReadBuffer(ReadBuffer & in_)
{
buf->setSubBuffer(in_);
}
RegexpSchemaReader::RegexpSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_)
: IRowSchemaReader(
buf,

View File

@ -61,8 +61,8 @@ public:
RegexpRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_);
String getName() const override { return "RegexpRowInputFormat"; }
void resetParser() override;
void setReadBuffer(ReadBuffer & in_) override;
void resetReadBuffer() override;
private:
RegexpRowInputFormat(std::unique_ptr<PeekableReadBuffer> buf_, const Block & header_, Params params_, const FormatSettings & format_settings_);

View File

@ -92,7 +92,7 @@ static bool readName(ReadBuffer & buf, StringRef & ref, String & tmp)
}
}
throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected end of stream while reading key name from TSKV format");
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected end of stream while reading key name from TSKV format");
}
@ -161,7 +161,7 @@ bool TSKVRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ex
if (in->eof())
{
throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected end of stream after field in TSKV format: {}", name_ref.toString());
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected end of stream after field in TSKV format: {}", name_ref.toString());
}
else if (*in->position() == '\t')
{

View File

@ -67,13 +67,14 @@ TabSeparatedRowInputFormat::TabSeparatedRowInputFormat(
void TabSeparatedRowInputFormat::setReadBuffer(ReadBuffer & in_)
{
buf->setSubBuffer(in_);
buf = std::make_unique<PeekableReadBuffer>(in_);
RowInputFormatWithNamesAndTypes::setReadBuffer(*buf);
}
void TabSeparatedRowInputFormat::resetParser()
void TabSeparatedRowInputFormat::resetReadBuffer()
{
RowInputFormatWithNamesAndTypes::resetParser();
buf->reset();
buf.reset();
RowInputFormatWithNamesAndTypes::resetReadBuffer();
}
TabSeparatedFormatReader::TabSeparatedFormatReader(PeekableReadBuffer & in_, const FormatSettings & format_settings_, bool is_raw_)

View File

@ -23,7 +23,7 @@ public:
String getName() const override { return "TabSeparatedRowInputFormat"; }
void setReadBuffer(ReadBuffer & in_) override;
void resetParser() override;
void resetReadBuffer() override;
private:
TabSeparatedRowInputFormat(const Block & header_, std::unique_ptr<PeekableReadBuffer> in_, const Params & params_,

View File

@ -21,7 +21,7 @@ namespace ErrorCodes
[[noreturn]] static void throwUnexpectedEof(size_t row_num)
{
throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected EOF while parsing row {}. "
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected EOF while parsing row {}. "
"Maybe last row has wrong format or input doesn't contain specified suffix before EOF.",
std::to_string(row_num));
}
@ -121,7 +121,7 @@ bool TemplateRowInputFormat::readRow(MutableColumns & columns, RowReadExtension
updateDiagnosticInfo();
if (likely(row_num != 1))
if (likely(getRowNum() != 0))
format_reader->skipRowBetweenDelimiter();
extra.read_columns.assign(columns.size(), false);
@ -160,7 +160,7 @@ bool TemplateRowInputFormat::deserializeField(const DataTypePtr & type,
catch (Exception & e)
{
if (e.code() == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF)
throwUnexpectedEof(row_num);
throwUnexpectedEof(getRowNum());
throw;
}
}
@ -198,7 +198,7 @@ bool TemplateRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & col
out << "\nUsing format string (from format_schema_rows): " << row_format.dump() << "\n";
out << "\nTrying to parse next row, because suffix does not match:\n";
if (likely(row_num != 1) && !parseDelimiterWithDiagnosticInfo(out, *buf, row_between_delimiter, "delimiter between rows", ignore_spaces))
if (likely(getRowNum() != 0) && !parseDelimiterWithDiagnosticInfo(out, *buf, row_between_delimiter, "delimiter between rows", ignore_spaces))
return false;
for (size_t i = 0; i < row_format.columnsCount(); ++i)
@ -288,12 +288,19 @@ void TemplateRowInputFormat::resetParser()
{
RowInputFormatWithDiagnosticInfo::resetParser();
end_of_stream = false;
buf->reset();
}
void TemplateRowInputFormat::setReadBuffer(ReadBuffer & in_)
{
buf->setSubBuffer(in_);
buf = std::make_unique<PeekableReadBuffer>(in_);
RowInputFormatWithDiagnosticInfo::setReadBuffer(*buf);
format_reader->setReadBuffer(*buf);
}
void TemplateRowInputFormat::resetReadBuffer()
{
buf.reset();
RowInputFormatWithDiagnosticInfo::resetReadBuffer();
}
TemplateFormatReader::TemplateFormatReader(

View File

@ -28,6 +28,8 @@ public:
String getName() const override { return "TemplateRowInputFormat"; }
void resetParser() override;
void setReadBuffer(ReadBuffer & in_) override;
void resetReadBuffer() override;
private:
TemplateRowInputFormat(const Block & header_, std::unique_ptr<PeekableReadBuffer> buf_, const Params & params_,
@ -50,8 +52,6 @@ private:
bool isGarbageAfterField(size_t after_col_idx, ReadBuffer::Position pos) override;
void setReadBuffer(ReadBuffer & in_) override;
std::unique_ptr<PeekableReadBuffer> buf;
const DataTypes data_types;

View File

@ -98,7 +98,7 @@ bool ValuesBlockInputFormat::skipToNextRow(ReadBuffer * buf, size_t min_chunk_by
return true;
}
Chunk ValuesBlockInputFormat::generate()
Chunk ValuesBlockInputFormat::read()
{
if (total_rows == 0)
readPrefix();
@ -642,13 +642,19 @@ void ValuesBlockInputFormat::resetParser()
IInputFormat::resetParser();
// I'm not resetting parser modes here.
// There is a good chance that all messages have the same format.
buf->reset();
total_rows = 0;
}
void ValuesBlockInputFormat::setReadBuffer(ReadBuffer & in_)
{
buf->setSubBuffer(in_);
buf = std::make_unique<PeekableReadBuffer>(in_);
IInputFormat::setReadBuffer(*buf);
}
void ValuesBlockInputFormat::resetReadBuffer()
{
buf.reset();
IInputFormat::resetReadBuffer();
}
ValuesSchemaReader::ValuesSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_)

View File

@ -34,6 +34,7 @@ public:
void resetParser() override;
void setReadBuffer(ReadBuffer & in_) override;
void resetReadBuffer() override;
/// TODO: remove context somehow.
void setContext(ContextPtr & context_) { context = Context::createCopy(context_); }
@ -57,7 +58,7 @@ private:
using ConstantExpressionTemplates = std::vector<std::optional<ConstantExpressionTemplate>>;
Chunk generate() override;
Chunk read() override;
void readRow(MutableColumns & columns, size_t row_num);
void readUntilTheEndOfRowAndReTokenize(size_t current_column_idx);

View File

@ -26,8 +26,6 @@ RowInputFormatWithDiagnosticInfo::RowInputFormatWithDiagnosticInfo(const Block &
void RowInputFormatWithDiagnosticInfo::updateDiagnosticInfo()
{
++row_num;
bytes_read_at_start_of_buffer_on_prev_row = bytes_read_at_start_of_buffer_on_current_row;
bytes_read_at_start_of_buffer_on_current_row = in->count() - in->offset();
@ -73,7 +71,7 @@ std::pair<String, String> RowInputFormatWithDiagnosticInfo::getDiagnosticAndRawD
{
in->position() = in->buffer().begin() + offset_of_prev_row;
out_diag << "\nRow " << (row_num - 1) << ":\n";
out_diag << "\nRow " << getRowNum() - 1 << ":\n";
if (!parseRowAndPrintDiagnosticInfo(columns, out_diag))
return std::make_pair(out_diag.str(), out_data.str());
}
@ -96,7 +94,7 @@ std::pair<String, String> RowInputFormatWithDiagnosticInfo::getDiagnosticAndRawD
++data;
}
out_diag << "\nRow " << row_num << ":\n";
out_diag << "\nRow " << getRowNum() << ":\n";
parseRowAndPrintDiagnosticInfo(columns, out_diag);
out_diag << "\n";
@ -193,7 +191,6 @@ bool RowInputFormatWithDiagnosticInfo::deserializeFieldAndPrintDiagnosticInfo(co
void RowInputFormatWithDiagnosticInfo::resetParser()
{
IRowInputFormat::resetParser();
row_num = 0;
bytes_read_at_start_of_buffer_on_current_row = 0;
bytes_read_at_start_of_buffer_on_prev_row = 0;
offset_of_current_row = std::numeric_limits<size_t>::max();

View File

@ -29,9 +29,6 @@ protected:
virtual void tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) = 0;
virtual bool isGarbageAfterField(size_t after_input_pos_idx, ReadBuffer::Position pos) = 0;
/// For convenient diagnostics in case of an error.
size_t row_num = 0;
private:
/// How many bytes were read, not counting those still in the buffer.
size_t bytes_read_at_start_of_buffer_on_current_row = 0;

View File

@ -66,11 +66,6 @@ RowInputFormatWithNamesAndTypes::RowInputFormatWithNamesAndTypes(
void RowInputFormatWithNamesAndTypes::readPrefix()
{
/// This is a bit of abstraction leakage, but we need it in parallel parsing:
/// we check if this InputFormat is working with the "real" beginning of the data.
if (getCurrentUnitNumber() != 0)
return;
/// Search and remove BOM only in textual formats (CSV, TSV etc), not in binary ones (RowBinary*).
/// Also, we assume that column name or type cannot contain BOM, so, if format has header,
/// then BOM at beginning of stream cannot be confused with name or type of field, and it is safe to skip it.
@ -206,7 +201,7 @@ bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadE
updateDiagnosticInfo();
if (likely(row_num != 1 || getCurrentUnitNumber() != 0 || (getCurrentUnitNumber() == 0 && (with_names || with_types || is_header_detected))))
if (likely(getRowNum() != 0 || with_names || with_types || is_header_detected))
format_reader->skipRowBetweenDelimiter();
format_reader->skipRowStartDelimiter();
@ -270,7 +265,7 @@ size_t RowInputFormatWithNamesAndTypes::countRows(size_t max_block_size)
return 0;
size_t num_rows = 0;
bool is_first_row = getTotalRows() == 0 && !with_names && !with_types && !is_header_detected;
bool is_first_row = getRowNum() == 0 && !with_names && !with_types && !is_header_detected;
while (!format_reader->checkForSuffix() && num_rows < max_block_size)
{
if (likely(!is_first_row))
@ -323,7 +318,7 @@ bool RowInputFormatWithNamesAndTypes::parseRowAndPrintDiagnosticInfo(MutableColu
if (!format_reader->tryParseSuffixWithDiagnosticInfo(out))
return false;
if (likely(row_num != 1) && !format_reader->parseRowBetweenDelimiterWithDiagnosticInfo(out))
if (likely(getRowNum() != 0) && !format_reader->parseRowBetweenDelimiterWithDiagnosticInfo(out))
return false;
if (!format_reader->parseRowStartWithDiagnosticInfo(out))

View File

@ -1,5 +1,6 @@
#include <Processors/Transforms/FillingTransform.h>
#include <Interpreters/convertFieldToType.h>
#include <Interpreters/ExpressionActions.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/IDataType.h>

View File

@ -9,6 +9,9 @@
namespace DB
{
class ExpressionActions;
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
/** Implements modifier WITH FILL of ORDER BY clause.
* It fills gaps in data stream by rows with missing values in columns with set WITH FILL and default values in other columns.
* Optionally FROM, TO and STEP values can be specified.

View File

@ -29,38 +29,38 @@ void ProxyV1Handler::run()
// read "PROXY"
if (!readWord(5, word, eol) || word != "PROXY" || eol)
throw ParsingException(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "PROXY protocol violation");
throw Exception(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "PROXY protocol violation");
// read "TCP4" or "TCP6" or "UNKNOWN"
if (!readWord(7, word, eol))
throw ParsingException(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "PROXY protocol violation");
throw Exception(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "PROXY protocol violation");
if (word != "TCP4" && word != "TCP6" && word != "UNKNOWN")
throw ParsingException(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "PROXY protocol violation");
throw Exception(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "PROXY protocol violation");
if (word == "UNKNOWN" && eol)
return;
if (eol)
throw ParsingException(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "PROXY protocol violation");
throw Exception(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "PROXY protocol violation");
// read address
if (!readWord(39, word, eol) || eol)
throw ParsingException(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "PROXY protocol violation");
throw Exception(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "PROXY protocol violation");
stack_data.forwarded_for = std::move(word);
// read address
if (!readWord(39, word, eol) || eol)
throw ParsingException(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "PROXY protocol violation");
throw Exception(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "PROXY protocol violation");
// read port
if (!readWord(5, word, eol) || eol)
throw ParsingException(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "PROXY protocol violation");
throw Exception(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "PROXY protocol violation");
// read port and "\r\n"
if (!readWord(5, word, eol) || !eol)
throw ParsingException(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "PROXY protocol violation");
throw Exception(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "PROXY protocol violation");
if (!stack_data.forwarded_for.empty())
LOG_TRACE(log, "Forwarded client address from PROXY header: {}", stack_data.forwarded_for);

View File

@ -2784,6 +2784,25 @@ void MergeTreeData::dropAllData()
column_sizes.clear();
auto detached_parts = getDetachedParts();
for (const auto & part : detached_parts)
{
bool is_zero_copy = supportsReplication() && part.disk->supportZeroCopyReplication()
&& settings_ptr->allow_remote_fs_zero_copy_replication;
try
{
bool keep_shared = removeDetachedPart(part.disk, fs::path(relative_data_path) / "detached" / part.dir_name / "", part.dir_name);
LOG_DEBUG(log, "Dropped detached part {}, keep shared data: {}", part.dir_name, keep_shared);
}
catch (...)
{
/// Without zero-copy-replication we will simply remove it recursively, but with zero-copy it will leave garbage on s3
if (is_zero_copy && isRetryableException(std::current_exception()))
throw;
tryLogCurrentException(log);
}
}
for (const auto & disk : getDisks())
{
if (disk->isBroken())
@ -2801,7 +2820,7 @@ void MergeTreeData::dropAllData()
disk->removeFileIfExists(fs::path(relative_data_path) / FORMAT_VERSION_FILE_NAME);
if (disk->exists(fs::path(relative_data_path) / DETACHED_DIR_NAME))
disk->removeRecursive(fs::path(relative_data_path) / DETACHED_DIR_NAME);
disk->removeSharedRecursive(fs::path(relative_data_path) / DETACHED_DIR_NAME, /*keep_all_shared_data*/ true, {});
if (disk->exists(fs::path(relative_data_path) / MOVING_DIR_NAME))
disk->removeRecursive(fs::path(relative_data_path) / MOVING_DIR_NAME);

View File

@ -216,6 +216,10 @@ size_t MergeTreeReaderCompact::readRows(
{
size_t rows_to_read = data_part_info_for_read->getIndexGranularity().getMarkRows(from_mark);
/// If we need to read multiple subcolumns from a single column in storage,
/// we will read it this column only once and then reuse to extract all subcolumns.
std::unordered_map<String, ColumnPtr> columns_cache_for_subcolumns;
for (size_t pos = 0; pos < num_columns; ++pos)
{
if (!res_columns[pos])
@ -226,7 +230,7 @@ size_t MergeTreeReaderCompact::readRows(
auto & column = res_columns[pos];
size_t column_size_before_reading = column->size();
readData(columns_to_read[pos], column, from_mark, current_task_last_mark, *column_positions[pos], rows_to_read, columns_for_offsets[pos]);
readData(columns_to_read[pos], column, from_mark, current_task_last_mark, *column_positions[pos], rows_to_read, columns_for_offsets[pos], columns_cache_for_subcolumns);
size_t read_rows_in_column = column->size() - column_size_before_reading;
if (read_rows_in_column != rows_to_read)
@ -265,7 +269,7 @@ size_t MergeTreeReaderCompact::readRows(
void MergeTreeReaderCompact::readData(
const NameAndTypePair & name_and_type, ColumnPtr & column,
size_t from_mark, size_t current_task_last_mark, size_t column_position, size_t rows_to_read,
ColumnNameLevel name_level_for_offsets)
ColumnNameLevel name_level_for_offsets, std::unordered_map<String, ColumnPtr> & columns_cache_for_subcolumns)
{
const auto & [name, type] = name_and_type;
std::optional<NameAndTypePair> column_for_offsets;
@ -327,34 +331,54 @@ void MergeTreeReaderCompact::readData(
ISerialization::DeserializeBinaryBulkSettings deserialize_settings;
deserialize_settings.avg_value_size_hint = avg_value_size_hints[name];
bool columns_cache_was_used = false;
if (name_and_type.isSubcolumn())
{
NameAndTypePair name_type_in_storage{name_and_type.getNameInStorage(), name_and_type.getTypeInStorage()};
ColumnPtr temp_column;
/// In case of reading onlys offset use the correct serialization for reading of the prefix
auto serialization = getSerializationInPart(name_type_in_storage);
ColumnPtr temp_column = name_type_in_storage.type->createColumn(*serialization);
if (column_for_offsets)
auto it = columns_cache_for_subcolumns.find(name_type_in_storage.name);
if (!column_for_offsets && it != columns_cache_for_subcolumns.end())
{
auto serialization_for_prefix = getSerializationInPart(*column_for_offsets);
temp_column = it->second;
auto subcolumn = name_type_in_storage.type->getSubcolumn(name_and_type.getSubcolumnName(), temp_column);
if (column->empty())
column = IColumn::mutate(subcolumn);
else
column->assumeMutable()->insertRangeFrom(*subcolumn, 0, subcolumn->size());
deserialize_settings.getter = buffer_getter_for_prefix;
serialization_for_prefix->deserializeBinaryBulkStatePrefix(deserialize_settings, state_for_prefix);
columns_cache_was_used = true;
}
deserialize_settings.getter = buffer_getter;
serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, state);
serialization->deserializeBinaryBulkWithMultipleStreams(temp_column, rows_to_read, deserialize_settings, state, nullptr);
auto subcolumn = name_type_in_storage.type->getSubcolumn(name_and_type.getSubcolumnName(), temp_column);
/// TODO: Avoid extra copying.
if (column->empty())
column = subcolumn;
else
column->assumeMutable()->insertRangeFrom(*subcolumn, 0, subcolumn->size());
{
/// In case of reading only offset use the correct serialization for reading of the prefix
auto serialization = getSerializationInPart(name_type_in_storage);
temp_column = name_type_in_storage.type->createColumn(*serialization);
if (column_for_offsets)
{
auto serialization_for_prefix = getSerializationInPart(*column_for_offsets);
deserialize_settings.getter = buffer_getter_for_prefix;
serialization_for_prefix->deserializeBinaryBulkStatePrefix(deserialize_settings, state_for_prefix);
}
deserialize_settings.getter = buffer_getter;
serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, state);
serialization->deserializeBinaryBulkWithMultipleStreams(temp_column, rows_to_read, deserialize_settings, state, nullptr);
if (!column_for_offsets)
columns_cache_for_subcolumns[name_type_in_storage.name] = temp_column;
auto subcolumn = name_type_in_storage.type->getSubcolumn(name_and_type.getSubcolumnName(), temp_column);
/// TODO: Avoid extra copying.
if (column->empty())
column = subcolumn;
else
column->assumeMutable()->insertRangeFrom(*subcolumn, 0, subcolumn->size());
}
}
else
{
@ -374,8 +398,8 @@ void MergeTreeReaderCompact::readData(
serialization->deserializeBinaryBulkWithMultipleStreams(column, rows_to_read, deserialize_settings, state, nullptr);
}
/// The buffer is left in inconsistent state after reading single offsets
if (name_level_for_offsets.has_value())
/// The buffer is left in inconsistent state after reading single offsets or using columns cache during subcolumns reading.
if (name_level_for_offsets.has_value() || columns_cache_was_used)
last_read_granule.reset();
else
last_read_granule.emplace(from_mark, column_position);

View File

@ -76,7 +76,7 @@ private:
void readData(const NameAndTypePair & name_and_type, ColumnPtr & column, size_t from_mark,
size_t current_task_last_mark, size_t column_position,
size_t rows_to_read, ColumnNameLevel name_level_for_offsets);
size_t rows_to_read, ColumnNameLevel name_level_for_offsets, std::unordered_map<String, ColumnPtr> & columns_cache_for_subcolumns);
/// Returns maximal value of granule size in compressed file from @mark_ranges.
/// This value is used as size of read buffer.

View File

@ -16,6 +16,7 @@
#include <IO/S3Common.h>
#include <Common/CurrentMetrics.h>
#include <Common/SipHash.h>
#include <Common/ZooKeeper/IKeeper.h>
#include <Poco/Net/NetException.h>
#if USE_AZURE_BLOB_STORAGE
@ -80,6 +81,11 @@ bool isRetryableException(const std::exception_ptr exception_ptr)
if (e.getErrno() == EMFILE)
return true;
}
catch (const Coordination::Exception & e)
{
if (Coordination::isHardwareError(e.code))
return true;
}
catch (const Exception & e)
{
if (isNotEnoughMemoryErrorCode(e.code()))

View File

@ -80,7 +80,6 @@ namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int NOT_IMPLEMENTED;
extern const int ILLEGAL_PREWHERE;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int SAMPLING_NOT_SUPPORTED;
extern const int ALTER_OF_COLUMN_IS_FORBIDDEN;
@ -88,6 +87,20 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
StorageMerge::DatabaseNameOrRegexp::DatabaseNameOrRegexp(
const String & source_database_name_or_regexp_,
bool database_is_regexp_,
std::optional<OptimizedRegularExpression> source_database_regexp_,
std::optional<OptimizedRegularExpression> source_table_regexp_,
std::optional<DBToTableSetMap> source_databases_and_tables_)
: source_database_name_or_regexp(source_database_name_or_regexp_)
, database_is_regexp(database_is_regexp_)
, source_database_regexp(std::move(source_database_regexp_))
, source_table_regexp(std::move(source_table_regexp_))
, source_databases_and_tables(std::move(source_databases_and_tables_))
{
}
StorageMerge::StorageMerge(
const StorageID & table_id_,
const ColumnsDescription & columns_,
@ -98,10 +111,11 @@ StorageMerge::StorageMerge(
ContextPtr context_)
: IStorage(table_id_)
, WithContext(context_->getGlobalContext())
, source_database_regexp(source_database_name_or_regexp_)
, source_databases_and_tables(source_databases_and_tables_)
, source_database_name_or_regexp(source_database_name_or_regexp_)
, database_is_regexp(database_is_regexp_)
, database_name_or_regexp(
source_database_name_or_regexp_,
database_is_regexp_,
source_database_name_or_regexp_, {},
source_databases_and_tables_)
{
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(columns_.empty() ? getColumnsDescriptionFromSourceTables() : columns_);
@ -119,10 +133,11 @@ StorageMerge::StorageMerge(
ContextPtr context_)
: IStorage(table_id_)
, WithContext(context_->getGlobalContext())
, source_database_regexp(source_database_name_or_regexp_)
, source_table_regexp(source_table_regexp_)
, source_database_name_or_regexp(source_database_name_or_regexp_)
, database_is_regexp(database_is_regexp_)
, database_name_or_regexp(
source_database_name_or_regexp_,
database_is_regexp_,
source_database_name_or_regexp_,
source_table_regexp_, {})
{
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(columns_.empty() ? getColumnsDescriptionFromSourceTables() : columns_);
@ -130,6 +145,11 @@ StorageMerge::StorageMerge(
setInMemoryMetadata(storage_metadata);
}
StorageMerge::DatabaseTablesIterators StorageMerge::getDatabaseIterators(ContextPtr context_) const
{
return database_name_or_regexp.getDatabaseIterators(context_);
}
ColumnsDescription StorageMerge::getColumnsDescriptionFromSourceTables() const
{
auto table = getFirstTable([](auto && t) { return t; });
@ -141,7 +161,7 @@ ColumnsDescription StorageMerge::getColumnsDescriptionFromSourceTables() const
template <typename F>
StoragePtr StorageMerge::getFirstTable(F && predicate) const
{
auto database_table_iterators = getDatabaseIterators(getContext());
auto database_table_iterators = database_name_or_regexp.getDatabaseIterators(getContext());
for (auto & iterator : database_table_iterators)
{
@ -236,7 +256,6 @@ std::optional<NameSet> StorageMerge::supportedPrewhereColumns() const
return supported_columns;
}
QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage(
ContextPtr local_context,
QueryProcessingStage::Enum to_stage,
@ -255,7 +274,7 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage(
auto stage_in_source_tables = QueryProcessingStage::FetchColumns;
DatabaseTablesIterators database_table_iterators = getDatabaseIterators(local_context);
DatabaseTablesIterators database_table_iterators = database_name_or_regexp.getDatabaseIterators(local_context);
size_t selected_table_size = 0;
@ -297,45 +316,6 @@ void StorageMerge::read(
*/
auto modified_context = Context::createCopy(local_context);
modified_context->setSetting("optimize_move_to_prewhere", false);
bool has_database_virtual_column = false;
bool has_table_virtual_column = false;
Names real_column_names;
real_column_names.reserve(column_names.size());
for (const auto & column_name : column_names)
{
if (column_name == "_database" && isVirtualColumn(column_name, storage_snapshot->metadata))
has_database_virtual_column = true;
else if (column_name == "_table" && isVirtualColumn(column_name, storage_snapshot->metadata))
has_table_virtual_column = true;
else
real_column_names.push_back(column_name);
}
StorageListWithLocks selected_tables
= getSelectedTables(modified_context, query_info.query, has_database_virtual_column, has_table_virtual_column);
InputOrderInfoPtr input_sorting_info;
if (query_info.order_optimizer)
{
for (auto it = selected_tables.begin(); it != selected_tables.end(); ++it)
{
auto storage_ptr = std::get<1>(*it);
auto storage_metadata_snapshot = storage_ptr->getInMemoryMetadataPtr();
auto current_info = query_info.order_optimizer->getInputOrder(storage_metadata_snapshot, modified_context);
if (it == selected_tables.begin())
input_sorting_info = current_info;
else if (!current_info || (input_sorting_info && *current_info != *input_sorting_info))
input_sorting_info.reset();
if (!input_sorting_info)
break;
}
query_info.input_order_info = input_sorting_info;
}
query_plan.addInterpreterContext(modified_context);
/// What will be result structure depending on query processed stage in source tables?
@ -343,10 +323,7 @@ void StorageMerge::read(
auto step = std::make_unique<ReadFromMerge>(
common_header,
std::move(selected_tables),
real_column_names,
has_database_virtual_column,
has_table_virtual_column,
column_names,
max_block_size,
num_streams,
shared_from_this(),
@ -358,43 +335,9 @@ void StorageMerge::read(
query_plan.addStep(std::move(step));
}
/// An object of this helper class is created
/// when processing a Merge table data source (subordinary table)
/// that has row policies
/// to guarantee that these row policies are applied
class ReadFromMerge::RowPolicyData
{
public:
RowPolicyData(RowPolicyFilterPtr, std::shared_ptr<DB::IStorage>, ContextPtr);
/// Add to data stream columns that are needed only for row policies
/// SELECT x from T if T has row policy y=42
/// required y in data pipeline
void extendNames(Names &) const;
/// Use storage facilities to filter data
/// optimization
/// does not guarantee accuracy, but reduces number of rows
void addStorageFilter(SourceStepWithFilter *) const;
/// Create explicit filter transform to exclude
/// rows that are not conform to row level policy
void addFilterTransform(QueryPipelineBuilder &) const;
private:
std::string filter_column_name; // complex filter, may contain logic operations
ActionsDAGPtr actions_dag;
ExpressionActionsPtr filter_actions;
StorageMetadataPtr storage_metadata_snapshot;
};
ReadFromMerge::ReadFromMerge(
Block common_header_,
StorageListWithLocks selected_tables_,
Names column_names_,
bool has_database_virtual_column_,
bool has_table_virtual_column_,
Names all_column_names_,
size_t max_block_size,
size_t num_streams,
StoragePtr storage,
@ -406,21 +349,19 @@ ReadFromMerge::ReadFromMerge(
, required_max_block_size(max_block_size)
, requested_num_streams(num_streams)
, common_header(std::move(common_header_))
, selected_tables(std::move(selected_tables_))
, column_names(std::move(column_names_))
, has_database_virtual_column(has_database_virtual_column_)
, has_table_virtual_column(has_table_virtual_column_)
, all_column_names(std::move(all_column_names_))
, storage_merge(std::move(storage))
, merge_storage_snapshot(std::move(storage_snapshot))
, query_info(query_info_)
, context(std::move(context_))
, common_processed_stage(processed_stage)
{
createChildPlans();
}
void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
{
filterTablesAndCreateChildrenPlans();
if (selected_tables.empty())
{
pipeline.init(Pipe(std::make_shared<NullSource>(output_stream->header)));
@ -430,13 +371,10 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu
QueryPlanResourceHolder resources;
std::vector<std::unique_ptr<QueryPipelineBuilder>> pipelines;
chassert(selected_tables.size() == child_plans.size());
chassert(selected_tables.size() == table_aliases.size());
chassert(selected_tables.size() == table_row_policy_data_opts.size());
auto table_it = selected_tables.begin();
for (size_t i = 0; i < selected_tables.size(); ++i, ++table_it)
{
auto & plan = child_plans.at(i);
auto & child_plan = child_plans->at(i);
const auto & table = *table_it;
const auto storage = std::get<1>(table);
@ -446,13 +384,13 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu
auto modified_query_info = getModifiedQueryInfo(query_info, context, table, nested_storage_snaphsot);
auto source_pipeline = createSources(
plan,
child_plan.plan,
nested_storage_snaphsot,
modified_query_info,
common_processed_stage,
common_header,
table_aliases.at(i),
table_row_policy_data_opts.at(i),
child_plan.table_aliases,
child_plan.row_policy_data_opt,
table,
context);
@ -490,10 +428,37 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu
pipeline.addResources(std::move(resources));
}
void ReadFromMerge::createChildPlans()
void ReadFromMerge::filterTablesAndCreateChildrenPlans()
{
if (child_plans)
return;
has_database_virtual_column = false;
has_table_virtual_column = false;
column_names.clear();
column_names.reserve(column_names.size());
for (const auto & column_name : all_column_names)
{
if (column_name == "_database" && storage_merge->isVirtualColumn(column_name, merge_storage_snapshot->metadata))
has_database_virtual_column = true;
else if (column_name == "_table" && storage_merge->isVirtualColumn(column_name, merge_storage_snapshot->metadata))
has_table_virtual_column = true;
else
column_names.push_back(column_name);
}
selected_tables = getSelectedTables(context, has_database_virtual_column, has_table_virtual_column);
child_plans = createChildrenPlans(query_info);
}
std::vector<ReadFromMerge::ChildPlan> ReadFromMerge::createChildrenPlans(SelectQueryInfo & query_info_) const
{
if (selected_tables.empty())
return;
return {};
std::vector<ChildPlan> res;
size_t tables_count = selected_tables.size();
Float64 num_streams_multiplier
@ -503,7 +468,7 @@ void ReadFromMerge::createChildPlans()
if (order_info)
{
query_info.input_order_info = order_info;
query_info_.input_order_info = order_info;
}
else if (query_info.order_optimizer)
{
@ -522,7 +487,7 @@ void ReadFromMerge::createChildPlans()
break;
}
query_info.input_order_info = input_sorting_info;
query_info_.input_order_info = input_sorting_info;
}
for (const auto & table : selected_tables)
@ -542,8 +507,10 @@ void ReadFromMerge::createChildPlans()
if (sampling_requested && !storage->supportsSampling())
throw Exception(ErrorCodes::SAMPLING_NOT_SUPPORTED, "Illegal SAMPLE: table {} doesn't support sampling", storage->getStorageID().getNameForLogs());
auto & aliases = table_aliases.emplace_back();
auto & row_policy_data_opt = table_row_policy_data_opts.emplace_back();
res.emplace_back();
auto & aliases = res.back().table_aliases;
auto & row_policy_data_opt = res.back().row_policy_data_opt;
auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr();
auto nested_storage_snaphsot = storage->getStorageSnapshot(storage_metadata_snapshot, context);
@ -616,7 +583,7 @@ void ReadFromMerge::createChildPlans()
}
}
child_plans.emplace_back(createPlanForTable(
res.back().plan = createPlanForTable(
nested_storage_snaphsot,
modified_query_info,
common_processed_stage,
@ -625,8 +592,10 @@ void ReadFromMerge::createChildPlans()
column_names_as_aliases.empty() ? std::move(real_column_names) : std::move(column_names_as_aliases),
row_policy_data_opt,
context,
current_streams));
current_streams);
}
return res;
}
SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const SelectQueryInfo & query_info,
@ -804,7 +773,7 @@ QueryPlan ReadFromMerge::createPlanForTable(
Names && real_column_names,
const RowPolicyDataOpt & row_policy_data_opt,
ContextMutablePtr modified_context,
size_t streams_num)
size_t streams_num) const
{
const auto & [database_name, storage, _, table_name] = storage_with_lock;
@ -967,21 +936,14 @@ void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPipelineBuilder & bui
});
}
StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(
StorageMerge::StorageListWithLocks ReadFromMerge::getSelectedTables(
ContextPtr query_context,
const ASTPtr & query /* = nullptr */,
bool filter_by_database_virtual_column /* = false */,
bool filter_by_table_virtual_column /* = false */) const
bool filter_by_database_virtual_column,
bool filter_by_table_virtual_column) const
{
/// FIXME: filtering does not work with allow_experimental_analyzer due to
/// different column names there (it has "table_name._table" not just
/// "_table")
assert(!filter_by_database_virtual_column || !filter_by_table_virtual_column || query);
const Settings & settings = query_context->getSettingsRef();
StorageListWithLocks selected_tables;
DatabaseTablesIterators database_table_iterators = getDatabaseIterators(getContext());
StorageListWithLocks res;
DatabaseTablesIterators database_table_iterators = assert_cast<StorageMerge &>(*storage_merge).getDatabaseIterators(query_context);
MutableColumnPtr database_name_virtual_column;
MutableColumnPtr table_name_virtual_column;
@ -1005,13 +967,10 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(
if (!storage)
continue;
if (query && query->as<ASTSelectQuery>()->prewhere() && !storage->supportsPrewhere())
throw Exception(ErrorCodes::ILLEGAL_PREWHERE, "Storage {} doesn't support PREWHERE.", storage->getName());
if (storage.get() != this)
if (storage.get() != storage_merge.get())
{
auto table_lock = storage->lockForShare(query_context->getCurrentQueryId(), settings.lock_acquire_timeout);
selected_tables.emplace_back(iterator->databaseName(), storage, std::move(table_lock), iterator->name());
res.emplace_back(iterator->databaseName(), storage, std::move(table_lock), iterator->name());
if (filter_by_table_virtual_column)
table_name_virtual_column->insert(iterator->name());
}
@ -1020,33 +979,42 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(
}
}
if (!filter_by_database_virtual_column && !filter_by_table_virtual_column)
return res;
auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context);
if (!filter_actions_dag)
return res;
const auto * predicate = filter_actions_dag->getOutputs().at(0);
if (filter_by_database_virtual_column)
{
/// Filter names of selected tables if there is a condition on "_database" virtual column in WHERE clause
Block virtual_columns_block
= Block{ColumnWithTypeAndName(std::move(database_name_virtual_column), std::make_shared<DataTypeString>(), "_database")};
VirtualColumnUtils::filterBlockWithQuery(query, virtual_columns_block, query_context);
VirtualColumnUtils::filterBlockWithPredicate(predicate, virtual_columns_block, query_context);
auto values = VirtualColumnUtils::extractSingleValueFromBlock<String>(virtual_columns_block, "_database");
/// Remove unused databases from the list
selected_tables.remove_if([&](const auto & elem) { return values.find(std::get<0>(elem)) == values.end(); });
res.remove_if([&](const auto & elem) { return values.find(std::get<0>(elem)) == values.end(); });
}
if (filter_by_table_virtual_column)
{
/// Filter names of selected tables if there is a condition on "_table" virtual column in WHERE clause
Block virtual_columns_block = Block{ColumnWithTypeAndName(std::move(table_name_virtual_column), std::make_shared<DataTypeString>(), "_table")};
VirtualColumnUtils::filterBlockWithQuery(query, virtual_columns_block, query_context);
VirtualColumnUtils::filterBlockWithPredicate(predicate, virtual_columns_block, query_context);
auto values = VirtualColumnUtils::extractSingleValueFromBlock<String>(virtual_columns_block, "_table");
/// Remove unused tables from the list
selected_tables.remove_if([&](const auto & elem) { return values.find(std::get<3>(elem)) == values.end(); });
res.remove_if([&](const auto & elem) { return values.find(std::get<3>(elem)) == values.end(); });
}
return selected_tables;
return res;
}
DatabaseTablesIteratorPtr StorageMerge::getDatabaseIterator(const String & database_name, ContextPtr local_context) const
DatabaseTablesIteratorPtr StorageMerge::DatabaseNameOrRegexp::getDatabaseIterator(const String & database_name, ContextPtr local_context) const
{
auto database = DatabaseCatalog::instance().getDatabase(database_name);
@ -1066,7 +1034,7 @@ DatabaseTablesIteratorPtr StorageMerge::getDatabaseIterator(const String & datab
return database->getTablesIterator(local_context, table_name_match);
}
StorageMerge::DatabaseTablesIterators StorageMerge::getDatabaseIterators(ContextPtr local_context) const
StorageMerge::DatabaseTablesIterators StorageMerge::DatabaseNameOrRegexp::getDatabaseIterators(ContextPtr local_context) const
{
try
{
@ -1191,8 +1159,16 @@ void ReadFromMerge::convertAndFilterSourceStream(
});
}
const ReadFromMerge::StorageListWithLocks & ReadFromMerge::getSelectedTables()
{
filterTablesAndCreateChildrenPlans();
return selected_tables;
}
bool ReadFromMerge::requestReadingInOrder(InputOrderInfoPtr order_info_)
{
filterTablesAndCreateChildrenPlans();
/// Disable read-in-order optimization for reverse order with final.
/// Otherwise, it can lead to incorrect final behavior because the implementation may rely on the reading in direct order).
if (order_info_->direction != 1 && InterpreterSelectQuery::isQueryWithFinal(query_info))
@ -1205,9 +1181,9 @@ bool ReadFromMerge::requestReadingInOrder(InputOrderInfoPtr order_info_)
};
bool ok = true;
for (const auto & plan : child_plans)
if (plan.isInitialized())
ok &= recursivelyApplyToReadingSteps(plan.getRootNode(), request_read_in_order);
for (const auto & child_plan : *child_plans)
if (child_plan.plan.isInitialized())
ok &= recursivelyApplyToReadingSteps(child_plan.plan.getRootNode(), request_read_in_order);
if (!ok)
return false;
@ -1234,9 +1210,11 @@ void ReadFromMerge::applyFilters(const QueryPlan & plan) const
void ReadFromMerge::applyFilters()
{
for (const auto & plan : child_plans)
if (plan.isInitialized())
applyFilters(plan);
filterTablesAndCreateChildrenPlans();
for (const auto & child_plan : *child_plans)
if (child_plan.plan.isInitialized())
applyFilters(child_plan.plan);
}
IStorage::ColumnSizeByName StorageMerge::getColumnSizes() const

View File

@ -12,6 +12,9 @@ namespace DB
struct QueryPlanResourceHolder;
struct RowPolicyFilter;
using RowPolicyFilterPtr = std::shared_ptr<const RowPolicyFilter>;
/** A table that represents the union of an arbitrary number of other tables.
* All tables must have the same structure.
*/
@ -78,24 +81,36 @@ public:
std::optional<UInt64> totalRows(const Settings & settings) const override;
std::optional<UInt64> totalBytes(const Settings & settings) const override;
using DatabaseTablesIterators = std::vector<DatabaseTablesIteratorPtr>;
DatabaseTablesIterators getDatabaseIterators(ContextPtr context) const;
private:
std::optional<OptimizedRegularExpression> source_database_regexp;
std::optional<OptimizedRegularExpression> source_table_regexp;
std::optional<DBToTableSetMap> source_databases_and_tables;
String source_database_name_or_regexp;
bool database_is_regexp = false;
/// (Database, Table, Lock, TableName)
using StorageWithLockAndName = std::tuple<String, StoragePtr, TableLockHolder, String>;
using StorageListWithLocks = std::list<StorageWithLockAndName>;
using DatabaseTablesIterators = std::vector<DatabaseTablesIteratorPtr>;
StorageMerge::StorageListWithLocks getSelectedTables(
ContextPtr query_context,
const ASTPtr & query = nullptr,
bool filter_by_database_virtual_column = false,
bool filter_by_table_virtual_column = false) const;
struct DatabaseNameOrRegexp
{
String source_database_name_or_regexp;
bool database_is_regexp = false;
std::optional<OptimizedRegularExpression> source_database_regexp;
std::optional<OptimizedRegularExpression> source_table_regexp;
std::optional<DBToTableSetMap> source_databases_and_tables;
DatabaseNameOrRegexp(
const String & source_database_name_or_regexp_,
bool database_is_regexp_,
std::optional<OptimizedRegularExpression> source_database_regexp_,
std::optional<OptimizedRegularExpression> source_table_regexp_,
std::optional<DBToTableSetMap> source_databases_and_tables_);
DatabaseTablesIteratorPtr getDatabaseIterator(const String & database_name, ContextPtr context) const;
DatabaseTablesIterators getDatabaseIterators(ContextPtr context) const;
};
DatabaseNameOrRegexp database_name_or_regexp;
template <typename F>
StoragePtr getFirstTable(F && predicate) const;
@ -103,10 +118,6 @@ private:
template <typename F>
void forEachTable(F && func) const;
DatabaseTablesIteratorPtr getDatabaseIterator(const String & database_name, ContextPtr context) const;
DatabaseTablesIterators getDatabaseIterators(ContextPtr context) const;
NamesAndTypesList getVirtuals() const override;
ColumnSizeByName getColumnSizes() const override;
@ -132,10 +143,7 @@ public:
ReadFromMerge(
Block common_header_,
StorageListWithLocks selected_tables_,
Names column_names_,
bool has_database_virtual_column_,
bool has_table_virtual_column_,
Names all_column_names_,
size_t max_block_size,
size_t num_streams,
StoragePtr storage,
@ -146,7 +154,7 @@ public:
void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override;
const StorageListWithLocks & getSelectedTables() const { return selected_tables; }
const StorageListWithLocks & getSelectedTables();
/// Returns `false` if requested reading cannot be performed.
bool requestReadingInOrder(InputOrderInfoPtr order_info_);
@ -159,16 +167,13 @@ private:
const Block common_header;
StorageListWithLocks selected_tables;
Names all_column_names;
Names column_names;
bool has_database_virtual_column;
bool has_table_virtual_column;
StoragePtr storage_merge;
StorageSnapshotPtr merge_storage_snapshot;
/// Store read plan for each child table.
/// It's needed to guarantee lifetime for child steps to be the same as for this step (mainly for EXPLAIN PIPELINE).
std::vector<QueryPlan> child_plans;
SelectQueryInfo query_info;
ContextMutablePtr context;
QueryProcessingStage::Enum common_processed_stage;
@ -184,14 +189,52 @@ private:
using Aliases = std::vector<AliasData>;
class RowPolicyData;
/// An object of this helper class is created
/// when processing a Merge table data source (subordinary table)
/// that has row policies
/// to guarantee that these row policies are applied
class RowPolicyData
{
public:
RowPolicyData(RowPolicyFilterPtr, std::shared_ptr<DB::IStorage>, ContextPtr);
/// Add to data stream columns that are needed only for row policies
/// SELECT x from T if T has row policy y=42
/// required y in data pipeline
void extendNames(Names &) const;
/// Use storage facilities to filter data
/// optimization
/// does not guarantee accuracy, but reduces number of rows
void addStorageFilter(SourceStepWithFilter *) const;
/// Create explicit filter transform to exclude
/// rows that are not conform to row level policy
void addFilterTransform(QueryPipelineBuilder &) const;
private:
std::string filter_column_name; // complex filter, may contain logic operations
ActionsDAGPtr actions_dag;
ExpressionActionsPtr filter_actions;
StorageMetadataPtr storage_metadata_snapshot;
};
using RowPolicyDataOpt = std::optional<RowPolicyData>;
std::vector<Aliases> table_aliases;
struct ChildPlan
{
QueryPlan plan;
Aliases table_aliases;
RowPolicyDataOpt row_policy_data_opt;
};
std::vector<RowPolicyDataOpt> table_row_policy_data_opts;
/// Store read plan for each child table.
/// It's needed to guarantee lifetime for child steps to be the same as for this step (mainly for EXPLAIN PIPELINE).
std::optional<std::vector<ChildPlan>> child_plans;
void createChildPlans();
std::vector<ChildPlan> createChildrenPlans(SelectQueryInfo & query_info_) const;
void filterTablesAndCreateChildrenPlans();
void applyFilters(const QueryPlan & plan) const;
@ -204,7 +247,7 @@ private:
Names && real_column_names,
const RowPolicyDataOpt & row_policy_data_opt,
ContextMutablePtr modified_context,
size_t streams_num);
size_t streams_num) const;
QueryPipelineBuilderPtr createSources(
QueryPlan & plan,
@ -231,6 +274,11 @@ private:
ContextPtr context,
QueryPipelineBuilder & builder,
QueryProcessingStage::Enum processed_stage);
StorageMerge::StorageListWithLocks getSelectedTables(
ContextPtr query_context,
bool filter_by_database_virtual_column,
bool filter_by_table_virtual_column) const;
};
}

View File

@ -502,9 +502,12 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs(
const ActionsDAG::Node * res = node_copy.children.front();
/// Expression like (not_allowed AND 256) can't be resuced to (and(256)) because AND requires
/// at least two arguments; also it can't be reduced to (256) because result type is different.
/// TODO: add CAST here
if (!res->result_type->equals(*node->result_type))
return nullptr;
{
ActionsDAG tmp_dag;
res = &tmp_dag.addCast(*res, node->result_type, {});
additional_nodes.splice(additional_nodes.end(), ActionsDAG::detachNodes(std::move(tmp_dag)));
}
return res;
}

View File

@ -294,7 +294,7 @@ def test_bad_messages_parsing_exception(kafka_cluster, max_retries=20):
]:
print(format_name)
kafka_create_topic(admin_client, f"{format_name}_err")
kafka_create_topic(admin_client, f"{format_name}_parsing_err")
instance.query(
f"""
@ -305,7 +305,7 @@ def test_bad_messages_parsing_exception(kafka_cluster, max_retries=20):
CREATE TABLE kafka_{format_name} (key UInt64, value UInt64)
ENGINE = Kafka
SETTINGS kafka_broker_list = 'kafka1:19092',
kafka_topic_list = '{format_name}_err',
kafka_topic_list = '{format_name}_parsing_err',
kafka_group_name = '{format_name}',
kafka_format = '{format_name}',
kafka_num_consumers = 1;
@ -316,16 +316,18 @@ def test_bad_messages_parsing_exception(kafka_cluster, max_retries=20):
)
kafka_produce(
kafka_cluster, f"{format_name}_err", ["qwertyuiop", "asdfghjkl", "zxcvbnm"]
kafka_cluster,
f"{format_name}_parsing_err",
["qwertyuiop", "asdfghjkl", "zxcvbnm"],
)
expected_result = """avro::Exception: Invalid data file. Magic does not match: : while parsing Kafka message (topic: Avro_err, partition: 0, offset: 0)\\'|1|1|1|default|kafka_Avro
Cannot parse input: expected \\'{\\' before: \\'qwertyuiop\\': while parsing Kafka message (topic: JSONEachRow_err, partition: 0, offset: 0|1|1|1|default|kafka_JSONEachRow
expected_result = """avro::Exception: Invalid data file. Magic does not match: : while parsing Kafka message (topic: Avro_parsing_err, partition: 0, offset: 0)\\'|1|1|1|default|kafka_Avro
Cannot parse input: expected \\'{\\' before: \\'qwertyuiop\\': (at row 1)\\n: while parsing Kafka message (topic: JSONEachRow_parsing_err, partition:|1|1|1|default|kafka_JSONEachRow
"""
# filter out stacktrace in exceptions.text[1] because it is hardly stable enough
result_system_kafka_consumers = instance.query_with_retry(
"""
SELECT substr(exceptions.text[1], 1, 131), length(exceptions.text) > 1 AND length(exceptions.text) < 15, length(exceptions.time) > 1 AND length(exceptions.time) < 15, abs(dateDiff('second', exceptions.time[1], now())) < 40, database, table FROM system.kafka_consumers WHERE table in('kafka_Avro', 'kafka_JSONEachRow') ORDER BY table, assignments.partition_id[1]
SELECT substr(exceptions.text[1], 1, 139), length(exceptions.text) > 1 AND length(exceptions.text) < 15, length(exceptions.time) > 1 AND length(exceptions.time) < 15, abs(dateDiff('second', exceptions.time[1], now())) < 40, database, table FROM system.kafka_consumers WHERE table in('kafka_Avro', 'kafka_JSONEachRow') ORDER BY table, assignments.partition_id[1]
""",
retry_count=max_retries,
sleep_time=1,
@ -338,7 +340,7 @@ Cannot parse input: expected \\'{\\' before: \\'qwertyuiop\\': while parsing Kaf
"Avro",
"JSONEachRow",
]:
kafka_delete_topic(admin_client, f"{format_name}_err")
kafka_delete_topic(admin_client, f"{format_name}_parsing_err")
def test_bad_messages_to_mv(kafka_cluster, max_retries=20):

View File

@ -0,0 +1,7 @@
@0x99f75f775fe63dae;
struct Message
{
key @0 : UInt64;
value @1 : UInt64;
}

View File

@ -0,0 +1,6 @@
syntax = "proto3";
message Message {
uint64 key = 1;
uint64 value = 1;
}

View File

@ -4834,6 +4834,103 @@ JSONExtractString(rdkafka_stat, 'type'): consumer
kafka_delete_topic(admin_client, topic)
def test_formats_errors(kafka_cluster):
admin_client = KafkaAdminClient(
bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)
)
for format_name in [
"Template",
"Regexp",
"TSV",
"TSVWithNamesAndTypes",
"TSKV",
"CSV",
"CSVWithNames",
"CSVWithNamesAndTypes",
"CustomSeparated",
"CustomSeparatedWithNames",
"CustomSeparatedWithNamesAndTypes",
"Values",
"JSON",
"JSONEachRow",
"JSONStringsEachRow",
"JSONCompactEachRow",
"JSONCompactEachRowWithNamesAndTypes",
"JSONObjectEachRow",
"Avro",
"RowBinary",
"RowBinaryWithNamesAndTypes",
"MsgPack",
"JSONColumns",
"JSONCompactColumns",
"JSONColumnsWithMetadata",
"BSONEachRow",
"Native",
"Arrow",
"Parquet",
"ORC",
"JSONCompactColumns",
"Npy",
"ParquetMetadata",
"CapnProto",
"Protobuf",
"ProtobufSingle",
"ProtobufList",
"DWARF",
"HiveText",
"MySQLDump",
]:
kafka_create_topic(admin_client, format_name)
table_name = f"kafka_{format_name}"
instance.query(
f"""
DROP TABLE IF EXISTS test.view;
DROP TABLE IF EXISTS test.{table_name};
CREATE TABLE test.{table_name} (key UInt64, value UInt64)
ENGINE = Kafka
SETTINGS kafka_broker_list = 'kafka1:19092',
kafka_topic_list = '{format_name}',
kafka_group_name = '{format_name}',
kafka_format = '{format_name}',
kafka_max_rows_per_message = 5,
format_template_row='template_row.format',
format_regexp='id: (.+?)',
input_format_with_names_use_header=0,
format_schema='key_value_message:Message';
CREATE MATERIALIZED VIEW test.view Engine=Log AS
SELECT key, value FROM test.{table_name};
"""
)
kafka_produce(
kafka_cluster,
format_name,
["Broken message\nBroken message\nBroken message\n"],
)
attempt = 0
num_errors = 0
while attempt < 200:
num_errors = int(
instance.query(
f"SELECT length(exceptions.text) from system.kafka_consumers where database = 'test' and table = '{table_name}'"
)
)
if num_errors > 0:
break
attempt += 1
assert num_errors > 0
kafka_delete_topic(admin_client, format_name)
instance.query(f"DROP TABLE test.{table_name}")
instance.query("DROP TABLE test.view")
if __name__ == "__main__":
cluster.start()
input("Cluster created, press any key to destroy...")

View File

@ -78,7 +78,9 @@ def test_details(started_cluster):
# 2. Time only is not supported
node.query("CREATE USER user_details_time_only VALID UNTIL '22:03:40'")
until_year = datetime.today().strftime("%Y")
assert (
node.query("SHOW CREATE USER user_details_time_only")
== "CREATE USER user_details_time_only VALID UNTIL \\'2023-01-01 22:03:40\\'\n"
== f"CREATE USER user_details_time_only VALID UNTIL \\'{until_year}-01-01 22:03:40\\'\n"
)

View File

@ -13,6 +13,8 @@ CREATE TABLE numbers5 ENGINE = MergeTree ORDER BY number AS SELECT number FROM n
SELECT count() FROM merge(currentDatabase(), '^numbers\\d+$');
SELECT DISTINCT count() FROM merge(currentDatabase(), '^numbers\\d+$') GROUP BY number;
SET optimize_aggregation_in_order = 0; -- FIXME : in order may happen before filter push down
SET max_rows_to_read = 1000;
SET max_threads = 'auto';

View File

@ -1,14 +1,14 @@
2.4
10.165
0.00012000000000000002
150.16500000000002
7.775900000000001
56.622689999999984
598.8376688440277
299.41883695311844
0.7485470860550345
2.2456412771483882
1.641386318314034
1.641386318314034
1.6413863258732018
1.6413863258732018
0.00012
150.165
7.7759
56.62269
598.837669
299.418837
0.748547
2.245641
1.641386
1.641386
1.641386
1.641386

View File

@ -1,9 +1,9 @@
SET optimize_arithmetic_operations_in_aggregate_functions = 0;
SELECT toDecimal32(2, 2) * 1.2;
SELECT toDecimal64(0.5, 2) * 20.33;
SELECT 0.00001 * toDecimal32(12, 2);
SELECT 30.033 * toDecimal32(5, 1);
SELECT round(toDecimal32(2, 2) * 1.2, 6);
SELECT round(toDecimal64(0.5, 2) * 20.33, 6);
SELECT round(0.00001 * toDecimal32(12, 2), 6);
SELECT round(30.033 * toDecimal32(5, 1), 6);
CREATE TABLE IF NOT EXISTS test01603 (
f64 Float64,
@ -13,17 +13,17 @@ CREATE TABLE IF NOT EXISTS test01603 (
INSERT INTO test01603(f64) SELECT 1 / (number + 1) FROM system.numbers LIMIT 1000;
SELECT sum(d * 1.1) FROM test01603;
SELECT sum(8.01 * d) FROM test01603;
SELECT round(sum(d * 1.1), 6) FROM test01603;
SELECT round(sum(8.01 * d), 6) FROM test01603;
SELECT sum(f64 * toDecimal64(80, 2)) FROM test01603;
SELECT sum(toDecimal64(40, 2) * f32) FROM test01603;
SELECT sum(f64 * toDecimal64(0.1, 2)) FROM test01603;
SELECT sum(toDecimal64(0.3, 2) * f32) FROM test01603;
SELECT round(sum(f64 * toDecimal64(80, 2)), 6) FROM test01603;
SELECT round(sum(toDecimal64(40, 2) * f32), 6) FROM test01603;
SELECT round(sum(f64 * toDecimal64(0.1, 2)), 6) FROM test01603;
SELECT round(sum(toDecimal64(0.3, 2) * f32), 6) FROM test01603;
SELECT sum(f64 * d) FROM test01603;
SELECT sum(d * f64) FROM test01603;
SELECT sum(f32 * d) FROM test01603;
SELECT sum(d * f32) FROM test01603;
SELECT round(sum(f64 * d), 6) FROM test01603;
SELECT round(sum(d * f64), 6) FROM test01603;
SELECT round(sum(f32 * d), 6) FROM test01603;
SELECT round(sum(d * f32), 6) FROM test01603;
DROP TABLE IF EXISTS test01603;

Some files were not shown because too many files have changed in this diff Show More