Merge pull request #35833 from ucasfl/context

Try remove unneed variable
This commit is contained in:
Kseniia Sumarokova 2022-04-01 16:58:42 +02:00 committed by GitHub
commit fe8c91541d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
2 changed files with 13 additions and 16 deletions

View File

@ -467,7 +467,7 @@ std::pair<BlocksPtr, Block> StorageWindowView::getNewBlocks(UInt32 watermark)
InterpreterSelectQuery fetch( InterpreterSelectQuery fetch(
getFetchColumnQuery(w_start, watermark), getFetchColumnQuery(w_start, watermark),
window_view_context, getContext(),
getInnerStorage(), getInnerStorage(),
nullptr, nullptr,
SelectQueryOptions(QueryProcessingStage::FetchColumns)); SelectQueryOptions(QueryProcessingStage::FetchColumns));
@ -509,11 +509,11 @@ std::pair<BlocksPtr, Block> StorageWindowView::getNewBlocks(UInt32 watermark)
return StorageBlocks::createStorage(blocks_id_global, required_columns, std::move(pipes), QueryProcessingStage::WithMergeableState); return StorageBlocks::createStorage(blocks_id_global, required_columns, std::move(pipes), QueryProcessingStage::WithMergeableState);
}; };
TemporaryTableHolder blocks_storage(window_view_context, creator); TemporaryTableHolder blocks_storage(getContext(), creator);
InterpreterSelectQuery select( InterpreterSelectQuery select(
getFinalQuery(), getFinalQuery(),
window_view_context, getContext(),
blocks_storage.getTable(), blocks_storage.getTable(),
blocks_storage.getTable()->getInMemoryMetadataPtr(), blocks_storage.getTable()->getInMemoryMetadataPtr(),
SelectQueryOptions(QueryProcessingStage::Complete)); SelectQueryOptions(QueryProcessingStage::Complete));
@ -617,8 +617,8 @@ std::shared_ptr<ASTCreateQuery> StorageWindowView::getInnerTableCreateQuery(
auto t_sample_block auto t_sample_block
= InterpreterSelectQuery( = InterpreterSelectQuery(
inner_select_query, window_view_context, getParentStorage(), nullptr, inner_select_query, getContext(), getParentStorage(), nullptr, SelectQueryOptions(QueryProcessingStage::WithMergeableState))
SelectQueryOptions(QueryProcessingStage::WithMergeableState)) .getSampleBlock(); .getSampleBlock();
auto columns_list = std::make_shared<ASTExpressionList>(); auto columns_list = std::make_shared<ASTExpressionList>();
@ -891,7 +891,7 @@ void StorageWindowView::updateMaxWatermark(UInt32 watermark)
inline void StorageWindowView::cleanup() inline void StorageWindowView::cleanup()
{ {
InterpreterAlterQuery alter_query(getCleanupQuery(), window_view_context); InterpreterAlterQuery alter_query(getCleanupQuery(), getContext());
alter_query.execute(); alter_query.execute();
std::lock_guard lock(fire_signal_mutex); std::lock_guard lock(fire_signal_mutex);
@ -999,9 +999,6 @@ StorageWindowView::StorageWindowView(
, WithContext(context_->getGlobalContext()) , WithContext(context_->getGlobalContext())
, log(&Poco::Logger::get(fmt::format("StorageWindowView({}.{})", table_id_.database_name, table_id_.table_name))) , log(&Poco::Logger::get(fmt::format("StorageWindowView({}.{})", table_id_.database_name, table_id_.table_name)))
{ {
window_view_context = Context::createCopy(getContext());
window_view_context->makeQueryContext();
StorageInMemoryMetadata storage_metadata; StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(columns_); storage_metadata.setColumns(columns_);
setInMemoryMetadata(storage_metadata); setInMemoryMetadata(storage_metadata);
@ -1089,11 +1086,11 @@ StorageWindowView::StorageWindowView(
clean_interval_ms = getContext()->getSettingsRef().window_view_clean_interval.totalMilliseconds(); clean_interval_ms = getContext()->getSettingsRef().window_view_clean_interval.totalMilliseconds();
next_fire_signal = getWindowUpperBound(std::time(nullptr)); next_fire_signal = getWindowUpperBound(std::time(nullptr));
clean_cache_task = window_view_context->getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncCleanup(); }); clean_cache_task = getContext()->getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncCleanup(); });
if (is_proctime) if (is_proctime)
fire_task = window_view_context->getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncFireProc(); }); fire_task = getContext()->getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncFireProc(); });
else else
fire_task = window_view_context->getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncFireEvent(); }); fire_task = getContext()->getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncFireEvent(); });
clean_cache_task->deactivate(); clean_cache_task->deactivate();
fire_task->deactivate(); fire_task->deactivate();
} }
@ -1424,9 +1421,10 @@ Block & StorageWindowView::getHeader() const
std::lock_guard lock(sample_block_lock); std::lock_guard lock(sample_block_lock);
if (!sample_block) if (!sample_block)
{ {
sample_block = InterpreterSelectQuery( sample_block
select_query->clone(), window_view_context, getParentStorage(), nullptr, = InterpreterSelectQuery(
SelectQueryOptions(QueryProcessingStage::Complete)).getSampleBlock(); select_query->clone(), getContext(), getParentStorage(), nullptr, SelectQueryOptions(QueryProcessingStage::Complete))
.getSampleBlock();
/// convert all columns to full columns /// convert all columns to full columns
/// in case some of them are constant /// in case some of them are constant
for (size_t i = 0; i < sample_block.columns(); ++i) for (size_t i = 0; i < sample_block.columns(); ++i)

View File

@ -157,7 +157,6 @@ private:
/// Used to fetch the mergeable state and generate the final result. e.g. SELECT * FROM * GROUP BY tumble(____timestamp, *) /// Used to fetch the mergeable state and generate the final result. e.g. SELECT * FROM * GROUP BY tumble(____timestamp, *)
ASTPtr final_query; ASTPtr final_query;
ContextMutablePtr window_view_context;
bool is_proctime{true}; bool is_proctime{true};
bool is_time_column_func_now; bool is_time_column_func_now;
bool is_tumble; // false if is hop bool is_tumble; // false if is hop