Skip to content

Commit

Permalink
fix ut
Browse files Browse the repository at this point in the history
  • Loading branch information
liuneng1994 committed Aug 16, 2024
1 parent c31c47f commit 7f50cbd
Show file tree
Hide file tree
Showing 15 changed files with 98 additions and 285 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -382,4 +382,5 @@ object CHBackendSettings extends BackendSettingsApi with Logging {

override def supportCartesianProductExec(): Boolean = true

override def supportNativeInputFileRelatedExpr(): Boolean = true
}
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import org.apache.spark.{InterruptibleIterator, SparkConf, TaskContext}
import org.apache.spark.affinity.CHAffinity
import org.apache.spark.executor.InputMetrics
import org.apache.spark.internal.Logging
import org.apache.spark.shuffle.CHColumnarShuffleWriter
import org.apache.spark.sql.connector.read.InputPartition
import org.apache.spark.sql.execution.datasources.FilePartition
import org.apache.spark.sql.execution.metric.SQLMetric
Expand Down Expand Up @@ -309,8 +310,10 @@ class CollectMetricIterator(
private var outputRowCount = 0L
private var outputVectorCount = 0L
private var metricsUpdated = false
private var wholeStagePipeline = true

override def hasNext: Boolean = {
wholeStagePipeline = false
nativeIterator.hasNext
}

Expand All @@ -334,6 +337,11 @@ class CollectMetricIterator(
private def collectStageMetrics(): Unit = {
if (!metricsUpdated) {
val nativeMetrics = nativeIterator.getMetrics.asInstanceOf[NativeMetrics]
if (wholeStagePipeline) {
outputRowCount = Math.max(outputRowCount, CHColumnarShuffleWriter.getTotalOutputRows())
outputVectorCount =
Math.max(outputVectorCount, CHColumnarShuffleWriter.getTotalOutputBatches())
}
nativeMetrics.setFinalOutputMetrics(outputRowCount, outputVectorCount)
updateNativeMetrics(nativeMetrics)
updateInputMetrics.foreach(_(inputMetrics))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import org.apache.gluten.execution.ColumnarNativeIterator
import org.apache.gluten.memory.CHThreadGroup
import org.apache.gluten.vectorized._

import org.apache.spark.SparkEnv
import org.apache.spark.{SparkEnv, TaskContext}
import org.apache.spark.internal.Logging
import org.apache.spark.scheduler.MapStatus
import org.apache.spark.sql.vectorized.ColumnarBatch
Expand Down Expand Up @@ -134,7 +134,7 @@ class CHColumnarShuffleWriter[K, V](
writeMetrics.incWriteTime(splitResult.getTotalWriteTime + splitResult.getTotalSpillTime)
partitionLengths = splitResult.getPartitionLengths
rawPartitionLengths = splitResult.getRawPartitionLengths

CHColumnarShuffleWriter.setOutputMetrics(splitResult)
try {
shuffleBlockResolver.writeMetadataFileAndCommit(
dep.shuffleId,
Expand Down Expand Up @@ -188,3 +188,30 @@ class CHColumnarShuffleWriter[K, V](
def getPartitionLengths(): Array[Long] = partitionLengths

}

object CHColumnarShuffleWriter {
def setOutputMetrics(splitResult: CHSplitResult): Unit = {
TaskContext
.get()
.getLocalProperties
.setProperty("total_output_rows", splitResult.getTotalRows.toString)
TaskContext
.get()
.getLocalProperties
.setProperty("total_output_batches", splitResult.getTotalBatches.toString)
}

def getTotalOutputRows(): Long = {
val output_rows = TaskContext.get().getLocalProperty("total_output_rows")
var output_rows_value = 0L
if (output_rows != null && output_rows.nonEmpty) output_rows_value = output_rows.toLong
output_rows_value
}

def getTotalOutputBatches(): Long = {
val output_batches = TaskContext.get().getLocalProperty("total_output_batches")
var output_batches_value = 0L
if (output_batches != null) output_batches_value = output_batches.toLong
output_batches_value
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -103,8 +103,8 @@ class GlutenClickHouseTPCHMetricsSuite extends GlutenClickHouseTPCHAbstractSuite
case scanExec: BasicScanExecTransformer => scanExec
}
assert(plans.size == 1)
// 1 block keep in SubstraitFileStep, and 4 blocks keep in other steps
assert(plans.head.metrics("numOutputRows").value === 5 * parquetMaxBlockSize)
// 1 block keep in SubstraitFileStep, and 5 blocks keep in other steps
assert(plans.head.metrics("numOutputRows").value === 6 * parquetMaxBlockSize)
assert(plans.head.metrics("outputVectors").value === 1)
assert(plans.head.metrics("outputBytes").value > 0)
}
Expand Down
36 changes: 0 additions & 36 deletions cpp-ch/local-engine/Common/common.cpp

This file was deleted.

151 changes: 8 additions & 143 deletions cpp-ch/local-engine/Parser/SerializedPlanParser.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,6 @@
#include <DataTypes/Serializations/ISerialization.h>
#include <DataTypes/getLeastSupertype.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Interpreters/ActionsDAG.h>
#include <Interpreters/ActionsVisitor.h>
#include <Interpreters/CollectJoinOnKeysVisitor.h>
Expand All @@ -60,17 +59,14 @@
#include <Parser/WriteRelParser.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ExpressionListParsers.h>
#include <Processors/Formats/Impl/ArrowBlockOutputFormat.h>
#include <Parser/LocalExecutor.h>
#include <Processors/QueryPlan/AggregatingStep.h>
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/QueryPlan/LimitStep.h>
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/ReadFromPreparedSource.h>
#include <Processors/Transforms/AggregatingTransform.h>
#include <QueryPipeline/Pipe.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <QueryPipeline/printPipeline.h>
#include <Storages/CustomStorageMergeTree.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/Output/FileWriterWrappers.h>
Expand All @@ -86,10 +82,9 @@
#include <Common/JNIUtils.h>
#include <Common/MergeTreeTool.h>
#include <Common/logger_useful.h>
#include <Common/QueryContext.h>
#include <Common/typeid_cast.h>
#include <Processors/Executors/PipelineExecutor.h>
#include <Processors/Executors/PullingAsyncPipelineExecutor.h>


namespace DB
{
Expand Down Expand Up @@ -319,11 +314,11 @@ QueryPlanStepPtr SerializedPlanParser::parseReadRealWithJavaIter(const substrait

GET_JNIENV(env)
SCOPE_EXIT({CLEAN_JNIENV});
auto * first_block = SourceFromJavaIter::peekBlock(env, input_iter);
auto first_block = SourceFromJavaIter::peekBlock(env, input_iter);

/// Try to decide header from the first block read from Java iterator. Thus AggregateFunction with parameters has more precise types.
auto header = first_block ? first_block->cloneEmpty() : TypeParser::buildBlockFromNamedStruct(rel.base_schema());
auto source = std::make_shared<SourceFromJavaIter>(context, std::move(header), input_iter, materialize_input, first_block);
auto header = first_block.has_value() ? first_block->cloneEmpty() : TypeParser::buildBlockFromNamedStruct(rel.base_schema());
auto source = std::make_shared<SourceFromJavaIter>(context, std::move(header), input_iter, materialize_input, std::move(first_block));

QueryPlanStepPtr source_step = std::make_unique<ReadFromPreparedSource>(Pipe(source));
source_step->setStepDescription("Read From Java Iter");
Expand Down Expand Up @@ -452,6 +447,9 @@ QueryPlanPtr SerializedPlanParser::parse(const substrait::Plan & plan)
return query_plan;
}

std::unique_ptr<LocalExecutor> SerializedPlanParser::createExecutor(const substrait::Plan & plan)
{ return createExecutor(parse(plan), plan); }

QueryPlanPtr SerializedPlanParser::parseOp(const substrait::Rel & rel, std::list<const substrait::Rel *> & rel_stack)
{
QueryPlanPtr query_plan;
Expand Down Expand Up @@ -1587,139 +1585,6 @@ void SerializedPlanParser::wrapNullable(

SharedContextHolder SerializedPlanParser::shared_context;

LocalExecutor::~LocalExecutor()
{
if (dump_pipeline)
LOG_INFO(&Poco::Logger::get("LocalExecutor"), "Dump pipeline:\n{}", dumpPipeline());

if (spark_buffer)
{
ch_column_to_spark_row->freeMem(spark_buffer->address, spark_buffer->size);
spark_buffer.reset();
}
}

std::unique_ptr<SparkRowInfo> LocalExecutor::writeBlockToSparkRow(const Block & block) const
{
return ch_column_to_spark_row->convertCHColumnToSparkRow(block);
}

void LocalExecutor::initPullingPipelineExecutor()
{
if (!executor)
{
query_pipeline = QueryPipelineBuilder::getPipeline(std::move(*query_pipeline_builder));
query_pipeline.setNumThreads(1);
executor = std::make_unique<PullingAsyncPipelineExecutor>(query_pipeline);
}
}

bool LocalExecutor::hasNext()
{
initPullingPipelineExecutor();
size_t columns = currentBlock().columns();
if (columns == 0 || isConsumed())
{
auto empty_block = header.cloneEmpty();
setCurrentBlock(empty_block);
bool has_next = executor->pull(currentBlock());
produce();
return has_next;
}
return true;
}

SparkRowInfoPtr LocalExecutor::next()
{
checkNextValid();
SparkRowInfoPtr row_info = writeBlockToSparkRow(currentBlock());
consume();
if (spark_buffer)
{
ch_column_to_spark_row->freeMem(spark_buffer->address, spark_buffer->size);
spark_buffer.reset();
}
spark_buffer = std::make_unique<SparkBuffer>();
spark_buffer->address = row_info->getBufferAddress();
spark_buffer->size = row_info->getTotalBytes();
return row_info;
}

Block * LocalExecutor::nextColumnar()
{
checkNextValid();
Block * columnar_batch;
if (currentBlock().columns() > 0)
{
columnar_batch = &currentBlock();
}
else
{
auto empty_block = header.cloneEmpty();
setCurrentBlock(empty_block);
columnar_batch = &currentBlock();
}
consume();
return columnar_batch;
}

void LocalExecutor::cancel()
{
if (executor)
executor->cancel();
if (push_executor)
push_executor->cancel();
}

void LocalExecutor::execute()
{
chassert(query_pipeline_builder || external_pipeline_builder);
if (external_pipeline_builder)
push_executor = external_pipeline_builder->execute();
else
push_executor = query_pipeline_builder->execute();
push_executor->execute(local_engine::QueryContextManager::instance().currentQueryContext()->getSettingsRef().max_threads, false);
}

Block LocalExecutor::getHeader()
{
return header;
}

LocalExecutor::LocalExecutor(QueryPlanPtr query_plan, QueryPipelineBuilderPtr pipeline_builder, bool dump_pipeline_)
: query_pipeline_builder(std::move(pipeline_builder))
, header(query_plan->getCurrentDataStream().header.cloneEmpty())
, dump_pipeline(dump_pipeline_)
, ch_column_to_spark_row(std::make_unique<CHColumnToSparkRow>())
, current_query_plan(std::move(query_plan))
{
chassert(!current_executor);
current_executor = this;
}
thread_local LocalExecutor * LocalExecutor::current_executor = nullptr;
std::string LocalExecutor::dumpPipeline() const
{
const auto & processors = query_pipeline.getProcessors();
for (auto & processor : processors)
{
WriteBufferFromOwnString buffer;
auto data_stats = processor->getProcessorDataStats();
buffer << "(";
buffer << "\nexecution time: " << processor->getElapsedNs() / 1000U << " us.";
buffer << "\ninput wait time: " << processor->getInputWaitElapsedNs() / 1000U << " us.";
buffer << "\noutput wait time: " << processor->getOutputWaitElapsedNs() / 1000U << " us.";
buffer << "\ninput rows: " << data_stats.input_rows;
buffer << "\ninput bytes: " << data_stats.input_bytes;
buffer << "\noutput rows: " << data_stats.output_rows;
buffer << "\noutput bytes: " << data_stats.output_bytes;
buffer << ")";
processor->setDescription(buffer.str());
}
WriteBufferFromOwnString out;
printPipeline(processors, out);
return out.str();
}

NonNullableColumnsResolver::NonNullableColumnsResolver(
const Block & header_, SerializedPlanParser & parser_, const substrait::Expression & cond_rel_)
: header(header_), parser(parser_), cond_rel(cond_rel_)
Expand Down
Loading

0 comments on commit 7f50cbd

Please sign in to comment.