Try remove unneed variable

This commit is contained in:
fenglv 2022-04-01 09:58:23 +00:00
parent 61a0becae5
commit 97523a3329
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(
getFetchColumnQuery(w_start, watermark),
window_view_context,
getContext(),
getInnerStorage(),
nullptr,
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);
};
TemporaryTableHolder blocks_storage(window_view_context, creator);
TemporaryTableHolder blocks_storage(getContext(), creator);
InterpreterSelectQuery select(
getFinalQuery(),
window_view_context,
getContext(),
blocks_storage.getTable(),
blocks_storage.getTable()->getInMemoryMetadataPtr(),
SelectQueryOptions(QueryProcessingStage::Complete));
@ -617,8 +617,8 @@ std::shared_ptr<ASTCreateQuery> StorageWindowView::getInnerTableCreateQuery(
auto t_sample_block
= InterpreterSelectQuery(
inner_select_query, window_view_context, getParentStorage(), nullptr,
SelectQueryOptions(QueryProcessingStage::WithMergeableState)) .getSampleBlock();
inner_select_query, getContext(), getParentStorage(), nullptr, SelectQueryOptions(QueryProcessingStage::WithMergeableState))
.getSampleBlock();
auto columns_list = std::make_shared<ASTExpressionList>();
@ -891,7 +891,7 @@ void StorageWindowView::updateMaxWatermark(UInt32 watermark)
inline void StorageWindowView::cleanup()
{
InterpreterAlterQuery alter_query(getCleanupQuery(), window_view_context);
InterpreterAlterQuery alter_query(getCleanupQuery(), getContext());
alter_query.execute();
std::lock_guard lock(fire_signal_mutex);
@ -999,9 +999,6 @@ StorageWindowView::StorageWindowView(
, WithContext(context_->getGlobalContext())
, 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;
storage_metadata.setColumns(columns_);
setInMemoryMetadata(storage_metadata);
@ -1089,11 +1086,11 @@ StorageWindowView::StorageWindowView(
clean_interval_ms = getContext()->getSettingsRef().window_view_clean_interval.totalMilliseconds();
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)
fire_task = window_view_context->getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncFireProc(); });
fire_task = getContext()->getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncFireProc(); });
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();
fire_task->deactivate();
}
@ -1424,9 +1421,10 @@ Block & StorageWindowView::getHeader() const
std::lock_guard lock(sample_block_lock);
if (!sample_block)
{
sample_block = InterpreterSelectQuery(
select_query->clone(), window_view_context, getParentStorage(), nullptr,
SelectQueryOptions(QueryProcessingStage::Complete)).getSampleBlock();
sample_block
= InterpreterSelectQuery(
select_query->clone(), getContext(), getParentStorage(), nullptr, SelectQueryOptions(QueryProcessingStage::Complete))
.getSampleBlock();
/// convert all columns to full columns
/// in case some of them are constant
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, *)
ASTPtr final_query;
ContextMutablePtr window_view_context;
bool is_proctime{true};
bool is_time_column_func_now;
bool is_tumble; // false if is hop