Merge pull request #6727 from yandex/merging_template_format

Merging Template format
This commit is contained in:
alexey-milovidov 2019-09-02 17:25:43 +03:00 committed by GitHub
commit 0967ecf03b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
38 changed files with 2412 additions and 1961 deletions

1
.gitignore vendored
View File

@ -90,7 +90,6 @@ dbms/src/Core/tests/field
dbms/src/Core/tests/rvo_test
dbms/src/Core/tests/string_pool
dbms/src/DataStreams/tests/aggregating_stream
dbms/src/DataStreams/tests/block_row_transforms
dbms/src/DataStreams/tests/block_tab_separated_streams
dbms/src/DataStreams/tests/collapsing_sorted_stream
dbms/src/DataStreams/tests/expression_stream

View File

@ -447,6 +447,7 @@ namespace ErrorCodes
extern const int QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW = 470;
extern const int SETTINGS_ARE_NOT_SUPPORTED = 471;
extern const int READONLY_SETTING = 472;
extern const int INVALID_TEMPLATE_FORMAT = 473;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;

View File

@ -216,6 +216,8 @@ struct Settings : public SettingsCollection<Settings>
M(SettingMilliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.") \
M(SettingMilliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.") \
M(SettingString, format_schema, "", "Schema identifier (used by schema-based formats)") \
M(SettingString, format_schema_rows, "", "Row format string for Template format") \
M(SettingString, format_schema_rows_between_delimiter, "\n", "Delimiter between rows for Template format") \
M(SettingBool, insert_allow_materialized_columns, 0, "If setting is enabled, Allow materialized columns in INSERT.") \
M(SettingSeconds, http_connection_timeout, DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT, "HTTP connection timeout.") \
M(SettingSeconds, http_send_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP send timeout") \

View File

@ -1,178 +0,0 @@
#include <Common/Exception.h>
#include <IO/WriteHelpers.h>
#include <Formats/BlockInputStreamFromRowInputStream.h>
#include <common/logger_useful.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED;
extern const int CANNOT_PARSE_QUOTED_STRING;
extern const int CANNOT_PARSE_DATE;
extern const int CANNOT_PARSE_DATETIME;
extern const int CANNOT_READ_ARRAY_FROM_TEXT;
extern const int CANNOT_PARSE_NUMBER;
extern const int CANNOT_PARSE_UUID;
extern const int TOO_LARGE_STRING_SIZE;
extern const int CANNOT_READ_ALL_DATA;
extern const int INCORRECT_DATA;
extern const int INCORRECT_NUMBER_OF_COLUMNS;
}
BlockInputStreamFromRowInputStream::BlockInputStreamFromRowInputStream(
const RowInputStreamPtr & row_input_,
const Block & sample_,
UInt64 max_block_size_,
UInt64 rows_portion_size_,
FormatFactory::ReadCallback callback,
const FormatSettings & settings)
: row_input(row_input_)
, sample(sample_)
, max_block_size(max_block_size_)
, rows_portion_size(rows_portion_size_)
, read_virtual_columns_callback(callback)
, allow_errors_num(settings.input_allow_errors_num)
, allow_errors_ratio(settings.input_allow_errors_ratio)
{
}
static bool isParseError(int code)
{
return code == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED
|| code == ErrorCodes::CANNOT_PARSE_QUOTED_STRING
|| code == ErrorCodes::CANNOT_PARSE_DATE
|| code == ErrorCodes::CANNOT_PARSE_DATETIME
|| code == ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT
|| code == ErrorCodes::CANNOT_PARSE_NUMBER
|| code == ErrorCodes::CANNOT_PARSE_UUID
|| code == ErrorCodes::TOO_LARGE_STRING_SIZE
|| code == ErrorCodes::CANNOT_READ_ALL_DATA
|| code == ErrorCodes::INCORRECT_DATA;
}
Block BlockInputStreamFromRowInputStream::readImpl()
{
size_t num_columns = sample.columns();
MutableColumns columns = sample.cloneEmptyColumns();
block_missing_values.clear();
try
{
for (size_t rows = 0, batch = 0; rows < max_block_size; ++rows, ++batch)
{
if (rows_portion_size && batch == rows_portion_size)
{
batch = 0;
if (!checkTimeLimit() || isCancelled())
break;
}
try
{
++total_rows;
RowReadExtension info_;
if (!row_input->read(columns, info_))
break;
if (read_virtual_columns_callback)
read_virtual_columns_callback();
for (size_t column_idx = 0; column_idx < info_.read_columns.size(); ++column_idx)
{
if (!info_.read_columns[column_idx])
{
size_t column_size = columns[column_idx]->size();
if (column_size == 0)
throw Exception("Unexpected empty column", ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS);
block_missing_values.setBit(column_idx, column_size - 1);
}
}
}
catch (Exception & e)
{
/// Logic for possible skipping of errors.
if (!isParseError(e.code()))
throw;
if (allow_errors_num == 0 && allow_errors_ratio == 0)
throw;
++num_errors;
Float32 current_error_ratio = static_cast<Float32>(num_errors) / total_rows;
if (num_errors > allow_errors_num
&& current_error_ratio > allow_errors_ratio)
{
e.addMessage("(Already have " + toString(num_errors) + " errors"
" out of " + toString(total_rows) + " rows"
", which is " + toString(current_error_ratio) + " of all rows)");
throw;
}
if (!row_input->allowSyncAfterError())
{
e.addMessage("(Input format doesn't allow to skip errors)");
throw;
}
row_input->syncAfterError();
/// Truncate all columns in block to minimal size (remove values, that was appended to only part of columns).
size_t min_size = std::numeric_limits<size_t>::max();
for (size_t column_idx = 0; column_idx < num_columns; ++column_idx)
min_size = std::min(min_size, columns[column_idx]->size());
for (size_t column_idx = 0; column_idx < num_columns; ++column_idx)
{
auto & column = columns[column_idx];
if (column->size() > min_size)
column->popBack(column->size() - min_size);
}
}
}
}
catch (Exception & e)
{
if (!isParseError(e.code()))
throw;
String verbose_diagnostic;
try
{
verbose_diagnostic = row_input->getDiagnosticInfo();
}
catch (...)
{
/// Error while trying to obtain verbose diagnostic. Ok to ignore.
}
e.addMessage("(at row " + toString(total_rows) + ")\n" + verbose_diagnostic);
throw;
}
if (columns.empty() || columns[0]->empty())
return {};
return sample.cloneWithColumns(std::move(columns));
}
void BlockInputStreamFromRowInputStream::readSuffix()
{
if (allow_errors_num > 0 || allow_errors_ratio > 0)
{
Logger * log = &Logger::get("BlockInputStreamFromRowInputStream");
LOG_TRACE(log, "Skipped " << num_errors << " rows with errors while reading the input stream");
}
row_input->readSuffix();
}
}

View File

@ -1,62 +0,0 @@
#pragma once
#include <Core/Defines.h>
#include <DataStreams/IBlockInputStream.h>
#include <Formats/FormatFactory.h>
#include <Formats/FormatSettings.h>
#include <Formats/IRowInputStream.h>
namespace DB
{
/** Makes block-oriented stream on top of row-oriented stream.
* It is used to read data from text formats.
*
* Also controls over parsing errors and prints diagnostic information about them.
*/
class BlockInputStreamFromRowInputStream : public IBlockInputStream
{
public:
/// |sample| is a block with zero rows, that structure describes how to interpret values
/// |rows_portion_size| is a number of rows to read before break and check limits
BlockInputStreamFromRowInputStream(
const RowInputStreamPtr & row_input_,
const Block & sample_,
UInt64 max_block_size_,
UInt64 rows_portion_size_,
FormatFactory::ReadCallback callback,
const FormatSettings & settings);
void readPrefix() override { row_input->readPrefix(); }
void readSuffix() override;
String getName() const override { return "BlockInputStreamFromRowInputStream"; }
RowInputStreamPtr & getRowInput() { return row_input; }
Block getHeader() const override { return sample; }
const BlockMissingValues & getMissingValues() const override { return block_missing_values; }
protected:
Block readImpl() override;
private:
RowInputStreamPtr row_input;
Block sample;
UInt64 max_block_size;
UInt64 rows_portion_size;
/// Callback used to setup virtual columns after reading each row.
FormatFactory::ReadCallback read_virtual_columns_callback;
BlockMissingValues block_missing_values;
UInt64 allow_errors_num;
Float32 allow_errors_ratio;
size_t total_rows = 0;
size_t num_errors = 0;
};
}

View File

@ -1,573 +0,0 @@
#include <Core/Defines.h>
#include <IO/ConcatReadBuffer.h>
#include <IO/ReadHelpers.h>
#include <IO/Operators.h>
#include <Formats/verbosePrintString.h>
#include <Formats/CSVRowInputStream.h>
#include <Formats/FormatFactory.h>
#include <Formats/BlockInputStreamFromRowInputStream.h>
#include <DataTypes/DataTypeNullable.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_DATA;
extern const int LOGICAL_ERROR;
}
static inline void skipEndOfLine(ReadBuffer & istr)
{
/// \n (Unix) or \r\n (DOS/Windows) or \n\r (Mac OS Classic)
if (*istr.position() == '\n')
{
++istr.position();
if (!istr.eof() && *istr.position() == '\r')
++istr.position();
}
else if (*istr.position() == '\r')
{
++istr.position();
if (!istr.eof() && *istr.position() == '\n')
++istr.position();
else
throw Exception("Cannot parse CSV format: found \\r (CR) not followed by \\n (LF)."
" Line must end by \\n (LF) or \\r\\n (CR LF) or \\n\\r.", ErrorCodes::INCORRECT_DATA);
}
else if (!istr.eof())
throw Exception("Expected end of line", ErrorCodes::INCORRECT_DATA);
}
static inline void skipDelimiter(ReadBuffer & istr, const char delimiter, bool is_last_column)
{
if (is_last_column)
{
if (istr.eof())
return;
/// we support the extra delimiter at the end of the line
if (*istr.position() == delimiter)
{
++istr.position();
if (istr.eof())
return;
}
skipEndOfLine(istr);
}
else
assertChar(delimiter, istr);
}
/// Skip `whitespace` symbols allowed in CSV.
static inline void skipWhitespacesAndTabs(ReadBuffer & buf)
{
while (!buf.eof()
&& (*buf.position() == ' '
|| *buf.position() == '\t'))
++buf.position();
}
static void skipRow(ReadBuffer & istr, const FormatSettings::CSV & settings, size_t num_columns)
{
String tmp;
for (size_t i = 0; i < num_columns; ++i)
{
skipWhitespacesAndTabs(istr);
readCSVString(tmp, istr, settings);
skipWhitespacesAndTabs(istr);
skipDelimiter(istr, settings.delimiter, i + 1 == num_columns);
}
}
CSVRowInputStream::CSVRowInputStream(ReadBuffer & istr_, const Block & header_, bool with_names_, const FormatSettings & format_settings_)
: istr(istr_), header(header_), with_names(with_names_), format_settings(format_settings_)
{
const auto num_columns = header.columns();
data_types.resize(num_columns);
column_indexes_by_names.reserve(num_columns);
column_idx_to_nullable_column_idx.resize(num_columns);
for (size_t i = 0; i < num_columns; ++i)
{
const auto & column_info = header.getByPosition(i);
data_types[i] = column_info.type;
column_indexes_by_names.emplace(column_info.name, i);
/// If input_format_null_as_default=1 we need ColumnNullable of type DataTypeNullable(nested_type)
/// to parse value as nullable before inserting it in corresponding column of not-nullable type.
/// Constructing temporary column for each row is slow, so we prepare it here
if (format_settings.csv.null_as_default && !column_info.type->isNullable() && column_info.type->canBeInsideNullable())
{
column_idx_to_nullable_column_idx[i] = nullable_columns.size();
nullable_types.emplace_back(std::make_shared<DataTypeNullable>(column_info.type));
nullable_columns.emplace_back(nullable_types.back()->createColumn());
}
}
}
/// Map an input file column to a table column, based on its name.
void CSVRowInputStream::addInputColumn(const String & column_name)
{
const auto column_it = column_indexes_by_names.find(column_name);
if (column_it == column_indexes_by_names.end())
{
if (format_settings.skip_unknown_fields)
{
column_indexes_for_input_fields.push_back(std::nullopt);
return;
}
throw Exception(
"Unknown field found in CSV header: '" + column_name + "' " +
"at position " + std::to_string(column_indexes_for_input_fields.size()) +
"\nSet the 'input_format_skip_unknown_fields' parameter explicitly to ignore and proceed",
ErrorCodes::INCORRECT_DATA
);
}
const auto column_index = column_it->second;
if (read_columns[column_index])
throw Exception("Duplicate field found while parsing CSV header: " + column_name, ErrorCodes::INCORRECT_DATA);
read_columns[column_index] = true;
column_indexes_for_input_fields.emplace_back(column_index);
}
void CSVRowInputStream::readPrefix()
{
/// In this format, we assume, that if first string field contain BOM as value, it will be written in quotes,
/// so BOM at beginning of stream cannot be confused with BOM in first string value, and it is safe to skip it.
skipBOMIfExists(istr);
if (with_names)
{
/// This CSV file has a header row with column names. Depending on the
/// settings, use it or skip it.
if (format_settings.with_names_use_header)
{
/// Look at the file header to see which columns we have there.
/// The missing columns are filled with defaults.
read_columns.assign(header.columns(), false);
do
{
String column_name;
skipWhitespacesAndTabs(istr);
readCSVString(column_name, istr, format_settings.csv);
skipWhitespacesAndTabs(istr);
addInputColumn(column_name);
}
while (checkChar(format_settings.csv.delimiter, istr));
skipDelimiter(istr, format_settings.csv.delimiter, true);
for (size_t column = 0; column < read_columns.size(); column++)
{
if (!read_columns[column])
{
have_always_default_columns = true;
break;
}
}
return;
}
else
{
skipRow(istr, format_settings.csv, header.columns());
}
}
/// The default: map each column of the file to the column of the table with
/// the same index.
read_columns.assign(header.columns(), true);
column_indexes_for_input_fields.resize(header.columns());
for (size_t i = 0; i < column_indexes_for_input_fields.size(); ++i)
{
column_indexes_for_input_fields[i] = i;
}
}
/** If you change this function, don't forget to change its counterpart
* with extended error reporting: parseRowAndPrintDiagnosticInfo().
*/
bool CSVRowInputStream::read(MutableColumns & columns, RowReadExtension & ext)
{
if (istr.eof())
return false;
updateDiagnosticInfo();
/// Track whether we have to fill any columns in this row with default
/// values. If not, we return an empty column mask to the caller, so that
/// it doesn't have to check it.
bool have_default_columns = have_always_default_columns;
const auto delimiter = format_settings.csv.delimiter;
for (size_t file_column = 0; file_column < column_indexes_for_input_fields.size(); ++file_column)
{
const auto & table_column = column_indexes_for_input_fields[file_column];
const bool is_last_file_column = file_column + 1 == column_indexes_for_input_fields.size();
if (table_column)
{
skipWhitespacesAndTabs(istr);
read_columns[*table_column] = readField(*columns[*table_column], data_types[*table_column],
is_last_file_column, *table_column);
if (!read_columns[*table_column])
have_default_columns = true;
skipWhitespacesAndTabs(istr);
}
else
{
/// We never read this column from the file, just skip it.
String tmp;
readCSVString(tmp, istr, format_settings.csv);
}
skipDelimiter(istr, delimiter, is_last_file_column);
}
if (have_default_columns)
{
for (size_t i = 0; i < read_columns.size(); i++)
{
if (!read_columns[i])
{
/// The column value for this row is going to be overwritten
/// with default by the caller, but the general assumption is
/// that the column size increases for each row, so we have
/// to insert something. Since we do not care about the exact
/// value, we do not have to use the default value specified by
/// the data type, and can just use IColumn::insertDefault().
columns[i]->insertDefault();
}
}
ext.read_columns = read_columns;
}
return true;
}
String CSVRowInputStream::getDiagnosticInfo()
{
if (istr.eof()) /// Buffer has gone, cannot extract information about what has been parsed.
return {};
WriteBufferFromOwnString out;
MutableColumns columns = header.cloneEmptyColumns();
/// It is possible to display detailed diagnostics only if the last and next to last rows are still in the read buffer.
size_t bytes_read_at_start_of_buffer = istr.count() - istr.offset();
if (bytes_read_at_start_of_buffer != bytes_read_at_start_of_buffer_on_prev_row)
{
out << "Could not print diagnostic info because two last rows aren't in buffer (rare case)\n";
return out.str();
}
size_t max_length_of_column_name = 0;
for (size_t i = 0; i < header.columns(); ++i)
if (header.safeGetByPosition(i).name.size() > max_length_of_column_name)
max_length_of_column_name = header.safeGetByPosition(i).name.size();
size_t max_length_of_data_type_name = 0;
for (size_t i = 0; i < header.columns(); ++i)
if (header.safeGetByPosition(i).type->getName().size() > max_length_of_data_type_name)
max_length_of_data_type_name = header.safeGetByPosition(i).type->getName().size();
/// Roll back the cursor to the beginning of the previous or current row and parse all over again. But now we derive detailed information.
if (pos_of_prev_row)
{
istr.position() = pos_of_prev_row;
out << "\nRow " << (row_num - 1) << ":\n";
if (!parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name))
return out.str();
}
else
{
if (!pos_of_current_row)
{
out << "Could not print diagnostic info because parsing of data hasn't started.\n";
return out.str();
}
istr.position() = pos_of_current_row;
}
out << "\nRow " << row_num << ":\n";
parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name);
out << "\n";
return out.str();
}
/** gcc-7 generates wrong code with optimization level greater than 1.
* See tests: dbms/src/IO/tests/write_int.cpp
* and dbms/tests/queries/0_stateless/00898_parsing_bad_diagnostic_message.sh
* This is compiler bug. The bug does not present in gcc-8 and clang-8.
* Nevertheless, we don't need high optimization of this function.
*/
bool OPTIMIZE(1) CSVRowInputStream::parseRowAndPrintDiagnosticInfo(MutableColumns & columns,
WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name)
{
const char delimiter = format_settings.csv.delimiter;
for (size_t file_column = 0; file_column < column_indexes_for_input_fields.size(); ++file_column)
{
if (file_column == 0 && istr.eof())
{
out << "<End of stream>\n";
return false;
}
if (column_indexes_for_input_fields[file_column].has_value())
{
const auto & table_column = *column_indexes_for_input_fields[file_column];
const auto & current_column_type = data_types[table_column];
const bool is_last_file_column =
file_column + 1 == column_indexes_for_input_fields.size();
const bool at_delimiter = !istr.eof() && *istr.position() == delimiter;
const bool at_last_column_line_end = is_last_file_column
&& (istr.eof() || *istr.position() == '\n' || *istr.position() == '\r');
out << "Column " << file_column << ", " << std::string((file_column < 10 ? 2 : file_column < 100 ? 1 : 0), ' ')
<< "name: " << header.safeGetByPosition(table_column).name << ", " << std::string(max_length_of_column_name - header.safeGetByPosition(table_column).name.size(), ' ')
<< "type: " << current_column_type->getName() << ", " << std::string(max_length_of_data_type_name - current_column_type->getName().size(), ' ');
if (format_settings.csv.empty_as_default
&& (at_delimiter || at_last_column_line_end))
{
columns[table_column]->insertDefault();
}
else
{
BufferBase::Position prev_position = istr.position();
BufferBase::Position curr_position = istr.position();
std::exception_ptr exception;
try
{
skipWhitespacesAndTabs(istr);
prev_position = istr.position();
readField(*columns[table_column], current_column_type, is_last_file_column, table_column);
curr_position = istr.position();
skipWhitespacesAndTabs(istr);
}
catch (...)
{
exception = std::current_exception();
}
if (curr_position < prev_position)
throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR);
if (isNativeNumber(current_column_type) || isDateOrDateTime(current_column_type))
{
/// An empty string instead of a value.
if (curr_position == prev_position)
{
out << "ERROR: text ";
verbosePrintString(prev_position, std::min(prev_position + 10, istr.buffer().end()), out);
out << " is not like " << current_column_type->getName() << "\n";
return false;
}
}
out << "parsed text: ";
verbosePrintString(prev_position, curr_position, out);
if (exception)
{
if (current_column_type->getName() == "DateTime")
out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n";
else if (current_column_type->getName() == "Date")
out << "ERROR: Date must be in YYYY-MM-DD format.\n";
else
out << "ERROR\n";
return false;
}
out << "\n";
if (current_column_type->haveMaximumSizeOfValue()
&& *curr_position != '\n' && *curr_position != '\r'
&& *curr_position != delimiter)
{
out << "ERROR: garbage after " << current_column_type->getName() << ": ";
verbosePrintString(curr_position, std::min(curr_position + 10, istr.buffer().end()), out);
out << "\n";
if (current_column_type->getName() == "DateTime")
out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n";
else if (current_column_type->getName() == "Date")
out << "ERROR: Date must be in YYYY-MM-DD format.\n";
return false;
}
}
}
else
{
static const String skipped_column_str = "<SKIPPED COLUMN>";
out << "Column " << file_column << ", " << std::string((file_column < 10 ? 2 : file_column < 100 ? 1 : 0), ' ')
<< "name: " << skipped_column_str << ", " << std::string(max_length_of_column_name - skipped_column_str.length(), ' ')
<< "type: " << skipped_column_str << ", " << std::string(max_length_of_data_type_name - skipped_column_str.length(), ' ');
String tmp;
readCSVString(tmp, istr, format_settings.csv);
}
/// Delimiters
if (file_column + 1 == column_indexes_for_input_fields.size())
{
if (istr.eof())
return false;
/// we support the extra delimiter at the end of the line
if (*istr.position() == delimiter)
{
++istr.position();
if (istr.eof())
break;
}
if (!istr.eof() && *istr.position() != '\n' && *istr.position() != '\r')
{
out << "ERROR: There is no line feed. ";
verbosePrintString(istr.position(), istr.position() + 1, out);
out << " found instead.\n"
" It's like your file has more columns than expected.\n"
"And if your file have right number of columns, maybe it have unquoted string value with comma.\n";
return false;
}
skipEndOfLine(istr);
}
else
{
try
{
assertChar(delimiter, istr);
}
catch (const DB::Exception &)
{
if (*istr.position() == '\n' || *istr.position() == '\r')
{
out << "ERROR: Line feed found where delimiter (" << delimiter << ") is expected."
" It's like your file has less columns than expected.\n"
"And if your file have right number of columns, maybe it have unescaped quotes in values.\n";
}
else
{
out << "ERROR: There is no delimiter (" << delimiter << "). ";
verbosePrintString(istr.position(), istr.position() + 1, out);
out << " found instead.\n";
}
return false;
}
}
}
return true;
}
void CSVRowInputStream::syncAfterError()
{
skipToNextLineOrEOF(istr);
}
void CSVRowInputStream::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 = istr.count() - istr.offset();
pos_of_prev_row = pos_of_current_row;
pos_of_current_row = istr.position();
}
bool CSVRowInputStream::readField(IColumn & column, const DataTypePtr & type, bool is_last_file_column, size_t column_idx)
{
const bool at_delimiter = !istr.eof() || *istr.position() == format_settings.csv.delimiter;
const bool at_last_column_line_end = is_last_file_column
&& (istr.eof() || *istr.position() == '\n' || *istr.position() == '\r');
if (format_settings.csv.empty_as_default
&& (at_delimiter || at_last_column_line_end))
{
/// Treat empty unquoted column value as default value, if
/// specified in the settings. Tuple columns might seem
/// problematic, because they are never quoted but still contain
/// commas, which might be also used as delimiters. However,
/// they do not contain empty unquoted fields, so this check
/// works for tuples as well.
return false;
}
else if (column_idx_to_nullable_column_idx[column_idx])
{
/// If value is null but type is not nullable then use default value instead.
const size_t nullable_idx = *column_idx_to_nullable_column_idx[column_idx];
auto & tmp_col = *nullable_columns[nullable_idx];
nullable_types[nullable_idx]->deserializeAsTextCSV(tmp_col, istr, format_settings);
Field value = tmp_col[0];
tmp_col.popBack(1); /// do not store copy of values in memory
if (value.isNull())
return false;
column.insert(value);
return true;
}
else
{
/// Read the column normally.
type->deserializeAsTextCSV(column, istr, format_settings);
return true;
}
}
void registerInputFormatCSV(FormatFactory & factory)
{
for (bool with_names : {false, true})
{
factory.registerInputFormat(with_names ? "CSVWithNames" : "CSV", [=](
ReadBuffer & buf,
const Block & sample,
const Context &,
UInt64 max_block_size,
UInt64 rows_portion_size,
FormatFactory::ReadCallback callback,
const FormatSettings & settings)
{
return std::make_shared<BlockInputStreamFromRowInputStream>(
std::make_shared<CSVRowInputStream>(buf, sample, with_names, settings),
sample, max_block_size, rows_portion_size, callback, settings);
});
}
}
}

View File

@ -1,83 +0,0 @@
#pragma once
#include <optional>
#include <unordered_map>
#include <Core/Block.h>
#include <Formats/IRowInputStream.h>
#include <Formats/FormatSettings.h>
namespace DB
{
class ReadBuffer;
/** A stream for inputting data in csv format.
* Does not conform with https://tools.ietf.org/html/rfc4180 because it skips spaces and tabs between values.
*/
class CSVRowInputStream : public IRowInputStream
{
public:
/** with_names - in the first line the header with column names
*/
CSVRowInputStream(ReadBuffer & istr_, const Block & header_, bool with_names_, const FormatSettings & format_settings_);
bool read(MutableColumns & columns, RowReadExtension & ext) override;
void readPrefix() override;
bool allowSyncAfterError() const override { return true; }
void syncAfterError() override;
std::string getDiagnosticInfo() override;
private:
ReadBuffer & istr;
Block header;
bool with_names;
DataTypes data_types;
const FormatSettings format_settings;
using IndexesMap = std::unordered_map<String, size_t>;
IndexesMap column_indexes_by_names;
/// Maps indexes of columns in the input file to indexes of table columns
using OptionalIndexes = std::vector<std::optional<size_t>>;
OptionalIndexes column_indexes_for_input_fields;
/// Tracks which colums we have read in a single read() call.
/// For columns that are never read, it is initialized to false when we
/// read the file header, and never changed afterwards.
/// For other columns, it is updated on each read() call.
std::vector<UInt8> read_columns;
/// Whether we have any columns that are not read from file at all,
/// and must be always initialized with defaults.
bool have_always_default_columns = false;
void addInputColumn(const String & column_name);
/// For convenient diagnostics in case of an error.
size_t row_num = 0;
/// How many bytes were read, not counting those that are still in the buffer.
size_t bytes_read_at_start_of_buffer_on_current_row = 0;
size_t bytes_read_at_start_of_buffer_on_prev_row = 0;
char * pos_of_current_row = nullptr;
char * pos_of_prev_row = nullptr;
/// For setting input_format_null_as_default
DataTypes nullable_types;
MutableColumns nullable_columns;
OptionalIndexes column_idx_to_nullable_column_idx;
void updateDiagnosticInfo();
bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns,
WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name);
bool readField(IColumn & column, const DataTypePtr & type, bool is_last_file_column, size_t column_idx);
};
}

View File

@ -47,6 +47,9 @@ static FormatSettings getInputFormatSetting(const Settings & settings)
format_settings.date_time_input_format = settings.date_time_input_format;
format_settings.input_allow_errors_num = settings.input_format_allow_errors_num;
format_settings.input_allow_errors_ratio = settings.input_format_allow_errors_ratio;
format_settings.template_settings.format = settings.format_schema;
format_settings.template_settings.row_format = settings.format_schema_rows;
format_settings.template_settings.row_between_delimiter = settings.format_schema_rows_between_delimiter;
return format_settings;
}
@ -63,6 +66,9 @@ static FormatSettings getOutputFormatSetting(const Settings & settings)
format_settings.pretty.max_rows = settings.output_format_pretty_max_rows;
format_settings.pretty.max_column_pad_width = settings.output_format_pretty_max_column_pad_width;
format_settings.pretty.color = settings.output_format_pretty_color;
format_settings.template_settings.format = settings.format_schema;
format_settings.template_settings.row_format = settings.format_schema_rows;
format_settings.template_settings.row_between_delimiter = settings.format_schema_rows_between_delimiter;
format_settings.write_statistics = settings.output_format_write_statistics;
format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size;
@ -220,8 +226,6 @@ void FormatFactory::registerOutputFormatProcessor(const String & name, OutputPro
void registerInputFormatNative(FormatFactory & factory);
void registerOutputFormatNative(FormatFactory & factory);
void registerInputFormatTabSeparated(FormatFactory & factory);
void registerInputFormatCSV(FormatFactory & factory);
void registerInputFormatProcessorNative(FormatFactory & factory);
void registerOutputFormatProcessorNative(FormatFactory & factory);
@ -242,6 +246,8 @@ void registerInputFormatProcessorORC(FormatFactory & factory);
void registerOutputFormatProcessorParquet(FormatFactory & factory);
void registerInputFormatProcessorProtobuf(FormatFactory & factory);
void registerOutputFormatProcessorProtobuf(FormatFactory & factory);
void registerInputFormatProcessorTemplate(FormatFactory & factory);
void registerOutputFormatProcessorTemplate(FormatFactory &factory);
/// Output only (presentational) formats.
@ -267,8 +273,6 @@ FormatFactory::FormatFactory()
{
registerInputFormatNative(*this);
registerOutputFormatNative(*this);
registerInputFormatTabSeparated(*this);
registerInputFormatCSV(*this);
registerOutputFormatProcessorJSONEachRowWithProgress(*this);
@ -292,6 +296,8 @@ FormatFactory::FormatFactory()
registerInputFormatProcessorORC(*this);
registerInputFormatProcessorParquet(*this);
registerOutputFormatProcessorParquet(*this);
registerInputFormatProcessorTemplate(*this);
registerOutputFormatProcessorTemplate(*this);
registerOutputFormatNull(*this);

View File

@ -50,6 +50,15 @@ struct FormatSettings
Values values;
struct Template
{
String format;
String row_format;
String row_between_delimiter;
};
Template template_settings;
bool skip_unknown_fields = false;
bool with_names_use_header = false;
bool write_statistics = true;

View File

@ -0,0 +1,217 @@
#include <Formats/ParsedTemplateFormatString.h>
#include <Formats/verbosePrintString.h>
#include <IO/ReadBufferFromMemory.h>
#include <IO/Operators.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INVALID_TEMPLATE_FORMAT;
}
ParsedTemplateFormatString::ParsedTemplateFormatString(const String & format_string, const ColumnIdxGetter & idx_by_name)
{
try
{
parse(format_string, idx_by_name);
}
catch (DB::Exception & e)
{
if (e.code() != ErrorCodes::INVALID_TEMPLATE_FORMAT)
throwInvalidFormat(e.message(), columnsCount());
else
throw;
}
}
void ParsedTemplateFormatString::parse(const String & format_string, const ColumnIdxGetter & idx_by_name)
{
enum ParserState
{
Delimiter,
Column,
Format
};
const char * pos = format_string.c_str();
const char * end = format_string.c_str() + format_string.size();
const char * token_begin = pos;
ParserState state = Delimiter;
delimiters.emplace_back();
for (; *pos; ++pos)
{
switch (state)
{
case Delimiter:
if (*pos == '$')
{
delimiters.back().append(token_begin, pos - token_begin);
++pos;
if (*pos == '{')
{
token_begin = pos + 1;
state = Column;
}
else if (*pos == '$')
{
token_begin = pos;
}
else
throwInvalidFormat("at pos " + std::to_string(pos - format_string.c_str()) +
": expected '{' or '$' after '$'", columnsCount());
}
break;
case Column:
column_names.emplace_back();
pos = readMayBeQuotedColumnNameInto(pos, end - pos, column_names.back());
if (*pos == ':')
state = Format;
else if (*pos == '}')
{
formats.push_back(ColumnFormat::None);
delimiters.emplace_back();
state = Delimiter;
}
else
throwInvalidFormat("Expected ':' or '}' after column name: \"" + column_names.back() + "\"", columnsCount());
token_begin = pos + 1;
format_idx_to_column_idx.emplace_back(idx_by_name(column_names.back()));
break;
case Format:
if (*pos == '}')
{
formats.push_back(stringToFormat(String(token_begin, pos - token_begin)));
token_begin = pos + 1;
delimiters.emplace_back();
state = Delimiter;
}
}
}
if (state != Delimiter)
throwInvalidFormat("Unbalanced parentheses", columnsCount());
delimiters.back().append(token_begin, pos - token_begin);
}
ParsedTemplateFormatString::ColumnFormat ParsedTemplateFormatString::stringToFormat(const String & col_format) const
{
if (col_format.empty())
return ColumnFormat::None;
else if (col_format == "None")
return ColumnFormat::None;
else if (col_format == "Escaped")
return ColumnFormat::Escaped;
else if (col_format == "Quoted")
return ColumnFormat::Quoted;
else if (col_format == "CSV")
return ColumnFormat::Csv;
else if (col_format == "JSON")
return ColumnFormat::Json;
else if (col_format == "XML")
return ColumnFormat::Xml;
else if (col_format == "Raw")
return ColumnFormat::Raw;
else
throwInvalidFormat("Unknown field format " + col_format, columnsCount());
}
size_t ParsedTemplateFormatString::columnsCount() const
{
return format_idx_to_column_idx.size();
}
String ParsedTemplateFormatString::formatToString(ParsedTemplateFormatString::ColumnFormat format)
{
switch (format)
{
case ColumnFormat::None:
return "None";
case ColumnFormat::Escaped:
return "Escaped";
case ColumnFormat::Quoted:
return "Quoted";
case ColumnFormat::Csv:
return "CSV";
case ColumnFormat::Json:
return "Json";
case ColumnFormat::Xml:
return "Xml";
case ColumnFormat::Raw:
return "Raw";
}
__builtin_unreachable();
}
const char * ParsedTemplateFormatString::readMayBeQuotedColumnNameInto(const char * pos, size_t size, String & s)
{
s.clear();
if (!size)
return pos;
ReadBufferFromMemory buf{pos, size};
if (*pos == '"')
readDoubleQuotedStringWithSQLStyle(s, buf);
else if (*pos == '`')
readBackQuotedStringWithSQLStyle(s, buf);
else if (isWordCharASCII(*pos))
{
size_t name_size = 1;
while (name_size < size && isWordCharASCII(*(pos + name_size)))
++name_size;
s = String{pos, name_size};
return pos + name_size;
}
return pos + buf.count();
}
String ParsedTemplateFormatString::dump() const
{
WriteBufferFromOwnString res;
res << "Delimiter " << 0 << ": ";
verbosePrintString(delimiters.front().c_str(), delimiters.front().c_str() + delimiters.front().size(), res);
size_t num_columns = std::max(formats.size(), format_idx_to_column_idx.size());
for (size_t i = 0; i < num_columns; ++i)
{
res << "\nColumn " << i << ": \"";
if (column_names.size() <= i)
res << "<ERROR>";
else if (column_names[i].empty())
res << "<SKIPPED>";
else
res << column_names[i];
res << "\" (mapped to table column ";
if (format_idx_to_column_idx.size() <= i)
res << "<ERROR>";
else if (!format_idx_to_column_idx[i])
res << "<SKIPPED>";
else
res << *format_idx_to_column_idx[i];
res << "), Format " << (i < formats.size() ? formatToString(formats[i]) : "<ERROR>");
res << "\nDelimiter " << i + 1 << ": ";
if (delimiters.size() <= i + 1)
res << "<ERROR>";
else
verbosePrintString(delimiters[i + 1].c_str(), delimiters[i + 1].c_str() + delimiters[i + 1].size(), res);
}
return res.str();
}
void ParsedTemplateFormatString::throwInvalidFormat(const String & message, size_t column) const
{
throw Exception("Invalid format string for Template: " + message + " (near column " + std::to_string(column) +
")" + ". Parsed format string:\n" + dump() + "\n",
ErrorCodes::INVALID_TEMPLATE_FORMAT);
}
}

View File

@ -0,0 +1,51 @@
#pragma once
#include <Core/Types.h>
#include <functional>
#include <optional>
namespace DB
{
struct ParsedTemplateFormatString
{
enum class ColumnFormat
{
None,
Escaped,
Quoted,
Csv,
Json,
Xml,
Raw
};
/// Format string has syntax: "Delimiter0 ${ColumnName0:Format0} Delimiter1 ${ColumnName1:Format1} Delimiter2"
/// The following vectors is filled with corresponding values, delimiters.size() - 1 = formats.size() = format_idx_to_column_idx.size()
/// If format_idx_to_column_idx[i] has no value, then TemplateRowInputFormat will skip i-th column.
std::vector<String> delimiters;
std::vector<ColumnFormat> formats;
std::vector<std::optional<size_t>> format_idx_to_column_idx;
/// For diagnostic info
Strings column_names;
typedef std::function<std::optional<size_t>(const String &)> ColumnIdxGetter;
ParsedTemplateFormatString() = default;
ParsedTemplateFormatString(const String & format_string, const ColumnIdxGetter & idx_by_name);
void parse(const String & format_string, const ColumnIdxGetter & idx_by_name);
ColumnFormat stringToFormat(const String & format) const;
static String formatToString(ColumnFormat format);
static const char * readMayBeQuotedColumnNameInto(const char * pos, size_t size, String & s);
size_t columnsCount() const;
String dump() const;
[[noreturn]] void throwInvalidFormat(const String & message, size_t column) const;
};
}

View File

@ -1,504 +0,0 @@
#include <string>
#include <Core/Defines.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <Formats/TabSeparatedRowInputStream.h>
#include <Formats/verbosePrintString.h>
#include <Formats/FormatFactory.h>
#include <Formats/BlockInputStreamFromRowInputStream.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_DATA;
extern const int LOGICAL_ERROR;
}
static void skipTSVRow(ReadBuffer & istr, const size_t num_columns)
{
NullSink null_sink;
for (size_t i = 0; i < num_columns; ++i)
{
readEscapedStringInto(null_sink, istr);
assertChar(i == num_columns - 1 ? '\n' : '\t', istr);
}
}
/** Check for a common error case - usage of Windows line feed.
*/
static void checkForCarriageReturn(ReadBuffer & istr)
{
if (istr.position()[0] == '\r' || (istr.position() != istr.buffer().begin() && istr.position()[-1] == '\r'))
throw Exception("\nYou have carriage return (\\r, 0x0D, ASCII 13) at end of first row."
"\nIt's like your input data has DOS/Windows style line separators, that are illegal in TabSeparated format."
" You must transform your file to Unix format."
"\nBut if you really need carriage return at end of string value of last column, you need to escape it as \\r.",
ErrorCodes::INCORRECT_DATA);
}
TabSeparatedRowInputStream::TabSeparatedRowInputStream(
ReadBuffer & istr_, const Block & header_, bool with_names_, bool with_types_, const FormatSettings & format_settings_)
: istr(istr_), header(header_), with_names(with_names_), with_types(with_types_), format_settings(format_settings_)
{
const auto num_columns = header.columns();
data_types.resize(num_columns);
column_indexes_by_names.reserve(num_columns);
for (size_t i = 0; i < num_columns; ++i)
{
const auto & column_info = header.getByPosition(i);
data_types[i] = column_info.type;
column_indexes_by_names.emplace(column_info.name, i);
}
column_indexes_for_input_fields.reserve(num_columns);
read_columns.assign(num_columns, false);
}
void TabSeparatedRowInputStream::setupAllColumnsByTableSchema()
{
read_columns.assign(header.columns(), true);
column_indexes_for_input_fields.resize(header.columns());
for (size_t i = 0; i < column_indexes_for_input_fields.size(); ++i)
column_indexes_for_input_fields[i] = i;
}
void TabSeparatedRowInputStream::addInputColumn(const String & column_name)
{
const auto column_it = column_indexes_by_names.find(column_name);
if (column_it == column_indexes_by_names.end())
{
if (format_settings.skip_unknown_fields)
{
column_indexes_for_input_fields.push_back(std::nullopt);
return;
}
throw Exception(
"Unknown field found in TSV header: '" + column_name + "' " +
"at position " + std::to_string(column_indexes_for_input_fields.size()) +
"\nSet the 'input_format_skip_unknown_fields' parameter explicitly to ignore and proceed",
ErrorCodes::INCORRECT_DATA
);
}
const auto column_index = column_it->second;
if (read_columns[column_index])
throw Exception("Duplicate field found while parsing TSV header: " + column_name, ErrorCodes::INCORRECT_DATA);
read_columns[column_index] = true;
column_indexes_for_input_fields.emplace_back(column_index);
}
void TabSeparatedRowInputStream::fillUnreadColumnsWithDefaults(MutableColumns & columns, RowReadExtension & row_read_extension)
{
/// It is safe to memorize this on the first run - the format guarantees this does not change
if (unlikely(row_num == 1))
{
columns_to_fill_with_default_values.clear();
for (size_t index = 0; index < read_columns.size(); ++index)
if (read_columns[index] == 0)
columns_to_fill_with_default_values.push_back(index);
}
for (const auto column_index : columns_to_fill_with_default_values)
data_types[column_index]->insertDefaultInto(*columns[column_index]);
row_read_extension.read_columns = read_columns;
}
void TabSeparatedRowInputStream::readPrefix()
{
if (with_names || with_types)
{
/// In this format, 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.
skipBOMIfExists(istr);
}
if (with_names)
{
if (format_settings.with_names_use_header)
{
String column_name;
do
{
readEscapedString(column_name, istr);
addInputColumn(column_name);
}
while (checkChar('\t', istr));
if (!istr.eof())
{
checkForCarriageReturn(istr);
assertChar('\n', istr);
}
}
else
{
setupAllColumnsByTableSchema();
skipTSVRow(istr, column_indexes_for_input_fields.size());
}
}
else
setupAllColumnsByTableSchema();
if (with_types)
{
skipTSVRow(istr, column_indexes_for_input_fields.size());
}
}
bool TabSeparatedRowInputStream::read(MutableColumns & columns, RowReadExtension & ext)
{
if (istr.eof())
return false;
updateDiagnosticInfo();
for (size_t input_position = 0; input_position < column_indexes_for_input_fields.size(); ++input_position)
{
const auto & column_index = column_indexes_for_input_fields[input_position];
if (column_index)
{
data_types[*column_index]->deserializeAsTextEscaped(*columns[*column_index], istr, format_settings);
}
else
{
NullSink null_sink;
readEscapedStringInto(null_sink, istr);
}
/// skip separators
if (input_position + 1 < column_indexes_for_input_fields.size())
{
assertChar('\t', istr);
}
else if (!istr.eof())
{
if (unlikely(row_num == 1))
checkForCarriageReturn(istr);
assertChar('\n', istr);
}
}
fillUnreadColumnsWithDefaults(columns, ext);
return true;
}
String TabSeparatedRowInputStream::getDiagnosticInfo()
{
if (istr.eof()) /// Buffer has gone, cannot extract information about what has been parsed.
return {};
WriteBufferFromOwnString out;
MutableColumns columns = header.cloneEmptyColumns();
/// It is possible to display detailed diagnostics only if the last and next to last lines are still in the read buffer.
size_t bytes_read_at_start_of_buffer = istr.count() - istr.offset();
if (bytes_read_at_start_of_buffer != bytes_read_at_start_of_buffer_on_prev_row)
{
out << "Could not print diagnostic info because two last rows aren't in buffer (rare case)\n";
return out.str();
}
size_t max_length_of_column_name = 0;
for (size_t i = 0; i < header.columns(); ++i)
if (header.safeGetByPosition(i).name.size() > max_length_of_column_name)
max_length_of_column_name = header.safeGetByPosition(i).name.size();
size_t max_length_of_data_type_name = 0;
for (size_t i = 0; i < header.columns(); ++i)
if (header.safeGetByPosition(i).type->getName().size() > max_length_of_data_type_name)
max_length_of_data_type_name = header.safeGetByPosition(i).type->getName().size();
/// Roll back the cursor to the beginning of the previous or current line and parse all over again. But now we derive detailed information.
if (pos_of_prev_row)
{
istr.position() = pos_of_prev_row;
out << "\nRow " << (row_num - 1) << ":\n";
if (!parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name))
return out.str();
}
else
{
if (!pos_of_current_row)
{
out << "Could not print diagnostic info because parsing of data hasn't started.\n";
return out.str();
}
istr.position() = pos_of_current_row;
}
out << "\nRow " << row_num << ":\n";
parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name);
out << "\n";
return out.str();
}
/** gcc-7 generates wrong code with optimization level greater than 1.
* See tests: dbms/src/IO/tests/write_int.cpp
* and dbms/tests/queries/0_stateless/00898_parsing_bad_diagnostic_message.sh
* This is compiler bug. The bug does not present in gcc-8 and clang-8.
* Nevertheless, we don't need high optimization of this function.
*/
bool OPTIMIZE(1) TabSeparatedRowInputStream::parseRowAndPrintDiagnosticInfo(
MutableColumns & columns, WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name)
{
for (size_t input_position = 0; input_position < column_indexes_for_input_fields.size(); ++input_position)
{
if (input_position == 0 && istr.eof())
{
out << "<End of stream>\n";
return false;
}
if (column_indexes_for_input_fields[input_position].has_value())
{
const auto & column_index = *column_indexes_for_input_fields[input_position];
const auto & current_column_type = data_types[column_index];
out << "Column " << input_position << ", " << std::string((input_position < 10 ? 2 : input_position < 100 ? 1 : 0), ' ')
<< "name: " << header.safeGetByPosition(column_index).name << ", " << std::string(max_length_of_column_name - header.safeGetByPosition(column_index).name.size(), ' ')
<< "type: " << current_column_type->getName() << ", " << std::string(max_length_of_data_type_name - current_column_type->getName().size(), ' ');
auto prev_position = istr.position();
std::exception_ptr exception;
try
{
current_column_type->deserializeAsTextEscaped(*columns[column_index], istr, format_settings);
}
catch (...)
{
exception = std::current_exception();
}
auto curr_position = istr.position();
if (curr_position < prev_position)
throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR);
if (isNativeNumber(current_column_type) || isDateOrDateTime(current_column_type))
{
/// An empty string instead of a value.
if (curr_position == prev_position)
{
out << "ERROR: text ";
verbosePrintString(prev_position, std::min(prev_position + 10, istr.buffer().end()), out);
out << " is not like " << current_column_type->getName() << "\n";
return false;
}
}
out << "parsed text: ";
verbosePrintString(prev_position, curr_position, out);
if (exception)
{
if (current_column_type->getName() == "DateTime")
out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n";
else if (current_column_type->getName() == "Date")
out << "ERROR: Date must be in YYYY-MM-DD format.\n";
else
out << "ERROR\n";
return false;
}
out << "\n";
if (current_column_type->haveMaximumSizeOfValue())
{
if (*curr_position != '\n' && *curr_position != '\t')
{
out << "ERROR: garbage after " << current_column_type->getName() << ": ";
verbosePrintString(curr_position, std::min(curr_position + 10, istr.buffer().end()), out);
out << "\n";
if (current_column_type->getName() == "DateTime")
out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n";
else if (current_column_type->getName() == "Date")
out << "ERROR: Date must be in YYYY-MM-DD format.\n";
return false;
}
}
}
else
{
static const String skipped_column_str = "<SKIPPED COLUMN>";
out << "Column " << input_position << ", " << std::string((input_position < 10 ? 2 : input_position < 100 ? 1 : 0), ' ')
<< "name: " << skipped_column_str << ", " << std::string(max_length_of_column_name - skipped_column_str.length(), ' ')
<< "type: " << skipped_column_str << ", " << std::string(max_length_of_data_type_name - skipped_column_str.length(), ' ');
NullSink null_sink;
readEscapedStringInto(null_sink, istr);
}
/// Delimiters
if (input_position + 1 == column_indexes_for_input_fields.size())
{
if (!istr.eof())
{
try
{
assertChar('\n', istr);
}
catch (const DB::Exception &)
{
if (*istr.position() == '\t')
{
out << "ERROR: Tab found where line feed is expected."
" It's like your file has more columns than expected.\n"
"And if your file have right number of columns, maybe it have unescaped tab in value.\n";
}
else if (*istr.position() == '\r')
{
out << "ERROR: Carriage return found where line feed is expected."
" It's like your file has DOS/Windows style line separators, that is illegal in TabSeparated format.\n";
}
else
{
out << "ERROR: There is no line feed. ";
verbosePrintString(istr.position(), istr.position() + 1, out);
out << " found instead.\n";
}
return false;
}
}
}
else
{
try
{
assertChar('\t', istr);
}
catch (const DB::Exception &)
{
if (*istr.position() == '\n')
{
out << "ERROR: Line feed found where tab is expected."
" It's like your file has less columns than expected.\n"
"And if your file have right number of columns, maybe it have unescaped backslash in value before tab, which cause tab has escaped.\n";
}
else if (*istr.position() == '\r')
{
out << "ERROR: Carriage return found where tab is expected.\n";
}
else
{
out << "ERROR: There is no tab. ";
verbosePrintString(istr.position(), istr.position() + 1, out);
out << " found instead.\n";
}
return false;
}
}
}
return true;
}
void TabSeparatedRowInputStream::syncAfterError()
{
skipToUnescapedNextLineOrEOF(istr);
}
void TabSeparatedRowInputStream::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 = istr.count() - istr.offset();
pos_of_prev_row = pos_of_current_row;
pos_of_current_row = istr.position();
}
void registerInputFormatTabSeparated(FormatFactory & factory)
{
for (auto name : {"TabSeparated", "TSV"})
{
factory.registerInputFormat(name, [](
ReadBuffer & buf,
const Block & sample,
const Context &,
UInt64 max_block_size,
UInt64 rows_portion_size,
FormatFactory::ReadCallback callback,
const FormatSettings & settings)
{
return std::make_shared<BlockInputStreamFromRowInputStream>(
std::make_shared<TabSeparatedRowInputStream>(buf, sample, false, false, settings),
sample, max_block_size, rows_portion_size, callback, settings);
});
}
for (auto name : {"TabSeparatedWithNames", "TSVWithNames"})
{
factory.registerInputFormat(name, [](
ReadBuffer & buf,
const Block & sample,
const Context &,
UInt64 max_block_size,
UInt64 rows_portion_size,
FormatFactory::ReadCallback callback,
const FormatSettings & settings)
{
return std::make_shared<BlockInputStreamFromRowInputStream>(
std::make_shared<TabSeparatedRowInputStream>(buf, sample, true, false, settings),
sample, max_block_size, rows_portion_size, callback, settings);
});
}
for (auto name : {"TabSeparatedWithNamesAndTypes", "TSVWithNamesAndTypes"})
{
factory.registerInputFormat(name, [](
ReadBuffer & buf,
const Block & sample,
const Context &,
UInt64 max_block_size,
UInt64 rows_portion_size,
FormatFactory::ReadCallback callback,
const FormatSettings & settings)
{
return std::make_shared<BlockInputStreamFromRowInputStream>(
std::make_shared<TabSeparatedRowInputStream>(buf, sample, true, true, settings),
sample, max_block_size, rows_portion_size, callback, settings);
});
}
}
}

View File

@ -1,73 +0,0 @@
#pragma once
#include <optional>
#include <unordered_map>
#include <Core/Block.h>
#include <Formats/FormatSettings.h>
#include <Formats/IRowInputStream.h>
namespace DB
{
class ReadBuffer;
/** A stream to input data in tsv format.
*/
class TabSeparatedRowInputStream : public IRowInputStream
{
public:
/** with_names - the first line is the header with the names of the columns
* with_types - on the next line header with type names
*/
TabSeparatedRowInputStream(
ReadBuffer & istr_, const Block & header_, bool with_names_, bool with_types_, const FormatSettings & format_settings_);
bool read(MutableColumns & columns, RowReadExtension & ext) override;
void readPrefix() override;
bool allowSyncAfterError() const override { return true; }
void syncAfterError() override;
std::string getDiagnosticInfo() override;
private:
ReadBuffer & istr;
Block header;
bool with_names;
bool with_types;
const FormatSettings format_settings;
DataTypes data_types;
using IndexesMap = std::unordered_map<String, size_t>;
IndexesMap column_indexes_by_names;
using OptionalIndexes = std::vector<std::optional<size_t>>;
OptionalIndexes column_indexes_for_input_fields;
std::vector<UInt8> read_columns;
std::vector<size_t> columns_to_fill_with_default_values;
void addInputColumn(const String & column_name);
void setupAllColumnsByTableSchema();
void fillUnreadColumnsWithDefaults(MutableColumns & columns, RowReadExtension& ext);
/// For convenient diagnostics in case of an error.
size_t row_num = 0;
/// How many bytes were read, not counting those still in the buffer.
size_t bytes_read_at_start_of_buffer_on_current_row = 0;
size_t bytes_read_at_start_of_buffer_on_prev_row = 0;
char * pos_of_current_row = nullptr;
char * pos_of_prev_row = nullptr;
void updateDiagnosticInfo();
bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns,
WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name);
};
}

View File

@ -2,6 +2,3 @@ set(SRCS )
add_executable (tab_separated_streams tab_separated_streams.cpp ${SRCS})
target_link_libraries (tab_separated_streams PRIVATE dbms)
add_executable (block_row_transforms block_row_transforms.cpp ${SRCS})
target_link_libraries (block_row_transforms PRIVATE dbms)

View File

@ -1,57 +0,0 @@
#include <string>
#include <iostream>
#include <fstream>
#include <Core/Block.h>
#include <Core/ColumnWithTypeAndName.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/WriteBufferFromFile.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <Formats/TabSeparatedRowInputStream.h>
#include <Formats/BlockInputStreamFromRowInputStream.h>
#include <DataStreams/copyData.h>
#include <Processors/Formats/Impl/TabSeparatedRowOutputFormat.h>
#include <Processors/Formats/OutputStreamToOutputFormat.h>
int main(int, char **)
try
{
using namespace DB;
Block sample;
ColumnWithTypeAndName col1;
col1.name = "col1";
col1.type = std::make_shared<DataTypeUInt64>();
col1.column = col1.type->createColumn();
sample.insert(col1);
ColumnWithTypeAndName col2;
col2.name = "col2";
col2.type = std::make_shared<DataTypeString>();
col2.column = col2.type->createColumn();
sample.insert(col2);
ReadBufferFromFile in_buf("test_in");
WriteBufferFromFile out_buf("test_out");
FormatSettings format_settings;
RowInputStreamPtr row_input = std::make_shared<TabSeparatedRowInputStream>(in_buf, sample, false, false, format_settings);
BlockInputStreamFromRowInputStream block_input(row_input, sample, DEFAULT_INSERT_BLOCK_SIZE, 0, []{}, format_settings);
BlockOutputStreamPtr block_output = std::make_shared<OutputStreamToOutputFormat>(std::make_shared<TabSeparatedRowOutputFormat>(out_buf, sample, false, false, []{}, format_settings));
copyData(block_input, *block_output);
}
catch (const DB::Exception & e)
{
std::cerr << e.what() << ", " << e.displayText() << std::endl;
return 1;
}

View File

@ -9,12 +9,12 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <Formats/TabSeparatedRowInputStream.h>
#include <Formats/BlockInputStreamFromRowInputStream.h>
#include <Processors/Formats/Impl/TabSeparatedRowInputFormat.h>
#include <DataStreams/copyData.h>
#include <Processors/Formats/OutputStreamToOutputFormat.h>
#include <Processors/Formats/Impl/TabSeparatedRowOutputFormat.h>
#include <Processors/Formats/InputStreamFromInputFormat.h>
using namespace DB;
@ -39,13 +39,15 @@ try
FormatSettings format_settings;
RowInputStreamPtr row_input = std::make_shared<TabSeparatedRowInputStream>(in_buf, sample, false, false, format_settings);
BlockInputStreamFromRowInputStream block_input(row_input, sample, DEFAULT_INSERT_BLOCK_SIZE, 0, []{}, format_settings);
RowInputFormatParams params{DEFAULT_INSERT_BLOCK_SIZE, 0, 0, 0, []{}};
InputFormatPtr input_format = std::make_shared<TabSeparatedRowInputFormat>(sample, in_buf, params, false, false, format_settings);
BlockInputStreamPtr block_input = std::make_shared<InputStreamFromInputFormat>(std::move(input_format));
BlockOutputStreamPtr block_output = std::make_shared<OutputStreamToOutputFormat>(
std::make_shared<TabSeparatedRowOutputFormat>(out_buf, sample, false, false, [] {}, format_settings));
copyData(block_input, *block_output);
copyData(*block_input, *block_output);
return 0;
}
catch (...)

View File

@ -0,0 +1,316 @@
#include <IO/PeekableReadBuffer.h>
namespace DB
{
PeekableReadBuffer::PeekableReadBuffer(ReadBuffer & sub_buf_, size_t start_size_ /*= DBMS_DEFAULT_BUFFER_SIZE*/,
size_t unread_limit_ /* = default_limit*/)
: BufferWithOwnMemory(start_size_), sub_buf(sub_buf_), unread_limit(unread_limit_)
{
padded &= sub_buf.isPadded();
/// Read from sub-buffer
Buffer & sub_working = sub_buf.buffer();
BufferBase::set(sub_working.begin(), sub_working.size(), sub_buf.offset());
checkStateCorrect();
}
bool PeekableReadBuffer::peekNext()
{
checkStateCorrect();
size_t bytes_read = 0;
Position copy_from = pos;
size_t bytes_to_copy = sub_buf.available();
if (useSubbufferOnly())
{
/// Don't have to copy all data from sub-buffer if there is no data in own memory (checkpoint and pos are in sub-buffer)
if (checkpoint)
copy_from = checkpoint;
bytes_read = copy_from - sub_buf.buffer().begin();
bytes_to_copy = sub_buf.buffer().end() - copy_from; /// sub_buf.available();
if (!bytes_to_copy)
{
bytes += bytes_read;
sub_buf.position() = copy_from;
/// Both checkpoint and pos are at the end of sub-buffer. Just load next part of data.
bool res = sub_buf.next();
BufferBase::set(sub_buf.buffer().begin(), sub_buf.buffer().size(), sub_buf.offset());
if (checkpoint)
checkpoint = pos;
checkStateCorrect();
return res;
}
}
/// May throw an exception
resizeOwnMemoryIfNecessary(bytes_to_copy);
if (useSubbufferOnly())
{
bytes += bytes_read;
sub_buf.position() = copy_from;
}
/// Save unread data from sub-buffer to own memory
memcpy(memory.data() + peeked_size, sub_buf.position(), bytes_to_copy);
/// If useSubbufferOnly() is false, then checkpoint is in own memory and it was updated in resizeOwnMemoryIfNecessary
/// Otherwise, checkpoint now at the beginning of own memory
if (checkpoint && useSubbufferOnly())
{
checkpoint = memory.data();
checkpoint_in_own_memory = true;
}
if (currentlyReadFromOwnMemory())
{
/// Update buffer size
BufferBase::set(memory.data(), peeked_size + bytes_to_copy, offset());
}
else
{
/// Switch to reading from own memory
size_t pos_offset = peeked_size + this->offset();
if (useSubbufferOnly())
{
if (checkpoint)
pos_offset = bytes_to_copy;
else
pos_offset = 0;
}
BufferBase::set(memory.data(), peeked_size + bytes_to_copy, pos_offset);
}
peeked_size += bytes_to_copy;
sub_buf.position() += bytes_to_copy;
checkStateCorrect();
return sub_buf.next();
}
void PeekableReadBuffer::setCheckpoint()
{
checkStateCorrect();
#ifndef NDEBUG
if (checkpoint)
throw DB::Exception("Does not support recursive checkpoints.", ErrorCodes::LOGICAL_ERROR);
#endif
checkpoint_in_own_memory = currentlyReadFromOwnMemory();
if (!checkpoint_in_own_memory)
{
/// Don't need to store unread data anymore
peeked_size = 0;
}
checkpoint = pos;
checkStateCorrect();
}
void PeekableReadBuffer::dropCheckpoint()
{
checkStateCorrect();
#ifndef NDEBUG
if (!checkpoint)
throw DB::Exception("There is no checkpoint", ErrorCodes::LOGICAL_ERROR);
#endif
if (!currentlyReadFromOwnMemory())
{
/// Don't need to store unread data anymore
peeked_size = 0;
}
checkpoint = nullptr;
checkpoint_in_own_memory = false;
checkStateCorrect();
}
void PeekableReadBuffer::rollbackToCheckpoint()
{
checkStateCorrect();
if (!checkpoint)
throw DB::Exception("There is no checkpoint", ErrorCodes::LOGICAL_ERROR);
else if (checkpointInOwnMemory() == currentlyReadFromOwnMemory())
pos = checkpoint;
else /// Checkpoint is in own memory and pos is not. Switch to reading from own memory
BufferBase::set(memory.data(), peeked_size, checkpoint - memory.data());
checkStateCorrect();
}
bool PeekableReadBuffer::nextImpl()
{
/// FIXME wrong bytes count because it can read the same data again after rollbackToCheckpoint()
/// However, changing bytes count on every call of next() (even after rollback) allows to determine if some pointers were invalidated.
checkStateCorrect();
bool res;
if (!checkpoint)
{
if (!useSubbufferOnly())
{
/// All copied data have been read from own memory, continue reading from sub_buf
peeked_size = 0;
res = sub_buf.hasPendingData() || sub_buf.next();
}
else
{
/// Load next data to sub_buf
sub_buf.position() = pos;
res = sub_buf.next();
}
Buffer & sub_working = sub_buf.buffer();
/// Switch to reading from sub_buf (or just update it if already switched)
BufferBase::set(sub_working.begin(), sub_working.size(), 0);
}
else
{
if (currentlyReadFromOwnMemory())
res = sub_buf.hasPendingData() || sub_buf.next();
else
res = peekNext();
Buffer & sub_working = sub_buf.buffer();
BufferBase::set(sub_working.begin(), sub_working.size(), 0);
}
checkStateCorrect();
return res;
}
bool PeekableReadBuffer::useSubbufferOnly() const
{
return !peeked_size;
}
void PeekableReadBuffer::checkStateCorrect() const
{
#ifndef NDEBUG
if (checkpoint)
{
if (checkpointInOwnMemory())
{
if (!peeked_size)
throw DB::Exception("Checkpoint in empty own buffer", ErrorCodes::LOGICAL_ERROR);
if (currentlyReadFromOwnMemory() && pos < checkpoint)
throw DB::Exception("Current position in own buffer before checkpoint in own buffer", ErrorCodes::LOGICAL_ERROR);
}
else
{
if (peeked_size)
throw DB::Exception("Own buffer is not empty", ErrorCodes::LOGICAL_ERROR);
if (currentlyReadFromOwnMemory())
throw DB::Exception("Current position in own buffer before checkpoint in subbuffer", ErrorCodes::LOGICAL_ERROR);
if (pos < checkpoint)
throw DB::Exception("Current position in subbuffer before checkpoint in subbuffer", ErrorCodes::LOGICAL_ERROR);
}
}
else
{
if (!currentlyReadFromOwnMemory() && peeked_size)
throw DB::Exception("Own buffer is not empty", ErrorCodes::LOGICAL_ERROR);
}
if (currentlyReadFromOwnMemory() && !peeked_size)
throw DB::Exception("Pos in empty own buffer", ErrorCodes::LOGICAL_ERROR);
if (unread_limit < memory.size())
throw DB::Exception("Size limit exceed", ErrorCodes::LOGICAL_ERROR);
#endif
}
size_t PeekableReadBuffer::resizeOwnMemoryIfNecessary(size_t bytes_to_append)
{
checkStateCorrect();
bool needUpdateCheckpoint = checkpointInOwnMemory();
bool needUpdatePos = currentlyReadFromOwnMemory();
size_t offset = 0;
if (needUpdateCheckpoint)
offset = checkpoint - memory.data();
else if (needUpdatePos)
offset = this->offset();
size_t new_size = peeked_size + bytes_to_append;
if (memory.size() < new_size)
{
if (bytes_to_append < offset && 2 * (peeked_size - offset) <= memory.size())
{
/// Move unread data to the beginning of own memory instead of resize own memory
peeked_size -= offset;
memmove(memory.data(), memory.data() + offset, peeked_size);
bytes += offset;
if (needUpdateCheckpoint)
checkpoint -= offset;
if (needUpdatePos)
pos -= offset;
checkStateCorrect();
return 0;
}
else
{
if (unread_limit < new_size)
throw DB::Exception("PeekableReadBuffer: Memory limit exceed", ErrorCodes::MEMORY_LIMIT_EXCEEDED);
size_t pos_offset = pos - memory.data();
size_t new_size_amortized = memory.size() * 2;
if (new_size_amortized < new_size)
new_size_amortized = new_size;
else if (unread_limit < new_size_amortized)
new_size_amortized = unread_limit;
memory.resize(new_size_amortized);
if (needUpdateCheckpoint)
checkpoint = memory.data() + offset;
if (needUpdatePos)
{
BufferBase::set(memory.data(), peeked_size, pos_offset);
}
}
}
checkStateCorrect();
return offset;
}
PeekableReadBuffer::~PeekableReadBuffer()
{
if (!currentlyReadFromOwnMemory())
sub_buf.position() = pos;
}
std::shared_ptr<BufferWithOwnMemory<ReadBuffer>> PeekableReadBuffer::takeUnreadData()
{
checkStateCorrect();
if (!currentlyReadFromOwnMemory())
return std::make_shared<BufferWithOwnMemory<ReadBuffer>>(0);
size_t unread_size = memory.data() + peeked_size - pos;
auto unread = std::make_shared<BufferWithOwnMemory<ReadBuffer>>(unread_size);
memcpy(unread->buffer().begin(), pos, unread_size);
unread->BufferBase::set(unread->buffer().begin(), unread_size, 0);
peeked_size = 0;
checkpoint = nullptr;
checkpoint_in_own_memory = false;
BufferBase::set(sub_buf.buffer().begin(), sub_buf.buffer().size(), sub_buf.offset());
checkStateCorrect();
return unread;
}
bool PeekableReadBuffer::currentlyReadFromOwnMemory() const
{
return working_buffer.begin() != sub_buf.buffer().begin();
}
bool PeekableReadBuffer::checkpointInOwnMemory() const
{
return checkpoint_in_own_memory;
}
void PeekableReadBuffer::assertCanBeDestructed() const
{
if (peeked_size && pos != memory.data() + peeked_size)
throw DB::Exception("There are data, which were extracted from sub-buffer, but not from peekable buffer. "
"Cannot destruct peekable buffer correctly because tha data will be lost."
"Most likely it's a bug.", ErrorCodes::LOGICAL_ERROR);
}
}

View File

@ -0,0 +1,96 @@
#pragma once
#include <IO/ReadBuffer.h>
#include <IO/BufferWithOwnMemory.h>
namespace DB
{
namespace ErrorCodes
{
extern const int MEMORY_LIMIT_EXCEEDED;
extern const int LOGICAL_ERROR;
}
/// Allows to peek next part of data from sub-buffer without extracting it.
/// Also allows to set checkpoint at some position in stream and come back to this position later,
/// even if next() was called.
/// Sub-buffer should not be accessed directly during the lifelime of peekable buffer.
/// If position() of peekable buffer is explicitly set to some position before checkpoint
/// (e.g. by istr.position() = prev_pos), behavior is undefined.
class PeekableReadBuffer : public BufferWithOwnMemory<ReadBuffer>
{
friend class PeekableReadBufferCheckpoint;
public:
explicit PeekableReadBuffer(ReadBuffer & sub_buf_, size_t start_size_ = DBMS_DEFAULT_BUFFER_SIZE,
size_t unread_limit_ = 16 * DBMS_DEFAULT_BUFFER_SIZE);
/// Use takeUnreadData() to extract unread data before destruct object
~PeekableReadBuffer() override;
/// Saves unread data to own memory, so it will be possible to read it later. Loads next data to sub-buffer.
/// Doesn't change checkpoint and position in stream,
/// but all pointers (such as this->buffer().end() and this->position()) may be invalidated
/// @returns false in case of EOF in sub-buffer, otherwise returns true
bool peekNext();
Buffer & lastPeeked() { return sub_buf.buffer(); }
/// Sets checkpoint at current position
void setCheckpoint();
/// Forget checkpoint and all data between checkpoint and position
void dropCheckpoint();
/// Sets position at checkpoint.
/// All pointers (such as this->buffer().end()) may be invalidated
void rollbackToCheckpoint();
/// If position is in own memory, returns buffer with data, which were extracted from sub-buffer,
/// but not from this buffer, so the data will not be lost after destruction of this buffer.
/// If position is in sub-buffer, returns empty buffer.
std::shared_ptr<BufferWithOwnMemory<ReadBuffer>> takeUnreadData();
void assertCanBeDestructed() const;
private:
bool nextImpl() override;
inline bool useSubbufferOnly() const;
inline bool currentlyReadFromOwnMemory() const;
inline bool checkpointInOwnMemory() const;
void checkStateCorrect() const;
/// Makes possible to append `bytes_to_append` bytes to data in own memory.
/// Updates all invalidated pointers and sizes.
/// @returns new offset of unread data in own memory
size_t resizeOwnMemoryIfNecessary(size_t bytes_to_append);
ReadBuffer & sub_buf;
const size_t unread_limit;
size_t peeked_size = 0;
Position checkpoint = nullptr;
bool checkpoint_in_own_memory = false;
};
class PeekableReadBufferCheckpoint : boost::noncopyable
{
PeekableReadBuffer & buf;
bool auto_rollback;
public:
explicit PeekableReadBufferCheckpoint(PeekableReadBuffer & buf_, bool auto_rollback_ = false)
: buf(buf_), auto_rollback(auto_rollback_) { buf.setCheckpoint(); }
~PeekableReadBufferCheckpoint()
{
if (!buf.checkpoint)
return;
if (auto_rollback)
buf.rollbackToCheckpoint();
buf.dropCheckpoint();
}
};
}

View File

@ -0,0 +1,131 @@
#include <gtest/gtest.h>
#include <Core/Types.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadBufferFromString.h>
#include <IO/ConcatReadBuffer.h>
#include <IO/PeekableReadBuffer.h>
void readAndAssert(DB::ReadBuffer & buf, const char * str)
{
size_t n = strlen(str);
char tmp[n];
buf.readStrict(tmp, n);
ASSERT_EQ(strncmp(tmp, str, n), 0);
}
void assertAvailable(DB::ReadBuffer & buf, const char * str)
{
size_t n = strlen(str);
ASSERT_EQ(buf.available(), n);
ASSERT_EQ(strncmp(buf.position(), str, n), 0);
}
TEST(PeekableReadBuffer, CheckpointsWorkCorrectly)
try
{
std::string s1 = "0123456789";
std::string s2 = "qwertyuiop";
std::string s3 = "asdfghjkl;";
std::string s4 = "zxcvbnm,./";
DB::ReadBufferFromString b1(s1);
DB::ReadBufferFromString b2(s2);
DB::ReadBufferFromString b3(s3);
DB::ReadBufferFromString b4(s4);
DB::ConcatReadBuffer concat({&b1, &b2, &b3, &b4});
DB::PeekableReadBuffer peekable(concat, 0, 16);
ASSERT_TRUE(!peekable.eof());
assertAvailable(peekable, "0123456789");
{
DB::PeekableReadBufferCheckpoint checkpoint{peekable};
readAndAssert(peekable, "01234");
}
bool exception = false;
try
{
peekable.rollbackToCheckpoint();
}
catch (DB::Exception & e)
{
if (e.code() != DB::ErrorCodes::LOGICAL_ERROR)
throw;
exception = true;
}
ASSERT_TRUE(exception);
assertAvailable(peekable, "56789");
readAndAssert(peekable, "56");
peekable.setCheckpoint();
readAndAssert(peekable, "789qwertyu");
peekable.rollbackToCheckpoint();
peekable.dropCheckpoint();
assertAvailable(peekable, "789");
peekable.peekNext();
assertAvailable(peekable, "789qwertyuiop");
ASSERT_EQ(peekable.lastPeeked().size(), 10);
ASSERT_EQ(strncmp(peekable.lastPeeked().begin(), "asdfghjkl;", 10), 0);
exception = false;
try
{
DB::PeekableReadBufferCheckpoint checkpoint{peekable, true};
peekable.ignore(30);
}
catch (DB::Exception & e)
{
if (e.code() != DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED)
throw;
exception = true;
}
ASSERT_TRUE(exception);
assertAvailable(peekable, "789qwertyuiop");
ASSERT_EQ(peekable.lastPeeked().size(), 10);
ASSERT_EQ(strncmp(peekable.lastPeeked().begin(), "asdfghjkl;", 10), 0);
readAndAssert(peekable, "789qwertyu");
peekable.setCheckpoint();
readAndAssert(peekable, "iopasdfghj");
assertAvailable(peekable, "kl;");
peekable.dropCheckpoint();
peekable.setCheckpoint();
readAndAssert(peekable, "kl;zxcvbnm,./");
ASSERT_TRUE(peekable.eof());
ASSERT_TRUE(peekable.eof());
ASSERT_TRUE(peekable.eof());
peekable.rollbackToCheckpoint();
readAndAssert(peekable, "kl;zxcvbnm");
peekable.dropCheckpoint();
exception = false;
try
{
peekable.assertCanBeDestructed();
}
catch (DB::Exception & e)
{
if (e.code() != DB::ErrorCodes::LOGICAL_ERROR)
throw;
exception = true;
}
ASSERT_TRUE(exception);
auto buf_ptr = peekable.takeUnreadData();
ASSERT_TRUE(peekable.eof());
ASSERT_TRUE(peekable.eof());
ASSERT_TRUE(peekable.eof());
readAndAssert(*buf_ptr, ",./");
ASSERT_TRUE(buf_ptr->eof());
peekable.assertCanBeDestructed();
}
catch (const DB::Exception & e)
{
std::cerr << e.what() << ", " << e.displayText() << std::endl;
throw;
}

View File

@ -174,7 +174,7 @@ Chunk IRowInputFormat::generate()
{
if (params.allow_errors_num > 0 || params.allow_errors_ratio > 0)
{
Logger * log = &Logger::get("BlockInputStreamFromRowInputStream");
Logger * log = &Logger::get("IRowInputFormat");
LOG_TRACE(log, "Skipped " << num_errors << " rows with errors while reading the input stream");
}

View File

@ -5,6 +5,7 @@
#include <Processors/Formats/Impl/CSVRowInputFormat.h>
#include <Formats/FormatFactory.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeNothing.h>
namespace DB
@ -17,9 +18,9 @@ namespace ErrorCodes
}
CSVRowInputFormat::CSVRowInputFormat(
ReadBuffer & in_, Block header_, Params params_, bool with_names_, const FormatSettings & format_settings_)
: IRowInputFormat(std::move(header_), in_, std::move(params_))
CSVRowInputFormat::CSVRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_,
bool with_names_, const FormatSettings & format_settings_)
: RowInputFormatWithDiagnosticInfo(header_, in_, params_)
, with_names(with_names_)
, format_settings(format_settings_)
{
@ -79,72 +80,72 @@ void CSVRowInputFormat::addInputColumn(const String & column_name)
column_indexes_for_input_fields.emplace_back(column_index);
}
static void skipEndOfLine(ReadBuffer & istr)
static void skipEndOfLine(ReadBuffer & in)
{
/// \n (Unix) or \r\n (DOS/Windows) or \n\r (Mac OS Classic)
if (*istr.position() == '\n')
if (*in.position() == '\n')
{
++istr.position();
if (!istr.eof() && *istr.position() == '\r')
++istr.position();
++in.position();
if (!in.eof() && *in.position() == '\r')
++in.position();
}
else if (*istr.position() == '\r')
else if (*in.position() == '\r')
{
++istr.position();
if (!istr.eof() && *istr.position() == '\n')
++istr.position();
++in.position();
if (!in.eof() && *in.position() == '\n')
++in.position();
else
throw Exception("Cannot parse CSV format: found \\r (CR) not followed by \\n (LF)."
" Line must end by \\n (LF) or \\r\\n (CR LF) or \\n\\r.", ErrorCodes::INCORRECT_DATA);
}
else if (!istr.eof())
else if (!in.eof())
throw Exception("Expected end of line", ErrorCodes::INCORRECT_DATA);
}
static void skipDelimiter(ReadBuffer & istr, const char delimiter, bool is_last_column)
static void skipDelimiter(ReadBuffer & in, const char delimiter, bool is_last_column)
{
if (is_last_column)
{
if (istr.eof())
if (in.eof())
return;
/// we support the extra delimiter at the end of the line
if (*istr.position() == delimiter)
if (*in.position() == delimiter)
{
++istr.position();
if (istr.eof())
++in.position();
if (in.eof())
return;
}
skipEndOfLine(istr);
skipEndOfLine(in);
}
else
assertChar(delimiter, istr);
assertChar(delimiter, in);
}
/// Skip `whitespace` symbols allowed in CSV.
static inline void skipWhitespacesAndTabs(ReadBuffer & buf)
static inline void skipWhitespacesAndTabs(ReadBuffer & in)
{
while (!buf.eof()
&& (*buf.position() == ' '
|| *buf.position() == '\t'))
++buf.position();
while (!in.eof()
&& (*in.position() == ' '
|| *in.position() == '\t'))
++in.position();
}
static void skipRow(ReadBuffer & istr, const FormatSettings::CSV & settings, size_t num_columns)
static void skipRow(ReadBuffer & in, const FormatSettings::CSV & settings, size_t num_columns)
{
String tmp;
for (size_t i = 0; i < num_columns; ++i)
{
skipWhitespacesAndTabs(istr);
readCSVString(tmp, istr, settings);
skipWhitespacesAndTabs(istr);
skipWhitespacesAndTabs(in);
readCSVString(tmp, in, settings);
skipWhitespacesAndTabs(in);
skipDelimiter(istr, settings.delimiter, i + 1 == num_columns);
skipDelimiter(in, settings.delimiter, i + 1 == num_columns);
}
}
@ -156,7 +157,6 @@ void CSVRowInputFormat::readPrefix()
skipBOMIfExists(in);
size_t num_columns = data_types.size();
String tmp;
auto & header = getPort().getHeader();
if (with_names)
@ -224,8 +224,7 @@ bool CSVRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext
for (size_t file_column = 0; file_column < column_indexes_for_input_fields.size(); ++file_column)
{
const auto & table_column = column_indexes_for_input_fields[file_column];
const bool is_last_file_column =
file_column + 1 == column_indexes_for_input_fields.size();
const bool is_last_file_column = file_column + 1 == column_indexes_for_input_fields.size();
if (table_column)
{
@ -267,71 +266,7 @@ bool CSVRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext
return true;
}
String CSVRowInputFormat::getDiagnosticInfo()
{
if (in.eof()) /// Buffer has gone, cannot extract information about what has been parsed.
return {};
WriteBufferFromOwnString out;
auto & header = getPort().getHeader();
MutableColumns columns = header.cloneEmptyColumns();
/// It is possible to display detailed diagnostics only if the last and next to last rows are still in the read buffer.
size_t bytes_read_at_start_of_buffer = in.count() - in.offset();
if (bytes_read_at_start_of_buffer != bytes_read_at_start_of_buffer_on_prev_row)
{
out << "Could not print diagnostic info because two last rows aren't in buffer (rare case)\n";
return out.str();
}
size_t max_length_of_column_name = 0;
for (size_t i = 0; i < header.columns(); ++i)
if (header.safeGetByPosition(i).name.size() > max_length_of_column_name)
max_length_of_column_name = header.safeGetByPosition(i).name.size();
size_t max_length_of_data_type_name = 0;
for (size_t i = 0; i < header.columns(); ++i)
if (header.safeGetByPosition(i).type->getName().size() > max_length_of_data_type_name)
max_length_of_data_type_name = header.safeGetByPosition(i).type->getName().size();
/// Roll back the cursor to the beginning of the previous or current row and parse all over again. But now we derive detailed information.
if (pos_of_prev_row)
{
in.position() = pos_of_prev_row;
out << "\nRow " << (row_num - 1) << ":\n";
if (!parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name))
return out.str();
}
else
{
if (!pos_of_current_row)
{
out << "Could not print diagnostic info because parsing of data hasn't started.\n";
return out.str();
}
in.position() = pos_of_current_row;
}
out << "\nRow " << row_num << ":\n";
parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name);
out << "\n";
return out.str();
}
/** gcc-7 generates wrong code with optimization level greater than 1.
* See tests: dbms/src/IO/tests/write_int.cpp
* and dbms/tests/queries/0_stateless/00898_parsing_bad_diagnostic_message.sh
* This is compiler bug. The bug does not present in gcc-8 and clang-8.
* Nevertheless, we don't need high optimization of this function.
*/
bool OPTIMIZE(1) CSVRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & columns,
WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name)
bool CSVRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out)
{
const char delimiter = format_settings.csv.delimiter;
@ -345,100 +280,19 @@ bool OPTIMIZE(1) CSVRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumn
if (column_indexes_for_input_fields[file_column].has_value())
{
const auto & table_column = *column_indexes_for_input_fields[file_column];
const auto & current_column_type = data_types[table_column];
const bool is_last_file_column =
file_column + 1 == column_indexes_for_input_fields.size();
const bool at_delimiter = !in.eof() && *in.position() == delimiter;
const bool at_last_column_line_end = is_last_file_column
&& (in.eof() || *in.position() == '\n' || *in.position() == '\r');
auto & header = getPort().getHeader();
out << "Column " << file_column << ", " << std::string((file_column < 10 ? 2 : file_column < 100 ? 1 : 0), ' ')
<< "name: " << header.safeGetByPosition(table_column).name << ", " << std::string(max_length_of_column_name - header.safeGetByPosition(table_column).name.size(), ' ')
<< "type: " << current_column_type->getName() << ", " << std::string(max_length_of_data_type_name - current_column_type->getName().size(), ' ');
if (format_settings.csv.empty_as_default
&& (at_delimiter || at_last_column_line_end))
{
columns[table_column]->insertDefault();
}
else
{
BufferBase::Position prev_position = in.position();
BufferBase::Position curr_position = in.position();
std::exception_ptr exception;
try
{
skipWhitespacesAndTabs(in);
prev_position = in.position();
readField(*columns[table_column], current_column_type, is_last_file_column, table_column);
curr_position = in.position();
skipWhitespacesAndTabs(in);
}
catch (...)
{
exception = std::current_exception();
}
if (curr_position < prev_position)
throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR);
if (isNativeNumber(current_column_type) || isDateOrDateTime(current_column_type))
{
/// An empty string instead of a value.
if (curr_position == prev_position)
{
out << "ERROR: text ";
verbosePrintString(prev_position, std::min(prev_position + 10, in.buffer().end()), out);
out << " is not like " << current_column_type->getName() << "\n";
return false;
}
}
out << "parsed text: ";
verbosePrintString(prev_position, curr_position, out);
if (exception)
{
if (current_column_type->getName() == "DateTime")
out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n";
else if (current_column_type->getName() == "Date")
out << "ERROR: Date must be in YYYY-MM-DD format.\n";
else
out << "ERROR\n";
return false;
}
out << "\n";
if (current_column_type->haveMaximumSizeOfValue()
&& *curr_position != '\n' && *curr_position != '\r'
&& *curr_position != delimiter)
{
out << "ERROR: garbage after " << current_column_type->getName() << ": ";
verbosePrintString(curr_position, std::min(curr_position + 10, in.buffer().end()), out);
out << "\n";
if (current_column_type->getName() == "DateTime")
out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n";
else if (current_column_type->getName() == "Date")
out << "ERROR: Date must be in YYYY-MM-DD format.\n";
return false;
}
}
size_t col_idx = column_indexes_for_input_fields[file_column].value();
if (!deserializeFieldAndPrintDiagnosticInfo(header.getByPosition(col_idx).name, data_types[col_idx], *columns[col_idx],
out, file_column))
return false;
}
else
{
static const String skipped_column_str = "<SKIPPED COLUMN>";
out << "Column " << file_column << ", " << std::string((file_column < 10 ? 2 : file_column < 100 ? 1 : 0), ' ')
<< "name: " << skipped_column_str << ", " << std::string(max_length_of_column_name - skipped_column_str.length(), ' ')
<< "type: " << skipped_column_str << ", " << std::string(max_length_of_data_type_name - skipped_column_str.length(), ' ');
String tmp;
readCSVString(tmp, in, format_settings.csv);
static const DataTypePtr skipped_column_type = std::make_shared<DataTypeNothing>();
static const MutableColumnPtr skipped_column = skipped_column_type->createColumn();
if (!deserializeFieldAndPrintDiagnosticInfo(skipped_column_str, skipped_column_type, *skipped_column, out, file_column))
return false;
}
/// Delimiters
@ -502,15 +356,26 @@ void CSVRowInputFormat::syncAfterError()
skipToNextLineOrEOF(in);
}
void CSVRowInputFormat::updateDiagnosticInfo()
void CSVRowInputFormat::tryDeserializeFiled(const DataTypePtr & type, IColumn & column, size_t file_column,
ReadBuffer::Position & prev_pos, ReadBuffer::Position & curr_pos)
{
++row_num;
skipWhitespacesAndTabs(in);
prev_pos = in.position();
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();
if (column_indexes_for_input_fields[file_column])
{
const bool is_last_file_column = file_column + 1 == column_indexes_for_input_fields.size();
if (!readField(column, type, is_last_file_column, *column_indexes_for_input_fields[file_column]))
column.insertDefault();
}
else
{
String tmp;
readCSVString(tmp, in, format_settings.csv);
}
pos_of_prev_row = pos_of_current_row;
pos_of_current_row = in.position();
curr_pos = in.position();
skipWhitespacesAndTabs(in);
}
bool CSVRowInputFormat::readField(IColumn & column, const DataTypePtr & type, bool is_last_file_column, size_t column_idx)
@ -563,7 +428,7 @@ void registerInputFormatProcessorCSV(FormatFactory & factory)
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<CSVRowInputFormat>(buf, sample, std::move(params), with_names, settings);
return std::make_shared<CSVRowInputFormat>(sample, buf, params, with_names, settings);
});
}
}

View File

@ -1,40 +1,38 @@
#pragma once
#include <optional>
#include <unordered_map>
#include <Core/Block.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <Processors/Formats/RowInputFormatWithDiagnosticInfo.h>
#include <Formats/FormatSettings.h>
namespace DB
{
class ReadBuffer;
/** A stream for inputting data in csv format.
* Does not conform with https://tools.ietf.org/html/rfc4180 because it skips spaces and tabs between values.
*/
class CSVRowInputFormat : public IRowInputFormat
class CSVRowInputFormat : public RowInputFormatWithDiagnosticInfo
{
public:
/** with_names - in the first line the header with column names
* with_types - on the next line header with type names
*/
CSVRowInputFormat(ReadBuffer & in_, Block header_, Params params_, bool with_names_, const FormatSettings & format_settings_);
CSVRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_,
bool with_names_, const FormatSettings & format_settings_);
String getName() const override { return "CSVRowInputFormat"; }
bool readRow(MutableColumns & columns, RowReadExtension &) override;
bool readRow(MutableColumns & columns, RowReadExtension & ext) override;
void readPrefix() override;
bool allowSyncAfterError() const override { return true; }
void syncAfterError() override;
std::string getDiagnosticInfo() override;
private:
bool with_names;
DataTypes data_types;
const FormatSettings format_settings;
DataTypes data_types;
using IndexesMap = std::unordered_map<String, size_t>;
IndexesMap column_indexes_by_names;
@ -43,7 +41,7 @@ private:
using OptionalIndexes = std::vector<std::optional<size_t>>;
OptionalIndexes column_indexes_for_input_fields;
/// Tracks which colums we have read in a single read() call.
/// Tracks which columns we have read in a single read() call.
/// For columns that are never read, it is initialized to false when we
/// read the file header, and never changed afterwards.
/// For other columns, it is updated on each read() call.
@ -55,26 +53,19 @@ private:
void addInputColumn(const String & column_name);
/// For convenient diagnostics in case of an error.
size_t row_num = 0;
/// How many bytes were read, not counting those that are still in the buffer.
size_t bytes_read_at_start_of_buffer_on_current_row = 0;
size_t bytes_read_at_start_of_buffer_on_prev_row = 0;
char * pos_of_current_row = nullptr;
char * pos_of_prev_row = nullptr;
bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) override;
void tryDeserializeFiled(const DataTypePtr & type, IColumn & column, size_t file_column,
ReadBuffer::Position & prev_pos, ReadBuffer::Position & curr_pos) override;
bool isGarbageAfterField(size_t, ReadBuffer::Position pos) override
{
return *pos != '\n' && *pos != '\r' && *pos != format_settings.csv.delimiter;
}
/// For setting input_format_null_as_default
DataTypes nullable_types;
MutableColumns nullable_columns;
OptionalIndexes column_idx_to_nullable_column_idx;
void updateDiagnosticInfo();
bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns,
WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name);
bool readField(IColumn & column, const DataTypePtr & type, bool is_last_file_column, size_t column_idx);
};

View File

@ -5,6 +5,7 @@
#include <Processors/Formats/Impl/TabSeparatedRowInputFormat.h>
#include <Formats/verbosePrintString.h>
#include <Formats/FormatFactory.h>
#include <DataTypes/DataTypeNothing.h>
namespace DB
{
@ -16,23 +17,23 @@ namespace ErrorCodes
}
static void skipTSVRow(ReadBuffer & istr, const size_t num_columns)
static void skipTSVRow(ReadBuffer & in, const size_t num_columns)
{
NullSink null_sink;
for (size_t i = 0; i < num_columns; ++i)
{
readEscapedStringInto(null_sink, istr);
assertChar(i == num_columns - 1 ? '\n' : '\t', istr);
readEscapedStringInto(null_sink, in);
assertChar(i == num_columns - 1 ? '\n' : '\t', in);
}
}
/** Check for a common error case - usage of Windows line feed.
*/
static void checkForCarriageReturn(ReadBuffer & istr)
static void checkForCarriageReturn(ReadBuffer & in)
{
if (istr.position()[0] == '\r' || (istr.position() != istr.buffer().begin() && istr.position()[-1] == '\r'))
if (in.position()[0] == '\r' || (in.position() != in.buffer().begin() && in.position()[-1] == '\r'))
throw Exception("\nYou have carriage return (\\r, 0x0D, ASCII 13) at end of first row."
"\nIt's like your input data has DOS/Windows style line separators, that are illegal in TabSeparated format."
" You must transform your file to Unix format."
@ -41,9 +42,9 @@ static void checkForCarriageReturn(ReadBuffer & istr)
}
TabSeparatedRowInputFormat::TabSeparatedRowInputFormat(
ReadBuffer & in_, Block header_, bool with_names_, bool with_types_, Params params_, const FormatSettings & format_settings_)
: IRowInputFormat(std::move(header_), in_, std::move(params_)), with_names(with_names_), with_types(with_types_), format_settings(format_settings_)
TabSeparatedRowInputFormat::TabSeparatedRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_,
bool with_names_, bool with_types_, const FormatSettings & format_settings_)
: RowInputFormatWithDiagnosticInfo(header_, in_, params_), with_names(with_names_), with_types(with_types_), format_settings(format_settings_)
{
auto & sample = getPort().getHeader();
size_t num_columns = sample.columns();
@ -173,9 +174,9 @@ bool TabSeparatedRowInputFormat::readRow(MutableColumns & columns, RowReadExtens
updateDiagnosticInfo();
for (size_t input_position = 0; input_position < column_indexes_for_input_fields.size(); ++input_position)
for (size_t file_column = 0; file_column < column_indexes_for_input_fields.size(); ++file_column)
{
const auto & column_index = column_indexes_for_input_fields[input_position];
const auto & column_index = column_indexes_for_input_fields[file_column];
if (column_index)
{
data_types[*column_index]->deserializeAsTextEscaped(*columns[*column_index], in, format_settings);
@ -187,7 +188,7 @@ bool TabSeparatedRowInputFormat::readRow(MutableColumns & columns, RowReadExtens
}
/// skip separators
if (input_position + 1 < column_indexes_for_input_fields.size())
if (file_column + 1 < column_indexes_for_input_fields.size())
{
assertChar('\t', in);
}
@ -205,160 +206,35 @@ bool TabSeparatedRowInputFormat::readRow(MutableColumns & columns, RowReadExtens
return true;
}
String TabSeparatedRowInputFormat::getDiagnosticInfo()
bool TabSeparatedRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out)
{
if (in.eof()) /// Buffer has gone, cannot extract information about what has been parsed.
return {};
auto & header = getPort().getHeader();
WriteBufferFromOwnString out;
MutableColumns columns = header.cloneEmptyColumns();
/// It is possible to display detailed diagnostics only if the last and next to last lines are still in the read buffer.
size_t bytes_read_at_start_of_buffer = in.count() - in.offset();
if (bytes_read_at_start_of_buffer != bytes_read_at_start_of_buffer_on_prev_row)
for (size_t file_column = 0; file_column < column_indexes_for_input_fields.size(); ++file_column)
{
out << "Could not print diagnostic info because two last rows aren't in buffer (rare case)\n";
return out.str();
}
size_t max_length_of_column_name = 0;
for (size_t i = 0; i < header.columns(); ++i)
if (header.safeGetByPosition(i).name.size() > max_length_of_column_name)
max_length_of_column_name = header.safeGetByPosition(i).name.size();
size_t max_length_of_data_type_name = 0;
for (size_t i = 0; i < header.columns(); ++i)
if (header.safeGetByPosition(i).type->getName().size() > max_length_of_data_type_name)
max_length_of_data_type_name = header.safeGetByPosition(i).type->getName().size();
/// Roll back the cursor to the beginning of the previous or current line and pars all over again. But now we derive detailed information.
if (pos_of_prev_row)
{
in.position() = pos_of_prev_row;
out << "\nRow " << (row_num - 1) << ":\n";
if (!parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name))
return out.str();
}
else
{
if (!pos_of_current_row)
{
out << "Could not print diagnostic info because parsing of data hasn't started.\n";
return out.str();
}
in.position() = pos_of_current_row;
}
out << "\nRow " << row_num << ":\n";
parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name);
out << "\n";
return out.str();
}
bool TabSeparatedRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & columns,
WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name)
{
for (size_t input_position = 0; input_position < column_indexes_for_input_fields.size(); ++input_position)
{
if (input_position == 0 && in.eof())
if (file_column == 0 && in.eof())
{
out << "<End of stream>\n";
return false;
}
if (column_indexes_for_input_fields[input_position].has_value())
if (column_indexes_for_input_fields[file_column].has_value())
{
const auto & column_index = *column_indexes_for_input_fields[input_position];
const auto & current_column_type = data_types[column_index];
const auto & header = getPort().getHeader();
out << "Column " << input_position << ", " << std::string((input_position < 10 ? 2 : input_position < 100 ? 1 : 0), ' ')
<< "name: " << header.safeGetByPosition(column_index).name << ", " << std::string(max_length_of_column_name - header.safeGetByPosition(column_index).name.size(), ' ')
<< "type: " << current_column_type->getName() << ", " << std::string(max_length_of_data_type_name - current_column_type->getName().size(), ' ');
auto prev_position = in.position();
std::exception_ptr exception;
try
{
current_column_type->deserializeAsTextEscaped(*columns[column_index], in, format_settings);
}
catch (...)
{
exception = std::current_exception();
}
auto curr_position = in.position();
if (curr_position < prev_position)
throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR);
if (isNativeNumber(current_column_type) || isDateOrDateTime(current_column_type))
{
/// An empty string instead of a value.
if (curr_position == prev_position)
{
out << "ERROR: text ";
verbosePrintString(prev_position, std::min(prev_position + 10, in.buffer().end()), out);
out << " is not like " << current_column_type->getName() << "\n";
return false;
}
}
out << "parsed text: ";
verbosePrintString(prev_position, curr_position, out);
if (exception)
{
if (current_column_type->getName() == "DateTime")
out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n";
else if (current_column_type->getName() == "Date")
out << "ERROR: Date must be in YYYY-MM-DD format.\n";
else
out << "ERROR\n";
auto & header = getPort().getHeader();
size_t col_idx = column_indexes_for_input_fields[file_column].value();
if (!deserializeFieldAndPrintDiagnosticInfo(header.getByPosition(col_idx).name, data_types[col_idx], *columns[col_idx],
out, file_column))
return false;
}
out << "\n";
if (current_column_type->haveMaximumSizeOfValue())
{
if (*curr_position != '\n' && *curr_position != '\t')
{
out << "ERROR: garbage after " << current_column_type->getName() << ": ";
verbosePrintString(curr_position, std::min(curr_position + 10, in.buffer().end()), out);
out << "\n";
if (current_column_type->getName() == "DateTime")
out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n";
else if (current_column_type->getName() == "Date")
out << "ERROR: Date must be in YYYY-MM-DD format.\n";
return false;
}
}
}
else
{
static const String skipped_column_str = "<SKIPPED COLUMN>";
out << "Column " << input_position << ", " << std::string((input_position < 10 ? 2 : input_position < 100 ? 1 : 0), ' ')
<< "name: " << skipped_column_str << ", " << std::string(max_length_of_column_name - skipped_column_str.length(), ' ')
<< "type: " << skipped_column_str << ", " << std::string(max_length_of_data_type_name - skipped_column_str.length(), ' ');
NullSink null_sink;
readEscapedStringInto(null_sink, in);
static const DataTypePtr skipped_column_type = std::make_shared<DataTypeNothing>();
static const MutableColumnPtr skipped_column = skipped_column_type->createColumn();
if (!deserializeFieldAndPrintDiagnosticInfo(skipped_column_str, skipped_column_type, *skipped_column, out, file_column))
return false;
}
/// Delimiters
if (input_position + 1 == column_indexes_for_input_fields.size())
if (file_column + 1 == column_indexes_for_input_fields.size())
{
if (!in.eof())
{
@ -401,7 +277,8 @@ bool TabSeparatedRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns &
{
out << "ERROR: Line feed found where tab is expected."
" It's like your file has less columns than expected.\n"
"And if your file have right number of columns, maybe it have unescaped backslash in value before tab, which cause tab has escaped.\n";
"And if your file have right number of columns, "
"maybe it have unescaped backslash in value before tab, which cause tab has escaped.\n";
}
else if (*in.position() == '\r')
{
@ -421,6 +298,19 @@ bool TabSeparatedRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns &
return true;
}
void TabSeparatedRowInputFormat::tryDeserializeFiled(const DataTypePtr & type, IColumn & column, size_t file_column,
ReadBuffer::Position & prev_pos, ReadBuffer::Position & curr_pos)
{
prev_pos = in.position();
if (column_indexes_for_input_fields[file_column])
type->deserializeAsTextEscaped(column, in, format_settings);
else
{
NullSink null_sink;
readEscapedStringInto(null_sink, in);
}
curr_pos = in.position();
}
void TabSeparatedRowInputFormat::syncAfterError()
{
@ -428,18 +318,6 @@ void TabSeparatedRowInputFormat::syncAfterError()
}
void TabSeparatedRowInputFormat::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();
pos_of_prev_row = pos_of_current_row;
pos_of_current_row = in.position();
}
void registerInputFormatProcessorTabSeparated(FormatFactory & factory)
{
for (auto name : {"TabSeparated", "TSV"})
@ -451,7 +329,7 @@ void registerInputFormatProcessorTabSeparated(FormatFactory & factory)
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<TabSeparatedRowInputFormat>(buf, sample, false, false, std::move(params), settings);
return std::make_shared<TabSeparatedRowInputFormat>(sample, buf, params, false, false, settings);
});
}
@ -464,7 +342,7 @@ void registerInputFormatProcessorTabSeparated(FormatFactory & factory)
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<TabSeparatedRowInputFormat>(buf, sample, true, false, std::move(params), settings);
return std::make_shared<TabSeparatedRowInputFormat>(sample, buf, params, true, false, settings);
});
}
@ -477,7 +355,7 @@ void registerInputFormatProcessorTabSeparated(FormatFactory & factory)
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<TabSeparatedRowInputFormat>(buf, sample, true, true, std::move(params), settings);
return std::make_shared<TabSeparatedRowInputFormat>(sample, buf, params, true, true, settings);
});
}
}

View File

@ -2,25 +2,22 @@
#include <Core/Block.h>
#include <Formats/FormatSettings.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <Processors/Formats/RowInputFormatWithDiagnosticInfo.h>
namespace DB
{
class ReadBuffer;
/** A stream to input data in tsv format.
*/
class TabSeparatedRowInputFormat : public IRowInputFormat
class TabSeparatedRowInputFormat : public RowInputFormatWithDiagnosticInfo
{
public:
/** with_names - the first line is the header with the names of the columns
* with_types - on the next line header with type names
*/
TabSeparatedRowInputFormat(
ReadBuffer & in_, Block header_, bool with_names_, bool with_types_, Params params_, const FormatSettings & format_settings_);
TabSeparatedRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_,
bool with_names_, bool with_types_, const FormatSettings & format_settings_);
String getName() const override { return "TabSeparatedRowInputFormat"; }
@ -29,8 +26,6 @@ public:
bool allowSyncAfterError() const override { return true; }
void syncAfterError() override;
std::string getDiagnosticInfo() override;
private:
bool with_names;
bool with_types;
@ -50,21 +45,10 @@ private:
void setupAllColumnsByTableSchema();
void fillUnreadColumnsWithDefaults(MutableColumns & columns, RowReadExtension& ext);
/// For convenient diagnostics in case of an error.
size_t row_num = 0;
/// How many bytes were read, not counting those still in the buffer.
size_t bytes_read_at_start_of_buffer_on_current_row = 0;
size_t bytes_read_at_start_of_buffer_on_prev_row = 0;
char * pos_of_current_row = nullptr;
char * pos_of_prev_row = nullptr;
void updateDiagnosticInfo();
bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns,
WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name);
bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) override;
void tryDeserializeFiled(const DataTypePtr & type, IColumn & column, size_t file_column,
ReadBuffer::Position & prev_pos, ReadBuffer::Position & curr_pos) override;
bool isGarbageAfterField(size_t, ReadBuffer::Position pos) override { return *pos != '\n' && *pos != '\t'; }
};
}

View File

@ -0,0 +1,243 @@
#include <Processors/Formats/Impl/TemplateBlockOutputFormat.h>
#include <Formats/FormatFactory.h>
#include <IO/WriteHelpers.h>
#include <DataTypes/DataTypesNumber.h>
namespace DB
{
namespace ErrorCodes
{
extern const int SYNTAX_ERROR;
}
TemplateBlockOutputFormat::TemplateBlockOutputFormat(const Block & header_, WriteBuffer & out_, const FormatSettings & settings_)
: IOutputFormat(header_, out_), settings(settings_)
{
auto & sample = getPort(PortKind::Main).getHeader();
size_t columns = sample.columns();
types.resize(columns);
for (size_t i = 0; i < columns; ++i)
types[i] = sample.safeGetByPosition(i).type;
/// Parse format string for whole output
static const String default_format("${data}");
const String & format_str = settings.template_settings.format.empty() ? default_format : settings.template_settings.format;
format = ParsedTemplateFormatString(format_str, [&](const String & partName)
{
return static_cast<size_t>(stringToOutputPart(partName));
});
/// Validate format string for whole output
size_t data_idx = format.format_idx_to_column_idx.size() + 1;
for (size_t i = 0; i < format.format_idx_to_column_idx.size(); ++i)
{
if (!format.format_idx_to_column_idx[i])
format.throwInvalidFormat("Output part name cannot be empty, it's a bug.", i);
switch (static_cast<OutputPart>(*format.format_idx_to_column_idx[i]))
{
case OutputPart::Data:
data_idx = i;
[[fallthrough]];
case OutputPart::Totals:
case OutputPart::ExtremesMin:
case OutputPart::ExtremesMax:
if (format.formats[i] != ColumnFormat::None)
format.throwInvalidFormat("Serialization type for data, totals, min and max must be empty or None", i);
break;
default:
if (format.formats[i] == ColumnFormat::None)
format.throwInvalidFormat("Serialization type for output part rows, rows_before_limit, time, "
"rows_read or bytes_read is not specified", i);
break;
}
}
if (data_idx != 0)
format.throwInvalidFormat("${data} must be the first output part", 0);
/// Parse format string for rows
row_format = ParsedTemplateFormatString(settings.template_settings.row_format, [&](const String & colName)
{
return sample.getPositionByName(colName);
});
/// Validate format string for rows
if (row_format.delimiters.size() == 1)
row_format.throwInvalidFormat("No columns specified", 0);
for (size_t i = 0; i < row_format.columnsCount(); ++i)
{
if (!row_format.format_idx_to_column_idx[i])
row_format.throwInvalidFormat("Cannot skip format field for output, it's a bug.", i);
if (row_format.formats[i] == ColumnFormat::None)
row_format.throwInvalidFormat("Serialization type for file column is not specified", i);
}
}
TemplateBlockOutputFormat::OutputPart TemplateBlockOutputFormat::stringToOutputPart(const String & part)
{
if (part == "data")
return OutputPart::Data;
else if (part == "totals")
return OutputPart::Totals;
else if (part == "min")
return OutputPart::ExtremesMin;
else if (part == "max")
return OutputPart::ExtremesMax;
else if (part == "rows")
return OutputPart::Rows;
else if (part == "rows_before_limit")
return OutputPart::RowsBeforeLimit;
else if (part == "time")
return OutputPart::TimeElapsed;
else if (part == "rows_read")
return OutputPart::RowsRead;
else if (part == "bytes_read")
return OutputPart::BytesRead;
else
throw Exception("Unknown output part " + part, ErrorCodes::SYNTAX_ERROR);
}
void TemplateBlockOutputFormat::writeRow(const Chunk & chunk, size_t row_num)
{
size_t columns = row_format.format_idx_to_column_idx.size();
for (size_t j = 0; j < columns; ++j)
{
writeString(row_format.delimiters[j], out);
size_t col_idx = *row_format.format_idx_to_column_idx[j];
serializeField(*chunk.getColumns()[col_idx], *types[col_idx], row_num, row_format.formats[j]);
}
writeString(row_format.delimiters[columns], out);
}
void TemplateBlockOutputFormat::serializeField(const IColumn & column, const IDataType & type, size_t row_num, ColumnFormat col_format)
{
switch (col_format)
{
case ColumnFormat::Escaped:
type.serializeAsTextEscaped(column, row_num, out, settings);
break;
case ColumnFormat::Quoted:
type.serializeAsTextQuoted(column, row_num, out, settings);
break;
case ColumnFormat::Csv:
type.serializeAsTextCSV(column, row_num, out, settings);
break;
case ColumnFormat::Json:
type.serializeAsTextJSON(column, row_num, out, settings);
break;
case ColumnFormat::Xml:
type.serializeAsTextXML(column, row_num, out, settings);
break;
case ColumnFormat::Raw:
type.serializeAsText(column, row_num, out, settings);
break;
default:
__builtin_unreachable();
}
}
template <typename U, typename V> void TemplateBlockOutputFormat::writeValue(U value, ColumnFormat col_format)
{
auto type = std::make_unique<V>();
auto col = type->createColumn();
col->insert(value);
serializeField(*col, *type, 0, col_format);
}
void TemplateBlockOutputFormat::consume(Chunk chunk)
{
doWritePrefix();
size_t rows = chunk.getNumRows();
for (size_t i = 0; i < rows; ++i)
{
if (row_count)
writeString(settings.template_settings.row_between_delimiter, out);
writeRow(chunk, i);
++row_count;
}
}
void TemplateBlockOutputFormat::doWritePrefix()
{
if (need_write_prefix)
{
writeString(format.delimiters.front(), out);
need_write_prefix = false;
}
}
void TemplateBlockOutputFormat::finalize()
{
if (finalized)
return;
doWritePrefix();
size_t parts = format.format_idx_to_column_idx.size();
for (size_t i = 0; i < parts; ++i)
{
auto type = std::make_shared<DataTypeUInt64>();
ColumnWithTypeAndName col(type->createColumnConst(1, row_count), type, String("tmp"));
switch (static_cast<OutputPart>(*format.format_idx_to_column_idx[i]))
{
case OutputPart::Totals:
if (!totals)
format.throwInvalidFormat("Cannot print totals for this request", i);
writeRow(totals, 0);
break;
case OutputPart::ExtremesMin:
if (!extremes)
format.throwInvalidFormat("Cannot print extremes for this request", i);
writeRow(extremes, 0);
break;
case OutputPart::ExtremesMax:
if (!extremes)
format.throwInvalidFormat("Cannot print extremes for this request", i);
writeRow(extremes, 1);
break;
case OutputPart::Rows:
writeValue<size_t, DataTypeUInt64>(row_count, format.formats[i]);
break;
case OutputPart::RowsBeforeLimit:
if (!rows_before_limit_set)
format.throwInvalidFormat("Cannot print rows_before_limit for this request", i);
writeValue<size_t, DataTypeUInt64>(rows_before_limit, format.formats[i]);
break;
case OutputPart::TimeElapsed:
writeValue<double, DataTypeFloat64>(watch.elapsedSeconds(), format.formats[i]);
break;
case OutputPart::RowsRead:
writeValue<size_t, DataTypeUInt64>(progress.read_rows.load(), format.formats[i]);
break;
case OutputPart::BytesRead:
writeValue<size_t, DataTypeUInt64>(progress.read_bytes.load(), format.formats[i]);
break;
default:
break;
}
writeString(format.delimiters[i + 1], out);
}
finalized = true;
}
void registerOutputFormatProcessorTemplate(FormatFactory & factory)
{
factory.registerOutputFormatProcessor("Template", [](
WriteBuffer & buf,
const Block & sample,
const Context &,
FormatFactory::WriteCallback,
const FormatSettings & settings)
{
return std::make_shared<TemplateBlockOutputFormat>(sample, buf, settings);
});
}
}

View File

@ -0,0 +1,68 @@
#pragma once
#include <Common/Stopwatch.h>
#include <Core/Block.h>
#include <Formats/FormatSettings.h>
#include <Processors/Formats/IOutputFormat.h>
#include <Formats/ParsedTemplateFormatString.h>
namespace DB
{
class TemplateBlockOutputFormat : public IOutputFormat
{
using ColumnFormat = ParsedTemplateFormatString::ColumnFormat;
public:
TemplateBlockOutputFormat(const Block & header_, WriteBuffer & out_, const FormatSettings & settings_);
String getName() const override { return "TemplateBlockOutputFormat"; }
void doWritePrefix() override;
void setRowsBeforeLimit(size_t rows_before_limit_) override { rows_before_limit = rows_before_limit_; rows_before_limit_set = true; }
void onProgress(const Progress & progress_) override { progress.incrementPiecewiseAtomically(progress_); }
protected:
void consume(Chunk chunk) override;
void consumeTotals(Chunk chunk) override { totals = std::move(chunk); }
void consumeExtremes(Chunk chunk) override { extremes = std::move(chunk); }
void finalize() override;
enum class OutputPart : size_t
{
Data,
Totals,
ExtremesMin,
ExtremesMax,
Rows,
RowsBeforeLimit,
TimeElapsed,
RowsRead,
BytesRead
};
OutputPart stringToOutputPart(const String & part);
void writeRow(const Chunk & chunk, size_t row_num);
void serializeField(const IColumn & column, const IDataType & type, size_t row_num, ColumnFormat format);
template <typename U, typename V> void writeValue(U value, ColumnFormat col_format);
protected:
const FormatSettings settings;
DataTypes types;
ParsedTemplateFormatString format;
ParsedTemplateFormatString row_format;
size_t rows_before_limit = 0;
bool rows_before_limit_set = false;
Chunk totals;
Chunk extremes;
Progress progress;
Stopwatch watch;
size_t row_count = 0;
bool need_write_prefix = true;
};
}

View File

@ -0,0 +1,520 @@
#include <Processors/Formats/Impl/TemplateRowInputFormat.h>
#include <Formats/FormatFactory.h>
#include <Formats/verbosePrintString.h>
#include <IO/Operators.h>
#include <DataTypes/DataTypeNothing.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ATTEMPT_TO_READ_AFTER_EOF;
extern const int CANNOT_READ_ALL_DATA;
extern const int CANNOT_PARSE_ESCAPE_SEQUENCE;
extern const int CANNOT_PARSE_QUOTED_STRING;
extern const int SYNTAX_ERROR;
}
TemplateRowInputFormat::TemplateRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_,
const FormatSettings & settings_, bool ignore_spaces_)
: RowInputFormatWithDiagnosticInfo(header_, buf, params_), buf(in_), data_types(header_.getDataTypes()),
settings(settings_), ignore_spaces(ignore_spaces_)
{
/// Parse format string for whole input
static const String default_format("${data}");
const String & format_str = settings.template_settings.format.empty() ? default_format : settings.template_settings.format;
format = ParsedTemplateFormatString(format_str, [&](const String & partName) -> std::optional<size_t>
{
if (partName == "data")
return 0;
else if (partName.empty()) /// For skipping some values in prefix and suffix
#if !__clang__
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wmaybe-uninitialized"
#endif
/// Suppress false-positive warning (bug in GCC 9: https://gcc.gnu.org/bugzilla/show_bug.cgi?id=86465)
return {};
#if !__clang__
#pragma GCC diagnostic pop
#endif
throw Exception("Unknown input part " + partName, ErrorCodes::SYNTAX_ERROR);
});
/// Validate format string for whole input
bool has_data = false;
for (size_t i = 0; i < format.columnsCount(); ++i)
{
if (format.format_idx_to_column_idx[i])
{
if (has_data)
format.throwInvalidFormat("${data} can occur only once", i);
if (format.formats[i] != ColumnFormat::None)
format.throwInvalidFormat("${data} must have empty or None deserialization type", i);
has_data = true;
format_data_idx = i;
}
else
{
if (format.formats[i] == ColumnFormat::Xml || format.formats[i] == ColumnFormat::Raw)
format.throwInvalidFormat("XML and Raw deserialization is not supported", i);
}
}
/// Parse format string for rows
row_format = ParsedTemplateFormatString(settings.template_settings.row_format, [&](const String & colName) -> std::optional<size_t>
{
if (colName.empty())
#if !__clang__
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wmaybe-uninitialized"
#endif
return {};
#if !__clang__
#pragma GCC diagnostic pop
#endif
return header_.getPositionByName(colName);
});
/// Validate format string for rows
std::vector<UInt8> column_in_format(header_.columns(), false);
for (size_t i = 0; i < row_format.columnsCount(); ++i)
{
if (row_format.formats[i] == ColumnFormat::Xml || row_format.formats[i] == ColumnFormat::Raw)
row_format.throwInvalidFormat("XML and Raw deserialization is not supported", i);
if (row_format.format_idx_to_column_idx[i])
{
if (row_format.formats[i] == ColumnFormat::None)
row_format.throwInvalidFormat("Column is not skipped, but deserialization type is None", i);
size_t col_idx = *row_format.format_idx_to_column_idx[i];
if (column_in_format[col_idx])
row_format.throwInvalidFormat("Duplicate column", i);
column_in_format[col_idx] = true;
}
}
}
void TemplateRowInputFormat::readPrefix()
{
size_t last_successfully_parsed_idx = 0;
try
{
tryReadPrefixOrSuffix<void>(last_successfully_parsed_idx, format_data_idx);
}
catch (Exception & e)
{
format.throwInvalidFormat(e.message() + " While parsing prefix", last_successfully_parsed_idx);
}
}
/// Asserts delimiters and skips fields in prefix or suffix.
/// tryReadPrefixOrSuffix<bool>(...) is used in checkForSuffix() to avoid throwing an exception after read of each row
/// (most likely false will be returned on first call of checkString(...))
template <typename ReturnType>
ReturnType TemplateRowInputFormat::tryReadPrefixOrSuffix(size_t & input_part_beg, size_t input_part_end)
{
static constexpr bool throw_exception = std::is_same_v<ReturnType, void>;
skipSpaces();
if constexpr (throw_exception)
assertString(format.delimiters[input_part_beg], buf);
else
{
if (likely(!checkString(format.delimiters[input_part_beg], buf)))
return ReturnType(false);
}
while (input_part_beg < input_part_end)
{
skipSpaces();
if constexpr (throw_exception)
skipField(format.formats[input_part_beg]);
else
{
try
{
skipField(format.formats[input_part_beg]);
}
catch (const Exception & e)
{
if (e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF &&
e.code() != ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE &&
e.code() != ErrorCodes::CANNOT_PARSE_QUOTED_STRING)
throw;
/// If it's parsing error, then suffix is not found
return ReturnType(false);
}
}
++input_part_beg;
skipSpaces();
if constexpr (throw_exception)
assertString(format.delimiters[input_part_beg], buf);
else
{
if (likely(!checkString(format.delimiters[input_part_beg], buf)))
return ReturnType(false);
}
}
if constexpr (!throw_exception)
return ReturnType(true);
}
bool TemplateRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & extra)
{
/// This function can be called again after it returned false
if (unlikely(end_of_stream))
return false;
skipSpaces();
if (unlikely(checkForSuffix()))
{
end_of_stream = true;
return false;
}
updateDiagnosticInfo();
if (likely(row_num != 1))
assertString(settings.template_settings.row_between_delimiter, buf);
extra.read_columns.assign(columns.size(), false);
for (size_t i = 0; i < row_format.columnsCount(); ++i)
{
skipSpaces();
assertString(row_format.delimiters[i], buf);
skipSpaces();
if (row_format.format_idx_to_column_idx[i])
{
size_t col_idx = *row_format.format_idx_to_column_idx[i];
deserializeField(*data_types[col_idx], *columns[col_idx], row_format.formats[i]);
extra.read_columns[col_idx] = true;
}
else
skipField(row_format.formats[i]);
}
skipSpaces();
assertString(row_format.delimiters.back(), buf);
for (size_t i = 0; i < columns.size(); ++i)
if (!extra.read_columns[i])
data_types[i]->insertDefaultInto(*columns[i]);
return true;
}
void TemplateRowInputFormat::deserializeField(const IDataType & type, IColumn & column, ColumnFormat col_format)
{
try
{
switch (col_format)
{
case ColumnFormat::Escaped:
type.deserializeAsTextEscaped(column, buf, settings);
break;
case ColumnFormat::Quoted:
type.deserializeAsTextQuoted(column, buf, settings);
break;
case ColumnFormat::Csv:
type.deserializeAsTextCSV(column, buf, settings);
break;
case ColumnFormat::Json:
type.deserializeAsTextJSON(column, buf, settings);
break;
default:
__builtin_unreachable();
}
}
catch (Exception & e)
{
if (e.code() == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF)
throwUnexpectedEof();
throw;
}
}
void TemplateRowInputFormat::skipField(TemplateRowInputFormat::ColumnFormat col_format)
{
String tmp;
constexpr const char * field_name = "<SKIPPED COLUMN>";
constexpr size_t field_name_len = 16;
try
{
switch (col_format)
{
case ColumnFormat::None:
/// Empty field, just skip spaces
break;
case ColumnFormat::Escaped:
readEscapedString(tmp, buf);
break;
case ColumnFormat::Quoted:
readQuotedString(tmp, buf);
break;
case ColumnFormat::Csv:
readCSVString(tmp, buf, settings.csv);
break;
case ColumnFormat::Json:
skipJSONField(buf, StringRef(field_name, field_name_len));
break;
default:
__builtin_unreachable();
}
}
catch (Exception & e)
{
if (e.code() == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF)
throwUnexpectedEof();
throw;
}
}
/// Returns true if all rows have been read i.e. there are only suffix and spaces (if ignore_spaces == true) before EOF.
/// Otherwise returns false
bool TemplateRowInputFormat::checkForSuffix()
{
PeekableReadBufferCheckpoint checkpoint{buf};
bool suffix_found = false;
size_t last_successfully_parsed_idx = format_data_idx + 1;
try
{
suffix_found = tryReadPrefixOrSuffix<bool>(last_successfully_parsed_idx, format.columnsCount());
}
catch (const Exception & e)
{
if (e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF &&
e.code() != ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE &&
e.code() != ErrorCodes::CANNOT_PARSE_QUOTED_STRING)
throw;
}
if (unlikely(suffix_found))
{
skipSpaces();
if (buf.eof())
return true;
}
buf.rollbackToCheckpoint();
return false;
}
bool TemplateRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out)
{
out << "Suffix does not match: ";
size_t last_successfully_parsed_idx = format_data_idx + 1;
const ReadBuffer::Position row_begin_pos = buf.position();
bool caught = false;
try
{
PeekableReadBufferCheckpoint checkpoint{buf, true};
tryReadPrefixOrSuffix<void>(last_successfully_parsed_idx, format.columnsCount());
}
catch (Exception & e)
{
out << e.message() << " Near column " << last_successfully_parsed_idx;
caught = true;
}
if (!caught)
{
out << " There is some data after suffix (EOF expected, got ";
verbosePrintString(buf.position(), std::min(buf.buffer().end(), buf.position() + 16), out);
out << "). ";
}
out << " Format string (from format_schema): \n" << format.dump() << "\n";
if (row_begin_pos != buf.position())
{
/// Pointers to buffer memory were invalidated during checking for suffix
out << "\nCannot print more diagnostic info.";
return false;
}
out << "\nUsing format string (from format_schema_rows): " << row_format.dump() << "\n";
out << "\nTrying to parse next row, because suffix does not match:\n";
try
{
if (likely(row_num != 1))
assertString(settings.template_settings.row_between_delimiter, buf);
}
catch (const DB::Exception &)
{
writeErrorStringForWrongDelimiter(out, "delimiter between rows", settings.template_settings.row_between_delimiter);
return false;
}
for (size_t i = 0; i < row_format.columnsCount(); ++i)
{
skipSpaces();
try
{
assertString(row_format.delimiters[i], buf);
}
catch (const DB::Exception &)
{
writeErrorStringForWrongDelimiter(out, "delimiter before field " + std::to_string(i), row_format.delimiters[i]);
return false;
}
skipSpaces();
if (row_format.format_idx_to_column_idx[i])
{
auto & header = getPort().getHeader();
size_t col_idx = *row_format.format_idx_to_column_idx[i];
if (!deserializeFieldAndPrintDiagnosticInfo(header.getByPosition(col_idx).name, data_types[col_idx],
*columns[col_idx], out, i))
{
out << "Maybe it's not possible to deserialize field " + std::to_string(i) +
" as " + ParsedTemplateFormatString::formatToString(row_format.formats[i]);
return false;
}
}
else
{
static const String skipped_column_str = "<SKIPPED COLUMN>";
static const DataTypePtr skipped_column_type = std::make_shared<DataTypeNothing>();
static const MutableColumnPtr skipped_column = skipped_column_type->createColumn();
if (!deserializeFieldAndPrintDiagnosticInfo(skipped_column_str, skipped_column_type, *skipped_column, out, i))
return false;
}
}
skipSpaces();
try
{
assertString(row_format.delimiters.back(), buf);
}
catch (const DB::Exception &)
{
writeErrorStringForWrongDelimiter(out, "delimiter after last field", row_format.delimiters.back());
return false;
}
return true;
}
void TemplateRowInputFormat::writeErrorStringForWrongDelimiter(WriteBuffer & out, const String & description, const String & delim)
{
out << "ERROR: There is no " << description << ": expected ";
verbosePrintString(delim.data(), delim.data() + delim.size(), out);
out << ", got ";
if (buf.eof())
out << "<End of stream>";
else
verbosePrintString(buf.position(), std::min(buf.position() + delim.size() + 10, buf.buffer().end()), out);
out << '\n';
}
void TemplateRowInputFormat::tryDeserializeFiled(const DataTypePtr & type, IColumn & column, size_t file_column,
ReadBuffer::Position & prev_pos, ReadBuffer::Position & curr_pos)
{
prev_pos = buf.position();
if (row_format.format_idx_to_column_idx[file_column])
deserializeField(*type, column, row_format.formats[file_column]);
else
skipField(row_format.formats[file_column]);
curr_pos = buf.position();
}
bool TemplateRowInputFormat::isGarbageAfterField(size_t, ReadBuffer::Position)
{
/// Garbage will be considered as wrong delimiter
return false;
}
bool TemplateRowInputFormat::allowSyncAfterError() const
{
return !row_format.delimiters.back().empty() || !settings.template_settings.row_between_delimiter.empty();
}
void TemplateRowInputFormat::syncAfterError()
{
bool at_beginning_of_row_or_eof = false;
while (!at_beginning_of_row_or_eof)
{
skipToNextDelimiterOrEof(row_format.delimiters.back());
if (buf.eof())
{
end_of_stream = true;
return;
}
buf.ignore(row_format.delimiters.back().size());
skipSpaces();
if (checkForSuffix())
return;
bool last_delimiter_in_row_found = !row_format.delimiters.back().empty();
if (last_delimiter_in_row_found && checkString(settings.template_settings.row_between_delimiter, buf))
at_beginning_of_row_or_eof = true;
else
skipToNextDelimiterOrEof(settings.template_settings.row_between_delimiter);
if (buf.eof())
at_beginning_of_row_or_eof = end_of_stream = true;
}
/// It can happen that buf.position() is not at the beginning of row
/// if some delimiters is similar to row_format.delimiters.back() and row_between_delimiter.
/// It will cause another parsing error.
}
/// Searches for delimiter in input stream and sets buffer position to the beginning of delimiter (if found) or EOF (if not)
void TemplateRowInputFormat::skipToNextDelimiterOrEof(const String & delimiter)
{
if (delimiter.empty())
return;
while (!buf.eof())
{
void * pos = memchr(buf.position(), delimiter[0], buf.available());
if (!pos)
{
buf.position() += buf.available();
continue;
}
buf.position() = static_cast<ReadBuffer::Position>(pos);
PeekableReadBufferCheckpoint checkpoint{buf};
if (checkString(delimiter, buf))
return;
buf.rollbackToCheckpoint();
++buf.position();
}
}
void TemplateRowInputFormat::throwUnexpectedEof()
{
throw Exception("Unexpected EOF while parsing row " + std::to_string(row_num) + ". "
"Maybe last row has wrong format or input doesn't contain specified suffix before EOF.",
ErrorCodes::CANNOT_READ_ALL_DATA);
}
void registerInputFormatProcessorTemplate(FormatFactory & factory)
{
for (bool ignore_spaces : {false, true})
{
factory.registerInputFormatProcessor(ignore_spaces ? "TemplateIgnoreSpaces" : "Template", [=](
ReadBuffer & buf,
const Block & sample,
const Context &,
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<TemplateRowInputFormat>(sample, buf, params, settings, ignore_spaces);
});
}
}
}

View File

@ -0,0 +1,61 @@
#pragma once
#include <Core/Block.h>
#include <Processors/Formats/RowInputFormatWithDiagnosticInfo.h>
#include <Formats/FormatSettings.h>
#include <Formats/ParsedTemplateFormatString.h>
#include <IO/ReadHelpers.h>
#include <IO/PeekableReadBuffer.h>
namespace DB
{
class TemplateRowInputFormat : public RowInputFormatWithDiagnosticInfo
{
using ColumnFormat = ParsedTemplateFormatString::ColumnFormat;
public:
TemplateRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_,
const FormatSettings & settings_, bool ignore_spaces_);
String getName() const override { return "TemplateRowInputFormat"; }
bool readRow(MutableColumns & columns, RowReadExtension & extra) override;
void readPrefix() override;
bool allowSyncAfterError() const override;
void syncAfterError() override;
private:
void deserializeField(const IDataType & type, IColumn & column, ColumnFormat col_format);
void skipField(ColumnFormat col_format);
inline void skipSpaces() { if (ignore_spaces) skipWhitespaceIfAny(buf); }
template <typename ReturnType = void>
ReturnType tryReadPrefixOrSuffix(size_t & input_part_beg, size_t input_part_end);
bool checkForSuffix();
[[noreturn]] void throwUnexpectedEof();
bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) override;
void tryDeserializeFiled(const DataTypePtr & type, IColumn & column, size_t file_column, ReadBuffer::Position & prev_pos,
ReadBuffer::Position & curr_pos) override;
bool isGarbageAfterField(size_t after_col_idx, ReadBuffer::Position pos) override;
void writeErrorStringForWrongDelimiter(WriteBuffer & out, const String & description, const String & delim);
void skipToNextDelimiterOrEof(const String & delimiter);
private:
PeekableReadBuffer buf;
DataTypes data_types;
FormatSettings settings;
ParsedTemplateFormatString format;
ParsedTemplateFormatString row_format;
const bool ignore_spaces;
size_t format_data_idx;
bool end_of_stream = false;
};
}

View File

@ -0,0 +1,167 @@
#include <Processors/Formats/RowInputFormatWithDiagnosticInfo.h>
#include <Formats/verbosePrintString.h>
#include <IO/Operators.h>
#include <IO/WriteBufferFromString.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
DB::RowInputFormatWithDiagnosticInfo::RowInputFormatWithDiagnosticInfo(const Block & header_, ReadBuffer & in_, const Params & params_)
: IRowInputFormat(header_, in_, params_)
{
}
void DB::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();
offset_of_prev_row = offset_of_current_row;
offset_of_current_row = in.offset();
}
String DB::RowInputFormatWithDiagnosticInfo::getDiagnosticInfo()
{
if (in.eof()) /// Buffer has gone, cannot extract information about what has been parsed.
return {};
WriteBufferFromOwnString out;
auto & header = getPort().getHeader();
MutableColumns columns = header.cloneEmptyColumns();
/// It is possible to display detailed diagnostics only if the last and next to last rows are still in the read buffer.
size_t bytes_read_at_start_of_buffer = in.count() - in.offset();
if (bytes_read_at_start_of_buffer != bytes_read_at_start_of_buffer_on_prev_row)
{
out << "Could not print diagnostic info because two last rows aren't in buffer (rare case)\n";
return out.str();
}
max_length_of_column_name = 0;
for (size_t i = 0; i < header.columns(); ++i)
if (header.safeGetByPosition(i).name.size() > max_length_of_column_name)
max_length_of_column_name = header.safeGetByPosition(i).name.size();
max_length_of_data_type_name = 0;
for (size_t i = 0; i < header.columns(); ++i)
if (header.safeGetByPosition(i).type->getName().size() > max_length_of_data_type_name)
max_length_of_data_type_name = header.safeGetByPosition(i).type->getName().size();
/// Roll back the cursor to the beginning of the previous or current row and parse all over again. But now we derive detailed information.
if (offset_of_prev_row <= in.buffer().size())
{
in.position() = in.buffer().begin() + offset_of_prev_row;
out << "\nRow " << (row_num - 1) << ":\n";
if (!parseRowAndPrintDiagnosticInfo(columns, out))
return out.str();
}
else
{
if (in.buffer().size() < offset_of_current_row)
{
out << "Could not print diagnostic info because parsing of data hasn't started.\n";
return out.str();
}
in.position() = in.buffer().begin() + offset_of_current_row;
}
out << "\nRow " << row_num << ":\n";
parseRowAndPrintDiagnosticInfo(columns, out);
out << "\n";
return out.str();
}
bool RowInputFormatWithDiagnosticInfo::deserializeFieldAndPrintDiagnosticInfo(const String & col_name,
const DataTypePtr & type,
IColumn & column,
WriteBuffer & out,
size_t file_column)
{
out << "Column " << file_column << ", " << std::string((file_column < 10 ? 2 : file_column < 100 ? 1 : 0), ' ')
<< "name: " << alignedName(col_name, max_length_of_column_name)
<< "type: " << alignedName(type->getName(), max_length_of_data_type_name);
auto prev_position = in.position();
auto curr_position = in.position();
std::exception_ptr exception;
try
{
tryDeserializeFiled(type, column, file_column, prev_position, curr_position);
}
catch (...)
{
exception = std::current_exception();
}
if (curr_position < prev_position)
throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR);
if (isNativeNumber(type) || isDateOrDateTime(type))
{
/// An empty string instead of a value.
if (curr_position == prev_position)
{
out << "ERROR: text ";
verbosePrintString(prev_position, std::min(prev_position + 10, in.buffer().end()), out);
out << " is not like " << type->getName() << "\n";
return false;
}
}
out << "parsed text: ";
verbosePrintString(prev_position, curr_position, out);
if (exception)
{
if (type->getName() == "DateTime")
out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n";
else if (type->getName() == "Date")
out << "ERROR: Date must be in YYYY-MM-DD format.\n";
else
out << "ERROR\n";
return false;
}
out << "\n";
if (type->haveMaximumSizeOfValue())
{
if (isGarbageAfterField(file_column, curr_position))
{
out << "ERROR: garbage after " << type->getName() << ": ";
verbosePrintString(curr_position, std::min(curr_position + 10, in.buffer().end()), out);
out << "\n";
if (type->getName() == "DateTime")
out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n";
else if (type->getName() == "Date")
out << "ERROR: Date must be in YYYY-MM-DD format.\n";
return false;
}
}
return true;
}
String RowInputFormatWithDiagnosticInfo::alignedName(const String & name, size_t max_length) const
{
size_t spaces_count = max_length >= name.size() ? max_length - name.size() : 0;
return name + ", " + std::string(spaces_count, ' ');
}
}

View File

@ -0,0 +1,46 @@
#pragma once
#include <Core/Block.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <IO/ReadBuffer.h>
#include <limits>
namespace DB
{
class RowInputFormatWithDiagnosticInfo : public IRowInputFormat
{
public:
RowInputFormatWithDiagnosticInfo(const Block & header_, ReadBuffer & in_, const Params & params_);
String getDiagnosticInfo() override;
protected:
void updateDiagnosticInfo();
bool deserializeFieldAndPrintDiagnosticInfo(const String & col_name, const DataTypePtr & type, IColumn & column,
WriteBuffer & out, size_t file_column);
String alignedName(const String & name, size_t max_length) const;
virtual bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) = 0;
virtual void tryDeserializeFiled(const DataTypePtr & type, IColumn & column, size_t file_column,
ReadBuffer::Position & prev_pos, ReadBuffer::Position & curr_pos) = 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;
size_t bytes_read_at_start_of_buffer_on_prev_row = 0;
size_t offset_of_current_row = std::numeric_limits<size_t>::max();
size_t offset_of_prev_row = std::numeric_limits<size_t>::max();
/// For alignment of diagnostic info.
size_t max_length_of_column_name = 0;
size_t max_length_of_data_type_name = 0;
};
}

View File

@ -0,0 +1,16 @@
{prefix}
n: "123", s1: qwe,rty, s2: 'as"df\'gh', s3: "", s4: "zx
cv bn m", d: 2016-01-01, n: 123 ;
n: "456", s1: as"df\'gh, s2: '', s3: "zx\ncv\tbn m", s4: "qwe,rty", d: 2016-01-02, n: 456 ;
n: "9876543210", s1: , s2: 'zx\ncv\tbn m', s3: "qwe,rty", s4: "as""df'gh", d: 2016-01-03, n: 9876543210 ;
n: "789", s1: zx\ncv\tbn m, s2: 'qwe,rty', s3: "as\"df'gh", s4: "", d: 2016-01-04, n: 789
------
n: "0", s1: , s2: '', s3: "", s4: "", d: 0000-00-00, n: 0
------
n: "123", s1: , s2: '', s3: "", s4: "", d: 2016-01-01, n: 123
------
n: "9876543210", s1: zx\ncv\tbn m, s2: 'zx\ncv\tbn m', s3: "zx\ncv\tbn m", s4: "zx
cv bn m", d: 2016-01-04, n: 9876543210
4 rows
before limit 4
read 4 $ suffix $

View File

@ -0,0 +1,12 @@
DROP TABLE IF EXISTS template;
CREATE TABLE template (s1 String, s2 String, `s 3` String, "s 4" String, n UInt64, d Date) ENGINE = Memory;
INSERT INTO template VALUES
('qwe,rty', 'as"df''gh', '', 'zx\ncv\tbn m', 123, '2016-01-01'),('as"df''gh', '', 'zx\ncv\tbn m', 'qwe,rty', 456, '2016-01-02'),('', 'zx\ncv\tbn m', 'qwe,rty', 'as"df''gh', 9876543210, '2016-01-03'),('zx\ncv\tbn m', 'qwe,rty', 'as"df''gh', '', 789, '2016-01-04');
SELECT * FROM template WITH TOTALS LIMIT 4 FORMAT Template SETTINGS
extremes = 1,
format_schema = '{prefix} \n${data:None}\n------\n${totals:}\n------\n${min}\n------\n${max}\n${rows:Escaped} rows\nbefore limit ${rows_before_limit:XML}\nread ${rows_read:Escaped} $$ suffix $$',
format_schema_rows = 'n:\t${n:JSON}, s1:\t${s1:Escaped}, s2:\t${s2:Quoted}, s3:\t${`s 3`:JSON}, s4:\t${"s 4":CSV}, d:\t${d:Escaped}, n:\t${n:Raw}\t',
format_schema_rows_between_delimiter = ';\n';
DROP TABLE template;

View File

@ -0,0 +1,25 @@
==== check escaping ====
"qwe,rty","as""df'gh","","zx
cv bn m",123,"2016-01-01"
"as""df'gh","","zx
cv bn m","qwe,rty",456,"2016-01-02"
"zx
cv bn m","qwe,rty","as""df'gh","",789,"2016-01-04"
"","zx
cv bn m","qwe,rty","as""df'gh",9876543210,"2016-01-03"
==== parse json (sophisticated template) ====
"qwe,rty","as""df'gh","","zx
cv bn m",123,"2016-01-01"
"as""df'gh","","zx
cv bn m","qwe,rty",456,"2016-01-02"
"zx
cv bn m","qwe,rty","as""df'gh","",789,"2016-01-04"
"","zx
cv bn m","qwe,rty","as""df'gh",9876543210,"2016-01-03"
==== parse json ====
"","","qwe,rty","",123,"2016-01-01"
"zx
cv bn m","","as""df'gh","",456,"2016-01-02"
"as""df'gh","","zx
cv bn m","",789,"2016-01-04"
"qwe,rty","","","",9876543210,"2016-01-03"

View File

@ -0,0 +1,46 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS template1";
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS template2";
$CLICKHOUSE_CLIENT --query="CREATE TABLE template1 (s1 String, s2 String, s3 String, s4 String, n UInt64, d Date) ENGINE = Memory";
$CLICKHOUSE_CLIENT --query="CREATE TABLE template2 (s1 String, s2 String, s3 String, s4 String, n UInt64, d Date) ENGINE = Memory";
echo "==== check escaping ===="
echo "{prefix}
n: 123, s1: qwe,rty , s2: 'as\"df\\'gh', s3: \"\", s4: \"zx
cv bn m\", d: 2016-01-01 ;
n: 456, s1: as\"df\\'gh , s2: '', s3: \"zx\\ncv\\tbn m\", s4: \"qwe,rty\", d: 2016-01-02 ;
n: 9876543210, s1: , s2: 'zx\\ncv\\tbn m', s3: \"qwe,rty\", s4: \"as\"\"df'gh\", d: 2016-01-03 ;
n: 789, s1: zx\\ncv\\tbn m , s2: 'qwe,rty', s3: \"as\\\"df'gh\", s4: \"\", d: 2016-01-04
$ suffix $" | $CLICKHOUSE_CLIENT --query="INSERT INTO template1 FORMAT Template SETTINGS \
format_schema = '{prefix} \n\${data}\n \$\$ suffix \$\$\n', \
format_schema_rows = 'n:\t\${n:Escaped}, s1:\t\${s1:Escaped}\t, s2:\t\${s2:Quoted}, s3:\t\${s3:JSON}, s4:\t\${s4:CSV}, d:\t\${d:Escaped}\t', \
format_schema_rows_between_delimiter = ';\n'";
$CLICKHOUSE_CLIENT --query="SELECT * FROM template1 ORDER BY n FORMAT CSV";
echo "==== parse json (sophisticated template) ===="
$CLICKHOUSE_CLIENT --query="SELECT * FROM template1 ORDER BY n FORMAT JSON" | $CLICKHOUSE_CLIENT --query="INSERT INTO template2 FORMAT TemplateIgnoreSpaces SETTINGS \
format_schema = '{\${:}\"meta\"\${:}:\${:}[\${:}{\${:}\"name\"\${:}:\${:}\"s1\"\${:},\${:}\"type\"\${:}:\${:}\"String\"\${:}}\${:},\${:}{\${:}\"name\"\${:}:\${:}\"s2\"\${:},\${:}\"type\"\${:}:\${:}\"String\"\${:}}\${:},\${:}{\${:}\"name\"\${:}:\${:}\"s3\"\${:},\${:}\"type\"\${:}:\${:}\"String\"\${:}}\${:},\${:}{\${:}\"name\"\${:}:\${:}\"s4\"\${:},\${:}\"type\"\${:}:\${:}\"String\"\${:}}\${:},\${:}{\${:}\"name\"\${:}:\${:}\"n\"\${:},\${:}\"type\"\${:}:\${:}\"UInt64\"\${:}}\${:},\${:}{\${:}\"name\"\${:}:\${:}\"d\"\${:},\${:}\"type\"\${:}:\${:}\"Date\"\${:}}\${:}]\${:},\${:}\"data\"\${:}:\${:}[\${data}]\${:},\${:}\"rows\"\${:}:\${:}\${:CSV}\${:},\${:}\"statistics\"\${:}:\${:}{\${:}\"elapsed\"\${:}:\${:}\${:CSV}\${:},\${:}\"rows_read\"\${:}:\${:}\${:CSV}\${:},\${:}\"bytes_read\"\${:}:\${:}\${:CSV}\${:}}\${:}}', \
format_schema_rows = '{\${:}\"s1\"\${:}:\${:}\${s1:JSON}\${:},\${:}\"s2\"\${:}:\${:}\${s2:JSON}\${:},\${:}\"s3\"\${:}:\${:}\${s3:JSON}\${:},\${:}\"s4\"\${:}:\${:}\${s4:JSON}\${:},\${:}\"n\"\${:}:\${:}\${n:JSON}\${:},\${:}\"d\"\${:}:\${:}\${d:JSON}\${:}\${:}}', \
format_schema_rows_between_delimiter = ','";
$CLICKHOUSE_CLIENT --query="SELECT * FROM template2 ORDER BY n FORMAT CSV";
$CLICKHOUSE_CLIENT --query="TRUNCATE TABLE template2";
echo "==== parse json ===="
$CLICKHOUSE_CLIENT --query="SELECT * FROM template1 ORDER BY n FORMAT JSON" | $CLICKHOUSE_CLIENT --query="INSERT INTO template2 FORMAT TemplateIgnoreSpaces SETTINGS \
format_schema = '{\${:}\"meta\"\${:}:\${:JSON},\${:}\"data\"\${:}:\${:}[\${data}]\${:},\${:}\"rows\"\${:}:\${:JSON},\${:}\"statistics\"\${:}:\${:JSON}\${:}}', \
format_schema_rows = '{\${:}\"s1\"\${:}:\${:}\${s3:JSON}\${:},\${:}\"s2\"\${:}:\${:}\${:JSON}\${:},\${:}\"s3\"\${:}:\${:}\${s1:JSON}\${:},\${:}\"s4\"\${:}:\${:}\${:JSON}\${:},\${:}\"n\"\${:}:\${:}\${n:JSON}\${:},\${:}\"d\"\${:}:\${:}\${d:JSON}\${:}\${:}}', \
format_schema_rows_between_delimiter = ','";
$CLICKHOUSE_CLIENT --query="SELECT * FROM template2 ORDER BY n FORMAT CSV";
$CLICKHOUSE_CLIENT --query="DROP TABLE template1";
$CLICKHOUSE_CLIENT --query="DROP TABLE template2";

View File

@ -11,6 +11,8 @@ The supported formats are:
| [TabSeparatedRaw](#tabseparatedraw) | ✗ | ✔ |
| [TabSeparatedWithNames](#tabseparatedwithnames) | ✔ | ✔ |
| [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes) | ✔ | ✔ |
| [Template](#template) | ✔ | ✔ |
| [TemplateIgnoreSpaces](#templateignorespaces) | ✔ | ✗ |
| [CSV](#csv) | ✔ | ✔ |
| [CSVWithNames](#csvwithnames) | ✔ | ✔ |
| [Values](#data-format-values) | ✔ | ✔ |
@ -119,6 +121,121 @@ During parsing, the first and second rows are completely ignored.
This format is also available under the name `TSVWithNamesAndTypes`.
## Template {#template}
This format allows to specify a custom format string with placeholders for values with specified escaping rule.
It uses settings `format_schema`, `format_schema_rows`, `format_schema_rows_between_delimiter` and some settings of other formats (e.g. `output_format_json_quote_64bit_integers` when using `JSON` escaping, see further)
Format string `format_schema_rows` specifies rows format with the following syntax:
`delimiter_1${column_1:serializeAs_1}delimiter_2${column_2:serializeAs_2} ... delimiter_N`,
where `delimiter_i` is a delimiter between values (`$` symbol can be escaped as `$$`),
`column_i` is a name of a column whose values are to be selected or inserted (if empty, then column will be skipped),
`serializeAs_i` is an escaping rule for the column values. The following escaping rules are supported:
- `CSV`, `JSON`, `XML` (similarly to the formats of the same names)
- `Escaped` (similarly to `TSV`)
- `Quoted` (similarly to `Values`)
- `Raw` (without escaping, similarly to `TSVRaw`)
- `None` (no escaping rule, see further)
If escaping rule is omitted, then`None` will be used. `XML` and `Raw` are suitable only for output.
So, for the following format string:
`Search phrase: ${SearchPhrase:Quoted}, count: ${c:Escaped}, ad price: $$${price:JSON};`
the values of `SearchPhrase`, `c` and `price` columns, which are escaped as `Quoted`, `Escaped` and `JSON` will be printed (for select) or will be expected (for insert) between `Search phrase: `, `, count: `, `, ad price: $` and `;` delimiters respectively. For example:
`Search phrase: 'bathroom interior design', count: 2166, ad price: $3;`
The `format_schema_rows_between_delimiter` setting specifies delimiter between rows, which is printed (or expected) after every row except the last one (`\n` by default)
Format string `format_schema` has the same syntax as `format_schema_rows` and allows to specify a prefix, a suffix and a way to print some additional information. It contains the following placeholders instead of column names:
- `data` is the rows with data in `format_schema_rows` format, separated by `format_schema_rows_between_delimiter`. This placeholder must be the first placeholder in the format string.
- `totals` is the row with total values in `format_schema_rows` format (when using WITH TOTALS)
- `min` is the row with minimum values in `format_schema_rows` format (when extremes is set to 1)
- `max` is the row with maximum values in `format_schema_rows` format (when extremes is set to 1)
- `rows` is the total number of output rows
- `rows_before_limit` is the minimal number of rows there would have been without LIMIT. Output only if the query contains LIMIT. If the query contains GROUP BY, rows_before_limit_at_least is the exact number of rows there would have been without a LIMIT.
- `time` is the request execution time in seconds
- `rows_read` is the number of rows have been read
- `bytes_read` is the number of bytes (uncompressed) have been read
The placeholders `data`, `totals`, `min` and `max` must not have escaping rule specified (or `None` must be specified explicitly). The remaining placeholders may have any escaping rule specified.
If the `format_schema` setting is an empty string, `${data}` is used as default value.
For insert queries format allows to skip some columns or some fields if prefix or suffix (see example).
`Select` example:
```sql
SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase ORDER BY c DESC LIMIT 5
FORMAT Template
SETTINGS format_schema = '<!DOCTYPE HTML>
<html> <head> <title>Search phrases</title> </head>
<body>
<table border="1"> <caption>Search phrases</caption>
<tr> <th>Search phrase</th> <th>Count</th> </tr>
${data}
</table>
<table border="1"> <caption>Max</caption>
${max}
</table>
<b>Processed ${rows_read:XML} rows in ${time:XML} sec</b>
</body>
</html>',
format_schema_rows = '<tr> <td>${SearchPhrase:XML}</td> <td>${с:XML}</td> </tr>',
format_schema_rows_between_delimiter = '\n '
```
```html
<!DOCTYPE HTML>
<html> <head> <title>Search phrases</title> </head>
<body>
<table border="1"> <caption>Search phrases</caption>
<tr> <th>Search phrase</th> <th>Count</th> </tr>
<tr> <td></td> <td>8267016</td> </tr>
<tr> <td>bathroom interior design</td> <td>2166</td> </tr>
<tr> <td>yandex</td> <td>1655</td> </tr>
<tr> <td>spring 2014 fashion</td> <td>1549</td> </tr>
<tr> <td>freeform photos</td> <td>1480</td> </tr>
</table>
<table border="1"> <caption>Max</caption>
<tr> <td></td> <td>8873898</td> </tr>
</table>
<b>Processed 3095973 rows in 0.1569913 sec</b>
</body>
</html>
```
`Insert` example:
```
Some header
Page views: 5, User id: 4324182021466249494, Useless field: hello, Duration: 146, Sign: -1
Page views: 6, User id: 4324182021466249494, Useless field: world, Duration: 185, Sign: 1
Total rows: 2
```
```sql
INSERT INTO UserActivity FORMAT Template SETTINGS
format_schema = 'Some header\n${data}\nTotal rows: ${:CSV}\n',
format_schema_rows = 'Page views: ${PageViews:CSV}, User id: ${UserID:CSV}, Useless field: ${:CSV}, Duration: ${Duration:CSV}, Sign: ${Sign:CSV}'
```
`PageViews`, `UserID`, `Duration` and `Sign` inside placeholders are names of columns in the table. Values after `Useless field` in rows and after `\nTotal rows: ` in suffix will be ignored.
All delimiters in the input data must be strictly equal to delimiters in specified format strings.
## TemplateIgnoreSpaces {#templateignorespaces}
This format is suitable only for input.
Similar to `Template`, but skips whitespace characters between delimiters and values in the input stream. However, if format strings contain whitespace characters, these characters will be expected in the input stream. Also allows to specify empty placeholders (`${}` or `${:None}`) to split some delimiter into separate parts to ignore spaces between them. Such placeholders are used only for skipping whitespace characters.
It's possible to read `JSON` using this format, if values of columns have the same order in all rows. For example, the following request can be used for inserting data from output example of format [JSON](#json):
```sql
INSERT INTO table_name FORMAT TemplateIgnoreSpaces SETTINGS
format_schema = '{${}"meta"${}:${:JSON},${}"data"${}:${}[${data}]${},${}"totals"${}:${:JSON},${}"extremes"${}:${:JSON},${}"rows"${}:${:JSON},${}"rows_before_limit_at_least"${}:${:JSON}${}}',
format_schema_rows = '{${}"SearchPhrase"${}:${}${phrase:JSON}${},${}"c"${}:${}${cnt:JSON}${}}',
format_schema_rows_between_delimiter = ','
```
## TSKV {#tskv}
Similar to TabSeparated, but outputs a value in name=value format. Names are escaped the same way as in TabSeparated format, and the = symbol is also escaped.

View File

@ -72,6 +72,9 @@ Works with tables in the MergeTree family.
If `force_primary_key=1`, ClickHouse checks to see if the query has a primary key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For more information about data ranges in MergeTree tables, see "[MergeTree](../../operations/table_engines/mergetree.md)".
## format_schema
This parameter is useful when you are using formats that require a schema definition, such as [Cap'n Proto](https://capnproto.org/), [Protobuf](https://developers.google.com/protocol-buffers/) or [Template](../../interfaces/formats.md#template-template). The value depends on the format.
## fsync_metadata
@ -573,10 +576,6 @@ If a query from the same user with the same 'query_id' already exists at this ti
Yandex.Metrica uses this parameter set to 1 for implementing suggestions for segmentation conditions. After entering the next character, if the old query hasn't finished yet, it should be canceled.
## schema
This parameter is useful when you are using formats that require a schema definition, such as [Cap'n Proto](https://capnproto.org/). The value depends on the format.
## stream_flush_interval_ms

View File

@ -10,6 +10,8 @@ ClickHouse может принимать (`INSERT`) и отдавать (`SELECT
| [TabSeparatedRaw](#tabseparatedraw) | ✗ | ✔ |
| [TabSeparatedWithNames](#tabseparatedwithnames) | ✔ | ✔ |
| [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes) | ✔ | ✔ |
| [Template](#template) | ✔ | ✔ |
| [TemplateIgnoreSpaces](#templateignorespaces) | ✔ | ✗ |
| [CSV](#csv) | ✔ | ✔ |
| [CSVWithNames](#csvwithnames) | ✔ | ✔ |
| [Values](#data-format-values) | ✔ | ✔ |
@ -118,6 +120,120 @@ world
Этот формат также доступен под именем `TSVWithNamesAndTypes`.
## Template {#template}
Этот формат позволяет указать произвольную форматную строку, в которую подставляются значения, сериализованные выбранным способом.
Для этого используются настройки `format_schema`, `format_schema_rows`, `format_schema_rows_between_delimiter` и настройки экранирования других форматов (например, `output_format_json_quote_64bit_integers` при экранировании как в `JSON`, см. далее)
Форматная строка `format_schema_rows` задаёт формат для строк таблицы и должна иметь вид:
`delimiter_1${column_1:serializeAs_1}delimiter_2${column_2:serializeAs_2} ... delimiter_N`,
где `delimiter_i` - разделители между значениями (символ `$` в разделителе экранируется как `$$`),
`column_i` - имена столбцов, значения которых должны быть выведены или считаны (если имя не указано - столбец пропускается),
`serializeAs_i` - тип экранирования для значений соответствующего столбца. Поддерживаются следующие типы экранирования:
- `CSV`, `JSON`, `XML` (как в одноимённых форматах)
- `Escaped` (как в `TSV`)
- `Quoted` (как в `Values`)
- `Raw` (без экранирования, как в `TSVRaw`)
- `None` (тип экранирования отсутствует, см. далее)
Если для столбца не указан тип экранирования, используется `None`. `XML` и `Raw` поддерживаются только для вывода.
Так, в форматной строке
`Search phrase: ${SearchPhrase:Quoted}, count: ${c:Escaped}, ad price: $$${price:JSON};`
между разделителями `Search phrase: `, `, count: `, `, ad price: $` и `;` при выводе будут подставлены (при вводе - будут ожидаться) значения столбцов `SearchPhrase`, `c` и `price`, сериализованные как `Quoted`, `Escaped` и `JSON` соответственно, например:
`Search phrase: 'bathroom interior design', count: 2166, ad price: $3;`
Настройка `format_schema_rows_between_delimiter` задаёт разделитель между строками, который выводится (или ожмдается при вводе) после каждой строки, кроме последней. По умолчанию `\n`.
Форматная строка `format_schema` имеет аналогичный `format_schema_rows` синтаксис и позволяет указать префикс, суффикс и способ вывода дополнительной информации. Вместо имён столбцов в ней указываются следующие имена подстановок:
- `data` - строки с данными в формате `format_schema_rows`, разделённые `format_schema_rows_between_delimiter`. Эта подстановка должна быть первой подстановкой в форматной строке.
- `totals` - строка с тотальными значениями в формате `format_schema_rows` (при использовании WITH TOTALS)
- `min` - строка с минимальными значениями в формате `format_schema_rows` (при настройке extremes, выставленной в 1)
- `max` - строка с максимальными значениями в формате `format_schema_rows` (при настройке extremes, выставленной в 1)
- `rows` - общее количество выведенных стрчек
- `rows_before_limit` - не менее скольких строчек получилось бы, если бы не было LIMIT-а. Выводится только если запрос содержит LIMIT. В случае, если запрос содержит GROUP BY, `rows_before_limit` - точное число строк, которое получилось бы, если бы не было LIMIT-а.
- `time` - время выполнения запроса в секундах
- `rows_read` - сколько строк было прочитано при выполнении запроса
- `bytes_read` - сколько байт (несжатых) было прочитано при выполнении запроса
У подстановок `data`, `totals`, `min` и `max` не должны быть указаны типы экранирования (или должен быть указан `None`). Остальные подстановки - это отдельные значения, для них может быть указан любой тип экранирования.
Если строка `format_schema` пустая, то по-умолчанию используется `${data}`.
Из всех перечисленных подстановок форматная строка `format_schema` для ввода может содержать только `data`.
Также при вводе формат поддерживает пропуск значений столбцов и пропуск значений в префиксе и суффиксе (см. пример).
Пример вывода:
```sql
SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase ORDER BY c DESC LIMIT 5
FORMAT Template
SETTINGS format_schema = '<!DOCTYPE HTML>
<html> <head> <title>Search phrases</title> </head>
<body>
<table border="1"> <caption>Search phrases</caption>
<tr> <th>Search phrase</th> <th>Count</th> </tr>
${data}
</table>
<table border="1"> <caption>Max</caption>
${max}
</table>
<b>Processed ${rows_read:XML} rows in ${time:XML} sec</b>
</body>
</html>',
format_schema_rows = '<tr> <td>${SearchPhrase:XML}</td> <td>${с:XML}</td> </tr>',
format_schema_rows_between_delimiter = '\n '
```
```html
<!DOCTYPE HTML>
<html> <head> <title>Search phrases</title> </head>
<body>
<table border="1"> <caption>Search phrases</caption>
<tr> <th>Search phrase</th> <th>Count</th> </tr>
<tr> <td></td> <td>8267016</td> </tr>
<tr> <td>bathroom interior design</td> <td>2166</td> </tr>
<tr> <td>yandex</td> <td>1655</td> </tr>
<tr> <td>spring 2014 fashion</td> <td>1549</td> </tr>
<tr> <td>freeform photos</td> <td>1480</td> </tr>
</table>
<table border="1"> <caption>Max</caption>
<tr> <td></td> <td>8873898</td> </tr>
</table>
<b>Processed 3095973 rows in 0.1569913 sec</b>
</body>
</html>
```
Пример ввода:
```
Some header
Page views: 5, User id: 4324182021466249494, Useless field: hello, Duration: 146, Sign: -1
Page views: 6, User id: 4324182021466249494, Useless field: world, Duration: 185, Sign: 1
Total rows: 2
```
```sql
INSERT INTO UserActivity FORMAT Template SETTINGS
format_schema = 'Some header\n${data}\nTotal rows: ${:CSV}\n',
format_schema_rows = 'Page views: ${PageViews:CSV}, User id: ${UserID:CSV}, Useless field: ${:CSV}, Duration: ${Duration:CSV}, Sign: ${Sign:CSV}'
```
`PageViews`, `UserID`, `Duration` и `Sign` внутри подстановок - имена столбцов в таблице, в которую вставляются данные. Значения после `Useless field` в строках и значение после `\nTotal rows: ` в суффиксе будут проигнорированы.
Все разделители во входных данных должны строго соответствовать разделителям в форматных строках.
## TemplateIgnoreSpaces {#templateignorespaces}
Подходит только для ввода. Отличается от формата `Template` тем, что пропускает пробельные символы между разделителями и значениями во входном потоке. Также в этом формате можно указать пустые подстановки с типом экранирования `None` (`${}` или `${:None}`), чтобы разбить разделители на несколько частей, пробелы между которыми должны игнорироваться. Такие подстановки используются только для пропуска пробелов. С помощью этого формата можно считывать `JSON`, если значения столбцов в нём всегда идут в одном порядке в каждой строке. Например, для вставки данных из примера вывода формата [JSON](#json) в таблицу со столбцами `phrase` и `cnt` можно использовать следующий запрос:
```sql
INSERT INTO table_name FORMAT TemplateIgnoreSpaces SETTINGS
format_schema = '{${}"meta"${}:${:JSON},${}"data"${}:${}[${data}]${},${}"totals"${}:${:JSON},${}"extremes"${}:${:JSON},${}"rows"${}:${:JSON},${}"rows_before_limit_at_least"${}:${:JSON}${}}',
format_schema_rows = '{${}"SearchPhrase"${}:${}${phrase:JSON}${},${}"c"${}:${}${cnt:JSON}${}}',
format_schema_rows_between_delimiter = ','
```
## TSKV {#tskv}
Похож на TabSeparated, но выводит значения в формате name=value. Имена экранируются так же, как строки в формате TabSeparated и, дополнительно, экранируется также символ =.

View File

@ -70,6 +70,10 @@ ClickHouse применяет настройку в тех случаях, ко
При `force_primary_key=1` ClickHouse проверяет, есть ли в запросе условие на первичный ключ, которое может использоваться для отсечения диапазонов данных. Если подходящего условия нет - кидается исключение. При этом не проверяется, действительно ли условие уменьшает объём данных для чтения. Подробнее про диапазоны данных в таблицах MergeTree читайте в разделе "[MergeTree](../../operations/table_engines/mergetree.md)".
## format_schema
Параметр применяется в том случае, когда используются форматы, требующие определения схемы, например [Cap'n Proto](https://capnproto.org/), [Protobuf](https://developers.google.com/protocol-buffers/) или [Template](../../interfaces/formats.md#template-template). Значение параметра зависит от формата.
## fsync_metadata
Включает или отключает [fsync](http://pubs.opengroup.org/onlinepubs/9699919799/functions/fsync.html) при записи `.sql` файлов. По умолчанию включено.
@ -579,10 +583,6 @@ ClickHouse использует этот параметр при чтении д
Эта настройка, выставленная в 1, используется в Яндекс.Метрике для реализации suggest-а значений для условий сегментации. После ввода очередного символа, если старый запрос ещё не выполнился, его следует отменить.
## schema
Параметр применяется в том случае, когда используются форматы, требующие определения схемы, например [Cap'n Proto](https://capnproto.org/). Значение параметра зависит от формата.
## stream_flush_interval_ms
Работает для таблиц со стриммингом в случае тайм-аута, или когда поток генерирует [max_insert_block_size](#settings-max_insert_block_size) строк.