mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
Revert "Merge pull request #39631 from ClickHouse/parallel-window"
This reverts commit33cc853d61
, reversing changes made tob05a3d02ed
.
This commit is contained in:
parent
cab764fff9
commit
22200566c8
@ -2942,7 +2942,6 @@ void InterpreterSelectQuery::executeWindow(QueryPlan & query_plan)
|
||||
auto sorting_step = std::make_unique<SortingStep>(
|
||||
query_plan.getCurrentDataStream(),
|
||||
window.full_sort_description,
|
||||
window.partition_by,
|
||||
0 /* LIMIT */,
|
||||
sort_settings,
|
||||
settings.optimize_sorting_by_input_stream_properties);
|
||||
|
@ -915,7 +915,6 @@ void addWindowSteps(QueryPlan & query_plan,
|
||||
auto sorting_step = std::make_unique<SortingStep>(
|
||||
query_plan.getCurrentDataStream(),
|
||||
window_description.full_sort_description,
|
||||
window_description.partition_by,
|
||||
0 /*limit*/,
|
||||
sort_settings,
|
||||
settings.optimize_sorting_by_input_stream_properties);
|
||||
|
@ -1,4 +1,3 @@
|
||||
#include <memory>
|
||||
#include <stdexcept>
|
||||
#include <IO/Operators.h>
|
||||
#include <Processors/Merges/MergingSortedTransform.h>
|
||||
@ -10,8 +9,6 @@
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <Common/JSONBuilder.h>
|
||||
|
||||
#include <Processors/ResizeProcessor.h>
|
||||
#include <Processors/Transforms/ScatterByPartitionTransform.h>
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
@ -79,21 +76,6 @@ SortingStep::SortingStep(
|
||||
output_stream->sort_scope = DataStream::SortScope::Global;
|
||||
}
|
||||
|
||||
SortingStep::SortingStep(
|
||||
const DataStream & input_stream,
|
||||
const SortDescription & description_,
|
||||
const SortDescription & partition_by_description_,
|
||||
UInt64 limit_,
|
||||
const Settings & settings_,
|
||||
bool optimize_sorting_by_input_stream_properties_)
|
||||
: SortingStep(input_stream, description_, limit_, settings_, optimize_sorting_by_input_stream_properties_)
|
||||
{
|
||||
partition_by_description = partition_by_description_;
|
||||
|
||||
output_stream->sort_description = result_description;
|
||||
output_stream->sort_scope = DataStream::SortScope::Stream;
|
||||
}
|
||||
|
||||
SortingStep::SortingStep(
|
||||
const DataStream & input_stream_,
|
||||
SortDescription prefix_description_,
|
||||
@ -135,11 +117,7 @@ void SortingStep::updateOutputStream()
|
||||
{
|
||||
output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits());
|
||||
output_stream->sort_description = result_description;
|
||||
|
||||
if (partition_by_description.empty())
|
||||
output_stream->sort_scope = DataStream::SortScope::Global;
|
||||
else
|
||||
output_stream->sort_scope = DataStream::SortScope::Stream;
|
||||
output_stream->sort_scope = DataStream::SortScope::Global;
|
||||
}
|
||||
|
||||
void SortingStep::updateLimit(size_t limit_)
|
||||
@ -157,55 +135,6 @@ void SortingStep::convertToFinishSorting(SortDescription prefix_description_)
|
||||
prefix_description = std::move(prefix_description_);
|
||||
}
|
||||
|
||||
void SortingStep::scatterByPartitionIfNeeded(QueryPipelineBuilder& pipeline)
|
||||
{
|
||||
size_t threads = pipeline.getNumThreads();
|
||||
size_t streams = pipeline.getNumStreams();
|
||||
|
||||
if (!partition_by_description.empty() && threads > 1)
|
||||
{
|
||||
Block stream_header = pipeline.getHeader();
|
||||
|
||||
ColumnNumbers key_columns;
|
||||
key_columns.reserve(partition_by_description.size());
|
||||
for (auto & col : partition_by_description)
|
||||
{
|
||||
key_columns.push_back(stream_header.getPositionByName(col.column_name));
|
||||
}
|
||||
|
||||
pipeline.transform([&](OutputPortRawPtrs ports)
|
||||
{
|
||||
Processors processors;
|
||||
for (auto * port : ports)
|
||||
{
|
||||
auto scatter = std::make_shared<ScatterByPartitionTransform>(stream_header, threads, key_columns);
|
||||
connect(*port, scatter->getInputs().front());
|
||||
processors.push_back(scatter);
|
||||
}
|
||||
return processors;
|
||||
});
|
||||
|
||||
if (streams > 1)
|
||||
{
|
||||
pipeline.transform([&](OutputPortRawPtrs ports)
|
||||
{
|
||||
Processors processors;
|
||||
for (size_t i = 0; i < threads; ++i)
|
||||
{
|
||||
size_t output_it = i;
|
||||
auto resize = std::make_shared<ResizeProcessor>(stream_header, streams, 1);
|
||||
auto & inputs = resize->getInputs();
|
||||
|
||||
for (auto input_it = inputs.begin(); input_it != inputs.end(); output_it += threads, ++input_it)
|
||||
connect(*ports[output_it], *input_it);
|
||||
processors.push_back(resize);
|
||||
}
|
||||
return processors;
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void SortingStep::finishSorting(
|
||||
QueryPipelineBuilder & pipeline, const SortDescription & input_sort_desc, const SortDescription & result_sort_desc, const UInt64 limit_)
|
||||
{
|
||||
@ -331,12 +260,10 @@ void SortingStep::fullSortStreams(
|
||||
void SortingStep::fullSort(
|
||||
QueryPipelineBuilder & pipeline, const SortDescription & result_sort_desc, const UInt64 limit_, const bool skip_partial_sort)
|
||||
{
|
||||
scatterByPartitionIfNeeded(pipeline);
|
||||
|
||||
fullSortStreams(pipeline, sort_settings, result_sort_desc, limit_, skip_partial_sort);
|
||||
|
||||
/// If there are several streams, then we merge them into one
|
||||
if (pipeline.getNumStreams() > 1 && (partition_by_description.empty() || pipeline.getNumThreads() == 1))
|
||||
if (pipeline.getNumStreams() > 1)
|
||||
{
|
||||
auto transform = std::make_shared<MergingSortedTransform>(
|
||||
pipeline.getHeader(),
|
||||
@ -368,7 +295,6 @@ void SortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build
|
||||
{
|
||||
bool need_finish_sorting = (prefix_description.size() < result_description.size());
|
||||
mergingSorted(pipeline, prefix_description, (need_finish_sorting ? 0 : limit));
|
||||
|
||||
if (need_finish_sorting)
|
||||
{
|
||||
finishSorting(pipeline, prefix_description, result_description, limit);
|
||||
|
@ -40,15 +40,6 @@ public:
|
||||
const Settings & settings_,
|
||||
bool optimize_sorting_by_input_stream_properties_);
|
||||
|
||||
/// Full with partitioning
|
||||
SortingStep(
|
||||
const DataStream & input_stream,
|
||||
const SortDescription & description_,
|
||||
const SortDescription & partition_by_description_,
|
||||
UInt64 limit_,
|
||||
const Settings & settings_,
|
||||
bool optimize_sorting_by_input_stream_properties_);
|
||||
|
||||
/// FinishSorting
|
||||
SortingStep(
|
||||
const DataStream & input_stream_,
|
||||
@ -92,24 +83,14 @@ public:
|
||||
bool skip_partial_sort = false);
|
||||
|
||||
private:
|
||||
void scatterByPartitionIfNeeded(QueryPipelineBuilder& pipeline);
|
||||
void updateOutputStream() override;
|
||||
|
||||
static void mergeSorting(
|
||||
QueryPipelineBuilder & pipeline,
|
||||
const Settings & sort_settings,
|
||||
const SortDescription & result_sort_desc,
|
||||
UInt64 limit_);
|
||||
static void
|
||||
mergeSorting(QueryPipelineBuilder & pipeline, const Settings & sort_settings, const SortDescription & result_sort_desc, UInt64 limit_);
|
||||
|
||||
void mergingSorted(
|
||||
QueryPipelineBuilder & pipeline,
|
||||
const SortDescription & result_sort_desc,
|
||||
UInt64 limit_);
|
||||
void mergingSorted(QueryPipelineBuilder & pipeline, const SortDescription & result_sort_desc, UInt64 limit_);
|
||||
void finishSorting(
|
||||
QueryPipelineBuilder & pipeline,
|
||||
const SortDescription & input_sort_desc,
|
||||
const SortDescription & result_sort_desc,
|
||||
UInt64 limit_);
|
||||
QueryPipelineBuilder & pipeline, const SortDescription & input_sort_desc, const SortDescription & result_sort_desc, UInt64 limit_);
|
||||
void fullSort(
|
||||
QueryPipelineBuilder & pipeline,
|
||||
const SortDescription & result_sort_desc,
|
||||
@ -120,9 +101,6 @@ private:
|
||||
|
||||
SortDescription prefix_description;
|
||||
const SortDescription result_description;
|
||||
|
||||
SortDescription partition_by_description;
|
||||
|
||||
UInt64 limit;
|
||||
bool always_read_till_end = false;
|
||||
|
||||
|
@ -67,8 +67,7 @@ void WindowStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQ
|
||||
// This resize is needed for cases such as `over ()` when we don't have a
|
||||
// sort node, and the input might have multiple streams. The sort node would
|
||||
// have resized it.
|
||||
if (window_description.full_sort_description.empty())
|
||||
pipeline.resize(1);
|
||||
pipeline.resize(1);
|
||||
|
||||
pipeline.addSimpleTransform(
|
||||
[&](const Block & /*header*/)
|
||||
|
@ -1,129 +0,0 @@
|
||||
#include <Processors/Transforms/ScatterByPartitionTransform.h>
|
||||
|
||||
#include <Common/PODArray.h>
|
||||
#include <Core/ColumnNumbers.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
ScatterByPartitionTransform::ScatterByPartitionTransform(Block header, size_t output_size_, ColumnNumbers key_columns_)
|
||||
: IProcessor(InputPorts{header}, OutputPorts{output_size_, header})
|
||||
, output_size(output_size_)
|
||||
, key_columns(std::move(key_columns_))
|
||||
, hash(0)
|
||||
{}
|
||||
|
||||
IProcessor::Status ScatterByPartitionTransform::prepare()
|
||||
{
|
||||
auto & input = getInputs().front();
|
||||
|
||||
/// Check all outputs are finished or ready to get data.
|
||||
|
||||
bool all_finished = true;
|
||||
for (auto & output : outputs)
|
||||
{
|
||||
if (output.isFinished())
|
||||
continue;
|
||||
|
||||
all_finished = false;
|
||||
}
|
||||
|
||||
if (all_finished)
|
||||
{
|
||||
input.close();
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
if (!all_outputs_processed)
|
||||
{
|
||||
auto output_it = outputs.begin();
|
||||
bool can_push = false;
|
||||
for (size_t i = 0; i < output_size; ++i, ++output_it)
|
||||
if (!was_output_processed[i] && output_it->canPush())
|
||||
can_push = true;
|
||||
if (!can_push)
|
||||
return Status::PortFull;
|
||||
return Status::Ready;
|
||||
}
|
||||
/// Try get chunk from input.
|
||||
|
||||
if (input.isFinished())
|
||||
{
|
||||
for (auto & output : outputs)
|
||||
output.finish();
|
||||
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
input.setNeeded();
|
||||
if (!input.hasData())
|
||||
return Status::NeedData;
|
||||
|
||||
chunk = input.pull();
|
||||
has_data = true;
|
||||
was_output_processed.assign(outputs.size(), false);
|
||||
|
||||
return Status::Ready;
|
||||
}
|
||||
|
||||
void ScatterByPartitionTransform::work()
|
||||
{
|
||||
if (all_outputs_processed)
|
||||
generateOutputChunks();
|
||||
all_outputs_processed = true;
|
||||
|
||||
size_t chunk_number = 0;
|
||||
for (auto & output : outputs)
|
||||
{
|
||||
auto & was_processed = was_output_processed[chunk_number];
|
||||
auto & output_chunk = output_chunks[chunk_number];
|
||||
++chunk_number;
|
||||
|
||||
if (was_processed)
|
||||
continue;
|
||||
|
||||
if (output.isFinished())
|
||||
continue;
|
||||
|
||||
if (!output.canPush())
|
||||
{
|
||||
all_outputs_processed = false;
|
||||
continue;
|
||||
}
|
||||
|
||||
output.push(std::move(output_chunk));
|
||||
was_processed = true;
|
||||
}
|
||||
|
||||
if (all_outputs_processed)
|
||||
{
|
||||
has_data = false;
|
||||
output_chunks.clear();
|
||||
}
|
||||
}
|
||||
|
||||
void ScatterByPartitionTransform::generateOutputChunks()
|
||||
{
|
||||
auto num_rows = chunk.getNumRows();
|
||||
const auto & columns = chunk.getColumns();
|
||||
|
||||
hash.reset(num_rows);
|
||||
|
||||
for (const auto & column_number : key_columns)
|
||||
columns[column_number]->updateWeakHash32(hash);
|
||||
|
||||
const auto & hash_data = hash.getData();
|
||||
IColumn::Selector selector(num_rows);
|
||||
|
||||
for (size_t row = 0; row < num_rows; ++row)
|
||||
selector[row] = hash_data[row] % output_size;
|
||||
|
||||
output_chunks.resize(output_size);
|
||||
for (const auto & column : columns)
|
||||
{
|
||||
auto filtered_columns = column->scatter(output_size, selector);
|
||||
for (size_t i = 0; i < output_size; ++i)
|
||||
output_chunks[i].addColumn(std::move(filtered_columns[i]));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -1,34 +0,0 @@
|
||||
#pragma once
|
||||
#include <Common/WeakHash.h>
|
||||
#include <Core/ColumnNumbers.h>
|
||||
#include <Processors/IProcessor.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct ScatterByPartitionTransform : IProcessor
|
||||
{
|
||||
ScatterByPartitionTransform(Block header, size_t output_size_, ColumnNumbers key_columns_);
|
||||
|
||||
String getName() const override { return "ScatterByPartitionTransform"; }
|
||||
|
||||
Status prepare() override;
|
||||
void work() override;
|
||||
|
||||
private:
|
||||
|
||||
void generateOutputChunks();
|
||||
|
||||
size_t output_size;
|
||||
ColumnNumbers key_columns;
|
||||
|
||||
bool has_data = false;
|
||||
bool all_outputs_processed = true;
|
||||
std::vector<char> was_output_processed;
|
||||
Chunk chunk;
|
||||
|
||||
WeakHash32 hash;
|
||||
Chunks output_chunks;
|
||||
};
|
||||
|
||||
}
|
@ -22,16 +22,6 @@ select sum(number) over w as x, max(number) over w as y from t_01568 window w as
|
||||
21 8
|
||||
21 8
|
||||
21 8
|
||||
select sum(number) over w, max(number) over w from t_01568 window w as (partition by p) order by p;
|
||||
3 2
|
||||
3 2
|
||||
3 2
|
||||
12 5
|
||||
12 5
|
||||
12 5
|
||||
21 8
|
||||
21 8
|
||||
21 8
|
||||
select sum(number) over w as x, max(number) over w as y from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by x, y;
|
||||
6 2
|
||||
6 2
|
||||
@ -51,25 +41,6 @@ select sum(number) over w as x, max(number) over w as y from remote('127.0.0.{1,
|
||||
42 8
|
||||
42 8
|
||||
42 8
|
||||
select sum(number) over w as x, max(number) over w as y from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by x, y SETTINGS max_threads = 1;
|
||||
6 2
|
||||
6 2
|
||||
6 2
|
||||
6 2
|
||||
6 2
|
||||
6 2
|
||||
24 5
|
||||
24 5
|
||||
24 5
|
||||
24 5
|
||||
24 5
|
||||
24 5
|
||||
42 8
|
||||
42 8
|
||||
42 8
|
||||
42 8
|
||||
42 8
|
||||
42 8
|
||||
select distinct sum(number) over w as x, max(number) over w as y from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by x, y;
|
||||
6 2
|
||||
24 5
|
||||
|
@ -15,12 +15,8 @@ from numbers(9);
|
||||
|
||||
select sum(number) over w as x, max(number) over w as y from t_01568 window w as (partition by p) order by x, y;
|
||||
|
||||
select sum(number) over w, max(number) over w from t_01568 window w as (partition by p) order by p;
|
||||
|
||||
select sum(number) over w as x, max(number) over w as y from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by x, y;
|
||||
|
||||
select sum(number) over w as x, max(number) over w as y from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by x, y SETTINGS max_threads = 1;
|
||||
|
||||
select distinct sum(number) over w as x, max(number) over w as y from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by x, y;
|
||||
|
||||
-- window functions + aggregation w/shards
|
||||
|
@ -1,100 +0,0 @@
|
||||
1
|
||||
-- { echoOn }
|
||||
|
||||
SELECT
|
||||
nw,
|
||||
sum(WR) AS R,
|
||||
sumIf(WR, uniq_rows = 1) AS UNR
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
uniq(nw) OVER (PARTITION BY ac) AS uniq_rows,
|
||||
AVG(wg) AS WR,
|
||||
ac,
|
||||
nw
|
||||
FROM window_funtion_threading
|
||||
GROUP BY ac, nw
|
||||
)
|
||||
GROUP BY nw
|
||||
ORDER BY nw ASC, R DESC
|
||||
LIMIT 10;
|
||||
0 2 0
|
||||
1 2 0
|
||||
2 2 0
|
||||
SELECT
|
||||
nw,
|
||||
sum(WR) AS R,
|
||||
sumIf(WR, uniq_rows = 1) AS UNR
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
uniq(nw) OVER (PARTITION BY ac) AS uniq_rows,
|
||||
AVG(wg) AS WR,
|
||||
ac,
|
||||
nw
|
||||
FROM window_funtion_threading
|
||||
GROUP BY ac, nw
|
||||
)
|
||||
GROUP BY nw
|
||||
ORDER BY nw ASC, R DESC
|
||||
LIMIT 10
|
||||
SETTINGS max_threads = 1;
|
||||
0 2 0
|
||||
1 2 0
|
||||
2 2 0
|
||||
SELECT
|
||||
nw,
|
||||
sum(WR) AS R,
|
||||
sumIf(WR, uniq_rows = 1) AS UNR
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
uniq(nw) OVER (PARTITION BY ac) AS uniq_rows,
|
||||
AVG(wg) AS WR,
|
||||
ac,
|
||||
nw
|
||||
FROM window_funtion_threading
|
||||
WHERE (ac % 4) = 0
|
||||
GROUP BY
|
||||
ac,
|
||||
nw
|
||||
UNION ALL
|
||||
SELECT
|
||||
uniq(nw) OVER (PARTITION BY ac) AS uniq_rows,
|
||||
AVG(wg) AS WR,
|
||||
ac,
|
||||
nw
|
||||
FROM window_funtion_threading
|
||||
WHERE (ac % 4) = 1
|
||||
GROUP BY
|
||||
ac,
|
||||
nw
|
||||
UNION ALL
|
||||
SELECT
|
||||
uniq(nw) OVER (PARTITION BY ac) AS uniq_rows,
|
||||
AVG(wg) AS WR,
|
||||
ac,
|
||||
nw
|
||||
FROM window_funtion_threading
|
||||
WHERE (ac % 4) = 2
|
||||
GROUP BY
|
||||
ac,
|
||||
nw
|
||||
UNION ALL
|
||||
SELECT
|
||||
uniq(nw) OVER (PARTITION BY ac) AS uniq_rows,
|
||||
AVG(wg) AS WR,
|
||||
ac,
|
||||
nw
|
||||
FROM window_funtion_threading
|
||||
WHERE (ac % 4) = 3
|
||||
GROUP BY
|
||||
ac,
|
||||
nw
|
||||
)
|
||||
GROUP BY nw
|
||||
ORDER BY nw ASC, R DESC
|
||||
LIMIT 10;
|
||||
0 2 0
|
||||
1 2 0
|
||||
2 2 0
|
@ -1,119 +0,0 @@
|
||||
CREATE TABLE window_funtion_threading
|
||||
Engine = MergeTree
|
||||
ORDER BY (ac, nw)
|
||||
AS SELECT
|
||||
toUInt64(toFloat32(number % 2) % 20000000) as ac,
|
||||
toFloat32(1) as wg,
|
||||
toUInt16(toFloat32(number % 3) % 400) as nw
|
||||
FROM numbers_mt(10000000);
|
||||
|
||||
SELECT count() FROM (EXPLAIN PIPELINE SELECT
|
||||
nw,
|
||||
sum(WR) AS R,
|
||||
sumIf(WR, uniq_rows = 1) AS UNR
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
uniq(nw) OVER (PARTITION BY ac) AS uniq_rows,
|
||||
AVG(wg) AS WR,
|
||||
ac,
|
||||
nw
|
||||
FROM window_funtion_threading
|
||||
GROUP BY ac, nw
|
||||
)
|
||||
GROUP BY nw
|
||||
ORDER BY nw ASC, R DESC
|
||||
LIMIT 10) where explain ilike '%ScatterByPartitionTransform%' SETTINGS max_threads = 4;
|
||||
|
||||
-- { echoOn }
|
||||
|
||||
SELECT
|
||||
nw,
|
||||
sum(WR) AS R,
|
||||
sumIf(WR, uniq_rows = 1) AS UNR
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
uniq(nw) OVER (PARTITION BY ac) AS uniq_rows,
|
||||
AVG(wg) AS WR,
|
||||
ac,
|
||||
nw
|
||||
FROM window_funtion_threading
|
||||
GROUP BY ac, nw
|
||||
)
|
||||
GROUP BY nw
|
||||
ORDER BY nw ASC, R DESC
|
||||
LIMIT 10;
|
||||
|
||||
SELECT
|
||||
nw,
|
||||
sum(WR) AS R,
|
||||
sumIf(WR, uniq_rows = 1) AS UNR
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
uniq(nw) OVER (PARTITION BY ac) AS uniq_rows,
|
||||
AVG(wg) AS WR,
|
||||
ac,
|
||||
nw
|
||||
FROM window_funtion_threading
|
||||
GROUP BY ac, nw
|
||||
)
|
||||
GROUP BY nw
|
||||
ORDER BY nw ASC, R DESC
|
||||
LIMIT 10
|
||||
SETTINGS max_threads = 1;
|
||||
|
||||
SELECT
|
||||
nw,
|
||||
sum(WR) AS R,
|
||||
sumIf(WR, uniq_rows = 1) AS UNR
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
uniq(nw) OVER (PARTITION BY ac) AS uniq_rows,
|
||||
AVG(wg) AS WR,
|
||||
ac,
|
||||
nw
|
||||
FROM window_funtion_threading
|
||||
WHERE (ac % 4) = 0
|
||||
GROUP BY
|
||||
ac,
|
||||
nw
|
||||
UNION ALL
|
||||
SELECT
|
||||
uniq(nw) OVER (PARTITION BY ac) AS uniq_rows,
|
||||
AVG(wg) AS WR,
|
||||
ac,
|
||||
nw
|
||||
FROM window_funtion_threading
|
||||
WHERE (ac % 4) = 1
|
||||
GROUP BY
|
||||
ac,
|
||||
nw
|
||||
UNION ALL
|
||||
SELECT
|
||||
uniq(nw) OVER (PARTITION BY ac) AS uniq_rows,
|
||||
AVG(wg) AS WR,
|
||||
ac,
|
||||
nw
|
||||
FROM window_funtion_threading
|
||||
WHERE (ac % 4) = 2
|
||||
GROUP BY
|
||||
ac,
|
||||
nw
|
||||
UNION ALL
|
||||
SELECT
|
||||
uniq(nw) OVER (PARTITION BY ac) AS uniq_rows,
|
||||
AVG(wg) AS WR,
|
||||
ac,
|
||||
nw
|
||||
FROM window_funtion_threading
|
||||
WHERE (ac % 4) = 3
|
||||
GROUP BY
|
||||
ac,
|
||||
nw
|
||||
)
|
||||
GROUP BY nw
|
||||
ORDER BY nw ASC, R DESC
|
||||
LIMIT 10;
|
Loading…
Reference in New Issue
Block a user