blob: 248109274b44da46f277e2289fab45d2e1ba7785 [file] [log] [blame]
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
**/
#include "expressions/aggregation/AggregationHandleAvg.hpp"
#include <cstddef>
#include <memory>
#include <vector>
#include "catalog/CatalogTypedefs.hpp"
#include "storage/HashTable.hpp"
#include "storage/HashTableFactory.hpp"
#include "threading/SpinMutex.hpp"
#include "types/Type.hpp"
#include "types/TypeFactory.hpp"
#include "types/TypeID.hpp"
#include "types/TypedValue.hpp"
#include "types/operations/binary_operations/BinaryOperation.hpp"
#include "types/operations/binary_operations/BinaryOperationFactory.hpp"
#include "types/operations/binary_operations/BinaryOperationID.hpp"
#include "glog/logging.h"
namespace quickstep {
class StorageManager;
AggregationHandleAvg::AggregationHandleAvg(const Type &type)
: argument_type_(type), block_update_(false) {
// We sum Int as Long and Float as Double so that we have more headroom when
// adding many values.
TypeID type_precision_id;
switch (type.getTypeID()) {
case kInt:
case kLong:
type_precision_id = kLong;
break;
case kFloat:
case kDouble:
type_precision_id = kDouble;
break;
default:
type_precision_id = type.getTypeID();
break;
}
const Type &sum_type = TypeFactory::GetType(type_precision_id);
blank_state_.sum_ = sum_type.makeZeroValue();
blank_state_.count_ = 0;
// Make operators to do arithmetic:
// Add operator for summing argument values.
fast_add_operator_.reset(
BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
.makeUncheckedBinaryOperatorForTypes(sum_type, argument_type_));
// Add operator for merging states.
merge_add_operator_.reset(
BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
.makeUncheckedBinaryOperatorForTypes(sum_type, sum_type));
// Divide operator for dividing sum by count to get final average.
divide_operator_.reset(
BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kDivide)
.makeUncheckedBinaryOperatorForTypes(sum_type,
TypeFactory::GetType(kDouble)));
// Result is nullable, because AVG() over 0 values (or all NULL values) is
// NULL.
result_type_ =
&(BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kDivide)
.resultTypeForArgumentTypes(sum_type, TypeFactory::GetType(kDouble))
->getNullableVersion());
}
AggregationStateHashTableBase* AggregationHandleAvg::createGroupByHashTable(
const HashTableImplType hash_table_impl,
const std::vector<const Type *> &group_by_types,
const std::size_t estimated_num_groups,
StorageManager *storage_manager) const {
return AggregationStateHashTableFactory<AggregationStateAvg>::CreateResizable(
hash_table_impl, group_by_types, estimated_num_groups, storage_manager);
}
AggregationState* AggregationHandleAvg::accumulateColumnVectors(
const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const {
DCHECK_EQ(1u, column_vectors.size())
<< "Got wrong number of ColumnVectors for AVG: " << column_vectors.size();
AggregationStateAvg *state = new AggregationStateAvg(blank_state_);
std::size_t count = 0;
state->sum_ = fast_add_operator_->accumulateColumnVector(
state->sum_, *column_vectors.front(), &count);
state->count_ = count;
return state;
}
#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
AggregationState* AggregationHandleAvg::accumulateValueAccessor(
ValueAccessor *accessor,
const std::vector<attribute_id> &accessor_ids) const {
DCHECK_EQ(1u, accessor_ids.size())
<< "Got wrong number of attributes for AVG: " << accessor_ids.size();
AggregationStateAvg *state = new AggregationStateAvg(blank_state_);
std::size_t count = 0;
state->sum_ = fast_add_operator_->accumulateValueAccessor(
state->sum_, accessor, accessor_ids.front(), &count);
state->count_ = count;
return state;
}
#endif
void AggregationHandleAvg::aggregateValueAccessorIntoHashTable(
ValueAccessor *accessor,
const std::vector<attribute_id> &argument_ids,
const std::vector<attribute_id> &group_by_key_ids,
AggregationStateHashTableBase *hash_table) const {
DCHECK_EQ(1u, argument_ids.size())
<< "Got wrong number of arguments for AVG: " << argument_ids.size();
}
void AggregationHandleAvg::mergeStates(const AggregationState &source,
AggregationState *destination) const {
const AggregationStateAvg &avg_source =
static_cast<const AggregationStateAvg &>(source);
AggregationStateAvg *avg_destination =
static_cast<AggregationStateAvg *>(destination);
SpinMutexLock lock(avg_destination->mutex_);
avg_destination->count_ += avg_source.count_;
avg_destination->sum_ = merge_add_operator_->applyToTypedValues(
avg_destination->sum_, avg_source.sum_);
}
void AggregationHandleAvg::mergeStatesFast(const std::uint8_t *source,
std::uint8_t *destination) const {
const TypedValue *src_sum_ptr =
reinterpret_cast<const TypedValue *>(source + blank_state_.sum_offset_);
const std::int64_t *src_count_ptr = reinterpret_cast<const std::int64_t *>(
source + blank_state_.count_offset_);
TypedValue *dst_sum_ptr =
reinterpret_cast<TypedValue *>(destination + blank_state_.sum_offset_);
std::int64_t *dst_count_ptr = reinterpret_cast<std::int64_t *>(
destination + blank_state_.count_offset_);
(*dst_count_ptr) += (*src_count_ptr);
*dst_sum_ptr =
merge_add_operator_->applyToTypedValues(*dst_sum_ptr, *src_sum_ptr);
}
TypedValue AggregationHandleAvg::finalize(const AggregationState &state) const {
const AggregationStateAvg &agg_state =
static_cast<const AggregationStateAvg &>(state);
if (agg_state.count_ == 0) {
// AVG() over no values is NULL.
return result_type_->makeNullValue();
} else {
// Divide sum by count to get final average.
return divide_operator_->applyToTypedValues(
agg_state.sum_, TypedValue(static_cast<double>(agg_state.count_)));
}
}
ColumnVector* AggregationHandleAvg::finalizeHashTable(
const AggregationStateHashTableBase &hash_table,
std::vector<std::vector<TypedValue>> *group_by_keys,
int index) const {
return finalizeHashTableHelperFast<AggregationHandleAvg,
AggregationStateFastHashTable>(
*result_type_, hash_table, group_by_keys, index);
}
AggregationState*
AggregationHandleAvg::aggregateOnDistinctifyHashTableForSingle(
const AggregationStateHashTableBase &distinctify_hash_table) const {
return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
AggregationHandleAvg,
AggregationStateAvg>(distinctify_hash_table);
}
void AggregationHandleAvg::aggregateOnDistinctifyHashTableForGroupBy(
const AggregationStateHashTableBase &distinctify_hash_table,
AggregationStateHashTableBase *aggregation_hash_table,
std::size_t index) const {
aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
AggregationHandleAvg,
AggregationStateFastHashTable>(
distinctify_hash_table, aggregation_hash_table, index);
}
} // namespace quickstep