mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 17:41:59 +00:00
Merge branch 'master' into database-filesystem-remove-catch
This commit is contained in:
commit
044eb0509f
@ -185,7 +185,7 @@ void SystemLogQueue<LogElement>::confirm(uint64_t to_flush_end)
|
||||
}
|
||||
|
||||
template <typename LogElement>
|
||||
SystemLogQueue<LogElement>::Index SystemLogQueue<LogElement>::pop(std::vector<LogElement>& output, bool& should_prepare_tables_anyway, bool& exit_this_thread)
|
||||
typename SystemLogQueue<LogElement>::Index SystemLogQueue<LogElement>::pop(std::vector<LogElement>& output, bool& should_prepare_tables_anyway, bool& exit_this_thread)
|
||||
{
|
||||
std::unique_lock lock(mutex);
|
||||
flush_event.wait_for(lock,
|
||||
|
@ -360,6 +360,9 @@ private:
|
||||
|
||||
struct DownloadInfo
|
||||
{
|
||||
DownloadInfo(const CacheMetadata::Key & key_, const size_t & offset_, const std::weak_ptr<FileSegment> & file_segment_)
|
||||
: key(key_), offset(offset_), file_segment(file_segment_) {}
|
||||
|
||||
CacheMetadata::Key key;
|
||||
size_t offset;
|
||||
/// We keep weak pointer to file segment
|
||||
|
@ -1011,7 +1011,7 @@ private:
|
||||
using ConfluentSchemaRegistry = AvroConfluentRowInputFormat::SchemaRegistry;
|
||||
#define SCHEMA_REGISTRY_CACHE_MAX_SIZE 1000
|
||||
/// Cache of Schema Registry URL -> SchemaRegistry
|
||||
static CacheBase<std::string, ConfluentSchemaRegistry> schema_registry_cache(SCHEMA_REGISTRY_CACHE_MAX_SIZE);
|
||||
static CacheBase<std::string, ConfluentSchemaRegistry> schema_registry_cache(SCHEMA_REGISTRY_CACHE_MAX_SIZE);
|
||||
|
||||
static std::shared_ptr<ConfluentSchemaRegistry> getConfluentSchemaRegistry(const FormatSettings & format_settings)
|
||||
{
|
||||
|
@ -303,14 +303,14 @@ void preparePrimitiveColumn(ColumnPtr column, DataTypePtr type, const std::strin
|
||||
}
|
||||
else
|
||||
{
|
||||
types(T::INT32, C::UINT_8 , int_type(8 , false));
|
||||
types(T::INT32, C::UINT_8, int_type(8, false));
|
||||
}
|
||||
break;
|
||||
case TypeIndex::UInt16: types(T::INT32, C::UINT_16, int_type(16, false)); break;
|
||||
case TypeIndex::UInt32: types(T::INT32, C::UINT_32, int_type(32, false)); break;
|
||||
case TypeIndex::UInt64: types(T::INT64, C::UINT_64, int_type(64, false)); break;
|
||||
case TypeIndex::Int8: types(T::INT32, C::INT_8 , int_type(8 , true)); break;
|
||||
case TypeIndex::Int16: types(T::INT32, C::INT_16 , int_type(16, true)); break;
|
||||
case TypeIndex::Int8: types(T::INT32, C::INT_8, int_type(8, true)); break;
|
||||
case TypeIndex::Int16: types(T::INT32, C::INT_16, int_type(16, true)); break;
|
||||
case TypeIndex::Int32: types(T::INT32); break;
|
||||
case TypeIndex::Int64: types(T::INT64); break;
|
||||
case TypeIndex::Float32: types(T::FLOAT); break;
|
||||
@ -319,8 +319,8 @@ void preparePrimitiveColumn(ColumnPtr column, DataTypePtr type, const std::strin
|
||||
/// These don't have suitable parquet logical types, so we write them as plain numbers.
|
||||
/// (Parquet has "enums" but they're just strings, with nowhere to declare all possible enum
|
||||
/// values in advance as part of the data type.)
|
||||
case TypeIndex::Enum8: types(T::INT32, C::INT_8 , int_type(8 , true)); break; // Int8
|
||||
case TypeIndex::Enum16: types(T::INT32, C::INT_16 , int_type(16, true)); break; // Int16
|
||||
case TypeIndex::Enum8: types(T::INT32, C::INT_8, int_type(8, true)); break; // Int8
|
||||
case TypeIndex::Enum16: types(T::INT32, C::INT_16, int_type(16, true)); break; // Int16
|
||||
case TypeIndex::IPv4: types(T::INT32, C::UINT_32, int_type(32, false)); break; // UInt32
|
||||
case TypeIndex::Date: types(T::INT32, C::UINT_16, int_type(16, false)); break; // UInt16
|
||||
case TypeIndex::DateTime: types(T::INT32, C::UINT_32, int_type(32, false)); break; // UInt32
|
||||
@ -392,8 +392,8 @@ void preparePrimitiveColumn(ColumnPtr column, DataTypePtr type, const std::strin
|
||||
case TypeIndex::Int256: fixed_string(32); break;
|
||||
case TypeIndex::IPv6: fixed_string(16); break;
|
||||
|
||||
case TypeIndex::Decimal32: decimal(4 , getDecimalPrecision(*type), getDecimalScale(*type)); break;
|
||||
case TypeIndex::Decimal64: decimal(8 , getDecimalPrecision(*type), getDecimalScale(*type)); break;
|
||||
case TypeIndex::Decimal32: decimal(4, getDecimalPrecision(*type), getDecimalScale(*type)); break;
|
||||
case TypeIndex::Decimal64: decimal(8, getDecimalPrecision(*type), getDecimalScale(*type)); break;
|
||||
case TypeIndex::Decimal128: decimal(16, getDecimalPrecision(*type), getDecimalScale(*type)); break;
|
||||
case TypeIndex::Decimal256: decimal(32, getDecimalPrecision(*type), getDecimalScale(*type)); break;
|
||||
|
||||
|
@ -755,20 +755,20 @@ void writeColumnChunkBody(ColumnChunkWriteState & s, const WriteOptions & option
|
||||
writeColumnImpl<parquet::BooleanType>(s, options, out,
|
||||
ConverterNumeric<ColumnVector<UInt8>, bool, bool>(s.primitive_column));
|
||||
else
|
||||
N(UInt8 , Int32Type);
|
||||
N(UInt8, Int32Type);
|
||||
break;
|
||||
case TypeIndex::UInt16 : N(UInt16, Int32Type); break;
|
||||
case TypeIndex::UInt32 : N(UInt32, Int32Type); break;
|
||||
case TypeIndex::UInt64 : N(UInt64, Int64Type); break;
|
||||
case TypeIndex::Int8 : N(Int8 , Int32Type); break;
|
||||
case TypeIndex::Int16 : N(Int16 , Int32Type); break;
|
||||
case TypeIndex::Int32 : N(Int32 , Int32Type); break;
|
||||
case TypeIndex::Int64 : N(Int64 , Int64Type); break;
|
||||
case TypeIndex::Int8 : N(Int8, Int32Type); break;
|
||||
case TypeIndex::Int16 : N(Int16, Int32Type); break;
|
||||
case TypeIndex::Int32 : N(Int32, Int32Type); break;
|
||||
case TypeIndex::Int64 : N(Int64, Int64Type); break;
|
||||
|
||||
case TypeIndex::Enum8: N(Int8 , Int32Type); break;
|
||||
case TypeIndex::Enum16: N(Int16 , Int32Type); break;
|
||||
case TypeIndex::Enum8: N(Int8, Int32Type); break;
|
||||
case TypeIndex::Enum16: N(Int16, Int32Type); break;
|
||||
case TypeIndex::Date: N(UInt16, Int32Type); break;
|
||||
case TypeIndex::Date32: N(Int32 , Int32Type); break;
|
||||
case TypeIndex::Date32: N(Int32, Int32Type); break;
|
||||
case TypeIndex::DateTime: N(UInt32, Int32Type); break;
|
||||
|
||||
#undef N
|
||||
|
@ -29,6 +29,8 @@ public:
|
||||
|
||||
struct PathWithInfo
|
||||
{
|
||||
PathWithInfo() = default;
|
||||
PathWithInfo(const String & path_, const std::optional<PathInfo> & info_) : path(path_), info(info_) {}
|
||||
String path;
|
||||
std::optional<PathInfo> info;
|
||||
};
|
||||
|
@ -243,6 +243,15 @@ void GinIndexStore::finalize()
|
||||
{
|
||||
if (!current_postings.empty())
|
||||
writeSegment();
|
||||
|
||||
if (metadata_file_stream)
|
||||
metadata_file_stream->finalize();
|
||||
|
||||
if (dict_file_stream)
|
||||
dict_file_stream->finalize();
|
||||
|
||||
if (postings_file_stream)
|
||||
postings_file_stream->finalize();
|
||||
}
|
||||
|
||||
void GinIndexStore::initFileStreams()
|
||||
@ -319,13 +328,8 @@ void GinIndexStore::writeSegment()
|
||||
current_segment.segment_id = getNextSegmentID();
|
||||
|
||||
metadata_file_stream->sync();
|
||||
metadata_file_stream->finalize();
|
||||
|
||||
dict_file_stream->sync();
|
||||
dict_file_stream->finalize();
|
||||
|
||||
postings_file_stream->sync();
|
||||
postings_file_stream->finalize();
|
||||
}
|
||||
|
||||
GinIndexStoreDeserializer::GinIndexStoreDeserializer(const GinIndexStorePtr & store_)
|
||||
|
@ -156,7 +156,7 @@ public:
|
||||
void checkTableCanBeDropped() const override {}
|
||||
|
||||
private:
|
||||
mutable std::mutex nested_mutex;
|
||||
mutable std::recursive_mutex nested_mutex;
|
||||
mutable GetNestedStorageFunc get_nested;
|
||||
mutable StoragePtr nested;
|
||||
const bool add_conversion;
|
||||
|
@ -0,0 +1 @@
|
||||
0
|
@ -0,0 +1,7 @@
|
||||
|
||||
drop table if exists t1;
|
||||
create table t1 as remote('localhost', 'system.one');
|
||||
rename table t1 to t2;
|
||||
select * from t2;
|
||||
rename table t2 to t1;
|
||||
drop table t1;
|
Loading…
Reference in New Issue
Block a user