Merge branch 'ClickHouse:master' into feature-password

This commit is contained in:
larryluogit 2022-04-19 08:36:00 -04:00 committed by GitHub
commit d8865f338c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 148 additions and 108 deletions

View File

@ -155,7 +155,12 @@ target_include_directories(_jemalloc SYSTEM PRIVATE
target_compile_definitions(_jemalloc PRIVATE -DJEMALLOC_NO_PRIVATE_NAMESPACE)
if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG")
target_compile_definitions(_jemalloc PRIVATE -DJEMALLOC_DEBUG=1)
target_compile_definitions(_jemalloc PRIVATE
-DJEMALLOC_DEBUG=1
# Usage examples:
# - MALLOC_CONF=log:.
# - MALLOC_CONF='log:core.malloc.exit|core.sallocx.entry|core.sdallocx.entry'
-DJEMALLOC_LOG=1)
endif ()
target_compile_definitions(_jemalloc PRIVATE -DJEMALLOC_PROF=1)

View File

@ -341,7 +341,7 @@ int Keeper::main(const std::vector<std::string> & /*args*/)
auto servers = std::make_shared<std::vector<ProtocolServerAdapter>>();
/// Initialize keeper RAFT. Do nothing if no keeper_server in config.
tiny_context.initializeKeeperDispatcher(/* start_async = */false);
tiny_context.initializeKeeperDispatcher(/* start_async = */ true);
FourLetterCommandFactory::registerCommands(*tiny_context.getKeeperDispatcher());
auto config_getter = [this] () -> const Poco::Util::AbstractConfiguration &

View File

@ -45,64 +45,64 @@ private:
/// This is internal method to use from COW.
/// It performs shallow copy with copy-ctor and not useful from outside.
/// If you want to copy column for modification, look at 'mutate' method.
virtual MutablePtr clone() const = 0;
[[nodiscard]] virtual MutablePtr clone() const = 0;
public:
/// Name of a Column. It is used in info messages.
virtual std::string getName() const { return getFamilyName(); }
[[nodiscard]] virtual std::string getName() const { return getFamilyName(); }
/// Name of a Column kind, without parameters (example: FixedString, Array).
virtual const char * getFamilyName() const = 0;
[[nodiscard]] virtual const char * getFamilyName() const = 0;
/// Type of data that column contains. It's an underlying type: UInt16 for Date, UInt32 for DateTime, so on.
virtual TypeIndex getDataType() const = 0;
[[nodiscard]] virtual TypeIndex getDataType() const = 0;
/** If column isn't constant, returns itself.
* If column is constant, transforms constant to full column (if column type allows such transform) and return it.
*/
virtual Ptr convertToFullColumnIfConst() const { return getPtr(); }
[[nodiscard]] virtual Ptr convertToFullColumnIfConst() const { return getPtr(); }
/// If column isn't ColumnLowCardinality, return itself.
/// If column is ColumnLowCardinality, transforms it to full column.
virtual Ptr convertToFullColumnIfLowCardinality() const { return getPtr(); }
[[nodiscard]] virtual Ptr convertToFullColumnIfLowCardinality() const { return getPtr(); }
/// If column isn't ColumnSparse, return itself.
/// If column is ColumnSparse, transforms it to full column.
virtual Ptr convertToFullColumnIfSparse() const { return getPtr(); }
[[nodiscard]] virtual Ptr convertToFullColumnIfSparse() const { return getPtr(); }
Ptr convertToFullIfNeeded() const
[[nodiscard]] Ptr convertToFullIfNeeded() const
{
return convertToFullColumnIfSparse()->convertToFullColumnIfConst()->convertToFullColumnIfLowCardinality();
}
/// Creates empty column with the same type.
virtual MutablePtr cloneEmpty() const { return cloneResized(0); }
[[nodiscard]] virtual MutablePtr cloneEmpty() const { return cloneResized(0); }
/// Creates column with the same type and specified size.
/// If size is less current size, then data is cut.
/// If size is greater, than default values are appended.
virtual MutablePtr cloneResized(size_t /*size*/) const { throw Exception("Cannot cloneResized() column " + getName(), ErrorCodes::NOT_IMPLEMENTED); }
[[nodiscard]] virtual MutablePtr cloneResized(size_t /*size*/) const { throw Exception("Cannot cloneResized() column " + getName(), ErrorCodes::NOT_IMPLEMENTED); }
/// Returns number of values in column.
virtual size_t size() const = 0;
[[nodiscard]] virtual size_t size() const = 0;
/// There are no values in columns.
bool empty() const { return size() == 0; }
[[nodiscard]] bool empty() const { return size() == 0; }
/// Returns value of n-th element in universal Field representation.
/// Is used in rare cases, since creation of Field instance is expensive usually.
virtual Field operator[](size_t n) const = 0;
[[nodiscard]] virtual Field operator[](size_t n) const = 0;
/// Like the previous one, but avoids extra copying if Field is in a container, for example.
virtual void get(size_t n, Field & res) const = 0;
/// If possible, returns pointer to memory chunk which contains n-th element (if it isn't possible, throws an exception)
/// Is used to optimize some computations (in aggregation, for example).
virtual StringRef getDataAt(size_t n) const = 0;
[[nodiscard]] virtual StringRef getDataAt(size_t n) const = 0;
/// Like getData, but has special behavior for columns that contain variable-length strings.
/// Returns zero-ending memory chunk (i.e. its size is 1 byte longer).
virtual StringRef getDataAtWithTerminatingZero(size_t n) const
[[nodiscard]] virtual StringRef getDataAtWithTerminatingZero(size_t n) const
{
return getDataAt(n);
}
@ -110,19 +110,19 @@ public:
/// If column stores integers, it returns n-th element transformed to UInt64 using static_cast.
/// If column stores floating point numbers, bits of n-th elements are copied to lower bits of UInt64, the remaining bits are zeros.
/// Is used to optimize some computations (in aggregation, for example).
virtual UInt64 get64(size_t /*n*/) const
[[nodiscard]] virtual UInt64 get64(size_t /*n*/) const
{
throw Exception("Method get64 is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
/// If column stores native numeric type, it returns n-th element casted to Float64
/// Is used in regression methods to cast each features into uniform type
virtual Float64 getFloat64(size_t /*n*/) const
[[nodiscard]] virtual Float64 getFloat64(size_t /*n*/) const
{
throw Exception("Method getFloat64 is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
virtual Float32 getFloat32(size_t /*n*/) const
[[nodiscard]] virtual Float32 getFloat32(size_t /*n*/) const
{
throw Exception("Method getFloat32 is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
@ -131,31 +131,31 @@ public:
* For NULL values of Nullable column it is allowed to return arbitrary value.
* Otherwise throw an exception.
*/
virtual UInt64 getUInt(size_t /*n*/) const
[[nodiscard]] virtual UInt64 getUInt(size_t /*n*/) const
{
throw Exception("Method getUInt is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
virtual Int64 getInt(size_t /*n*/) const
[[nodiscard]] virtual Int64 getInt(size_t /*n*/) const
{
throw Exception("Method getInt is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
virtual bool isDefaultAt(size_t n) const = 0;
virtual bool isNullAt(size_t /*n*/) const { return false; }
[[nodiscard]] virtual bool isDefaultAt(size_t n) const = 0;
[[nodiscard]] virtual bool isNullAt(size_t /*n*/) const { return false; }
/** If column is numeric, return value of n-th element, casted to bool.
* For NULL values of Nullable column returns false.
* Otherwise throw an exception.
*/
virtual bool getBool(size_t /*n*/) const
[[nodiscard]] virtual bool getBool(size_t /*n*/) const
{
throw Exception("Method getBool is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
/// Removes all elements outside of specified range.
/// Is used in LIMIT operation, for example.
virtual Ptr cut(size_t start, size_t length) const
[[nodiscard]] virtual Ptr cut(size_t start, size_t length) const
{
MutablePtr res = cloneEmpty();
res->insertRangeFrom(*this, start, length);
@ -249,7 +249,7 @@ public:
* otherwise (i.e. < 0), makes reserve() using size of source column.
*/
using Filter = PaddedPODArray<UInt8>;
virtual Ptr filter(const Filter & filt, ssize_t result_size_hint) const = 0;
[[nodiscard]] virtual Ptr filter(const Filter & filt, ssize_t result_size_hint) const = 0;
/** Expand column by mask inplace. After expanding column will
* satisfy the following: if we filter it by given mask, we will
@ -262,11 +262,11 @@ public:
/// Permutes elements using specified permutation. Is used in sorting.
/// limit - if it isn't 0, puts only first limit elements in the result.
using Permutation = PaddedPODArray<size_t>;
virtual Ptr permute(const Permutation & perm, size_t limit) const = 0;
[[nodiscard]] virtual Ptr permute(const Permutation & perm, size_t limit) const = 0;
/// Creates new column with values column[indexes[:limit]]. If limit is 0, all indexes are used.
/// Indexes must be one of the ColumnUInt. For default implementation, see selectIndexImpl from ColumnsCommon.h
virtual Ptr index(const IColumn & indexes, size_t limit) const = 0;
[[nodiscard]] virtual Ptr index(const IColumn & indexes, size_t limit) const = 0;
/** Compares (*this)[n] and rhs[m]. Column rhs should have the same type.
* Returns negative number, 0, or positive number (*this)[n] is less, equal, greater than rhs[m] respectively.
@ -279,10 +279,10 @@ public:
*
* For non Nullable and non floating point types, nan_direction_hint is ignored.
*/
virtual int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const = 0;
[[nodiscard]] virtual int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const = 0;
/// Equivalent to compareAt, but collator is used to compare values.
virtual int compareAtWithCollation(size_t, size_t, const IColumn &, int, const Collator &) const
[[nodiscard]] virtual int compareAtWithCollation(size_t, size_t, const IColumn &, int, const Collator &) const
{
throw Exception("Collations could be specified only for String, LowCardinality(String), Nullable(String) or for Array or Tuple, containing it.", ErrorCodes::BAD_COLLATION);
}
@ -297,7 +297,7 @@ public:
int direction, int nan_direction_hint) const = 0;
/// Check if all elements in the column have equal values. Return true if column is empty.
virtual bool hasEqualValues() const = 0;
[[nodiscard]] virtual bool hasEqualValues() const = 0;
enum class PermutationSortDirection : uint8_t
{
@ -353,7 +353,7 @@ public:
*/
using Offset = UInt64;
using Offsets = PaddedPODArray<Offset>;
virtual Ptr replicate(const Offsets & offsets) const = 0;
[[nodiscard]] virtual Ptr replicate(const Offsets & offsets) const = 0;
/** Split column to smaller columns. Each value goes to column index, selected by corresponding element of 'selector'.
* Selector must contain values from 0 to num_columns - 1.
@ -361,7 +361,7 @@ public:
*/
using ColumnIndex = UInt64;
using Selector = PaddedPODArray<ColumnIndex>;
virtual std::vector<MutablePtr> scatter(ColumnIndex num_columns, const Selector & selector) const = 0;
[[nodiscard]] virtual std::vector<MutablePtr> scatter(ColumnIndex num_columns, const Selector & selector) const = 0;
/// Insert data from several other columns according to source mask (used in vertical merge).
/// For now it is a helper to de-virtualize calls to insert*() functions inside gather loop
@ -385,15 +385,15 @@ public:
virtual void ensureOwnership() {}
/// Size of column data in memory (may be approximate) - for profiling. Zero, if could not be determined.
virtual size_t byteSize() const = 0;
[[nodiscard]] virtual size_t byteSize() const = 0;
/// Size of single value in memory (for accounting purposes)
virtual size_t byteSizeAt(size_t /*n*/) const = 0;
[[nodiscard]] virtual size_t byteSizeAt(size_t /*n*/) const = 0;
/// Size of memory, allocated for column.
/// This is greater or equals to byteSize due to memory reservation in containers.
/// Zero, if could not be determined.
virtual size_t allocatedBytes() const = 0;
[[nodiscard]] virtual size_t allocatedBytes() const = 0;
/// Make memory region readonly with mprotect if it is large enough.
/// The operation is slow and performed only for debug builds.
@ -406,14 +406,14 @@ public:
/// Columns have equal structure.
/// If true - you can use "compareAt", "insertFrom", etc. methods.
virtual bool structureEquals(const IColumn &) const
[[nodiscard]] virtual bool structureEquals(const IColumn &) const
{
throw Exception("Method structureEquals is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
/// Returns ration of values in column, that equal to default value of column.
/// Checks only @sample_ratio ratio of rows.
virtual double getRatioOfDefaultRows(double sample_ratio = 1.0) const = 0; /// NOLINT
[[nodiscard]] virtual double getRatioOfDefaultRows(double sample_ratio = 1.0) const = 0; /// NOLINT
/// Returns indices of values in column, that not equal to default value of column.
virtual void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const = 0;
@ -423,13 +423,13 @@ public:
/// Other values are filled by @default_value.
/// @shift means how much rows to skip from the beginning of current column.
/// Used to create full column from sparse.
virtual Ptr createWithOffsets(const Offsets & offsets, const Field & default_field, size_t total_rows, size_t shift) const;
[[nodiscard]] virtual Ptr createWithOffsets(const Offsets & offsets, const Field & default_field, size_t total_rows, size_t shift) const;
virtual SerializationInfoPtr getSerializationInfo() const;
[[nodiscard]] virtual SerializationInfoPtr getSerializationInfo() const;
/// Compress column in memory to some representation that allows to decompress it back.
/// Return itself if compression is not applicable for this column type.
virtual Ptr compress() const
[[nodiscard]] virtual Ptr compress() const
{
/// No compression by default.
return getPtr();
@ -437,13 +437,13 @@ public:
/// If it's CompressedColumn, decompress it and return.
/// Otherwise return itself.
virtual Ptr decompress() const
[[nodiscard]] virtual Ptr decompress() const
{
return getPtr();
}
static MutablePtr mutate(Ptr ptr)
[[nodiscard]] static MutablePtr mutate(Ptr ptr)
{
MutablePtr res = ptr->shallowMutate(); /// Now use_count is 2.
ptr.reset(); /// Reset use_count to 1.
@ -463,10 +463,10 @@ public:
/// Various properties on behaviour of column type.
/// True if column contains something nullable inside. It's true for ColumnNullable, can be true or false for ColumnConst, etc.
virtual bool isNullable() const { return false; }
[[nodiscard]] virtual bool isNullable() const { return false; }
/// It's a special kind of column, that contain single value, but is not a ColumnConst.
virtual bool isDummy() const { return false; }
[[nodiscard]] virtual bool isDummy() const { return false; }
/** Memory layout properties.
*
@ -486,32 +486,32 @@ public:
*/
/// Values in column have fixed size (including the case when values span many memory segments).
virtual bool valuesHaveFixedSize() const { return isFixedAndContiguous(); }
[[nodiscard]] virtual bool valuesHaveFixedSize() const { return isFixedAndContiguous(); }
/// Values in column are represented as continuous memory segment of fixed size. Implies valuesHaveFixedSize.
virtual bool isFixedAndContiguous() const { return false; }
[[nodiscard]] virtual bool isFixedAndContiguous() const { return false; }
/// If isFixedAndContiguous, returns the underlying data array, otherwise throws an exception.
virtual StringRef getRawData() const { throw Exception("Column " + getName() + " is not a contiguous block of memory", ErrorCodes::NOT_IMPLEMENTED); }
[[nodiscard]] virtual StringRef getRawData() const { throw Exception("Column " + getName() + " is not a contiguous block of memory", ErrorCodes::NOT_IMPLEMENTED); }
/// If valuesHaveFixedSize, returns size of value, otherwise throw an exception.
virtual size_t sizeOfValueIfFixed() const { throw Exception("Values of column " + getName() + " are not fixed size.", ErrorCodes::CANNOT_GET_SIZE_OF_FIELD); }
[[nodiscard]] virtual size_t sizeOfValueIfFixed() const { throw Exception("Values of column " + getName() + " are not fixed size.", ErrorCodes::CANNOT_GET_SIZE_OF_FIELD); }
/// Column is ColumnVector of numbers or ColumnConst of it. Note that Nullable columns are not numeric.
virtual bool isNumeric() const { return false; }
[[nodiscard]] virtual bool isNumeric() const { return false; }
/// If the only value column can contain is NULL.
/// Does not imply type of object, because it can be ColumnNullable(ColumnNothing) or ColumnConst(ColumnNullable(ColumnNothing))
virtual bool onlyNull() const { return false; }
[[nodiscard]] virtual bool onlyNull() const { return false; }
/// Can be inside ColumnNullable.
virtual bool canBeInsideNullable() const { return false; }
[[nodiscard]] virtual bool canBeInsideNullable() const { return false; }
virtual bool lowCardinality() const { return false; }
[[nodiscard]] virtual bool lowCardinality() const { return false; }
virtual bool isSparse() const { return false; }
[[nodiscard]] virtual bool isSparse() const { return false; }
virtual bool isCollationSupported() const { return false; }
[[nodiscard]] virtual bool isCollationSupported() const { return false; }
virtual ~IColumn() = default;
IColumn() = default;
@ -519,7 +519,7 @@ public:
/** Print column name, size, and recursively print all subcolumns.
*/
String dumpStructure() const;
[[nodiscard]] String dumpStructure() const;
protected:
/// Template is to devirtualize calls to insertFrom method.

View File

@ -29,8 +29,8 @@ struct Settings;
M(UInt64, reserved_log_items, 100000, "How many log items to store (don't remove during compaction)", 0) \
M(UInt64, snapshot_distance, 100000, "How many log items we have to collect to write new snapshot", 0) \
M(Bool, auto_forwarding, true, "Allow to forward write requests from followers to leader", 0) \
M(Milliseconds, shutdown_timeout, 5000, "How many time we will until RAFT shutdown", 0) \
M(Milliseconds, startup_timeout, 180000, "How many time we will until RAFT to start", 0) \
M(Milliseconds, shutdown_timeout, 5000, "How much time we will wait until RAFT shutdown", 0) \
M(Milliseconds, startup_timeout, 180000, "How much time we will wait until RAFT to start.", 0) \
M(LogsLevel, raft_logs_level, LogsLevel::information, "Log internal RAFT logs into main server log level. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'fatal', 'none'", 0) \
M(UInt64, rotate_log_storage_interval, 100000, "How many records will be stored in one log storage file", 0) \
M(UInt64, snapshots_to_keep, 3, "How many compressed snapshots to keep on disk", 0) \

View File

@ -44,6 +44,11 @@ int32_t IFourLetterCommand::toCode(const String & name)
return __builtin_bswap32(res);
}
bool IFourLetterCommand::serverIsActive() const
{
return keeper_dispatcher.hasLeader();
}
IFourLetterCommand::~IFourLetterCommand() = default;
FourLetterCommandFactory & FourLetterCommandFactory::instance()
@ -198,6 +203,8 @@ void print(IFourLetterCommand::StringBuffer & buf, const String & key, uint64_t
print(buf, key, toString(value));
}
constexpr auto * SERVER_NOT_ACTIVE_MSG = "This instance is not currently serving requests";
}
String MonitorCommand::run()
@ -205,11 +212,11 @@ String MonitorCommand::run()
auto & stats = keeper_dispatcher.getKeeperConnectionStats();
Keeper4LWInfo keeper_info = keeper_dispatcher.getKeeper4LWInfo();
if (!keeper_info.has_leader)
return "This instance is not currently serving requests";
const auto & state_machine = keeper_dispatcher.getStateMachine();
if (!keeper_info.has_leader)
return SERVER_NOT_ACTIVE_MSG;
StringBuffer ret;
print(ret, "version", String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH);
@ -247,6 +254,9 @@ String MonitorCommand::run()
String StatResetCommand::run()
{
if (!serverIsActive())
return SERVER_NOT_ACTIVE_MSG;
keeper_dispatcher.resetConnectionStats();
return "Server stats reset.\n";
}
@ -258,6 +268,9 @@ String NopCommand::run()
String ConfCommand::run()
{
if (!serverIsActive())
return SERVER_NOT_ACTIVE_MSG;
StringBuffer buf;
keeper_dispatcher.getKeeperConfigurationAndSettings()->dump(buf);
return buf.str();
@ -265,6 +278,9 @@ String ConfCommand::run()
String ConsCommand::run()
{
if (!serverIsActive())
return SERVER_NOT_ACTIVE_MSG;
StringBuffer buf;
KeeperTCPHandler::dumpConnections(buf, false);
return buf.str();
@ -272,12 +288,18 @@ String ConsCommand::run()
String RestConnStatsCommand::run()
{
if (!serverIsActive())
return SERVER_NOT_ACTIVE_MSG;
KeeperTCPHandler::resetConnsStats();
return "Connection stats reset.\n";
}
String ServerStatCommand::run()
{
if (!serverIsActive())
return SERVER_NOT_ACTIVE_MSG;
StringBuffer buf;
auto write = [&buf](const String & key, const String & value)
@ -310,6 +332,9 @@ String ServerStatCommand::run()
String StatCommand::run()
{
if (!serverIsActive())
return SERVER_NOT_ACTIVE_MSG;
StringBuffer buf;
auto write = [&buf] (const String & key, const String & value) { buf << key << ": " << value << '\n'; };
@ -340,6 +365,9 @@ String StatCommand::run()
String BriefWatchCommand::run()
{
if (!serverIsActive())
return SERVER_NOT_ACTIVE_MSG;
StringBuffer buf;
const auto & state_machine = keeper_dispatcher.getStateMachine();
buf << state_machine.getSessionsWithWatchesCount() << " connections watching "
@ -350,6 +378,9 @@ String BriefWatchCommand::run()
String WatchCommand::run()
{
if (!serverIsActive())
return SERVER_NOT_ACTIVE_MSG;
StringBuffer buf;
const auto & state_machine = keeper_dispatcher.getStateMachine();
state_machine.dumpWatches(buf);
@ -358,6 +389,9 @@ String WatchCommand::run()
String WatchByPathCommand::run()
{
if (!serverIsActive())
return SERVER_NOT_ACTIVE_MSG;
StringBuffer buf;
const auto & state_machine = keeper_dispatcher.getStateMachine();
state_machine.dumpWatchesByPath(buf);
@ -366,6 +400,9 @@ String WatchByPathCommand::run()
String DataSizeCommand::run()
{
if (!serverIsActive())
return SERVER_NOT_ACTIVE_MSG;
StringBuffer buf;
buf << "snapshot_dir_size: " << keeper_dispatcher.getSnapDirSize() << '\n';
buf << "log_dir_size: " << keeper_dispatcher.getLogDirSize() << '\n';
@ -374,6 +411,9 @@ String DataSizeCommand::run()
String DumpCommand::run()
{
if (!serverIsActive())
return SERVER_NOT_ACTIVE_MSG;
StringBuffer buf;
const auto & state_machine = keeper_dispatcher.getStateMachine();
state_machine.dumpSessionsAndEphemerals(buf);

View File

@ -32,6 +32,9 @@ public:
static String toName(int32_t code);
static inline int32_t toCode(const String & name);
// Return true if server is running and serving requests
bool serverIsActive() const;
protected:
KeeperDispatcher & keeper_dispatcher;
};

View File

@ -582,7 +582,7 @@ class IColumn;
M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \
M(String, insert_deduplication_token, "", "If not empty, used for duplicate detection instead of data digest", 0) \
M(Bool, throw_on_unsupported_query_inside_transaction, true, "Throw exception if unsupported query is used inside transaction", 0) \
M(Bool, throw_if_no_data_to_insert, true, "Enables or disables empty INSERTs, disable by default", 0) \
M(Bool, throw_if_no_data_to_insert, true, "Enables or disables empty INSERTs, enabled by default", 0) \
// End of COMMON_SETTINGS
// Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS.

View File

@ -110,18 +110,19 @@ StoragePtr DatabaseAtomic::detachTable(ContextPtr /* context */, const String &
void DatabaseAtomic::dropTable(ContextPtr local_context, const String & table_name, bool no_delay)
{
auto storage = tryGetTable(table_name, local_context);
auto table = tryGetTable(table_name, local_context);
/// Remove the inner table (if any) to avoid deadlock
/// (due to attempt to execute DROP from the worker thread)
if (storage)
storage->dropInnerTableIfAny(no_delay, local_context);
if (table)
table->dropInnerTableIfAny(no_delay, local_context);
else
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist",
backQuote(database_name), backQuote(table_name));
String table_metadata_path = getObjectMetadataPath(table_name);
String table_metadata_path_drop;
StoragePtr table;
{
std::unique_lock lock(mutex);
table = getTableUnlocked(table_name, lock);
table_metadata_path_drop = DatabaseCatalog::instance().getPathForDroppedMetadata(table->getStorageID());
auto txn = local_context->getZooKeeperMetadataTransaction();
if (txn && !local_context->isInternalSubquery())

View File

@ -94,7 +94,7 @@ ColumnsDescription readSchemaFromFormat(
}
}
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} file format doesn't support schema inference", format_name);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} file format doesn't support schema inference. You must specify the structure manually", format_name);
return ColumnsDescription(names_and_types);
}

View File

@ -345,7 +345,11 @@ void KeeperTCPHandler::runImpl()
return;
}
if (keeper_dispatcher->checkInit() && keeper_dispatcher->hasLeader())
// we store the checks because they can change during the execution
// leading to weird results
const auto is_initialized = keeper_dispatcher->checkInit();
const auto has_leader = keeper_dispatcher->hasLeader();
if (is_initialized && has_leader)
{
try
{
@ -366,9 +370,9 @@ void KeeperTCPHandler::runImpl()
else
{
String reason;
if (!keeper_dispatcher->checkInit() && !keeper_dispatcher->hasLeader())
if (!is_initialized && !has_leader)
reason = "server is not initialized yet and no alive leader exists";
else if (!keeper_dispatcher->checkInit())
else if (!is_initialized)
reason = "server is not initialized yet";
else
reason = "no alive leader exists";

View File

@ -25,24 +25,6 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
}
namespace
{
void checkIfFormatSupportsAutoStructure(const String & name, const String & format)
{
if (name == "file" && format == "Distributed")
return;
if (FormatFactory::instance().checkIfFormatHasAnySchemaReader(format))
return;
throw Exception(
"Table function '" + name
+ "' allows automatic structure determination only for formats that support schema inference and for Distributed format in table function "
"'file'",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}
}
void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, ContextPtr context)
{
/// Parse args
@ -68,18 +50,13 @@ void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, Context
format = FormatFactory::instance().getFormatFromFileName(filename, true);
if (args.size() <= 2)
{
checkIfFormatSupportsAutoStructure(getName(), format);
return;
}
if (args.size() != 3 && args.size() != 4)
throw Exception("Table function '" + getName() + "' requires 1, 2, 3 or 4 arguments: filename, format (default auto), structure (default auto) and compression method (default auto)",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
structure = args[2]->as<ASTLiteral &>().value.safeGet<String>();
if (structure == "auto")
checkIfFormatSupportsAutoStructure(getName(), format);
if (structure.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS,

View File

@ -17,15 +17,3 @@ while [[ $counter -lt $retries ]]; do
done
echo 'Ok'
counter=0
I=0
while [[ $counter -lt $retries ]]; do
I=$((I + 1))
TYPE=$(perl -e "print 'Array(' x $I; print 'UInt8'; print ')' x $I")
${CLICKHOUSE_CLIENT} --prefer_localhost_replica=0 --max_parser_depth 1000000 --query "SELECT * FROM remote('127.0.0.{1,2}', generateRandom('x $TYPE', 1, 1, 1)) LIMIT 1 FORMAT Null" 2>&1 | grep -q -F 'Maximum parse depth' && break;
((++counter))
done
#echo "I = ${I}"
echo 'Ok'

View File

@ -0,0 +1 @@
Ok

View File

@ -0,0 +1,19 @@
#!/usr/bin/env bash
# Tags: long
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
counter=0 retries=60
I=0
while [[ $counter -lt $retries ]]; do
I=$((I + 1))
TYPE=$(perl -e "print 'Array(' x $I; print 'UInt8'; print ')' x $I")
${CLICKHOUSE_CLIENT} --prefer_localhost_replica=0 --max_parser_depth 1000000 --query "SELECT * FROM remote('127.0.0.{1,2}', generateRandom('x $TYPE', 1, 1, 1)) LIMIT 1 FORMAT Null" 2>&1 | grep -q -F 'Maximum parse depth' && break;
((++counter))
done
#echo "I = ${I}"
echo 'Ok'

View File

@ -0,0 +1,2 @@
insert into function file('02269_data', 'RowBinary') select 1;
select * from file('02269_data', 'RowBinary', 'x UInt8');