You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@quickstep.apache.org by ji...@apache.org on 2016/08/11 20:27:41 UTC
[01/16] incubator-quickstep git commit: Minor updates to Shiftboss.
[Forced Update!]
Repository: incubator-quickstep
Updated Branches:
refs/heads/LIP-for-tpch abc9c0238 -> 39f574999 (forced update)
Minor updates to Shiftboss.
Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/e443b2b8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/e443b2b8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/e443b2b8
Branch: refs/heads/LIP-for-tpch
Commit: e443b2b8409a128cc5ba2bdf1a6d01ebf79e7e74
Parents: bd01748
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Mon Aug 8 10:33:10 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Mon Aug 8 10:33:10 2016 -0700
----------------------------------------------------------------------
query_execution/QueryExecutionMessages.proto | 1 +
query_execution/Shiftboss.cpp | 89 ++++++++++++++++-------
query_execution/Shiftboss.hpp | 3 +
3 files changed, 65 insertions(+), 28 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e443b2b8/query_execution/QueryExecutionMessages.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionMessages.proto b/query_execution/QueryExecutionMessages.proto
index f6a8b73..f680d35 100644
--- a/query_execution/QueryExecutionMessages.proto
+++ b/query_execution/QueryExecutionMessages.proto
@@ -87,6 +87,7 @@ message ShiftbossRegistrationMessage {
}
message ShiftbossRegistrationResponseMessage {
+ required uint64 shiftboss_index = 1;
}
message QueryInitiateMessage {
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e443b2b8/query_execution/Shiftboss.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Shiftboss.cpp b/query_execution/Shiftboss.cpp
index 120e8fb..24c91fe 100644
--- a/query_execution/Shiftboss.cpp
+++ b/query_execution/Shiftboss.cpp
@@ -56,6 +56,7 @@ using std::string;
using std::unique_ptr;
using std::vector;
+using tmb::MessageBus;
using tmb::TaggedMessage;
namespace quickstep {
@@ -78,6 +79,13 @@ void Shiftboss::run() {
switch (annotated_message.tagged_message.message_type()) {
case kShiftbossRegistrationResponseMessage: {
foreman_client_id_ = annotated_message.sender;
+
+ const TaggedMessage &tagged_message = annotated_message.tagged_message;
+
+ serialization::ShiftbossRegistrationResponseMessage proto;
+ CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
+
+ shiftboss_index_ = proto.shiftboss_index();
break;
}
case kQueryInitiateMessage: {
@@ -117,10 +125,14 @@ void Shiftboss::run() {
<< "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
<< "') from Foreman to worker " << worker_index;
- QueryExecutionUtil::SendTMBMessage(bus_,
- shiftboss_client_id_,
- workers_->getClientID(worker_index),
- move(worker_tagged_message));
+ const MessageBus::SendStatus send_status =
+ QueryExecutionUtil::SendTMBMessage(bus_,
+ shiftboss_client_id_,
+ workers_->getClientID(worker_index),
+ move(worker_tagged_message));
+ CHECK(send_status == MessageBus::SendStatus::kOK)
+ << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
+ << " to Worker with TMB client ID " << workers_->getClientID(worker_index);
break;
}
case kInitiateRebuildMessage: {
@@ -147,10 +159,14 @@ void Shiftboss::run() {
<< "' message from worker (client " << annotated_message.sender << ") to Foreman";
DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
- QueryExecutionUtil::SendTMBMessage(bus_,
- shiftboss_client_id_,
- foreman_client_id_,
- move(annotated_message.tagged_message));
+ const MessageBus::SendStatus send_status =
+ QueryExecutionUtil::SendTMBMessage(bus_,
+ shiftboss_client_id_,
+ foreman_client_id_,
+ move(annotated_message.tagged_message));
+ CHECK(send_status == MessageBus::SendStatus::kOK)
+ << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
+ << " to Foreman with TMB client ID " << foreman_client_id_;
break;
}
case kSaveQueryResultMessage: {
@@ -178,10 +194,14 @@ void Shiftboss::run() {
LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
<< "') sent SaveQueryResultResponseMessage (typed '" << kSaveQueryResultResponseMessage
<< "') to Foreman";
- QueryExecutionUtil::SendTMBMessage(bus_,
- shiftboss_client_id_,
- foreman_client_id_,
- move(message_response));
+ const MessageBus::SendStatus send_status =
+ QueryExecutionUtil::SendTMBMessage(bus_,
+ shiftboss_client_id_,
+ foreman_client_id_,
+ move(message_response));
+ CHECK(send_status == MessageBus::SendStatus::kOK)
+ << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
+ << " to Foreman with TMB client ID " << foreman_client_id_;
break;
}
case kPoisonMessage: {
@@ -192,12 +212,14 @@ void Shiftboss::run() {
tmb::MessageStyle broadcast_style;
broadcast_style.Broadcast(true);
- tmb::MessageBus::SendStatus send_status =
+ const MessageBus::SendStatus send_status =
bus_->Send(shiftboss_client_id_,
worker_addresses_,
broadcast_style,
move(annotated_message.tagged_message));
- DCHECK(send_status == tmb::MessageBus::SendStatus::kOK);
+ CHECK(send_status == MessageBus::SendStatus::kOK)
+ << "Message could not be broadcast from Shiftboss with TMB client ID " << shiftboss_client_id_
+ << " to All workers";
return;
}
default: {
@@ -280,10 +302,14 @@ void Shiftboss::processQueryInitiateMessage(
kQueryInitiateResponseMessage);
free(proto_bytes);
- QueryExecutionUtil::SendTMBMessage(bus_,
- shiftboss_client_id_,
- foreman_client_id_,
- move(message_response));
+ const MessageBus::SendStatus send_status =
+ QueryExecutionUtil::SendTMBMessage(bus_,
+ shiftboss_client_id_,
+ foreman_client_id_,
+ move(message_response));
+ CHECK(send_status == MessageBus::SendStatus::kOK)
+ << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
+ << " to Foreman with TMB client ID " << foreman_client_id_;
}
void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
@@ -307,8 +333,7 @@ void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
proto.set_query_id(query_id);
proto.set_operator_index(op_index);
proto.set_num_rebuild_work_orders(partially_filled_block_refs.size());
- // TODO(zuyu): Multiple Shiftboss support.
- proto.set_shiftboss_index(0);
+ proto.set_shiftboss_index(shiftboss_index_);
const size_t proto_length = proto.ByteSize();
char *proto_bytes = static_cast<char*>(malloc(proto_length));
@@ -319,10 +344,14 @@ void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
kInitiateRebuildResponseMessage);
free(proto_bytes);
- QueryExecutionUtil::SendTMBMessage(bus_,
- shiftboss_client_id_,
- foreman_client_id_,
- move(message_response));
+ const MessageBus::SendStatus send_status =
+ QueryExecutionUtil::SendTMBMessage(bus_,
+ shiftboss_client_id_,
+ foreman_client_id_,
+ move(message_response));
+ CHECK(send_status == MessageBus::SendStatus::kOK)
+ << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
+ << " to Foreman with TMB client ID " << foreman_client_id_;
for (size_t i = 0; i < partially_filled_block_refs.size(); ++i) {
// NOTE(zuyu): Worker releases the memory after the execution of
@@ -347,10 +376,14 @@ void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
<< "') sent RebuildWorkOrderMessage (typed '" << kRebuildWorkOrderMessage
<< "') to worker " << worker_index;
- QueryExecutionUtil::SendTMBMessage(bus_,
- shiftboss_client_id_,
- workers_->getClientID(worker_index),
- move(worker_tagged_message));
+ const MessageBus::SendStatus send_status =
+ QueryExecutionUtil::SendTMBMessage(bus_,
+ shiftboss_client_id_,
+ workers_->getClientID(worker_index),
+ move(worker_tagged_message));
+ CHECK(send_status == MessageBus::SendStatus::kOK)
+ << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
+ << " to Worker with TMB client ID " << workers_->getClientID(worker_index);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e443b2b8/query_execution/Shiftboss.hpp
----------------------------------------------------------------------
diff --git a/query_execution/Shiftboss.hpp b/query_execution/Shiftboss.hpp
index 32d2408..9464a4d 100644
--- a/query_execution/Shiftboss.hpp
+++ b/query_execution/Shiftboss.hpp
@@ -223,6 +223,9 @@ class Shiftboss : public Thread {
tmb::client_id shiftboss_client_id_, foreman_client_id_;
+ // Unique per Shiftboss instance.
+ std::uint64_t shiftboss_index_;
+
// TMB recipients for all workers managed by this Shiftboss.
tmb::Address worker_addresses_;
[08/16] incubator-quickstep git commit: Initial commit
Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/PackedRowStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/PackedRowStoreValueAccessor.hpp b/storage/PackedRowStoreValueAccessor.hpp
index 80edecd..1520b3a 100644
--- a/storage/PackedRowStoreValueAccessor.hpp
+++ b/storage/PackedRowStoreValueAccessor.hpp
@@ -20,6 +20,8 @@
#ifndef QUICKSTEP_STORAGE_PACKED_ROW_STORE_VALUE_ACCESSOR_HPP_
#define QUICKSTEP_STORAGE_PACKED_ROW_STORE_VALUE_ACCESSOR_HPP_
+#include <utility>
+
#include "catalog/CatalogRelationSchema.hpp"
#include "catalog/CatalogTypedefs.hpp"
#include "storage/StorageBlockInfo.hpp"
@@ -42,7 +44,8 @@ class PackedRowStoreValueAccessorHelper {
: relation_(relation),
num_tuples_(num_tuples),
tuple_storage_(tuple_storage),
- null_bitmap_(null_bitmap) {
+ null_bitmap_(null_bitmap),
+ attr_max_lengths_(relation.getMaximumAttributeByteLengths()) {
}
inline tuple_id numPackedTuples() const {
@@ -67,6 +70,25 @@ class PackedRowStoreValueAccessorHelper {
+ relation_.getFixedLengthAttributeOffset(attr); // Attribute offset within tuple.
}
+ template <bool check_null>
+ inline std::pair<const void*, std::size_t> getAttributeValueAndByteLength(const tuple_id tuple,
+ const attribute_id attr) const {
+ DEBUG_ASSERT(tuple < num_tuples_);
+ DEBUG_ASSERT(relation_.hasAttributeWithId(attr));
+ if (check_null) {
+ const int nullable_idx = relation_.getNullableAttributeIndex(attr);
+ if ((nullable_idx != -1)
+ && null_bitmap_->getBit(tuple * relation_.numNullableAttributes() + nullable_idx)) {
+ return std::make_pair(nullptr, 0);
+ }
+ }
+
+ return std::make_pair(static_cast<const char*>(tuple_storage_)
+ + (tuple * relation_.getFixedByteLength())
+ + relation_.getFixedLengthAttributeOffset(attr),
+ attr_max_lengths_[attr]);
+ }
+
inline TypedValue getAttributeValueTyped(const tuple_id tuple,
const attribute_id attr) const {
const Type &attr_type = relation_.getAttributeById(attr)->getType();
@@ -81,6 +103,7 @@ class PackedRowStoreValueAccessorHelper {
const tuple_id num_tuples_;
const void *tuple_storage_;
const BitVector<false> *null_bitmap_;
+ const std::vector<std::size_t> &attr_max_lengths_;
DISALLOW_COPY_AND_ASSIGN(PackedRowStoreValueAccessorHelper);
};
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/SplitRowStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/SplitRowStoreValueAccessor.hpp b/storage/SplitRowStoreValueAccessor.hpp
index 61bb7bf..e2d2b47 100644
--- a/storage/SplitRowStoreValueAccessor.hpp
+++ b/storage/SplitRowStoreValueAccessor.hpp
@@ -102,6 +102,11 @@ class SplitRowStoreValueAccessor : public ValueAccessor {
return getUntypedValueAtAbsolutePosition<check_null>(attr_id, current_position_);
}
+ template <bool check_null = true>
+ inline std::pair<const void*, std::size_t> getUntypedValueAndByteLength(const attribute_id attr_id) const {
+ return getUntypedValueAndByteLengthAtAbsolutePosition<check_null>(attr_id, current_position_);
+ }
+
inline TypedValue getTypedValue(const attribute_id attr_id) const {
return getTypedValueAtAbsolutePosition(attr_id, current_position_);
}
@@ -142,6 +147,44 @@ class SplitRowStoreValueAccessor : public ValueAccessor {
}
}
+ template <bool check_null = true>
+ inline std::pair<const void*, std::size_t> getUntypedValueAndByteLengthAtAbsolutePosition(const attribute_id attr_id,
+ const tuple_id tid) const {
+ DEBUG_ASSERT(occupancy_bitmap_.getBit(tid));
+ DEBUG_ASSERT(relation_.hasAttributeWithId(attr_id));
+ const char *tuple_slot = static_cast<const char*>(tuple_storage_)
+ + tuple_slot_bytes_ * tid;
+ if (check_null) {
+ const int nullable_idx = relation_.getNullableAttributeIndex(attr_id);
+ if (nullable_idx != -1) {
+ // const_cast is safe here. We will only be using read-only methods of
+ // BitVector.
+ BitVector<true> tuple_null_bitmap(const_cast<void*>(static_cast<const void*>(tuple_slot)),
+ relation_.numNullableAttributes());
+ if (tuple_null_bitmap.getBit(nullable_idx)) {
+ return std::make_pair(nullptr, 0);
+ }
+ }
+ }
+
+ const int variable_length_idx = relation_.getVariableLengthAttributeIndex(attr_id);
+ if (variable_length_idx == -1) {
+ // Fixed-length, stored in-line in slot.
+ return std::make_pair(tuple_slot + per_tuple_null_bitmap_bytes_
+ + relation_.getFixedLengthAttributeOffset(attr_id),
+ attr_max_lengths_[attr_id]);
+
+ } else {
+ // Variable-length, stored at back of block.
+ const std::uint32_t *pos_ptr = reinterpret_cast<const std::uint32_t*>(
+ tuple_slot + per_tuple_null_bitmap_bytes_
+ + relation_.getFixedByteLength()
+ + variable_length_idx * 2 * sizeof(std::uint32_t));
+ return std::make_pair(static_cast<const char*>(tuple_storage_) + pos_ptr[0],
+ pos_ptr[1]);
+ }
+ }
+
inline TypedValue getTypedValueAtAbsolutePosition(const attribute_id attr_id,
const tuple_id tid) const {
DEBUG_ASSERT(occupancy_bitmap_.getBit(tid));
@@ -319,6 +362,7 @@ class SplitRowStoreValueAccessor : public ValueAccessor {
tuple_storage_(tuple_storage),
tuple_slot_bytes_(tuple_slot_bytes),
per_tuple_null_bitmap_bytes_(per_tuple_null_bitmap_bytes),
+ attr_max_lengths_(relation.getMaximumAttributeByteLengths()),
current_position_(std::numeric_limits<std::size_t>::max()) {
}
@@ -329,6 +373,7 @@ class SplitRowStoreValueAccessor : public ValueAccessor {
const void *tuple_storage_;
const std::size_t tuple_slot_bytes_;
const std::size_t per_tuple_null_bitmap_bytes_;
+ const std::vector<std::size_t> &attr_max_lengths_;
std::size_t current_position_;
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/StorageBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.cpp b/storage/StorageBlock.cpp
index 21aa12c..8370418 100644
--- a/storage/StorageBlock.cpp
+++ b/storage/StorageBlock.cpp
@@ -389,15 +389,7 @@ AggregationState* StorageBlock::aggregate(
const AggregationHandle &handle,
const std::vector<std::unique_ptr<const Scalar>> &arguments,
const std::vector<attribute_id> *arguments_as_attributes,
- const Predicate *predicate,
std::unique_ptr<TupleIdSequence> *reuse_matches) const {
- // If there is a filter predicate that hasn't already been evaluated,
- // evaluate it now and save the results for other aggregates on this same
- // block.
- if (predicate && !*reuse_matches) {
- reuse_matches->reset(getMatchesForPredicate(predicate));
- }
-
#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
// If all the arguments to this aggregate are plain relation attributes,
// aggregate directly on a ValueAccessor from this block to avoid a copy.
@@ -418,7 +410,6 @@ void StorageBlock::aggregateGroupBy(
const AggregationHandle &handle,
const std::vector<std::unique_ptr<const Scalar>> &arguments,
const std::vector<std::unique_ptr<const Scalar>> &group_by,
- const Predicate *predicate,
AggregationStateHashTableBase *hash_table,
std::unique_ptr<TupleIdSequence> *reuse_matches,
std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const {
@@ -440,14 +431,7 @@ void StorageBlock::aggregateGroupBy(
ColumnVectorsValueAccessor temp_result;
{
std::unique_ptr<ValueAccessor> accessor;
- if (predicate) {
- if (!*reuse_matches) {
- // If there is a filter predicate that hasn't already been evaluated,
- // evaluate it now and save the results for other aggregates on this
- // same block.
- reuse_matches->reset(getMatchesForPredicate(predicate));
- }
-
+ if (reuse_matches) {
// Create a filtered ValueAccessor that only iterates over predicate
// matches.
accessor.reset(tuple_store_->createValueAccessor(reuse_matches->get()));
@@ -499,7 +483,6 @@ void StorageBlock::aggregateDistinct(
const std::vector<std::unique_ptr<const Scalar>> &arguments,
const std::vector<attribute_id> *arguments_as_attributes,
const std::vector<std::unique_ptr<const Scalar>> &group_by,
- const Predicate *predicate,
AggregationStateHashTableBase *distinctify_hash_table,
std::unique_ptr<TupleIdSequence> *reuse_matches,
std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const {
@@ -514,14 +497,7 @@ void StorageBlock::aggregateDistinct(
ColumnVectorsValueAccessor temp_result;
{
std::unique_ptr<ValueAccessor> accessor;
- if (predicate) {
- if (!*reuse_matches) {
- // If there is a filter predicate that hasn't already been evaluated,
- // evaluate it now and save the results for other aggregates on this
- // same block.
- reuse_matches->reset(getMatchesForPredicate(predicate));
- }
-
+ if (reuse_matches) {
// Create a filtered ValueAccessor that only iterates over predicate
// matches.
accessor.reset(tuple_store_->createValueAccessor(reuse_matches->get()));
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/StorageBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.hpp b/storage/StorageBlock.hpp
index 97b4773..2a20cb5 100644
--- a/storage/StorageBlock.hpp
+++ b/storage/StorageBlock.hpp
@@ -412,7 +412,6 @@ class StorageBlock : public StorageBlockBase {
const AggregationHandle &handle,
const std::vector<std::unique_ptr<const Scalar>> &arguments,
const std::vector<attribute_id> *arguments_as_attributes,
- const Predicate *predicate,
std::unique_ptr<TupleIdSequence> *reuse_matches) const;
/**
@@ -462,7 +461,6 @@ class StorageBlock : public StorageBlockBase {
void aggregateGroupBy(const AggregationHandle &handle,
const std::vector<std::unique_ptr<const Scalar>> &arguments,
const std::vector<std::unique_ptr<const Scalar>> &group_by,
- const Predicate *predicate,
AggregationStateHashTableBase *hash_table,
std::unique_ptr<TupleIdSequence> *reuse_matches,
std::vector<std::unique_ptr<ColumnVector>>
@@ -507,7 +505,6 @@ class StorageBlock : public StorageBlockBase {
const std::vector<std::unique_ptr<const Scalar>> &arguments,
const std::vector<attribute_id> *arguments_as_attributes,
const std::vector<std::unique_ptr<const Scalar>> &group_by,
- const Predicate *predicate,
AggregationStateHashTableBase *distinctify_hash_table,
std::unique_ptr<TupleIdSequence> *reuse_matches,
std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const;
@@ -590,6 +587,8 @@ class StorageBlock : public StorageBlockBase {
**/
const std::size_t getNumTuples() const;
+ TupleIdSequence* getMatchesForPredicate(const Predicate *predicate) const;
+
private:
static TupleStorageSubBlock* CreateTupleStorageSubBlock(
const CatalogRelationSchema &relation,
@@ -629,8 +628,6 @@ class StorageBlock : public StorageBlockBase {
// StorageBlock's header.
bool rebuildIndexes(bool short_circuit);
- TupleIdSequence* getMatchesForPredicate(const Predicate *predicate) const;
-
std::unordered_map<attribute_id, TypedValue>* generateUpdatedValues(
const ValueAccessor &accessor,
const tuple_id tuple,
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/ValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/ValueAccessor.hpp b/storage/ValueAccessor.hpp
index 70d4405..b107390 100644
--- a/storage/ValueAccessor.hpp
+++ b/storage/ValueAccessor.hpp
@@ -377,6 +377,11 @@ class TupleIdSequenceAdapterValueAccessor : public ValueAccessor {
return accessor_->template getUntypedValueAtAbsolutePosition<check_null>(attr_id, *current_position_);
}
+ template <bool check_null = true>
+ inline std::pair<const void*, std::size_t> getUntypedValueAndByteLength(const attribute_id attr_id) const {
+ return accessor_->template getUntypedValueAndByteLengthAtAbsolutePosition<check_null>(attr_id, *current_position_);
+ }
+
inline TypedValue getTypedValue(const attribute_id attr_id) const {
return accessor_->getTypedValueAtAbsolutePosition(attr_id, *current_position_);
}
@@ -389,6 +394,13 @@ class TupleIdSequenceAdapterValueAccessor : public ValueAccessor {
}
// Pass-through.
+ template <bool check_null = true>
+ inline std::pair<const void*, std::size_t> getUntypedValueAndByteLengthAtAbsolutePosition(const attribute_id attr_id,
+ const tuple_id tid) const {
+ return accessor_->template getUntypedValueAndByteLengthAtAbsolutePosition<check_null>(attr_id, tid);
+ }
+
+ // Pass-through.
inline TypedValue getTypedValueAtAbsolutePosition(const attribute_id attr_id,
const tuple_id tid) const {
return accessor_->getTypedValueAtAbsolutePosition(attr_id, tid);
@@ -562,6 +574,12 @@ class OrderedTupleIdSequenceAdapterValueAccessor : public ValueAccessor {
id_sequence_[current_position_]);
}
+ template <bool check_null = true>
+ inline std::pair<const void*, std::size_t> getUntypedValueAndByteLength(const attribute_id attr_id) const {
+ return accessor_->template getUntypedValueAndByteLengthAtAbsolutePosition<check_null>(
+ attr_id, id_sequence_[current_position_]);
+ }
+
inline TypedValue getTypedValue(const attribute_id attr_id) const {
return accessor_->getTypedValueAtAbsolutePosition(attr_id, id_sequence_[current_position_]);
}
@@ -573,6 +591,13 @@ class OrderedTupleIdSequenceAdapterValueAccessor : public ValueAccessor {
"OrderedTupleIdSequenceAdapterValueAccessor");
}
+ template <bool check_null = true>
+ inline std::pair<const void*, std::size_t> getUntypedValueAndByteLengthAtAbsolutePosition(const attribute_id attr_id,
+ const tuple_id tid) const {
+ FATAL_ERROR("getUntypedValueAndByteLengthAtAbsolutePosition() not implemented in "
+ "OrderedTupleIdSequenceAdapterValueAccessor");
+ }
+
inline TypedValue getTypedValueAtAbsolutePosition(const attribute_id attr_id,
const tuple_id tid) const {
FATAL_ERROR("getTypedValueAtAbsolutePosition() not implemented in "
@@ -739,6 +764,11 @@ class PackedTupleStorageSubBlockValueAccessor : public ValueAccessor {
return getUntypedValueAtAbsolutePosition<check_null>(attr_id, current_tuple_);
}
+ template <bool check_null = true>
+ inline std::pair<const void*, std::size_t> getUntypedValueAndByteLength(const attribute_id attr_id) const {
+ return getUntypedValueAndByteLengthAtAbsolutePosition<check_null>(attr_id, current_tuple_);
+ }
+
inline TypedValue getTypedValue(const attribute_id attr_id) const {
return getTypedValueAtAbsolutePosition(attr_id, current_tuple_);
}
@@ -749,6 +779,12 @@ class PackedTupleStorageSubBlockValueAccessor : public ValueAccessor {
return helper_.template getAttributeValue<check_null>(tid, attr_id);
}
+ template <bool check_null = true>
+ inline std::pair<const void*, std::size_t> getUntypedValueAndByteLengthAtAbsolutePosition(const attribute_id attr_id,
+ const tuple_id tid) const {
+ return helper_.template getAttributeValueAndByteLength<check_null>(tid, attr_id);
+ }
+
inline TypedValue getTypedValueAtAbsolutePosition(const attribute_id attr_id,
const tuple_id tid) const {
return helper_.getAttributeValueTyped(tid, attr_id);
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/types/containers/ColumnVector.hpp
----------------------------------------------------------------------
diff --git a/types/containers/ColumnVector.hpp b/types/containers/ColumnVector.hpp
index fc65656..0817054 100644
--- a/types/containers/ColumnVector.hpp
+++ b/types/containers/ColumnVector.hpp
@@ -195,6 +195,22 @@ class NativeColumnVector : public ColumnVector {
}
/**
+ * @brief Get the untyped pointer to a value as well as the value's byte length
+ * in this NativeColumnVector as a pair.
+ *
+ * @param position The position of the value to get.
+ * @return A pair containing the untyped pointer to the value at position and
+ * the value's byte length.
+ **/
+ template <bool check_null = true>
+ inline std::pair<const void*, std::size_t> getUntypedValueAndByteLength(const std::size_t position) const {
+ DCHECK_LT(position, actual_length_);
+ return (check_null && null_bitmap_ && null_bitmap_->getBit(position))
+ ? std::make_pair(nullptr, 0)
+ : std::make_pair(static_cast<const char*>(values_) + (position * type_length_), type_length_);
+ }
+
+ /**
* @brief Get a value in this NativeColumnVector as a TypedValue.
*
* @param position The position of the value to get.
@@ -455,6 +471,25 @@ class IndirectColumnVector : public ColumnVector {
}
/**
+ * @brief Get the untyped pointer to a value as well as the value's byte length
+ * in this IndirectColumnVector as a pair.
+ *
+ * @param position The position of the value to get.
+ * @return A pair containing the untyped pointer to the value at position and
+ * the value's byte length.
+ **/
+ template <bool check_null = true>
+ inline std::pair<const void*, std::size_t> getUntypedValueAndByteLength(const std::size_t position) const {
+ DCHECK_LT(position, values_.size());
+ if (check_null && type_is_nullable_ && values_[position].isNull()) {
+ return std::make_pair(nullptr, 0);
+ } else {
+ const TypedValue &value = values_[position];
+ return std::make_pair(value.getDataPtr(), value.getDataSize());
+ }
+ }
+
+ /**
* @brief Get a value in this IndirectColumnVector as a TypedValue.
*
* @param position The position of the value to get.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/types/containers/ColumnVectorsValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/types/containers/ColumnVectorsValueAccessor.hpp b/types/containers/ColumnVectorsValueAccessor.hpp
index 2300f3b..19e57a9 100644
--- a/types/containers/ColumnVectorsValueAccessor.hpp
+++ b/types/containers/ColumnVectorsValueAccessor.hpp
@@ -126,6 +126,11 @@ class ColumnVectorsValueAccessor : public ValueAccessor {
return getUntypedValueAtAbsolutePosition<check_null>(attr_id, current_position_);
}
+ template <bool check_null = true>
+ inline std::pair<const void*, std::size_t> getUntypedValueAndByteLength(const attribute_id attr_id) const {
+ return getUntypedValueAndByteLengthAtAbsolutePosition<check_null>(attr_id, current_position_);
+ }
+
inline TypedValue getTypedValue(const attribute_id attr_id) const {
return getTypedValueAtAbsolutePosition(attr_id, current_position_);
}
@@ -142,6 +147,18 @@ class ColumnVectorsValueAccessor : public ValueAccessor {
}
}
+ template <bool check_null = true>
+ inline std::pair<const void*, std::size_t> getUntypedValueAndByteLengthAtAbsolutePosition(const attribute_id attr_id,
+ const tuple_id tid) const {
+ DCHECK(attributeIdInRange(attr_id));
+ DCHECK(tupleIdInRange(tid));
+ if (column_native_[attr_id]) {
+ return static_cast<const NativeColumnVector&>(*columns_[attr_id]).getUntypedValueAndByteLength<check_null>(tid);
+ } else {
+ return static_cast<const IndirectColumnVector&>(*columns_[attr_id]).getUntypedValueAndByteLength<check_null>(tid);
+ }
+ }
+
inline TypedValue getTypedValueAtAbsolutePosition(const attribute_id attr_id,
const tuple_id tid) const {
DCHECK(attributeIdInRange(attr_id));
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/utility/BloomFilter.hpp
----------------------------------------------------------------------
diff --git a/utility/BloomFilter.hpp b/utility/BloomFilter.hpp
index 8d62da9..749d33a 100644
--- a/utility/BloomFilter.hpp
+++ b/utility/BloomFilter.hpp
@@ -23,6 +23,7 @@
#include <algorithm>
#include <cstddef>
#include <cstdint>
+#include <cstring>
#include <memory>
#include <utility>
#include <vector>
@@ -41,11 +42,358 @@ namespace quickstep {
* @{
*/
+class BloomFilterOriginal;
+class BloomFilterBlocked;
+typedef BloomFilterBlocked BloomFilter;
+
+/**
+ * @brief A "blocked" version of Bloom Filter based on this paper:
+ * Putze, Felix, Peter Sanders, and Johannes Singler.
+ * "Cache-, hash-and space-efficient bloom filters."
+ * International Workshop on Experimental and Efficient Algorithms.
+ * Springer Berlin Heidelberg, 2007.
+ **/
+class BloomFilterBlocked {
+ public:
+ static const std::uint8_t kNumBitsPerByte = 8;
+ static const std::uint8_t kMaxNumHashFns = 4;
+
+ // This union allows us to read/write position in convenient fashion,
+ // through nested structs and their bitfield members
+ //
+ // A position can simply be a 32-bit hash
+ // Or it can be a cache line (block of 512 bits) and position within it
+ // Or it can be a byte (block of 8 bits) and position within it
+ union Position {
+ std::uint32_t hash;
+ struct CacheLinePosition {
+ unsigned index_in_line : 9;
+ unsigned line_num : 23;
+ } cache_line_pos;
+ struct BytePosition {
+ unsigned index_in_byte : 3;
+ unsigned byte_num : 29;
+ } byte_pos;
+ };
+
+ // This Bloom filter implementation requires the bit array to be a
+ // multiple of the cache-line size. So we either have to round up to a
+ // multiple (default behavior) or round down to a multiple.
+ // Rounding up is usually preferable but rounding down is necessary when
+ // we are given a bit array that we don't control the size of, in the
+ // constructor.
+ static std::uint64_t getNearestAllowedSize(
+ const std::uint64_t approx_size,
+ bool round_down = false) {
+ if (round_down)
+ return (approx_size / kCacheLineBytes) * kCacheLineBytes;
+ return ((approx_size + kCacheLineBytes - 1)/ kCacheLineBytes) * kCacheLineBytes;
+ }
+
+
+ /**
+ * @brief Constructor.
+ * @note When no bit_array is being passed to the constructor,
+ * then the bit_array is owned and managed by this class.
+ *
+ * @param hash_fn_count The number of hash functions used by this bloom filter.
+ * @param bit_array_size_in_bytes Size of the bit array.
+ **/
+ BloomFilterBlocked(const std::uint8_t hash_fn_count,
+ const std::uint64_t bit_array_size_in_bytes)
+ : hash_fn_count_(hash_fn_count),
+ array_size_in_bytes_(getNearestAllowedSize(bit_array_size_in_bytes)),
+ is_bit_array_owner_(true),
+ bit_array_(new std::uint8_t[array_size_in_bytes_]) {
+ reset();
+ }
+
+ /**
+ * @brief Constructor.
+ * @note When a bit_array is passed as an argument to the constructor,
+ * then the ownership of the bit array lies with the caller.
+ *
+ * @param hash_fn_count The number of hash functions used by this bloom filter.
+ * @param bit_array_size_in_bytes Size of the bit array.
+ * @param bit_array A pointer to the memory region that is used to store bit array.
+ * @param is_initialized A boolean that indicates whether to zero-out the region
+ * before use or not.
+ **/
+ BloomFilterBlocked(const std::uint8_t hash_fn_count,
+ const std::uint64_t bit_array_size_in_bytes,
+ std::uint8_t *bit_array,
+ const bool is_initialized)
+ : hash_fn_count_(hash_fn_count),
+ array_size_in_bytes_(getNearestAllowedSize(bit_array_size_in_bytes, true)),
+ is_bit_array_owner_(false),
+ bit_array_(bit_array) { // Owned by the calling method.
+ if (!is_initialized) {
+ reset();
+ }
+ }
+
+ /**
+ * @brief Constructor.
+ * @note When a bloom filter proto is passed as an initializer,
+ * then the bit_array is owned and managed by this class.
+ *
+ * @param bloom_filter_proto The protobuf representation of a
+ * bloom filter configuration.
+ **/
+ explicit BloomFilterBlocked(const serialization::BloomFilter &bloom_filter_proto)
+ : hash_fn_count_(bloom_filter_proto.number_of_hashes()),
+ array_size_in_bytes_(bloom_filter_proto.bloom_filter_size()),
+ is_bit_array_owner_(true),
+ bit_array_(new std::uint8_t[array_size_in_bytes_]) {
+ reset();
+ }
+
+ /**
+ * @brief Destructor.
+ **/
+ ~BloomFilterBlocked() {
+ if (is_bit_array_owner_) {
+ bit_array_.reset();
+ } else {
+ bit_array_.release();
+ }
+ }
+
+ static bool ProtoIsValid(const serialization::BloomFilter &bloom_filter_proto) {
+ return bloom_filter_proto.IsInitialized();
+ }
+
+ /**
+ * @brief Zeros out the contents of the bit array.
+ **/
+ inline void reset() {
+ // Initialize the bit_array with all zeros.
+ std::fill_n(bit_array_.get(), array_size_in_bytes_, 0x00);
+ inserted_element_count_ = 0;
+ }
+
+ /**
+ * @brief Get the number of hash functions used in this bloom filter.
+ *
+ * @return Returns the number of hash functions.
+ **/
+ inline std::uint8_t getNumberOfHashes() const {
+ return hash_fn_count_;
+ }
+
+ /**
+ * @brief Get the size of the bit array in bytes for this bloom filter.
+ *
+ * @return Returns the bit array size (in bytes).
+ **/
+ inline std::uint64_t getBitArraySize() const {
+ return array_size_in_bytes_;
+ }
+
+ /**
+ * @brief Get the constant pointer to the bit array for this bloom filter
+ *
+ * @return Returns constant pointer to the bit array.
+ **/
+ inline const std::uint8_t* getBitArray() const {
+ return bit_array_.get();
+ }
+
+ template <typename T>
+ void insert(const T &value) {
+ insert(reinterpret_cast<const std::uint8_t *>(&value), sizeof(T));
+ }
+
+ /**
+ * @brief Inserts a given value into the bloom filter in a thread-safe manner.
+ *
+ * @param key_begin A pointer to the value being inserted.
+ * @param length Size of the value being inserted in bytes.
+ */
+ inline void insert(const std::uint8_t *key_begin, const std::size_t length) {
+ SpinSharedMutexExclusiveLock<false> exclusive_writer_lock(bloom_filter_insert_mutex_);
+ insertUnSafe(key_begin, length);
+ }
+
+ template <typename T>
+ void insertUnSafe(const T &value) {
+ insertUnSafe(reinterpret_cast<const std::uint8_t *>(&value), sizeof(T));
+ }
+
+ /**
+ * @brief Inserts a given value into the bloom filter.
+ * @Warning This is a faster thread-unsafe version of the insert() function.
+ * The caller needs to ensure the thread safety.
+ *
+ * @param key_begin A pointer to the value being inserted.
+ * @param length Size of the value being inserted in bytes.
+ */
+ inline void insertUnSafe(const std::uint8_t *key_begin, const std::size_t length) {
+ Position first_pos = getFirstPosition(key_begin, length);
+ setBitAtPosition(first_pos);
+ Position other_pos;
+ for (std::uint8_t i = 1; i <hash_fn_count_; ++i) {
+ other_pos = getOtherPosition(key_begin, length, first_pos, i);
+ setBitAtPosition(other_pos);
+ }
+ ++inserted_element_count_;
+ }
+
+ template <typename T>
+ bool contains(const T &value) {
+ return contains(reinterpret_cast<const std::uint8_t *>(&value), sizeof(T));
+ }
+
+ /**
+ * @brief Test membership of a given value in the bloom filter.
+ * If true is returned, then a value may or may not be present in the bloom filter.
+ * If false is returned, a value is certainly not present in the bloom filter.
+ *
+ * @note The membersip test does not require any locks, because the assumption is that
+ * the bloom filter will only be used after it has been built.
+ *
+ * @param key_begin A pointer to the value being tested for membership.
+ * @param length Size of the value being inserted in bytes.
+ */
+ inline bool contains(
+ const std::uint8_t *__restrict__ key_begin,
+ const std::size_t length) const {
+ Position first_pos = getFirstPosition(key_begin, length);
+ if (!getBitAtPosition(first_pos)) {
+ return false;
+ }
+ Position other_pos;
+ for (std::uint8_t i = 1; i < hash_fn_count_; ++i) {
+ other_pos = getOtherPosition(key_begin, length, first_pos, i);
+ if (!getBitAtPosition(other_pos)) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ /**
+ * @brief Perform a bitwise-OR of the given Bloom filter with this bloom filter.
+ * Essentially, it does a union of this bloom filter with the passed bloom filter.
+ *
+ * @param bloom_filter A const pointer to the bloom filter object to do bitwise-OR with.
+ */
+ inline void bitwiseOr(const BloomFilterBlocked *bloom_filter) {
+ SpinSharedMutexExclusiveLock<false> exclusive_writer_lock(bloom_filter_insert_mutex_);
+ for (std::size_t byte_index = 0; byte_index < bloom_filter->getBitArraySize(); ++byte_index) {
+ (bit_array_.get())[byte_index] |= bloom_filter->getBitArray()[byte_index];
+ }
+ }
+
+ /**
+ * @brief Return the number of elements currently inserted into bloom filter.
+ *
+ * @return The number of elements inserted into bloom filter.
+ **/
+ inline std::uint32_t element_count() const {
+ return inserted_element_count_;
+ }
+
+ protected:
+ Position getFirstPosition(const std::uint8_t *begin, std::size_t length) const {
+ Position pos;
+ pos.hash = hash_identity(begin, length);
+ return pos;
+ }
+
+ Position getOtherPosition(
+ const std::uint8_t *begin,
+ std::size_t length,
+ const Position first_pos,
+ const std::uint8_t index) const {
+ Position pos;
+ pos.hash = hash_multiplicative(begin, length, hash_fn_[index-1]);
+ pos.cache_line_pos.line_num = first_pos.cache_line_pos.line_num;
+ return pos;
+ }
+
+ void fillPosition(
+ const std::uint8_t *begin,
+ std::size_t length,
+ const std::uint8_t index,
+ Position positions[]) const {
+ if (index == 0)
+ positions[0].hash = hash_identity(begin, length);
+ else {
+ positions[index].hash = hash_multiplicative(begin, length, hash_fn_[index-1]);
+ positions[index].cache_line_pos.line_num = positions[0].cache_line_pos.line_num;
+ }
+ }
+
+ void setBitAtPosition(const Position &pos) {
+ (bit_array_.get())[pos.byte_pos.byte_num] |= (1 << pos.byte_pos.index_in_byte);
+ }
+
+ bool getBitAtPosition(const Position &pos) const {
+ return (bit_array_.get())[pos.byte_pos.byte_num] & (1 << pos.byte_pos.index_in_byte);
+ }
+
+ inline std::uint32_t hash_identity(
+ const std::uint8_t *__restrict__ begin,
+ std::size_t length) const {
+ std::uint32_t hash;
+ if (length >= 4)
+ hash = *reinterpret_cast<const std::uint32_t*> (begin);
+ else
+ std::memcpy(&hash, begin, length);
+ return hash % (array_size_in_bytes_ * kNumBitsPerByte);
+ }
+
+ inline std::uint32_t hash_multiplicative(
+ const std::uint8_t *__restrict__ begin,
+ std::size_t length,
+ const std::uint64_t multiplier) const {
+ std::uint32_t hash = 0;
+ std::size_t bytes_hashed = 0;
+ if (length >= 4) {
+ while (bytes_hashed < length) {
+ auto val = *reinterpret_cast<const std::uint32_t *>(begin + bytes_hashed);
+ hash += (multiplier * val) >> 24;
+ bytes_hashed += 4;
+ }
+ }
+ while (bytes_hashed < length) {
+ std::uint8_t val = *(begin + bytes_hashed);
+ hash += (multiplier * val) >> 24;
+ bytes_hashed++;
+ }
+ return hash;// % (array_size_in_bytes_ * kNumBitsPerByte);
+ }
+
+ private:
+ const std::uint32_t hash_fn_count_;
+ const std::uint64_t array_size_in_bytes_;
+ std::uint32_t inserted_element_count_;
+ const bool is_bit_array_owner_;
+
+ static constexpr std::uint64_t kKnuthGoldenRatioNumber = 2654435761;
+ const std::uint64_t hash_fn_[kMaxNumHashFns] = { // hash_fn_[i] is 2**(i+1) - 1
+ 0x00000001 * kKnuthGoldenRatioNumber, // 0x00000003, 0x00000007, 0x0000000f,
+ // 0x0000001f * kKnuthGoldenRatioNumber, // 0x0000003f, 0x0000007f, 0x000000ff,
+ 0x000001ff * kKnuthGoldenRatioNumber, // 0x000003ff, 0x000007ff, 0x00000fff,
+ // 0x00001fff * kKnuthGoldenRatioNumber, // 0x00003fff, 0x00007fff, 0x0000ffff,
+ 0x0001ffff * kKnuthGoldenRatioNumber, // 0x0003ffff, 0x0007ffff, 0x000fffff,
+ // 0x001fffff * kKnuthGoldenRatioNumber, // 0x003fffff, 0x007fffff, 0x00ffffff,
+ 0x01ffffff * kKnuthGoldenRatioNumber, // 0x03ffffff, 0x07ffffff, 0x0fffffff,
+ // 0x1fffffff * kKnuthGoldenRatioNumber // 0x3fffffff, 0x7fffffff, 0xffffffff
+ };
+
+ alignas(kCacheLineBytes) std::unique_ptr<std::uint8_t> bit_array_;
+ alignas(kCacheLineBytes) mutable SpinSharedMutex<false> bloom_filter_insert_mutex_;
+
+ DISALLOW_COPY_AND_ASSIGN(BloomFilterBlocked);
+};
+
/**
* @brief A simple Bloom Filter implementation with basic primitives
* based on Partow's Bloom Filter implementation.
**/
-class BloomFilter {
+class BloomFilterOriginal {
public:
static const uint32_t kNumBitsPerByte = 8;
@@ -54,21 +402,17 @@ class BloomFilter {
* @note When no bit_array is being passed to the constructor,
* then the bit_array is owned and managed by this class.
*
- * @param random_seed A random_seed that generates unique hash functions.
* @param hash_fn_count The number of hash functions used by this bloom filter.
* @param bit_array_size_in_bytes Size of the bit array.
**/
- BloomFilter(const std::uint64_t random_seed,
- const std::size_t hash_fn_count,
+ BloomFilterOriginal(const std::size_t hash_fn_count,
const std::uint64_t bit_array_size_in_bytes)
- : random_seed_(random_seed),
- hash_fn_count_(hash_fn_count),
+ : hash_fn_count_(hash_fn_count),
array_size_in_bytes_(bit_array_size_in_bytes),
array_size_(array_size_in_bytes_ * kNumBitsPerByte),
bit_array_(new std::uint8_t[array_size_in_bytes_]),
is_bit_array_owner_(true) {
reset();
- generate_unique_hash_fn();
}
/**
@@ -76,20 +420,17 @@ class BloomFilter {
* @note When a bit_array is passed as an argument to the constructor,
* then the ownership of the bit array lies with the caller.
*
- * @param random_seed A random_seed that generates unique hash functions.
* @param hash_fn_count The number of hash functions used by this bloom filter.
* @param bit_array_size_in_bytes Size of the bit array.
* @param bit_array A pointer to the memory region that is used to store bit array.
* @param is_initialized A boolean that indicates whether to zero-out the region
* before use or not.
**/
- BloomFilter(const std::uint64_t random_seed,
- const std::size_t hash_fn_count,
+ BloomFilterOriginal(const std::size_t hash_fn_count,
const std::uint64_t bit_array_size_in_bytes,
std::uint8_t *bit_array,
const bool is_initialized)
- : random_seed_(random_seed),
- hash_fn_count_(hash_fn_count),
+ : hash_fn_count_(hash_fn_count),
array_size_in_bytes_(bit_array_size_in_bytes),
array_size_(bit_array_size_in_bytes * kNumBitsPerByte),
bit_array_(bit_array), // Owned by the calling method.
@@ -97,7 +438,6 @@ class BloomFilter {
if (!is_initialized) {
reset();
}
- generate_unique_hash_fn();
}
/**
@@ -108,21 +448,19 @@ class BloomFilter {
* @param bloom_filter_proto The protobuf representation of a
* bloom filter configuration.
**/
- explicit BloomFilter(const serialization::BloomFilter &bloom_filter_proto)
- : random_seed_(bloom_filter_proto.bloom_filter_seed()),
- hash_fn_count_(bloom_filter_proto.number_of_hashes()),
+ explicit BloomFilterOriginal(const serialization::BloomFilter &bloom_filter_proto)
+ : hash_fn_count_(bloom_filter_proto.number_of_hashes()),
array_size_in_bytes_(bloom_filter_proto.bloom_filter_size()),
array_size_(array_size_in_bytes_ * kNumBitsPerByte),
bit_array_(new std::uint8_t[array_size_in_bytes_]),
is_bit_array_owner_(true) {
reset();
- generate_unique_hash_fn();
}
/**
* @brief Destructor.
**/
- ~BloomFilter() {
+ ~BloomFilterOriginal() {
if (is_bit_array_owner_) {
bit_array_.reset();
} else {
@@ -144,15 +482,6 @@ class BloomFilter {
}
/**
- * @brief Get the random seed that was used to initialize this bloom filter.
- *
- * @return Returns the random seed.
- **/
- inline std::uint64_t getRandomSeed() const {
- return random_seed_;
- }
-
- /**
* @brief Get the number of hash functions used in this bloom filter.
*
* @return Returns the number of hash functions.
@@ -195,7 +524,7 @@ class BloomFilter {
// Determine all the bit positions that are required to be set.
for (std::size_t i = 0; i < hash_fn_count_; ++i) {
- compute_indices(hash_ap(key_begin, length, hash_fn_[i]), &bit_index, &bit);
+ compute_indices(hash_multiplicative(key_begin, length, hash_fn_[i]), &bit_index, &bit);
modified_bit_positions.push_back(std::make_pair(bit_index, bit));
}
@@ -240,7 +569,7 @@ class BloomFilter {
std::size_t bit = 0;
for (std::size_t i = 0; i < hash_fn_count_; ++i) {
- compute_indices(hash_ap(key_begin, length, hash_fn_[i]), &bit_index, &bit);
+ compute_indices(hash_multiplicative(key_begin, length, hash_fn_[i]), &bit_index, &bit);
(bit_array_.get())[bit_index / kNumBitsPerByte] |= (1 << bit);
}
@@ -262,7 +591,7 @@ class BloomFilter {
std::size_t bit_index = 0;
std::size_t bit = 0;
for (std::size_t i = 0; i < hash_fn_count_; ++i) {
- compute_indices(hash_ap(key_begin, length, hash_fn_[i]), &bit_index, &bit);
+ compute_indices(hash_multiplicative(key_begin, length, hash_fn_[i]), &bit_index, &bit);
if (((bit_array_.get())[bit_index / kNumBitsPerByte] & (1 << bit)) != (1 << bit)) {
return false;
}
@@ -276,7 +605,7 @@ class BloomFilter {
*
* @param bloom_filter A const pointer to the bloom filter object to do bitwise-OR with.
*/
- inline void bitwiseOr(const BloomFilter *bloom_filter) {
+ inline void bitwiseOr(const BloomFilterOriginal *bloom_filter) {
SpinSharedMutexExclusiveLock<false> exclusive_writer_lock(bloom_filter_insert_mutex_);
for (std::size_t byte_index = 0; byte_index < bloom_filter->getBitArraySize(); ++byte_index) {
(bit_array_.get())[byte_index] |= bloom_filter->getBitArray()[byte_index];
@@ -298,95 +627,28 @@ class BloomFilter {
*bit = *bit_index % kNumBitsPerByte;
}
- void generate_unique_hash_fn() {
- hash_fn_.reserve(hash_fn_count_);
- const std::uint32_t predef_hash_fn_count = 128;
- static const std::uint32_t predef_hash_fn[predef_hash_fn_count] = {
- 0xAAAAAAAA, 0x55555555, 0x33333333, 0xCCCCCCCC,
- 0x66666666, 0x99999999, 0xB5B5B5B5, 0x4B4B4B4B,
- 0xAA55AA55, 0x55335533, 0x33CC33CC, 0xCC66CC66,
- 0x66996699, 0x99B599B5, 0xB54BB54B, 0x4BAA4BAA,
- 0xAA33AA33, 0x55CC55CC, 0x33663366, 0xCC99CC99,
- 0x66B566B5, 0x994B994B, 0xB5AAB5AA, 0xAAAAAA33,
- 0x555555CC, 0x33333366, 0xCCCCCC99, 0x666666B5,
- 0x9999994B, 0xB5B5B5AA, 0xFFFFFFFF, 0xFFFF0000,
- 0xB823D5EB, 0xC1191CDF, 0xF623AEB3, 0xDB58499F,
- 0xC8D42E70, 0xB173F616, 0xA91A5967, 0xDA427D63,
- 0xB1E8A2EA, 0xF6C0D155, 0x4909FEA3, 0xA68CC6A7,
- 0xC395E782, 0xA26057EB, 0x0CD5DA28, 0x467C5492,
- 0xF15E6982, 0x61C6FAD3, 0x9615E352, 0x6E9E355A,
- 0x689B563E, 0x0C9831A8, 0x6753C18B, 0xA622689B,
- 0x8CA63C47, 0x42CC2884, 0x8E89919B, 0x6EDBD7D3,
- 0x15B6796C, 0x1D6FDFE4, 0x63FF9092, 0xE7401432,
- 0xEFFE9412, 0xAEAEDF79, 0x9F245A31, 0x83C136FC,
- 0xC3DA4A8C, 0xA5112C8C, 0x5271F491, 0x9A948DAB,
- 0xCEE59A8D, 0xB5F525AB, 0x59D13217, 0x24E7C331,
- 0x697C2103, 0x84B0A460, 0x86156DA9, 0xAEF2AC68,
- 0x23243DA5, 0x3F649643, 0x5FA495A8, 0x67710DF8,
- 0x9A6C499E, 0xDCFB0227, 0x46A43433, 0x1832B07A,
- 0xC46AFF3C, 0xB9C8FFF0, 0xC9500467, 0x34431BDF,
- 0xB652432B, 0xE367F12B, 0x427F4C1B, 0x224C006E,
- 0x2E7E5A89, 0x96F99AA5, 0x0BEB452A, 0x2FD87C39,
- 0x74B2E1FB, 0x222EFD24, 0xF357F60C, 0x440FCB1E,
- 0x8BBE030F, 0x6704DC29, 0x1144D12F, 0x948B1355,
- 0x6D8FD7E9, 0x1C11A014, 0xADD1592F, 0xFB3C712E,
- 0xFC77642F, 0xF9C4CE8C, 0x31312FB9, 0x08B0DD79,
- 0x318FA6E7, 0xC040D23D, 0xC0589AA7, 0x0CA5C075,
- 0xF874B172, 0x0CF914D5, 0x784D3280, 0x4E8CFEBC,
- 0xC569F575, 0xCDB2A091, 0x2CC016B4, 0x5C5F4421
- };
- if (hash_fn_count_ <= predef_hash_fn_count) {
- std::copy(predef_hash_fn, predef_hash_fn + hash_fn_count_, hash_fn_.begin());
- for (std::uint32_t i = 0; i < hash_fn_.size(); ++i) {
- hash_fn_[i] = hash_fn_[i] * hash_fn_[(i + 3) % hash_fn_count_] + static_cast<std::uint32_t>(random_seed_);
+ inline std::uint32_t hash_multiplicative(
+ const std::uint8_t *begin,
+ std::size_t remaining_length,
+ const std::uint64_t multiplier) const {
+ std::uint32_t hash = 0;
+ std::size_t bytes_hashed = 0;
+ if (remaining_length >= 4) {
+ while (bytes_hashed < remaining_length) {
+ auto val = *reinterpret_cast<const std::uint32_t *>(begin + bytes_hashed);
+ hash += (multiplier * val) >> 32;
+ bytes_hashed += 4;
}
- } else {
- LOG(FATAL) << "Requested number of hash functions is too large.";
}
- }
-
- inline std::uint32_t hash_ap(const std::uint8_t *begin, std::size_t remaining_length, std::uint32_t hash) const {
- const std::uint8_t *itr = begin;
- std::uint32_t loop = 0;
- while (remaining_length >= 8) {
- const std::uint32_t &i1 = *(reinterpret_cast<const std::uint32_t*>(itr)); itr += sizeof(std::uint32_t);
- const std::uint32_t &i2 = *(reinterpret_cast<const std::uint32_t*>(itr)); itr += sizeof(std::uint32_t);
- hash ^= (hash << 7) ^ i1 * (hash >> 3) ^ (~((hash << 11) + (i2 ^ (hash >> 5))));
- remaining_length -= 8;
- }
- if (remaining_length) {
- if (remaining_length >= 4) {
- const std::uint32_t &i = *(reinterpret_cast<const std::uint32_t*>(itr));
- if (loop & 0x01) {
- hash ^= (hash << 7) ^ i * (hash >> 3);
- } else {
- hash ^= (~((hash << 11) + (i ^ (hash >> 5))));
- }
- ++loop;
- remaining_length -= 4;
- itr += sizeof(std::uint32_t);
- }
- if (remaining_length >= 2) {
- const std::uint16_t &i = *(reinterpret_cast<const std::uint16_t*>(itr));
- if (loop & 0x01) {
- hash ^= (hash << 7) ^ i * (hash >> 3);
- } else {
- hash ^= (~((hash << 11) + (i ^ (hash >> 5))));
- }
- ++loop;
- remaining_length -= 2;
- itr += sizeof(std::uint16_t);
- }
- if (remaining_length) {
- hash += ((*itr) ^ (hash * 0xA5A5A5A5)) + loop;
- }
+ while (bytes_hashed < remaining_length) {
+ std::uint8_t val = *(begin + bytes_hashed);
+ hash += (multiplier * val) >> 32;
+ bytes_hashed++;
}
return hash;
}
private:
- const std::uint64_t random_seed_;
- std::vector<std::uint32_t> hash_fn_;
const std::uint32_t hash_fn_count_;
std::uint64_t array_size_in_bytes_;
std::uint64_t array_size_;
@@ -394,9 +656,21 @@ class BloomFilter {
std::uint32_t inserted_element_count_;
const bool is_bit_array_owner_;
+ static constexpr std::uint64_t kKnuthGoldenRatioNumber = 2654435761;
+ static constexpr std::size_t kMaxNumHashFns = 8;
+ const std::uint64_t hash_fn_[kMaxNumHashFns] = { // hash_fn_[i] is 2**(i+1) - 1
+ 0x00000001 * kKnuthGoldenRatioNumber, // 0x00000003, 0x00000007, 0x0000000f,
+ 0x0000001f * kKnuthGoldenRatioNumber, // 0x0000003f, 0x0000007f, 0x000000ff,
+ 0x000001ff * kKnuthGoldenRatioNumber, // 0x000003ff, 0x000007ff, 0x00000fff,
+ 0x00001fff * kKnuthGoldenRatioNumber, // 0x00003fff, 0x00007fff, 0x0000ffff,
+ 0x0001ffff * kKnuthGoldenRatioNumber, // 0x0003ffff, 0x0007ffff, 0x000fffff,
+ 0x001fffff * kKnuthGoldenRatioNumber, // 0x003fffff, 0x007fffff, 0x00ffffff,
+ 0x01ffffff * kKnuthGoldenRatioNumber, // 0x03ffffff, 0x07ffffff, 0x0fffffff,
+ 0x1fffffff * kKnuthGoldenRatioNumber // 0x3fffffff, 0x7fffffff, 0xffffffff
+ };
alignas(kCacheLineBytes) mutable SpinSharedMutex<false> bloom_filter_insert_mutex_;
- DISALLOW_COPY_AND_ASSIGN(BloomFilter);
+ DISALLOW_COPY_AND_ASSIGN(BloomFilterOriginal);
};
/** @} */
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/utility/BloomFilter.proto
----------------------------------------------------------------------
diff --git a/utility/BloomFilter.proto b/utility/BloomFilter.proto
index 0f67878..1a8dbf2 100644
--- a/utility/BloomFilter.proto
+++ b/utility/BloomFilter.proto
@@ -23,10 +23,8 @@ message BloomFilter {
// The default values were determined from empirical experiments.
// These values control the amount of false positivity that
// is expected from Bloom Filter.
- // - Default seed for initializing family of hashes = 0xA5A5A5A55A5A5A5A.
// - Default bloom filter size = 10 KB.
// - Default number of hash functions used in bloom filter = 5.
- optional fixed64 bloom_filter_seed = 1 [default = 0xA5A5A5A55A5A5A5A];
- optional uint32 bloom_filter_size = 2 [default = 10000];
- optional uint32 number_of_hashes = 3 [default = 5];
+ optional uint32 bloom_filter_size = 1 [default = 10000];
+ optional uint32 number_of_hashes = 2 [default = 5];
}
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/utility/BloomFilterAdapter.hpp
----------------------------------------------------------------------
diff --git a/utility/BloomFilterAdapter.hpp b/utility/BloomFilterAdapter.hpp
new file mode 100644
index 0000000..f094307
--- /dev/null
+++ b/utility/BloomFilterAdapter.hpp
@@ -0,0 +1,142 @@
+/**
+ * Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ * University of Wisconsin\u2014Madison.
+ *
+ * Licensed 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.
+ **/
+
+#ifndef QUICKSTEP_UTILITY_BLOOM_FILTER_ADAPTER_HPP
+#define QUICKSTEP_UTILITY_BLOOM_FILTER_ADAPTER_HPP
+
+#include <algorithm>
+#include <cstddef>
+#include <cstdint>
+#include <memory>
+#include <utility>
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "utility/BloomFilter.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Utility
+ * @{
+ */
+
+class BloomFilterAdapter {
+ public:
+ BloomFilterAdapter(const std::vector<const BloomFilter*> &bloom_filters,
+ const std::vector<attribute_id> &attribute_ids,
+ const std::vector<std::size_t> &attr_sizes) {
+ DCHECK_EQ(bloom_filters.size(), attribute_ids.size());
+ DCHECK_EQ(bloom_filters.size(), attr_sizes.size());
+
+ bloom_filter_entries_.reserve(bloom_filters.size());
+ for (std::size_t i = 0; i < bloom_filters.size(); ++i) {
+ bloom_filter_entries_.emplace_back(
+ new BloomFilterEntry(
+ bloom_filters[i], attribute_ids[i], attr_sizes[i]));
+ }
+ }
+
+ ~BloomFilterAdapter() {
+ for (auto &entry : bloom_filter_entries_) {
+ delete entry;
+ }
+ }
+
+ template <bool adapt_filters, typename ValueAccessorT>
+ inline std::size_t bulkProbe(const ValueAccessorT *accessor,
+ std::vector<tuple_id> &batch,
+ const std::size_t batch_size) {
+ std::size_t out_size = batch_size;
+ for (auto &entry : bloom_filter_entries_) {
+ out_size = bulkProbeBloomFilterEntry<adapt_filters>(*entry, accessor, batch, out_size);
+ }
+ adaptEntryOrder();
+ return out_size;
+ }
+
+ private:
+ struct BloomFilterEntry {
+ BloomFilterEntry(const BloomFilter *in_bloom_filter,
+ const attribute_id &in_attribute_id,
+ const std::size_t &in_attribute_size)
+ : bloom_filter(in_bloom_filter),
+ attribute_id(in_attribute_id),
+ attribute_size(in_attribute_size),
+ miss(0),
+ cnt(0) {
+ }
+
+ static bool isBetterThan(const BloomFilterEntry *a,
+ const BloomFilterEntry *b) {
+ return a->miss_rate > b->miss_rate;
+ }
+
+ const BloomFilter *bloom_filter;
+ const attribute_id attribute_id;
+ const std::size_t attribute_size;
+ std::uint32_t miss;
+ std::uint32_t cnt;
+ float miss_rate;
+ };
+
+ template <bool adapt_filters, typename ValueAccessorT>
+ inline std::size_t bulkProbeBloomFilterEntry(
+ BloomFilterEntry &entry,
+ const ValueAccessorT *accessor,
+ std::vector<tuple_id> &batch,
+ const std::size_t in_size) {
+ std::size_t out_size = 0;
+ const BloomFilter *bloom_filter = entry.bloom_filter;
+
+ for (std::size_t t = 0; t < in_size; ++t) {
+ const tuple_id tid = batch[t];
+ const auto value = static_cast<const std::uint8_t*>(
+ accessor->getUntypedValueAtAbsolutePosition(entry.attribute_id, tid));
+ if (bloom_filter->contains(value, entry.attribute_size)) {
+ batch[out_size] = tid;
+ ++out_size;
+ }
+ }
+ if (adapt_filters) {
+ entry.cnt += in_size;
+ entry.miss += (in_size - out_size);
+ }
+ return out_size;
+ }
+
+ inline void adaptEntryOrder() {
+ for (auto &entry : bloom_filter_entries_) {
+ entry->miss_rate = static_cast<float>(entry->miss) / entry->cnt;
+ }
+ std::sort(bloom_filter_entries_.begin(),
+ bloom_filter_entries_.end(),
+ BloomFilterEntry::isBetterThan);
+ }
+
+ std::vector<BloomFilterEntry *> bloom_filter_entries_;
+
+ DISALLOW_COPY_AND_ASSIGN(BloomFilterAdapter);
+};
+
+/** @} */
+
+} // namespace quickstep
+
+#endif // QUICKSTEP_UTILITY_BLOOM_FILTER_ADAPTER_HPP
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/utility/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/CMakeLists.txt b/utility/CMakeLists.txt
index ae1179d..46389f0 100644
--- a/utility/CMakeLists.txt
+++ b/utility/CMakeLists.txt
@@ -161,6 +161,7 @@ add_library(quickstep_utility_Alignment ../empty_src.cpp Alignment.hpp)
add_library(quickstep_utility_BitManipulation ../empty_src.cpp BitManipulation.hpp)
add_library(quickstep_utility_BitVector ../empty_src.cpp BitVector.hpp)
add_library(quickstep_utility_BloomFilter ../empty_src.cpp BloomFilter.hpp)
+add_library(quickstep_utility_BloomFilterAdapter ../empty_src.cpp BloomFilterAdapter.hpp)
add_library(quickstep_utility_BloomFilter_proto
${quickstep_utility_BloomFilter_proto_srcs}
${quickstep_utility_BloomFilter_proto_hdrs})
@@ -168,6 +169,8 @@ add_library(quickstep_utility_CalculateInstalledMemory CalculateInstalledMemory.
add_library(quickstep_utility_Cast ../empty_src.cpp Cast.hpp)
add_library(quickstep_utility_CheckSnprintf ../empty_src.cpp CheckSnprintf.hpp)
add_library(quickstep_utility_DAG ../empty_src.cpp DAG.hpp)
+add_library(quickstep_utility_DisjointTreeForest ../empty_src.cpp DisjointTreeForest.hpp)
+add_library(quickstep_utility_EventProfiler EventProfiler.cpp EventProfiler.hpp)
add_library(quickstep_utility_EqualsAnyConstant ../empty_src.cpp EqualsAnyConstant.hpp)
add_library(quickstep_utility_ExecutionDAGVisualizer
ExecutionDAGVisualizer.cpp
@@ -221,6 +224,10 @@ target_link_libraries(quickstep_utility_BloomFilter
quickstep_threading_SpinSharedMutex
quickstep_utility_BloomFilter_proto
quickstep_utility_Macros)
+target_link_libraries(quickstep_utility_BloomFilterAdapter
+ quickstep_catalog_CatalogTypedefs
+ quickstep_utility_BloomFilter
+ quickstep_utility_Macros)
target_link_libraries(quickstep_utility_BloomFilter_proto
${PROTOBUF_LIBRARY})
target_link_libraries(quickstep_utility_CalculateInstalledMemory
@@ -230,6 +237,9 @@ target_link_libraries(quickstep_utility_CheckSnprintf
target_link_libraries(quickstep_utility_DAG
glog
quickstep_utility_Macros)
+target_link_libraries(quickstep_utility_DisjointTreeForest)
+target_link_libraries(quickstep_utility_EventProfiler
+ quickstep_threading_Mutex)
target_link_libraries(quickstep_utility_ExecutionDAGVisualizer
quickstep_catalog_CatalogRelationSchema
quickstep_queryexecution_QueryExecutionTypedefs
@@ -312,11 +322,14 @@ target_link_libraries(quickstep_utility
quickstep_utility_BitManipulation
quickstep_utility_BitVector
quickstep_utility_BloomFilter
+ quickstep_utility_BloomFilterAdapter
quickstep_utility_BloomFilter_proto
quickstep_utility_CalculateInstalledMemory
quickstep_utility_Cast
quickstep_utility_CheckSnprintf
quickstep_utility_DAG
+ quickstep_utility_DisjointTreeForest
+ quickstep_utility_EventProfiler
quickstep_utility_EqualsAnyConstant
quickstep_utility_ExecutionDAGVisualizer
quickstep_utility_Glob
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/utility/DisjointTreeForest.hpp
----------------------------------------------------------------------
diff --git a/utility/DisjointTreeForest.hpp b/utility/DisjointTreeForest.hpp
new file mode 100644
index 0000000..f5722ba
--- /dev/null
+++ b/utility/DisjointTreeForest.hpp
@@ -0,0 +1,116 @@
+/**
+ * Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ * University of Wisconsin\u2014Madison.
+ *
+ * Licensed 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.
+ **/
+
+#ifndef QUICKSTEP_UTILITY_DISJOINT_TREE_FOREST_HPP_
+#define QUICKSTEP_UTILITY_DISJOINT_TREE_FOREST_HPP_
+
+#include <cstddef>
+#include <limits>
+#include <utility>
+#include <unordered_map>
+
+namespace quickstep {
+
+/** \addtogroup Utility
+ * @{
+ */
+
+/**
+ * @brief A.k.a. union-find set.
+ */
+template <typename ElementT,
+ class MapperT = std::unordered_map<ElementT, std::size_t>>
+class DisjointTreeForest {
+ public:
+ inline bool hasElement(const ElementT &element) const {
+ return elements_map_.find(element) != elements_map_.end();
+ }
+
+ inline void makeSet(const ElementT &element) {
+ if (!hasElement(element)) {
+ std::size_t loc = nodes_.size();
+ nodes_.emplace_back(0, loc);
+ elements_map_.emplace(element, loc);
+ }
+ }
+
+ inline std::size_t find(const ElementT &element) {
+ const std::size_t node_id = elements_map_.at(element);
+ std::size_t root_id = node_id;
+ std::size_t parent_id;
+ while ((parent_id = nodes_[root_id].parent) != root_id) {
+ root_id = parent_id;
+ }
+ compress_path(node_id, root_id);
+ return root_id;
+ }
+
+ inline void merge(const ElementT &element1, const ElementT &element2) {
+ std::size_t root_id1 = find(element1);
+ std::size_t root_id2 = find(element2);
+ if (root_id1 != root_id2) {
+ Node &n1 = nodes_[root_id1];
+ Node &n2 = nodes_[root_id2];
+ if (n1.rank > n2.rank) {
+ n2.parent = root_id1;
+ } else if (n1.rank < n2.rank) {
+ n1.parent = root_id2;
+ } else {
+ n1.parent = root_id2;
+ n2.rank += 1;
+ }
+ }
+ }
+
+ inline bool isConnected(const ElementT &element1, const ElementT &element2) {
+ return find(element1) == find(element2);
+ }
+
+ private:
+ struct Node {
+ Node(const std::size_t rank_in, const std::size_t parent_in)
+ : rank(rank_in), parent(parent_in) {
+ }
+ std::size_t rank;
+ std::size_t parent;
+ };
+
+ inline void compress_path(const std::size_t leaf_node_id, const std::size_t root_node_id) {
+ std::size_t node_id = leaf_node_id;
+ std::size_t max_rank = 0;
+ while (node_id != root_node_id) {
+ const Node &node = nodes_[node_id];
+ max_rank = std::max(max_rank, node.rank);
+
+ const std::size_t parent_id = node.parent;
+ nodes_[node_id].parent = root_node_id;
+ node_id = parent_id;
+ }
+ nodes_[root_node_id].rank = max_rank + 1;
+ }
+
+ std::vector<Node> nodes_;
+ MapperT elements_map_;
+
+ static constexpr std::size_t kInvalid = std::numeric_limits<std::size_t>::max();
+};
+
+/** @} */
+
+} // namespace quickstep
+
+#endif // QUICKSTEP_UTILITY_DISJOINT_TREE_FOREST_HPP_
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/utility/EventProfiler.cpp
----------------------------------------------------------------------
diff --git a/utility/EventProfiler.cpp b/utility/EventProfiler.cpp
new file mode 100644
index 0000000..728ebff
--- /dev/null
+++ b/utility/EventProfiler.cpp
@@ -0,0 +1,29 @@
+/**
+ * Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ * University of Wisconsin\u2014Madison.
+ *
+ * Licensed 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 "utility/EventProfiler.hpp"
+
+#include <cstddef>
+#include <string>
+#include <vector>
+
+namespace quickstep {
+
+EventProfiler<int, std::size_t> simple_profiler;
+EventProfiler<std::size_t> relop_profiler;
+
+} // namespace quickstep
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/utility/EventProfiler.hpp
----------------------------------------------------------------------
diff --git a/utility/EventProfiler.hpp b/utility/EventProfiler.hpp
new file mode 100644
index 0000000..70024e6
--- /dev/null
+++ b/utility/EventProfiler.hpp
@@ -0,0 +1,188 @@
+/**
+ * Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ * University of Wisconsin\u2014Madison.
+ *
+ * Licensed 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.
+ **/
+
+#ifndef QUICKSTEP_UTILITY_EVENT_PROFILER_HPP_
+#define QUICKSTEP_UTILITY_EVENT_PROFILER_HPP_
+
+#include <chrono>
+#include <cstddef>
+#include <cstring>
+#include <ctime>
+#include <iomanip>
+#include <map>
+#include <ostream>
+#include <thread>
+#include <type_traits>
+#include <utility>
+#include <vector>
+
+#include "threading/Mutex.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Utility
+ * @{
+ */
+
+using clock = std::chrono::steady_clock;
+
+template <typename TagT, typename ...PayloadT>
+class EventProfiler {
+
+ public:
+ EventProfiler()
+ : zero_time_(clock::now()) {
+ }
+
+ struct EventInfo {
+ clock::time_point start_time;
+ clock::time_point end_time;
+ bool is_finished;
+ std::tuple<PayloadT...> payload;
+
+ explicit EventInfo(const clock::time_point &start_time_in)
+ : start_time(start_time_in),
+ is_finished(false) {
+ }
+
+ EventInfo()
+ : start_time(clock::now()),
+ is_finished(false) {
+ }
+
+ inline void setPayload(PayloadT &&...in_payload) {
+ payload = std::make_tuple(in_payload...);
+ }
+
+ inline void endEvent() {
+ end_time = clock::now();
+ is_finished = true;
+ }
+ };
+
+ struct EventContainer {
+ EventContainer()
+ : context(0) {}
+
+ inline void startEvent(const TagT &tag) {
+ events[tag].emplace_back(clock::now());
+ }
+
+ inline void endEvent(const TagT &tag) {
+ auto &event_info = events.at(tag).back();
+ event_info.is_finished = true;
+ event_info.end_time = clock::now();
+ }
+
+ inline std::vector<EventInfo> *getEventLine(const TagT &tag) {
+ return &events[tag];
+ }
+
+ inline void setContext(int context_in) {
+ context = context_in;
+ }
+
+ inline int getContext() const {
+ return context;
+ }
+
+ std::map<TagT, std::vector<EventInfo>> events;
+ int context;
+ };
+
+ EventContainer *getContainer() {
+ MutexLock lock(mutex_);
+ return &thread_map_[std::this_thread::get_id()];
+ }
+
+ void writeToStream(std::ostream &os) const {
+ time_t rawtime;
+ time(&rawtime);
+ char event_id[32];
+ strftime(event_id, sizeof event_id, "%Y-%m-%d %H:%M:%S", localtime(&rawtime));
+
+ int thread_id = 0;
+ for (const auto &thread_ctx : thread_map_) {
+ for (const auto &event_group : thread_ctx.second.events) {
+ for (const auto &event_info : event_group.second) {
+ CHECK(event_info.is_finished) << "Unfinished profiling event";
+
+ os << std::setprecision(12)
+ << event_id << ","
+ << thread_id << "," << event_group.first << ",";
+
+ PrintTuple(os, event_info.payload, ",");
+
+ os << std::chrono::duration<double>(event_info.start_time - zero_time_).count()
+ << ","
+ << std::chrono::duration<double>(event_info.end_time - zero_time_).count()
+ << "\n";
+ }
+ }
+ ++thread_id;
+ }
+ }
+
+ void clear() {
+ zero_time_ = clock::now();
+ thread_map_.clear();
+ }
+
+ const std::map<std::thread::id, EventContainer> &containers() {
+ return thread_map_;
+ }
+
+ const clock::time_point &zero_time() {
+ return zero_time_;
+ }
+
+ private:
+ template<class Tuple, std::size_t N>
+ struct TuplePrinter {
+ static void Print(std::ostream &os, const Tuple &t, const std::string &sep) {
+ TuplePrinter<Tuple, N-1>::Print(os, t, sep);
+ os << std::get<N-1>(t) << sep;
+ }
+ };
+
+ template<class Tuple>
+ struct TuplePrinter<Tuple, 1> {
+ static void Print(std::ostream &os, const Tuple &t, const std::string &sep) {
+ os << std::get<0>(t) << sep;
+ }
+ };
+
+ template<class... Args>
+ static void PrintTuple(std::ostream &os, const std::tuple<Args...>& t, const std::string &sep) {
+ TuplePrinter<decltype(t), sizeof...(Args)>::Print(os, t, sep);
+ }
+
+ clock::time_point zero_time_;
+ std::map<std::thread::id, EventContainer> thread_map_;
+ Mutex mutex_;
+};
+
+extern EventProfiler<int, std::size_t> simple_profiler;
+extern EventProfiler<std::size_t> relop_profiler;
+
+/** @} */
+
+} // namespace quickstep
+
+#endif // QUICKSTEP_UTILITY_EVENT_PROFILER_HPP_
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/utility/PlanVisualizer.cpp
----------------------------------------------------------------------
diff --git a/utility/PlanVisualizer.cpp b/utility/PlanVisualizer.cpp
index 50cf7f0..b90a8dc 100644
--- a/utility/PlanVisualizer.cpp
+++ b/utility/PlanVisualizer.cpp
@@ -21,6 +21,7 @@
#include <cstddef>
#include <memory>
+#include <set>
#include <sstream>
#include <string>
#include <unordered_map>
@@ -30,6 +31,7 @@
#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/physical/Aggregate.hpp"
#include "query_optimizer/physical/HashJoin.hpp"
#include "query_optimizer/physical/Physical.hpp"
#include "query_optimizer/physical/PhysicalType.hpp"
@@ -103,6 +105,10 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
int node_id = ++id_counter_;
node_id_map_.emplace(input, node_id);
+ std::set<E::ExprId> referenced_ids;
+ for (const auto &attr : input->getReferencedAttributes()) {
+ referenced_ids.emplace(attr->id());
+ }
for (const auto &child : input->children()) {
visit(child);
@@ -113,10 +119,8 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
edge_info.src_node_id = child_id;
edge_info.dst_node_id = node_id;
- // Print output attributes except for TableReference -- there are just too many
- // attributes out of TableReference.
- if (child->getPhysicalType() != P::PhysicalType::kTableReference) {
- for (const auto &attr : child->getOutputAttributes()) {
+ for (const auto &attr : child->getOutputAttributes()) {
+ if (referenced_ids.find(attr->id()) != referenced_ids.end()) {
edge_info.labels.emplace_back(attr->attribute_alias());
}
}
@@ -147,6 +151,36 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
node_info.labels.emplace_back(
left_attributes[i]->attribute_alias() + " = " + right_attributes[i]->attribute_alias());
}
+ if (hash_join->left()->impliesUniqueAttributes(left_attributes)) {
+ node_info.labels.emplace_back("LEFT join attrs unique");
+ }
+ if (hash_join->right()->impliesUniqueAttributes(right_attributes)) {
+ node_info.labels.emplace_back("RIGHT join attrs unique");
+ }
+
+ const auto &bf_config = hash_join->bloom_filter_config();
+ for (const auto &bf : bf_config.build_side_bloom_filters) {
+ node_info.labels.emplace_back(
+ std::string("[BF build] ") + bf.attribute->attribute_alias());
+ }
+ for (const auto &bf : bf_config.probe_side_bloom_filters) {
+ node_info.labels.emplace_back(
+ std::string("[BF probe] ") + bf.attribute->attribute_alias());
+ }
+
+ break;
+ }
+ case P::PhysicalType::kAggregate: {
+ const P::AggregatePtr aggregate =
+ std::static_pointer_cast<const P::Aggregate>(input);
+ node_info.labels.emplace_back(input->getName());
+
+ const auto &bf_config = aggregate->bloom_filter_config();
+ for (const auto &bf : bf_config.probe_side_bloom_filters) {
+ node_info.labels.emplace_back(
+ std::string("[BF probe] ") + bf.attribute->attribute_alias());
+ }
+
break;
}
default: {
[15/16] incubator-quickstep git commit: Attach bloom filters to select
Posted by ji...@apache.org.
Attach bloom filters to select
Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/ef4cd1ed
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/ef4cd1ed
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/ef4cd1ed
Branch: refs/heads/LIP-for-tpch
Commit: ef4cd1ed9b0a48fdde7a753dbb309b213a7b2701
Parents: c588775
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Thu Aug 4 18:05:13 2016 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Thu Aug 11 15:23:00 2016 -0500
----------------------------------------------------------------------
expressions/Expressions.proto | 9 +-
expressions/predicate/BloomFiltersPredicate.cpp | 68 +++++++++++
expressions/predicate/BloomFiltersPredicate.hpp | 94 +++++++++++++++
query_optimizer/ExecutionGenerator.cpp | 15 +++
query_optimizer/ExecutionHeuristics.cpp | 24 ++++
query_optimizer/ExecutionHeuristics.hpp | 28 +++++
query_optimizer/physical/Selection.hpp | 16 ++-
query_optimizer/rules/AttachBloomFilters.cpp | 117 +++++++++++++++---
query_optimizer/rules/AttachBloomFilters.hpp | 6 +-
.../StarSchemaHashJoinOrderOptimization.cpp | 34 +++---
.../StarSchemaHashJoinOrderOptimization.hpp | 14 ++-
relational_operators/SelectOperator.cpp | 57 ++++++++-
relational_operators/SelectOperator.hpp | 25 ++++
storage/AggregationOperationState.cpp | 2 +
storage/StorageBlock.cpp | 119 ++++++++++++++++++-
storage/StorageBlock.hpp | 8 +-
utility/PlanVisualizer.cpp | 22 +++-
utility/PlanVisualizer.hpp | 1 +
18 files changed, 604 insertions(+), 55 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/expressions/Expressions.proto
----------------------------------------------------------------------
diff --git a/expressions/Expressions.proto b/expressions/Expressions.proto
index 8d923c5..34c49e0 100644
--- a/expressions/Expressions.proto
+++ b/expressions/Expressions.proto
@@ -30,10 +30,11 @@ message Predicate {
enum PredicateType {
TRUE = 0;
FALSE = 1;
- COMPARISON = 2;
- NEGATION = 3;
- CONJUNCTION = 4;
- DISJUNCTION = 5;
+ BLOOM_FILTERS = 2;
+ COMPARISON = 3;
+ NEGATION = 4;
+ CONJUNCTION = 5;
+ DISJUNCTION = 6;
}
required PredicateType predicate_type = 1;
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/expressions/predicate/BloomFiltersPredicate.cpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/BloomFiltersPredicate.cpp b/expressions/predicate/BloomFiltersPredicate.cpp
new file mode 100644
index 0000000..17ff796
--- /dev/null
+++ b/expressions/predicate/BloomFiltersPredicate.cpp
@@ -0,0 +1,68 @@
+/**
+ * Copyright 2011-2015 Quickstep Technologies LLC.
+ * Copyright 2015 Pivotal Software, Inc.
+ *
+ * Licensed 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/predicate/BloomFiltersPredicate.hpp"
+
+#include "expressions/Expressions.pb.h"
+#include "expressions/predicate/Predicate.hpp"
+#include "storage/TupleIdSequence.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+class ValueAccessor;
+
+serialization::Predicate BloomFiltersPredicate::getProto() const {
+ serialization::Predicate proto;
+ proto.set_predicate_type(serialization::Predicate::BLOOM_FILTERS);
+ return proto;
+}
+
+Predicate* BloomFiltersPredicate::clone() const {
+ LOG(FATAL) << "Not implemented\n";
+ return nullptr;
+}
+
+bool BloomFiltersPredicate::matchesForSingleTuple(const ValueAccessor &accessor,
+ const tuple_id tuple) const {
+ LOG(FATAL) << "Not implemented\n";
+ return false;
+}
+
+bool BloomFiltersPredicate::matchesForJoinedTuples(
+ const ValueAccessor &left_accessor,
+ const relation_id left_relation_id,
+ const tuple_id left_tuple_id,
+ const ValueAccessor &right_accessor,
+ const relation_id right_relation_id,
+ const tuple_id right_tuple_id) const {
+ LOG(FATAL) << "Not implemented\n";
+ return false;
+}
+
+TupleIdSequence* BloomFiltersPredicate::getAllMatches(
+ ValueAccessor *accessor,
+ const SubBlocksReference *sub_blocks_ref,
+ const TupleIdSequence *filter,
+ const TupleIdSequence *existence_map) const {
+ LOG(FATAL) << "Not implemented\n";
+ return nullptr;
+}
+
+} // namespace quickstep
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/expressions/predicate/BloomFiltersPredicate.hpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/BloomFiltersPredicate.hpp b/expressions/predicate/BloomFiltersPredicate.hpp
new file mode 100644
index 0000000..3c3acf4
--- /dev/null
+++ b/expressions/predicate/BloomFiltersPredicate.hpp
@@ -0,0 +1,94 @@
+/**
+ * Copyright 2011-2015 Quickstep Technologies LLC.
+ * Copyright 2015 Pivotal Software, Inc.
+ *
+ * Licensed 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.
+ **/
+
+#ifndef QUICKSTEP_EXPRESSIONS_PREDICATE_BLOOM_FILTERS_PREDICATE_HPP_
+#define QUICKSTEP_EXPRESSIONS_PREDICATE_BLOOM_FILTERS_PREDICATE_HPP_
+
+#include <memory>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "expressions/Expressions.pb.h"
+#include "expressions/predicate/Predicate.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "utility/BloomFilter.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+class TupleIdSequence;
+class ValueAccessor;
+
+struct SubBlocksReference;
+
+/** \addtogroup Expressions
+ * @{
+ */
+
+class BloomFiltersPredicate : public Predicate {
+ public:
+ BloomFiltersPredicate() {
+ }
+
+ ~BloomFiltersPredicate() override {
+ }
+
+ serialization::Predicate getProto() const override;
+
+ Predicate* clone() const override;
+
+ PredicateType getPredicateType() const override {
+ return kBloomFilters;
+ }
+
+ bool matchesForSingleTuple(const ValueAccessor &accessor,
+ const tuple_id tuple) const override;
+
+ bool matchesForJoinedTuples(
+ const ValueAccessor &left_accessor,
+ const relation_id left_relation_id,
+ const tuple_id left_tuple_id,
+ const ValueAccessor &right_accessor,
+ const relation_id right_relation_id,
+ const tuple_id right_tuple_id) const override;
+
+ TupleIdSequence* getAllMatches(ValueAccessor *accessor,
+ const SubBlocksReference *sub_blocks_ref,
+ const TupleIdSequence *filter,
+ const TupleIdSequence *existence_map) const override;
+
+ void addBloomFilter(const BloomFilter *bloom_filter) {
+ bloom_filters_.emplace_back(bloom_filter);
+ }
+
+ void addAttributeId(const attribute_id probe_attribute_id) {
+ bloom_filter_attribute_ids_.push_back(probe_attribute_id);
+ }
+
+ private:
+ std::vector<const BloomFilter *> bloom_filters_;
+ std::vector<attribute_id> bloom_filter_attribute_ids_;
+
+ friend class PredicateTest;
+
+ DISALLOW_COPY_AND_ASSIGN(BloomFiltersPredicate);
+};
+
+/** @} */
+
+} // namespace quickstep
+
+#endif // QUICKSTEP_EXPRESSIONS_PREDICATE_BLOOM_FILTERS_PREDICATE_HPP_
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index a194d46..21c6e30 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -507,6 +507,16 @@ void ExecutionGenerator::convertSelection(
}
}
+ const P::BloomFilterConfig &bloom_filter_config =
+ physical_selection->bloom_filter_config();
+ std::vector<attribute_id> bloom_filter_attribute_ids;
+
+ for (const auto &bf : bloom_filter_config.probe_side_bloom_filters) {
+ const CatalogAttribute *bf_catalog_attribute
+ = attribute_substitution_map_[bf.attribute->id()];
+ bloom_filter_attribute_ids.emplace_back(bf_catalog_attribute->getID());
+ }
+
// Convert the project expressions proto.
const QueryContext::scalar_group_id project_expressions_group_index =
query_context_proto_->scalar_groups_size();
@@ -572,6 +582,11 @@ void ExecutionGenerator::convertSelection(
std::forward_as_tuple(select_index,
output_relation));
temporary_relation_info_vec_.emplace_back(select_index, output_relation);
+
+ execution_heuristics_->addSelectInfo(select_index,
+ bloom_filter_config,
+ std::move(bloom_filter_attribute_ids),
+ op);
}
void ExecutionGenerator::convertSharedSubplanReference(const physical::SharedSubplanReferencePtr &physical_plan) {
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/query_optimizer/ExecutionHeuristics.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionHeuristics.cpp b/query_optimizer/ExecutionHeuristics.cpp
index 0bef716..d5d7640 100644
--- a/query_optimizer/ExecutionHeuristics.cpp
+++ b/query_optimizer/ExecutionHeuristics.cpp
@@ -127,6 +127,30 @@ void ExecutionHeuristics::optimizeExecutionPlan(QueryPlan *query_plan,
true /* is_pipeline_breaker */);
}
}
+
+ for (const auto &info : selects_) {
+ const auto &bloom_filter_config = info.bloom_filter_config_;
+
+ for (std::size_t i = 0; i < info.bloom_filter_ids_.size(); ++i) {
+ const auto &bf =
+ bloom_filter_config.probe_side_bloom_filters[i];
+ std::cerr << "Select probe " << bf.attribute->toString()
+ << " @" << bf.builder << "\n";
+
+ const auto &build_side_info =
+ bloom_filter_map.at(
+ std::make_pair(bf.source_attribute->id(),
+ bf.builder));
+ info.select_operator_->addBloomFilter(
+ build_side_info.first, info.bloom_filter_ids_[i]);
+// std::cerr << "Select probe attr_id = "
+// << info.bloom_filter_ids_[i] << "\n";
+
+ query_plan->addDirectDependency(info.select_operator_index_,
+ build_side_info.second,
+ true /* is_pipeline_breaker */);
+ }
+ }
}
void ExecutionHeuristics::setBloomFilterProperties(serialization::BloomFilter *bloom_filter_proto,
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/query_optimizer/ExecutionHeuristics.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionHeuristics.hpp b/query_optimizer/ExecutionHeuristics.hpp
index 9e5efc5..9b021a3 100644
--- a/query_optimizer/ExecutionHeuristics.hpp
+++ b/query_optimizer/ExecutionHeuristics.hpp
@@ -28,6 +28,7 @@
#include "query_execution/QueryContext.pb.h"
#include "query_optimizer/QueryPlan.hpp"
#include "query_optimizer/physical/HashJoin.hpp"
+#include "relational_operators/SelectOperator.hpp"
#include "utility/Macros.hpp"
#include "glog/logging.h"
@@ -112,6 +113,22 @@ class ExecutionHeuristics {
const QueryContext::aggregation_state_id aggregate_state_id_;
};
+ struct SelectInfo {
+ SelectInfo(const QueryPlan::DAGNodeIndex select_operator_index,
+ const physical::BloomFilterConfig &bloom_filter_config,
+ std::vector<attribute_id> &&bloom_filter_ids,
+ SelectOperator *select_operator)
+ : select_operator_index_(select_operator_index),
+ bloom_filter_config_(bloom_filter_config),
+ bloom_filter_ids_(bloom_filter_ids),
+ select_operator_(select_operator) {
+ }
+
+ const QueryPlan::DAGNodeIndex select_operator_index_;
+ const physical::BloomFilterConfig &bloom_filter_config_;
+ const std::vector<attribute_id> bloom_filter_ids_;
+ SelectOperator *select_operator_;
+ };
/**
* @brief Constructor.
@@ -161,6 +178,16 @@ class ExecutionHeuristics {
aggregate_state_id);
}
+ inline void addSelectInfo(const QueryPlan::DAGNodeIndex select_operator_index,
+ const physical::BloomFilterConfig &bloom_filter_config,
+ std::vector<attribute_id> &&bloom_filter_ids,
+ SelectOperator *select_operator) {
+ selects_.emplace_back(select_operator_index,
+ bloom_filter_config,
+ std::move(bloom_filter_ids),
+ select_operator);
+ }
+
/**
* @brief Optimize the execution plan based on heuristics generated
* during physical plan to execution plan conversion.
@@ -184,6 +211,7 @@ class ExecutionHeuristics {
private:
std::vector<HashJoinInfo> hash_joins_;
std::vector<AggregateInfo> aggregates_;
+ std::vector<SelectInfo> selects_;
DISALLOW_COPY_AND_ASSIGN(ExecutionHeuristics);
};
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/query_optimizer/physical/Selection.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Selection.hpp b/query_optimizer/physical/Selection.hpp
index f42fc71..bb50314 100644
--- a/query_optimizer/physical/Selection.hpp
+++ b/query_optimizer/physical/Selection.hpp
@@ -89,6 +89,10 @@ class Selection : public Physical {
bool impliesUniqueAttributes(
const std::vector<expressions::AttributeReferencePtr> &attributes) const override;
+ const BloomFilterConfig &bloom_filter_config() const {
+ return bloom_filter_config_;
+ }
+
/**
* @brief Creates a Selection.
*
@@ -100,9 +104,10 @@ class Selection : public Physical {
static SelectionPtr Create(
const PhysicalPtr &input,
const std::vector<expressions::NamedExpressionPtr> &project_expressions,
- const expressions::PredicatePtr &filter_predicate) {
+ const expressions::PredicatePtr &filter_predicate,
+ const BloomFilterConfig bloom_filter_config = BloomFilterConfig()) {
return SelectionPtr(
- new Selection(input, project_expressions, filter_predicate));
+ new Selection(input, project_expressions, filter_predicate, bloom_filter_config));
}
/**
@@ -143,15 +148,18 @@ class Selection : public Physical {
Selection(
const PhysicalPtr &input,
const std::vector<expressions::NamedExpressionPtr> &project_expressions,
- const expressions::PredicatePtr &filter_predicate)
+ const expressions::PredicatePtr &filter_predicate,
+ const BloomFilterConfig &bloom_filter_config)
: project_expressions_(project_expressions),
- filter_predicate_(filter_predicate) {
+ filter_predicate_(filter_predicate),
+ bloom_filter_config_(bloom_filter_config) {
addChild(input);
}
std::vector<expressions::NamedExpressionPtr> project_expressions_;
// Can be NULL. If NULL, the filter predicate is treated as the literal true.
expressions::PredicatePtr filter_predicate_;
+ BloomFilterConfig bloom_filter_config_;
DISALLOW_COPY_AND_ASSIGN(Selection);
};
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/query_optimizer/rules/AttachBloomFilters.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/AttachBloomFilters.cpp b/query_optimizer/rules/AttachBloomFilters.cpp
index 03a42a0..10ed512 100644
--- a/query_optimizer/rules/AttachBloomFilters.cpp
+++ b/query_optimizer/rules/AttachBloomFilters.cpp
@@ -17,6 +17,8 @@
#include "query_optimizer/rules/AttachBloomFilters.hpp"
+#include <algorithm>
+#include <iterator>
#include <memory>
#include <set>
#include <unordered_set>
@@ -34,8 +36,6 @@
#include "query_optimizer/physical/PhysicalType.hpp"
#include "query_optimizer/physical/TopLevelPlan.hpp"
-#include "glog/logging.h"
-
namespace quickstep {
namespace optimizer {
@@ -70,7 +70,10 @@ P::PhysicalPtr AttachBloomFilters::apply(const P::PhysicalPtr &input) {
// std::cerr << "********\n";
// }
- return visitAndAttach(input);
+ std::set<E::ExprId> used_bloom_filters;
+ decideAttach(input, &used_bloom_filters);
+
+ return performAttach(input);
}
void AttachBloomFilters::visitProducer(const P::PhysicalPtr &node, const int depth) {
@@ -140,6 +143,52 @@ void AttachBloomFilters::visitConsumer(const P::PhysicalPtr &node) {
// Bloom filters from parent
const auto &parent_bloom_filters = consumers_[node];
if (!parent_bloom_filters.empty()) {
+// if (node->getPhysicalType() == P::PhysicalType::kHashJoin) {
+// const P::HashJoinPtr hash_join =
+// std::static_pointer_cast<const P::HashJoin>(node);
+// const std::vector<const std::vector<E::AttributeReferencePtr>*> join_attributes =
+// { &hash_join->left_join_attributes(), &hash_join->right_join_attributes() };
+//
+// for (std::size_t i = 0; i < 2; ++i) {
+// const auto child = hash_join->children()[i];
+// std::unordered_set<E::ExprId> child_output_attribute_ids;
+// for (const auto &attr : child->getOutputAttributes()) {
+// child_output_attribute_ids.emplace(attr->id());
+// }
+//
+// std::unordered_map<E::ExprId, E::AttributeReferencePtr> join_attribute_map;
+// for (std::size_t k = 0; k < hash_join->left_join_attributes().size(); ++k) {
+// join_attribute_map.emplace(
+// join_attributes[1-i]->at(k)->id(),
+// join_attributes[i]->at(k));
+// }
+//
+// std::vector<BloomFilterInfo> bloom_filters;
+// for (const auto &info : parent_bloom_filters) {
+// if (child_output_attribute_ids.find(info.attribute->id())
+// != child_output_attribute_ids.end()) {
+// bloom_filters.emplace_back(info.source,
+// info.attribute,
+// info.depth,
+// info.selectivity,
+// false,
+// info.source_attribute);
+// } else {
+// auto attr_it = join_attribute_map.find(info.attribute->id());
+// if (attr_it != join_attribute_map.end()) {
+// bloom_filters.emplace_back(info.source,
+// attr_it->second,
+// info.depth,
+// info.selectivity,
+// false,
+// info.source_attribute);
+//
+// }
+// }
+// }
+// consumers_.emplace(child, std::move(bloom_filters));
+// }
+// }
for (const auto &child : node->children()) {
std::unordered_set<E::ExprId> child_output_attribute_ids;
for (const auto &attr : child->getOutputAttributes()) {
@@ -195,6 +244,21 @@ void AttachBloomFilters::visitConsumer(const P::PhysicalPtr &node) {
}
}
+ for (const auto &child : node->children()) {
+ visitConsumer(child);
+ }
+}
+
+void AttachBloomFilters::decideAttach(
+ const P::PhysicalPtr &node,
+ std::set<E::ExprId> *used_bloom_filters) {
+ for (const auto &child : node->children()) {
+ std::set<E::ExprId> child_bloom_filters;
+ decideAttach(child, &child_bloom_filters);
+ used_bloom_filters->insert(child_bloom_filters.begin(),
+ child_bloom_filters.end());
+ }
+
P::PhysicalPtr consumer_child = nullptr;
if (node->getPhysicalType() == P::PhysicalType::kHashJoin) {
consumer_child = std::static_pointer_cast<const P::HashJoin>(node)->left();
@@ -202,6 +266,9 @@ void AttachBloomFilters::visitConsumer(const P::PhysicalPtr &node) {
if (node->getPhysicalType() == P::PhysicalType::kAggregate) {
consumer_child = std::static_pointer_cast<const P::Aggregate>(node)->input();
}
+ if (node->getPhysicalType() == P::PhysicalType::kSelection) {
+ consumer_child = std::static_pointer_cast<const P::Selection>(node)->input();
+ }
if (consumer_child != nullptr) {
// Decide attaches
@@ -222,27 +289,27 @@ void AttachBloomFilters::visitConsumer(const P::PhysicalPtr &node) {
auto &probe_attaches = getBloomFilterConfig(node);
for (const auto &pair : filters) {
- auto &build_attaches = getBloomFilterConfig(pair.second->source);
- build_attaches.addBuildSideBloomFilter(
- pair.second->source_attribute);
- probe_attaches.addProbeSideBloomFilter(
- pair.first,
- pair.second->source_attribute,
- pair.second->source);
+ const E::ExprId source_attr_id = pair.second->source_attribute->id();
+ if (used_bloom_filters->find(source_attr_id) == used_bloom_filters->end()) {
+ auto &build_attaches = getBloomFilterConfig(pair.second->source);
+ build_attaches.addBuildSideBloomFilter(
+ pair.second->source_attribute);
+ probe_attaches.addProbeSideBloomFilter(
+ pair.first,
+ pair.second->source_attribute,
+ pair.second->source);
+ used_bloom_filters->emplace(source_attr_id);
+ }
}
}
}
-
- for (const auto &child : node->children()) {
- visitConsumer(child);
- }
}
-P::PhysicalPtr AttachBloomFilters::visitAndAttach(const physical::PhysicalPtr &node) {
+P::PhysicalPtr AttachBloomFilters::performAttach(const physical::PhysicalPtr &node) {
std::vector<P::PhysicalPtr> new_children;
bool has_changed = false;
for (const auto &child : node->children()) {
- P::PhysicalPtr new_child = visitAndAttach(child);
+ P::PhysicalPtr new_child = performAttach(child);
if (new_child != child) {
has_changed = true;
}
@@ -290,6 +357,24 @@ P::PhysicalPtr AttachBloomFilters::visitAndAttach(const physical::PhysicalPtr &n
}
}
+ if (node->getPhysicalType() == P::PhysicalType::kSelection) {
+ const auto attach_it = attaches_.find(node);
+ if (attach_it != attaches_.end()) {
+// for (const auto& item : attach_it->second.probe_side_bloom_filters) {
+// std::cout << "Attach probe from " << item.builder
+// << " to " << node << "\n";
+// }
+
+ const P::SelectionPtr selection =
+ std::static_pointer_cast<const P::Selection>(node);
+ return P::Selection::Create(
+ selection->input(),
+ selection->project_expressions(),
+ selection->filter_predicate(),
+ attach_it->second);
+ }
+ }
+
if (has_changed) {
return node->copyWithNewChildren(new_children);
}
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/query_optimizer/rules/AttachBloomFilters.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/AttachBloomFilters.hpp b/query_optimizer/rules/AttachBloomFilters.hpp
index e4437f7..5131bdd 100644
--- a/query_optimizer/rules/AttachBloomFilters.hpp
+++ b/query_optimizer/rules/AttachBloomFilters.hpp
@@ -21,6 +21,7 @@
#include <algorithm>
#include <cstddef>
#include <memory>
+#include <set>
#include <string>
#include <unordered_map>
#include <unordered_set>
@@ -97,7 +98,10 @@ class AttachBloomFilters : public Rule<physical::Physical> {
void visitConsumer(const physical::PhysicalPtr &node);
- physical::PhysicalPtr visitAndAttach(const physical::PhysicalPtr &node);
+ void decideAttach(const physical::PhysicalPtr &node,
+ std::set<expressions::ExprId> *used_bloom_filters);
+
+ physical::PhysicalPtr performAttach(const physical::PhysicalPtr &node);
physical::BloomFilterConfig &getBloomFilterConfig(const physical::PhysicalPtr &node);
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
index 15a7154..9e8d794 100644
--- a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
+++ b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
@@ -221,11 +221,14 @@ physical::PhysicalPtr StarSchemaHashJoinOrderOptimization::generatePlan(
attribute_id_to_reference_map.at(build_it->second));
}
}
- if (!build_attrs.empty()
- && build_table_info->table->impliesUniqueAttributes(build_attrs)) {
- std::unique_ptr<JoinPair> new_join(
- new JoinPair(probe_table_info, build_table_info));
- if (best_join == nullptr || new_join->isBetterThan(*best_join)) {
+
+ const bool build_side_unique =
+ !build_attrs.empty() && build_table_info->table->impliesUniqueAttributes(build_attrs);
+ std::unique_ptr<JoinPair> new_join(
+ new JoinPair(probe_table_info,
+ build_table_info,
+ build_side_unique));
+ if (best_join == nullptr || new_join->isBetterThan(*best_join)) {
// if (best_join != nullptr) {
// std::cerr << "(" << best_join->probe->estimated_selectivity
// << ", " << best_join->probe->estimated_cardinality << ")"
@@ -241,25 +244,24 @@ physical::PhysicalPtr StarSchemaHashJoinOrderOptimization::generatePlan(
// << "(" << new_join->build->estimated_selectivity
// << ", " << new_join->build->estimated_cardinality << ")"
// << "\n****\n";
- best_join.reset(new_join.release());
- }
+ best_join.reset(new_join.release());
}
}
}
}
- TableInfo *selected_probe_table_info = nullptr;
- TableInfo *selected_build_table_info = nullptr;
+ CHECK(best_join != nullptr);
- if (best_join != nullptr) {
- selected_probe_table_info = best_join->probe;
- selected_build_table_info = best_join->build;
+ TableInfo *selected_probe_table_info = best_join->probe;
+ TableInfo *selected_build_table_info = best_join->build;
+ std::cerr << "card: " << selected_probe_table_info->estimated_cardinality << "\n";
+ std::cerr << "card: " << selected_build_table_info->estimated_cardinality << "\n";
+ std::cerr << "--------\n";
+ if (!best_join->build_side_unique &&
+ selected_probe_table_info->estimated_cardinality < selected_build_table_info->estimated_cardinality) {
+ std::swap(selected_probe_table_info, selected_build_table_info);
}
- // TODO(jianqiao): Handle the case when there is no primary key-foreign key information available.
- CHECK(selected_probe_table_info != nullptr);
- CHECK(selected_build_table_info != nullptr);
-
// std::cerr << selected_probe_table_info->estimated_selectivity
// << " -- "
// << selected_build_table_info->estimated_selectivity
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
index eb21d03..0466765 100644
--- a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
+++ b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
@@ -96,12 +96,21 @@ class StarSchemaHashJoinOrderOptimization : public Rule<physical::Physical> {
};
struct JoinPair {
- JoinPair(TableInfo *probe_in, TableInfo *build_in)
- : probe(probe_in), build(build_in) {
+ JoinPair(TableInfo *probe_in,
+ TableInfo *build_in,
+ const bool build_side_unique_in)
+ : probe(probe_in),
+ build(build_in),
+ build_side_unique(build_side_unique_in) {
}
inline bool isBetterThan(const JoinPair &rhs) const {
const auto &lhs = *this;
+
+ if (lhs.build_side_unique != rhs.build_side_unique) {
+ return lhs.build_side_unique;
+ }
+
const bool lhs_has_large_output =
lhs.build->estimated_num_output_attributes
+ lhs.probe->estimated_num_output_attributes > 5;
@@ -151,6 +160,7 @@ class StarSchemaHashJoinOrderOptimization : public Rule<physical::Physical> {
TableInfo *probe;
TableInfo *build;
+ const bool build_side_unique;
};
physical::PhysicalPtr applyInternal(const physical::PhysicalPtr &input,
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/relational_operators/SelectOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.cpp b/relational_operators/SelectOperator.cpp
index d56326e..5a67e2d 100644
--- a/relational_operators/SelectOperator.cpp
+++ b/relational_operators/SelectOperator.cpp
@@ -42,6 +42,8 @@ class Predicate;
void SelectOperator::addWorkOrders(WorkOrdersContainer *container,
StorageManager *storage_manager,
const Predicate *predicate,
+ const std::vector<const BloomFilter *> &bloom_filters,
+ const std::vector<attribute_id> &bloom_filter_attribute_ids,
const std::vector<std::unique_ptr<const Scalar>> *selection,
InsertDestination *output_destination) {
if (input_relation_is_stored_) {
@@ -50,6 +52,8 @@ void SelectOperator::addWorkOrders(WorkOrdersContainer *container,
input_relation_,
input_block_id,
predicate,
+ bloom_filters,
+ bloom_filter_attribute_ids,
simple_projection_,
simple_selection_,
selection,
@@ -65,6 +69,8 @@ void SelectOperator::addWorkOrders(WorkOrdersContainer *container,
input_relation_,
input_relation_block_ids_[num_workorders_generated_],
predicate,
+ bloom_filters,
+ bloom_filter_attribute_ids,
simple_projection_,
simple_selection_,
selection,
@@ -80,6 +86,8 @@ void SelectOperator::addWorkOrders(WorkOrdersContainer *container,
void SelectOperator::addPartitionAwareWorkOrders(WorkOrdersContainer *container,
StorageManager *storage_manager,
const Predicate *predicate,
+ const std::vector<const BloomFilter *> &bloom_filters,
+ const std::vector<attribute_id> &bloom_filter_attribute_ids,
const std::vector<std::unique_ptr<const Scalar>> *selection,
InsertDestination *output_destination) {
DCHECK(placement_scheme_ != nullptr);
@@ -94,6 +102,8 @@ void SelectOperator::addPartitionAwareWorkOrders(WorkOrdersContainer *container,
input_relation_,
input_block_id,
predicate,
+ bloom_filters,
+ bloom_filter_attribute_ids,
simple_projection_,
simple_selection_,
selection,
@@ -115,6 +125,8 @@ void SelectOperator::addPartitionAwareWorkOrders(WorkOrdersContainer *container,
input_relation_,
block_in_partition,
predicate,
+ bloom_filters,
+ bloom_filter_attribute_ids,
simple_projection_,
simple_selection_,
selection,
@@ -137,6 +149,15 @@ bool SelectOperator::getAllWorkOrders(
tmb::MessageBus *bus) {
DCHECK(query_context != nullptr);
+ if (bloom_filters_ == nullptr) {
+ bloom_filters_.reset(new std::vector<const BloomFilter*>());
+ for (const auto bloom_filter_id : bloom_filter_ids_) {
+ // Add the pointer to the probe bloom filter within the list of probe bloom filters to use.
+ bloom_filters_->emplace_back(
+ query_context->getBloomFilter(bloom_filter_id));
+ }
+ }
+
const Predicate *predicate =
query_context->getPredicate(predicate_index_);
const std::vector<std::unique_ptr<const Scalar>> *selection =
@@ -151,11 +172,23 @@ bool SelectOperator::getAllWorkOrders(
if (input_relation_.hasPartitionScheme()) {
#ifdef QUICKSTEP_HAVE_LIBNUMA
if (input_relation_.hasNUMAPlacementScheme()) {
- addPartitionAwareWorkOrders(container, storage_manager, predicate, selection, output_destination);
+ addPartitionAwareWorkOrders(container,
+ storage_manager,
+ predicate,
+ *bloom_filters_,
+ bloom_filter_attribute_ids_,
+ selection,
+ output_destination);
}
#endif
} else {
- addWorkOrders(container, storage_manager, predicate, selection, output_destination);
+ addWorkOrders(container,
+ storage_manager,
+ predicate,
+ *bloom_filters_,
+ bloom_filter_attribute_ids_,
+ selection,
+ output_destination);
}
started_ = true;
}
@@ -164,11 +197,23 @@ bool SelectOperator::getAllWorkOrders(
if (input_relation_.hasPartitionScheme()) {
#ifdef QUICKSTEP_HAVE_LIBNUMA
if (input_relation_.hasNUMAPlacementScheme()) {
- addPartitionAwareWorkOrders(container, storage_manager, predicate, selection, output_destination);
+ addPartitionAwareWorkOrders(container,
+ storage_manager,
+ predicate,
+ *bloom_filters_,
+ bloom_filter_attribute_ids_,
+ selection,
+ output_destination);
}
#endif
} else {
- addWorkOrders(container, storage_manager, predicate, selection, output_destination);
+ addWorkOrders(container,
+ storage_manager,
+ predicate,
+ *bloom_filters_,
+ bloom_filter_attribute_ids_,
+ selection,
+ output_destination);
}
return done_feeding_input_relation_;
}
@@ -222,10 +267,14 @@ void SelectWorkOrder::execute() {
if (simple_projection_) {
block->selectSimple(simple_selection_,
predicate_,
+ bloom_filters_,
+ bloom_filter_attribute_ids_,
output_destination_);
} else {
block->select(*DCHECK_NOTNULL(selection_),
predicate_,
+ bloom_filters_,
+ bloom_filter_attribute_ids_,
output_destination_);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/relational_operators/SelectOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.hpp b/relational_operators/SelectOperator.hpp
index 0f5c712..bc587a1 100644
--- a/relational_operators/SelectOperator.hpp
+++ b/relational_operators/SelectOperator.hpp
@@ -37,6 +37,7 @@
#include "relational_operators/RelationalOperator.hpp"
#include "relational_operators/WorkOrder.hpp"
#include "storage/StorageBlockInfo.hpp"
+#include "utility/BloomFilterAdapter.hpp"
#include "utility/Macros.hpp"
#include "glog/logging.h"
@@ -47,6 +48,7 @@ namespace tmb { class MessageBus; }
namespace quickstep {
+class BloomFitler;
class CatalogRelationSchema;
class InsertDestination;
class Predicate;
@@ -246,15 +248,25 @@ class SelectOperator : public RelationalOperator {
return output_relation_.getID();
}
+ void addBloomFilter(const QueryContext::bloom_filter_id bloom_filter_id,
+ const attribute_id bloom_filter_attribute_id) {
+ bloom_filter_ids_.emplace_back(bloom_filter_id);
+ bloom_filter_attribute_ids_.emplace_back(bloom_filter_attribute_id);
+ }
+
void addWorkOrders(WorkOrdersContainer *container,
StorageManager *storage_manager,
const Predicate *predicate,
+ const std::vector<const BloomFilter *> &bloom_filters,
+ const std::vector<attribute_id> &bloom_filter_attribute_ids,
const std::vector<std::unique_ptr<const Scalar>> *selection,
InsertDestination *output_destination);
void addPartitionAwareWorkOrders(WorkOrdersContainer *container,
StorageManager *storage_manager,
const Predicate *predicate,
+ const std::vector<const BloomFilter *> &bloom_filters,
+ const std::vector<attribute_id> &bloom_filter_attribute_ids,
const std::vector<std::unique_ptr<const Scalar>> *selection,
InsertDestination *output_destination);
@@ -270,6 +282,9 @@ class SelectOperator : public RelationalOperator {
const CatalogRelation &output_relation_;
const QueryContext::insert_destination_id output_destination_index_;
const QueryContext::predicate_id predicate_index_;
+ std::vector<QueryContext::bloom_filter_id> bloom_filter_ids_;
+ std::vector<attribute_id> bloom_filter_attribute_ids_;
+ std::unique_ptr<std::vector<const BloomFilter*>> bloom_filters_;
const QueryContext::scalar_group_id selection_index_;
const std::vector<attribute_id> simple_selection_;
@@ -323,6 +338,8 @@ class SelectWorkOrder : public WorkOrder {
const CatalogRelationSchema &input_relation,
const block_id input_block_id,
const Predicate *predicate,
+ const std::vector<const BloomFilter *> &bloom_filters,
+ const std::vector<attribute_id> &bloom_filter_attribute_ids,
const bool simple_projection,
const std::vector<attribute_id> &simple_selection,
const std::vector<std::unique_ptr<const Scalar>> *selection,
@@ -333,6 +350,8 @@ class SelectWorkOrder : public WorkOrder {
input_relation_(input_relation),
input_block_id_(input_block_id),
predicate_(predicate),
+ bloom_filters_(bloom_filters),
+ bloom_filter_attribute_ids_(bloom_filter_attribute_ids),
simple_projection_(simple_projection),
simple_selection_(simple_selection),
selection_(selection),
@@ -365,6 +384,8 @@ class SelectWorkOrder : public WorkOrder {
const CatalogRelationSchema &input_relation,
const block_id input_block_id,
const Predicate *predicate,
+ const std::vector<const BloomFilter *> &bloom_filters,
+ const std::vector<attribute_id> &bloom_filter_attribute_ids,
const bool simple_projection,
std::vector<attribute_id> &&simple_selection,
const std::vector<std::unique_ptr<const Scalar>> *selection,
@@ -375,6 +396,8 @@ class SelectWorkOrder : public WorkOrder {
input_relation_(input_relation),
input_block_id_(input_block_id),
predicate_(predicate),
+ bloom_filters_(bloom_filters),
+ bloom_filter_attribute_ids_(bloom_filter_attribute_ids),
simple_projection_(simple_projection),
simple_selection_(std::move(simple_selection)),
selection_(selection),
@@ -399,6 +422,8 @@ class SelectWorkOrder : public WorkOrder {
const CatalogRelationSchema &input_relation_;
const block_id input_block_id_;
const Predicate *predicate_;
+ const std::vector<const BloomFilter *> &bloom_filters_;
+ const std::vector<attribute_id> &bloom_filter_attribute_ids_;
const bool simple_projection_;
const std::vector<attribute_id> simple_selection_;
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index d225258..d85b5c4 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -54,6 +54,8 @@
#include "types/containers/Tuple.hpp"
#include "utility/BloomFilterAdapter.hpp"
+#include "gflags/gflags.h"
+
#include "glog/logging.h"
using std::unique_ptr;
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/storage/StorageBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.cpp b/storage/StorageBlock.cpp
index 8370418..74ff5b6 100644
--- a/storage/StorageBlock.cpp
+++ b/storage/StorageBlock.cpp
@@ -59,8 +59,12 @@
#include "types/containers/ColumnVectorsValueAccessor.hpp"
#include "types/containers/Tuple.hpp"
#include "types/operations/comparisons/ComparisonUtil.hpp"
+#include "utility/BloomFilter.hpp"
+#include "utility/BloomFilterAdapter.hpp"
#include "utility/Macros.hpp"
+#include "gflags/gflags.h"
+
#include "glog/logging.h"
#ifdef QUICKSTEP_HAVE_BITWEAVING
@@ -78,6 +82,8 @@ using std::vector;
namespace quickstep {
+DECLARE_int64(bloom_adapter_batch_size);
+
class Type;
StorageBlock::StorageBlock(const CatalogRelationSchema &relation,
@@ -341,6 +347,8 @@ void StorageBlock::sample(const bool is_block_sample,
void StorageBlock::select(const vector<unique_ptr<const Scalar>> &selection,
const Predicate *predicate,
+ const std::vector<const BloomFilter *> &bloom_filters,
+ const std::vector<attribute_id> &bloom_filter_attribute_ids,
InsertDestinationInterface *destination) const {
ColumnVectorsValueAccessor temp_result;
{
@@ -350,10 +358,58 @@ void StorageBlock::select(const vector<unique_ptr<const Scalar>> &selection,
std::unique_ptr<TupleIdSequence> matches;
std::unique_ptr<ValueAccessor> accessor;
- if (predicate == nullptr) {
+
+ if (bloom_filters.size() > 0) {
+ const std::size_t num_tuples = tuple_store_->numTuples();
+ matches.reset(new TupleIdSequence(num_tuples));
+// std::cerr << "Before: " << num_tuples << "\n";
accessor.reset(tuple_store_->createValueAccessor());
+ InvokeOnAnyValueAccessor(
+ accessor.get(),
+ [&](auto *accessor) -> void { // NOLINT(build/c++11)
+ std::vector<std::size_t> attr_size_vector;
+ attr_size_vector.reserve(bloom_filter_attribute_ids.size());
+ for (const auto &attr : bloom_filter_attribute_ids) {
+ auto val_and_size =
+ accessor->template getUntypedValueAndByteLengthAtAbsolutePosition<false>(0, attr);
+ attr_size_vector.emplace_back(val_and_size.second);
+ }
+
+ std::unique_ptr<BloomFilterAdapter> bloom_filter_adapter;
+ bloom_filter_adapter.reset(new BloomFilterAdapter(
+ bloom_filters, bloom_filter_attribute_ids, attr_size_vector));
+
+ std::uint32_t batch_size_try = FLAGS_bloom_adapter_batch_size;
+ std::uint32_t num_tuples_left = accessor->getNumTuples();
+ std::vector<tuple_id> batch(num_tuples_left);
+
+ do {
+ std::uint32_t batch_size =
+ batch_size_try < num_tuples_left ? batch_size_try : num_tuples_left;
+ for (std::size_t i = 0; i < batch_size; ++i) {
+ accessor->next();
+ batch.push_back(accessor->getCurrentPosition());
+ }
+
+ std::size_t num_hits =
+ bloom_filter_adapter->bulkProbe<true>(accessor, batch, batch_size);
+ for (std::size_t t = 0; t < num_hits; ++t){
+ matches->set(batch[t], true);
+ }
+
+ batch.clear();
+ num_tuples_left -= batch_size;
+ batch_size_try = batch_size * 2;
+ } while (num_tuples_left > 0);
+ });
+// std::cerr << "After: " << matches->numTuples() << "\n";
+ }
+
+ if (predicate == nullptr) {
+ accessor.reset(tuple_store_->createValueAccessor(matches.get()));
} else {
- matches.reset(getMatchesForPredicate(predicate));
+ auto *new_matches = getMatchesForPredicate(predicate, matches.get());
+ matches.reset(new_matches);
accessor.reset(tuple_store_->createValueAccessor(matches.get()));
}
@@ -371,13 +427,63 @@ void StorageBlock::select(const vector<unique_ptr<const Scalar>> &selection,
void StorageBlock::selectSimple(const std::vector<attribute_id> &selection,
const Predicate *predicate,
+ const std::vector<const BloomFilter *> &bloom_filters,
+ const std::vector<attribute_id> &bloom_filter_attribute_ids,
InsertDestinationInterface *destination) const {
std::unique_ptr<ValueAccessor> accessor;
std::unique_ptr<TupleIdSequence> matches;
- if (predicate == nullptr) {
+
+ if (bloom_filters.size() > 0) {
+ const std::size_t num_tuples = tuple_store_->numTuples();
+ matches.reset(new TupleIdSequence(num_tuples));
+// std::cerr << "Before: " << num_tuples << "\n";
accessor.reset(tuple_store_->createValueAccessor());
+ InvokeOnAnyValueAccessor(
+ accessor.get(),
+ [&](auto *accessor) -> void { // NOLINT(build/c++11)
+ std::vector<std::size_t> attr_size_vector;
+ attr_size_vector.reserve(bloom_filter_attribute_ids.size());
+ for (const auto &attr : bloom_filter_attribute_ids) {
+ auto val_and_size =
+ accessor->template getUntypedValueAndByteLengthAtAbsolutePosition<false>(0, attr);
+ attr_size_vector.emplace_back(val_and_size.second);
+ }
+
+ std::unique_ptr<BloomFilterAdapter> bloom_filter_adapter;
+ bloom_filter_adapter.reset(new BloomFilterAdapter(
+ bloom_filters, bloom_filter_attribute_ids, attr_size_vector));
+
+ std::uint32_t batch_size_try = FLAGS_bloom_adapter_batch_size;
+ std::uint32_t num_tuples_left = accessor->getNumTuples();
+ std::vector<tuple_id> batch(num_tuples_left);
+
+ do {
+ std::uint32_t batch_size =
+ batch_size_try < num_tuples_left ? batch_size_try : num_tuples_left;
+ for (std::size_t i = 0; i < batch_size; ++i) {
+ accessor->next();
+ batch.push_back(accessor->getCurrentPosition());
+ }
+
+ std::size_t num_hits =
+ bloom_filter_adapter->bulkProbe<true>(accessor, batch, batch_size);
+ for (std::size_t t = 0; t < num_hits; ++t){
+ matches->set(batch[t], true);
+ }
+
+ batch.clear();
+ num_tuples_left -= batch_size;
+ batch_size_try = batch_size * 2;
+ } while (num_tuples_left > 0);
+ });
+// std::cerr << "After: " << matches->numTuples() << "\n";
+ }
+
+ if (predicate == nullptr) {
+ accessor.reset(tuple_store_->createValueAccessor(matches.get()));
} else {
- matches.reset(getMatchesForPredicate(predicate));
+ auto *new_matches = getMatchesForPredicate(predicate, matches.get());
+ matches.reset(new_matches);
accessor.reset(tuple_store_->createValueAccessor(matches.get()));
}
@@ -1219,12 +1325,13 @@ bool StorageBlock::rebuildIndexes(bool short_circuit) {
return all_indices_consistent_;
}
-TupleIdSequence* StorageBlock::getMatchesForPredicate(const Predicate *predicate) const {
+TupleIdSequence* StorageBlock::getMatchesForPredicate(const Predicate *predicate,
+ const TupleIdSequence *sequence) const {
if (predicate == nullptr) {
return tuple_store_->getExistenceMap();
}
- std::unique_ptr<ValueAccessor> value_accessor(tuple_store_->createValueAccessor());
+ std::unique_ptr<ValueAccessor> value_accessor(tuple_store_->createValueAccessor(sequence));
std::unique_ptr<TupleIdSequence> existence_map;
if (!tuple_store_->isPacked()) {
existence_map.reset(tuple_store_->getExistenceMap());
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/storage/StorageBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.hpp b/storage/StorageBlock.hpp
index 2a20cb5..5cca51c 100644
--- a/storage/StorageBlock.hpp
+++ b/storage/StorageBlock.hpp
@@ -34,6 +34,7 @@
#include "storage/StorageBlockLayout.pb.h"
#include "storage/TupleIdSequence.hpp"
#include "storage/TupleStorageSubBlock.hpp"
+#include "utility/BloomFilter.hpp"
#include "utility/Macros.hpp"
#include "utility/PtrVector.hpp"
@@ -349,6 +350,8 @@ class StorageBlock : public StorageBlockBase {
**/
void select(const std::vector<std::unique_ptr<const Scalar>> &selection,
const Predicate *predicate,
+ const std::vector<const BloomFilter *> &bloom_filters,
+ const std::vector<attribute_id> &bloom_filter_attribute_ids,
InsertDestinationInterface *destination) const;
/**
@@ -372,6 +375,8 @@ class StorageBlock : public StorageBlockBase {
**/
void selectSimple(const std::vector<attribute_id> &selection,
const Predicate *predicate,
+ const std::vector<const BloomFilter *> &bloom_filters,
+ const std::vector<attribute_id> &bloom_filter_attribute_ids,
InsertDestinationInterface *destination) const;
/**
@@ -587,7 +592,8 @@ class StorageBlock : public StorageBlockBase {
**/
const std::size_t getNumTuples() const;
- TupleIdSequence* getMatchesForPredicate(const Predicate *predicate) const;
+ TupleIdSequence* getMatchesForPredicate(const Predicate *predicate,
+ const TupleIdSequence *sequence = nullptr) const;
private:
static TupleStorageSubBlock* CreateTupleStorageSubBlock(
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/utility/PlanVisualizer.cpp
----------------------------------------------------------------------
diff --git a/utility/PlanVisualizer.cpp b/utility/PlanVisualizer.cpp
index b90a8dc..9af00b4 100644
--- a/utility/PlanVisualizer.cpp
+++ b/utility/PlanVisualizer.cpp
@@ -35,6 +35,7 @@
#include "query_optimizer/physical/HashJoin.hpp"
#include "query_optimizer/physical/Physical.hpp"
#include "query_optimizer/physical/PhysicalType.hpp"
+#include "query_optimizer/physical/Selection.hpp"
#include "query_optimizer/physical/TableReference.hpp"
#include "query_optimizer/physical/TopLevelPlan.hpp"
#include "utility/StringUtil.hpp"
@@ -88,6 +89,9 @@ std::string PlanVisualizer::visualize(const P::PhysicalPtr &input) {
for (const EdgeInfo &edge_info : edges_) {
graph_oss << " " << edge_info.src_node_id << " -> "
<< edge_info.dst_node_id << " [";
+ if (edge_info.dashed) {
+ graph_oss << "style=dashed ";
+ }
if (!edge_info.labels.empty()) {
graph_oss << "label=\""
<< EscapeSpecialChars(JoinToString(edge_info.labels, " "))
@@ -118,6 +122,12 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
EdgeInfo &edge_info = edges_.back();
edge_info.src_node_id = child_id;
edge_info.dst_node_id = node_id;
+ edge_info.dashed = false;
+
+ if (input->getPhysicalType() == P::PhysicalType::kHashJoin &&
+ child == input->children()[1]) {
+ edge_info.dashed = true;
+ }
for (const auto &attr : child->getOutputAttributes()) {
if (referenced_ids.find(attr->id()) != referenced_ids.end()) {
@@ -167,7 +177,6 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
node_info.labels.emplace_back(
std::string("[BF probe] ") + bf.attribute->attribute_alias());
}
-
break;
}
case P::PhysicalType::kAggregate: {
@@ -180,7 +189,18 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
node_info.labels.emplace_back(
std::string("[BF probe] ") + bf.attribute->attribute_alias());
}
+ break;
+ }
+ case P::PhysicalType::kSelection: {
+ const P::SelectionPtr selection =
+ std::static_pointer_cast<const P::Selection>(input);
+ node_info.labels.emplace_back(input->getName());
+ const auto &bf_config = selection->bloom_filter_config();
+ for (const auto &bf : bf_config.probe_side_bloom_filters) {
+ node_info.labels.emplace_back(
+ std::string("[BF probe] ") + bf.attribute->attribute_alias());
+ }
break;
}
default: {
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/utility/PlanVisualizer.hpp
----------------------------------------------------------------------
diff --git a/utility/PlanVisualizer.hpp b/utility/PlanVisualizer.hpp
index 1c0df77..e4e3957 100644
--- a/utility/PlanVisualizer.hpp
+++ b/utility/PlanVisualizer.hpp
@@ -73,6 +73,7 @@ class PlanVisualizer {
int src_node_id;
int dst_node_id;
std::vector<std::string> labels;
+ bool dashed;
};
void visit(const optimizer::physical::PhysicalPtr &input);
[10/16] incubator-quickstep git commit: Initial commit
Posted by ji...@apache.org.
Initial commit
Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/b7150fb1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/b7150fb1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/b7150fb1
Branch: refs/heads/LIP-for-tpch
Commit: b7150fb13b117841919ac8dfcc7aca9043a30e82
Parents: d9135a8
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Sat Jun 11 23:14:00 2016 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Thu Aug 11 15:23:00 2016 -0500
----------------------------------------------------------------------
CMakeLists.txt | 1 +
catalog/CMakeLists.txt | 9 +
catalog/Catalog.proto | 5 +
catalog/CatalogRelationConstraints.cpp | 55 ++
catalog/CatalogRelationConstraints.hpp | 110 ++++
catalog/CatalogRelationSchema.cpp | 15 +
catalog/CatalogRelationSchema.hpp | 16 +-
cli/CommandExecutor.cpp | 25 +-
cli/QuickstepCli.cpp | 65 +++
compression/CompressionDictionaryLite.hpp | 42 ++
query_execution/CMakeLists.txt | 1 +
query_execution/QueryContext.cpp | 11 +-
query_execution/Worker.cpp | 5 +
query_optimizer/CMakeLists.txt | 2 +
query_optimizer/ExecutionGenerator.cpp | 74 +--
query_optimizer/ExecutionGenerator.hpp | 2 +-
query_optimizer/ExecutionHeuristics.cpp | 171 ++++---
query_optimizer/ExecutionHeuristics.hpp | 79 ++-
query_optimizer/PhysicalGenerator.cpp | 7 +-
query_optimizer/cost_model/SimpleCostModel.cpp | 4 +-
.../cost_model/StarSchemaSimpleCostModel.cpp | 42 +-
query_optimizer/expressions/ExpressionUtil.hpp | 8 +-
query_optimizer/physical/Aggregate.cpp | 5 +
query_optimizer/physical/Aggregate.hpp | 23 +-
query_optimizer/physical/HashJoin.cpp | 27 +
query_optimizer/physical/HashJoin.hpp | 23 +-
query_optimizer/physical/Physical.hpp | 55 ++
query_optimizer/physical/Selection.cpp | 6 +
query_optimizer/physical/Selection.hpp | 3 +
query_optimizer/physical/TableReference.cpp | 18 +
query_optimizer/physical/TableReference.hpp | 3 +
query_optimizer/rules/AttachBloomFilters.cpp | 308 ++++++++++++
query_optimizer/rules/AttachBloomFilters.hpp | 118 +++++
query_optimizer/rules/CMakeLists.txt | 17 +
.../StarSchemaHashJoinOrderOptimization.cpp | 277 ++++++----
.../StarSchemaHashJoinOrderOptimization.hpp | 100 ++--
.../tests/ExecutionHeuristics_unittest.cpp | 3 +-
relational_operators/HashJoinOperator.cpp | 10 +
relational_operators/HashJoinOperator.hpp | 25 +-
relational_operators/WorkOrder.hpp | 11 +-
storage/AggregationOperationState.cpp | 98 +++-
storage/AggregationOperationState.hpp | 10 +-
storage/AggregationOperationState.proto | 6 +
storage/BasicColumnStoreValueAccessor.hpp | 26 +-
storage/BloomFilterIndexSubBlock.cpp | 4 +-
storage/BloomFilterIndexSubBlock.hpp | 6 -
storage/CMakeLists.txt | 2 +
storage/CompressedColumnStoreValueAccessor.hpp | 22 +
.../CompressedPackedRowStoreValueAccessor.hpp | 22 +
storage/HashTable.hpp | 185 ++++---
storage/HashTable.proto | 10 +-
storage/HashTableFactory.hpp | 23 +-
storage/PackedRowStoreValueAccessor.hpp | 25 +-
storage/SplitRowStoreValueAccessor.hpp | 45 ++
storage/StorageBlock.cpp | 28 +-
storage/StorageBlock.hpp | 7 +-
storage/ValueAccessor.hpp | 36 ++
types/containers/ColumnVector.hpp | 35 ++
types/containers/ColumnVectorsValueAccessor.hpp | 17 +
utility/BloomFilter.hpp | 502 ++++++++++++++-----
utility/BloomFilter.proto | 6 +-
utility/BloomFilterAdapter.hpp | 142 ++++++
utility/CMakeLists.txt | 13 +
utility/DisjointTreeForest.hpp | 116 +++++
utility/EventProfiler.cpp | 29 ++
utility/EventProfiler.hpp | 188 +++++++
utility/PlanVisualizer.cpp | 42 +-
67 files changed, 2867 insertions(+), 559 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 487aaf9..27db15e 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -770,6 +770,7 @@ target_link_libraries(quickstep_cli_shell
quickstep_queryoptimizer_QueryProcessor
quickstep_storage_PreloaderThread
quickstep_threading_ThreadIDBasedMap
+ quickstep_utility_EventProfiler
quickstep_utility_ExecutionDAGVisualizer
quickstep_utility_Macros
quickstep_utility_PtrVector
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/catalog/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/catalog/CMakeLists.txt b/catalog/CMakeLists.txt
index dd4ef99..fc01458 100644
--- a/catalog/CMakeLists.txt
+++ b/catalog/CMakeLists.txt
@@ -35,6 +35,9 @@ add_library(quickstep_catalog_CatalogDatabaseCache CatalogDatabaseCache.cpp Cata
add_library(quickstep_catalog_CatalogDatabaseLite ../empty_src.cpp CatalogDatabaseLite.hpp)
add_library(quickstep_catalog_CatalogErrors ../empty_src.cpp CatalogErrors.hpp)
add_library(quickstep_catalog_CatalogRelation CatalogRelation.cpp CatalogRelation.hpp)
+add_library(quickstep_catalog_CatalogRelationConstraints
+ CatalogRelationConstraints.cpp
+ CatalogRelationConstraints.hpp)
add_library(quickstep_catalog_CatalogRelationSchema
CatalogRelationSchema.cpp
CatalogRelationSchema.hpp)
@@ -117,6 +120,10 @@ target_link_libraries(quickstep_catalog_CatalogRelation
quickstep_threading_SpinSharedMutex
quickstep_utility_Macros
quickstep_utility_PtrVector)
+target_link_libraries(quickstep_catalog_CatalogRelationConstraints
+ quickstep_catalog_CatalogTypedefs
+ quickstep_catalog_Catalog_proto
+ quickstep_utility_Macros)
target_link_libraries(quickstep_catalog_CatalogRelationStatistics
quickstep_catalog_CatalogTypedefs
quickstep_catalog_Catalog_proto
@@ -136,6 +143,7 @@ target_link_libraries(quickstep_catalog_CatalogRelationSchema
glog
quickstep_catalog_CatalogAttribute
quickstep_catalog_CatalogErrors
+ quickstep_catalog_CatalogRelationConstraints
quickstep_catalog_CatalogTypedefs
quickstep_catalog_Catalog_proto
quickstep_types_Type
@@ -182,6 +190,7 @@ target_link_libraries(quickstep_catalog
quickstep_catalog_CatalogDatabaseLite
quickstep_catalog_CatalogErrors
quickstep_catalog_CatalogRelation
+ quickstep_catalog_CatalogRelationConstraints
quickstep_catalog_CatalogRelationSchema
quickstep_catalog_CatalogRelationStatistics
quickstep_catalog_CatalogTypedefs
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/catalog/Catalog.proto
----------------------------------------------------------------------
diff --git a/catalog/Catalog.proto b/catalog/Catalog.proto
index 90ce37e..a4cc3ce 100644
--- a/catalog/Catalog.proto
+++ b/catalog/Catalog.proto
@@ -80,6 +80,10 @@ message IndexScheme {
repeated IndexEntry index_entries = 1;
}
+message CatalogRelationConstraints {
+ repeated int32 primary_key = 1;
+}
+
message CatalogRelationStatistics {
optional fixed64 num_tuples = 1;
@@ -96,6 +100,7 @@ message CatalogRelationSchema {
required bool temporary = 3;
repeated CatalogAttribute attributes = 4;
+ optional CatalogRelationConstraints constraints = 5;
extensions 16 to max;
}
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/catalog/CatalogRelationConstraints.cpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationConstraints.cpp b/catalog/CatalogRelationConstraints.cpp
new file mode 100644
index 0000000..4525a98
--- /dev/null
+++ b/catalog/CatalogRelationConstraints.cpp
@@ -0,0 +1,55 @@
+/**
+ * Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ * University of Wisconsin\u2014Madison.
+ *
+ * Licensed 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 "catalog/CatalogRelationConstraints.hpp"
+
+#include "catalog/Catalog.pb.h"
+
+namespace quickstep {
+
+CatalogRelationConstraints::CatalogRelationConstraints(
+ const serialization::CatalogRelationConstraints &proto) {
+ if (proto.primary_key_size() > 0) {
+ primary_key_.reset(new std::set<attribute_id>());
+ for (std::size_t i = 0; i < proto.primary_key_size(); ++i) {
+ primary_key_->emplace(proto.primary_key(i));
+ }
+ }
+}
+
+serialization::CatalogRelationConstraints CatalogRelationConstraints::getProto() const {
+ serialization::CatalogRelationConstraints proto;
+ if (primary_key_ != nullptr) {
+ for (const auto attr_id : *primary_key_) {
+ proto.add_primary_key(attr_id);
+ }
+ }
+ return proto;
+}
+
+bool CatalogRelationConstraints::ProtoIsValid(
+ const serialization::CatalogRelationConstraints &proto,
+ const std::size_t num_attributes) {
+ for (std::size_t i = 0; i < proto.primary_key_size(); ++i) {
+ if (proto.primary_key(i) >= num_attributes) {
+ return false;
+ }
+ }
+ return true;
+}
+
+} // namespace quickstep
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/catalog/CatalogRelationConstraints.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationConstraints.hpp b/catalog/CatalogRelationConstraints.hpp
new file mode 100644
index 0000000..896c072
--- /dev/null
+++ b/catalog/CatalogRelationConstraints.hpp
@@ -0,0 +1,110 @@
+/**
+ * Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ * University of Wisconsin\u2014Madison.
+ *
+ * Licensed 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.
+ **/
+
+#ifndef QUICKSTEP_CATALOG_CATALOG_RELATION_CONSTRAINTS_HPP_
+#define QUICKSTEP_CATALOG_CATALOG_RELATION_CONSTRAINTS_HPP_
+
+#include <algorithm>
+#include <cstddef>
+#include <memory>
+#include <set>
+#include <utility>
+
+#include "catalog/Catalog.pb.h"
+#include "catalog/CatalogTypedefs.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Catalog
+ * @{
+ */
+
+/**
+ * @brief Constraints on a catalog relation.
+ **/
+class CatalogRelationConstraints {
+ public:
+ /**
+ * @brief Constructor.
+ **/
+ CatalogRelationConstraints() {}
+
+ /**
+ * @brief Reconstruct a CatalogRelationConstraints object from its serialized
+ * Protocol Buffer form.
+ *
+ * @param proto The Protocol Buffer serialization of a CatalogRelationConstraints
+ * object, previously produced by getProto().
+ **/
+ explicit CatalogRelationConstraints(const serialization::CatalogRelationConstraints &proto);
+
+ /**
+ * @brief Serialize the CatalogRelationConstraints object as Protocol Buffer.
+ *
+ * @return The Protocol Buffer representation of the CatalogRelationConstraints
+ * object.
+ **/
+ serialization::CatalogRelationConstraints getProto() const;
+
+ static bool ProtoIsValid(const serialization::CatalogRelationConstraints &proto,
+ const std::size_t num_attributes);
+
+ bool hasPrimaryKey() const {
+ return (primary_key_ != nullptr);
+ }
+
+ const std::set<attribute_id>* getPrimaryKey() const {
+ return primary_key_.get();
+ }
+
+ template <typename IterableT>
+ void setPrimaryKey(IterableT &&primary_key) {
+ CHECK(!primary_key.empty());
+ primary_key_.reset(
+ new std::set<attribute_id>(primary_key.begin(), primary_key.end()));
+ }
+
+ void removePrimaryKey() {
+ primary_key_.reset();
+ }
+
+ bool impliesUniqueAttributes(const std::set<attribute_id> &attributes) const {
+ if (primary_key_ == nullptr) {
+ return false;
+ }
+
+ std::vector<attribute_id> attr_intersection;
+ std::set_intersection(primary_key_->begin(), primary_key_->end(),
+ attributes.begin(), attributes.end(),
+ std::back_inserter(attr_intersection));
+ return (attr_intersection.size() == primary_key_->size());
+ }
+
+ private:
+ std::unique_ptr<std::set<attribute_id>> primary_key_;
+
+ DISALLOW_COPY_AND_ASSIGN(CatalogRelationConstraints);
+};
+
+/** @} */
+
+} // namespace quickstep
+
+#endif // QUICKSTEP_CATALOG_CATALOG_RELATION_CONSTRAINTS_HPP_
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/catalog/CatalogRelationSchema.cpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationSchema.cpp b/catalog/CatalogRelationSchema.cpp
index 5568cef..d474ef6 100644
--- a/catalog/CatalogRelationSchema.cpp
+++ b/catalog/CatalogRelationSchema.cpp
@@ -27,6 +27,7 @@
#include "catalog/Catalog.pb.h"
#include "catalog/CatalogAttribute.hpp"
#include "catalog/CatalogErrors.hpp"
+#include "catalog/CatalogRelationConstraints.hpp"
#include "catalog/CatalogTypedefs.hpp"
#include "types/Type.hpp"
#include "utility/PtrVector.hpp"
@@ -70,6 +71,12 @@ CatalogRelationSchema::CatalogRelationSchema(const serialization::CatalogRelatio
attr_vec_.push_back(nullptr);
}
}
+
+ if (proto.has_constraints()) {
+ constraints_.reset(new CatalogRelationConstraints(proto.constraints()));
+ } else {
+ constraints_.reset(new CatalogRelationConstraints());
+ }
}
bool CatalogRelationSchema::ProtoIsValid(const serialization::CatalogRelationSchema &proto) {
@@ -84,6 +91,12 @@ bool CatalogRelationSchema::ProtoIsValid(const serialization::CatalogRelationSch
}
}
+ if (proto.has_constraints()
+ && !CatalogRelationConstraints::ProtoIsValid(proto.constraints(),
+ proto.attributes_size())) {
+ return false;
+ }
+
return true;
}
@@ -104,6 +117,8 @@ serialization::CatalogRelationSchema CatalogRelationSchema::getProto() const {
}
}
+ proto.mutable_constraints()->CopyFrom(constraints_->getProto());
+
return proto;
}
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/catalog/CatalogRelationSchema.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationSchema.hpp b/catalog/CatalogRelationSchema.hpp
index d8b6128..0c7e97b 100644
--- a/catalog/CatalogRelationSchema.hpp
+++ b/catalog/CatalogRelationSchema.hpp
@@ -21,12 +21,14 @@
#define QUICKSTEP_CATALOG_CATALOG_RELATION_SCHEMA_HPP_
#include <cstddef>
+#include <memory>
#include <string>
#include <unordered_map>
#include <vector>
#include "catalog/Catalog.pb.h"
#include "catalog/CatalogAttribute.hpp"
+#include "catalog/CatalogRelationConstraints.hpp"
#include "catalog/CatalogTypedefs.hpp"
#include "utility/Macros.hpp"
#include "utility/PtrVector.hpp"
@@ -427,6 +429,14 @@ class CatalogRelationSchema {
return max_byte_lengths_;
}
+ const CatalogRelationConstraints& getConstraints() const {
+ return *constraints_;
+ }
+
+ CatalogRelationConstraints* getConstraintsMutable() {
+ return constraints_.get();
+ }
+
protected:
/**
* @brief Create a new relation.
@@ -456,7 +466,8 @@ class CatalogRelationSchema {
min_variable_byte_length_excluding_nullable_(0),
estimated_variable_byte_length_(0),
current_nullable_attribute_index_(-1),
- current_variable_length_attribute_index_(-1) {
+ current_variable_length_attribute_index_(-1),
+ constraints_(new CatalogRelationConstraints()) {
}
/**
@@ -532,6 +543,9 @@ class CatalogRelationSchema {
std::vector<int> variable_length_attribute_indices_;
int current_variable_length_attribute_index_;
+ // Primary key, foreign keys, etc.
+ std::unique_ptr<CatalogRelationConstraints> constraints_;
+
private:
friend class CatalogDatabase;
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/cli/CommandExecutor.cpp
----------------------------------------------------------------------
diff --git a/cli/CommandExecutor.cpp b/cli/CommandExecutor.cpp
index 78fbe6f..f27f1ef 100644
--- a/cli/CommandExecutor.cpp
+++ b/cli/CommandExecutor.cpp
@@ -258,7 +258,8 @@ inline TypedValue executeQueryForSingleResult(
return value;
}
-void executeAnalyze(const tmb::client_id main_thread_client_id,
+void executeAnalyze(const PtrVector<ParseString> *arguments,
+ const tmb::client_id main_thread_client_id,
const tmb::client_id foreman_client_id,
MessageBus *bus,
QueryProcessor *query_processor,
@@ -267,8 +268,19 @@ void executeAnalyze(const tmb::client_id main_thread_client_id,
StorageManager *storage_manager = query_processor->getStorageManager();
std::unique_ptr<SqlParserWrapper> parser_wrapper(new SqlParserWrapper());
- std::vector<std::reference_wrapper<const CatalogRelation>> relations(
- database.begin(), database.end());
+ std::vector<std::reference_wrapper<const CatalogRelation>> relations;
+ if (arguments->size() == 0) {
+ relations.insert(relations.begin(), database.begin(), database.end());
+ } else {
+ for (const auto &rel_name : *arguments) {
+ const CatalogRelation *rel = database.getRelationByName(rel_name.value());
+ if (rel == nullptr) {
+ THROW_SQL_ERROR_AT(&rel_name) << "Table does not exist";
+ } else {
+ relations.emplace_back(*rel);
+ }
+ }
+ }
// Analyze each relation in the database.
for (const CatalogRelation &relation : relations) {
@@ -348,8 +360,11 @@ void executeCommand(const ParseStatement &statement,
executeDescribeTable(arguments, catalog_database, out);
}
} else if (command_str == C::kAnalyzeCommand) {
- executeAnalyze(
- main_thread_client_id, foreman_client_id, bus, query_processor, out);
+ executeAnalyze(arguments,
+ main_thread_client_id,
+ foreman_client_id,
+ bus,
+ query_processor, out);
} else {
THROW_SQL_ERROR_AT(command.command()) << "Invalid Command";
}
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index 8269197..a3dfae4 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -52,6 +52,9 @@ typedef quickstep::LineReaderDumb LineReaderImpl;
#include <gperftools/profiler.h>
#endif
+#include "catalog/CatalogDatabase.hpp"
+#include "catalog/CatalogRelation.hpp"
+#include "catalog/CatalogRelationConstraints.hpp"
#include "cli/DefaultsConfigurator.hpp"
#include "cli/InputParserUtil.hpp"
#include "cli/PrintToScreen.hpp"
@@ -75,6 +78,7 @@ typedef quickstep::LineReaderDumb LineReaderImpl;
#include "storage/PreloaderThread.hpp"
#include "threading/ThreadIDBasedMap.hpp"
+#include "utility/EventProfiler.hpp"
#include "utility/ExecutionDAGVisualizer.hpp"
#include "utility/Macros.hpp"
#include "utility/PtrVector.hpp"
@@ -90,6 +94,8 @@ typedef quickstep::LineReaderDumb LineReaderImpl;
#include "tmb/message_bus.h"
#include "tmb/message_style.h"
+#include "google/protobuf/text_format.h"
+
namespace quickstep {
class CatalogRelation;
}
@@ -190,9 +196,48 @@ DEFINE_bool(visualize_execution_dag, false,
"If true, visualize the execution plan DAG into a graph in DOT "
"format (DOT is a plain text graph description language) which is "
"then printed via stderr.");
+DEFINE_string(profile_output, "",
+ "Output file name for writing the profiled events.");
} // namespace quickstep
+void addPrimaryKeyInfoForTPCHTables(quickstep::CatalogDatabase *database) {
+ const std::vector<std::pair<std::string, std::vector<std::string>>> rel_pkeys = {
+ { "region", { "r_regionkey" } },
+ { "nation", { "n_nationkey" } },
+ { "supplier", { "s_suppkey" } },
+ { "customer", { "c_custkey" } },
+ { "part", { "p_partkey" } },
+ { "partsupp", { "ps_partkey", "ps_suppkey" } },
+ { "orders", { "o_orderkey" } }
+ };
+ for (const auto &rel_pair : rel_pkeys) {
+ CatalogRelation *rel = database->getRelationByNameMutable(rel_pair.first);
+ std::vector<quickstep::attribute_id> attrs;
+ for (const auto &pkey : rel_pair.second) {
+ attrs.emplace_back(rel->getAttributeByName(pkey)->getID());
+ }
+ rel->getConstraintsMutable()->setPrimaryKey(attrs);
+ }
+}
+
+void addPrimaryKeyInfoForSSBTables(quickstep::CatalogDatabase *database) {
+ const std::vector<std::pair<std::string, std::vector<std::string>>> rel_pkeys = {
+ { "supplier", { "s_suppkey" } },
+ { "customer", { "c_custkey" } },
+ { "part", { "p_partkey" } },
+ { "ddate", { "d_datekey" } }
+ };
+ for (const auto &rel_pair : rel_pkeys) {
+ CatalogRelation *rel = database->getRelationByNameMutable(rel_pair.first);
+ std::vector<quickstep::attribute_id> attrs;
+ for (const auto &pkey : rel_pair.second) {
+ attrs.emplace_back(rel->getAttributeByName(pkey)->getID());
+ }
+ rel->getConstraintsMutable()->setPrimaryKey(attrs);
+ }
+}
+
int main(int argc, char* argv[]) {
google::InitGoogleLogging(argv[0]);
gflags::ParseCommandLineFlags(&argc, &argv, true);
@@ -300,6 +345,15 @@ int main(int argc, char* argv[]) {
LOG(FATAL) << "NON-STANDARD EXCEPTION DURING STARTUP";
}
+// addPrimaryKeyInfoForTPCHTables(query_processor->getDefaultDatabase());
+// addPrimaryKeyInfoForSSBTables(query_processor->getDefaultDatabase());
+// std::string proto_str;
+// google::protobuf::TextFormat::PrintToString(
+// query_processor->getDefaultDatabase()->getProto(), &proto_str);
+// std::cerr << proto_str << "\n";
+// query_processor->markCatalogAltered();
+// query_processor->saveCatalog();
+
// Parse the CPU affinities for workers and the preloader thread, if enabled
// to warm up the buffer pool.
const vector<int> worker_cpu_affinities =
@@ -449,6 +503,7 @@ int main(int argc, char* argv[]) {
new quickstep::ExecutionDAGVisualizer(*query_handle->getQueryPlanMutable()));
}
+ quickstep::simple_profiler.clear();
start = std::chrono::steady_clock::now();
QueryExecutionUtil::ConstructAndSendAdmitRequestMessage(
main_thread_client_id,
@@ -461,6 +516,11 @@ int main(int argc, char* argv[]) {
main_thread_client_id, &bus);
end = std::chrono::steady_clock::now();
+ if (quickstep::FLAGS_visualize_dag) {
+ quickstep::DAGVisualizer visualizer(*query_handle->getQueryPlanMutable());
+ std::cerr << "\n" << visualizer.toDOT() << "\n";
+ }
+
const CatalogRelation *query_result_relation = query_handle->getQueryResultRelation();
if (query_result_relation) {
PrintToScreen::PrintRelation(*query_result_relation,
@@ -492,6 +552,11 @@ int main(int argc, char* argv[]) {
dag_visualizer->bindProfilingStats(profiling_stats);
std::cerr << "\n" << dag_visualizer->toDOT() << "\n";
}
+ if (!quickstep::FLAGS_profile_output.empty()) {
+ std::ofstream ofs(quickstep::FLAGS_profile_output, std::ios::out);
+ quickstep::simple_profiler.writeToStream(ofs);
+ ofs.close();
+ }
} catch (const std::exception &e) {
fprintf(stderr, "QUERY EXECUTION ERROR: %s\n", e.what());
break;
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/compression/CompressionDictionaryLite.hpp
----------------------------------------------------------------------
diff --git a/compression/CompressionDictionaryLite.hpp b/compression/CompressionDictionaryLite.hpp
index c4c338e..7eb0e34 100644
--- a/compression/CompressionDictionaryLite.hpp
+++ b/compression/CompressionDictionaryLite.hpp
@@ -176,6 +176,15 @@ class CompressionDictionaryLite {
}
}
+ template <bool check_null = true>
+ inline std::pair<const void*, std::size_t> getUntypedValueAndByteLengthForCode(const std::uint32_t code) const {
+ if (type_is_variable_length_) {
+ return variableLengthGetUntypedValueAndByteLengthHelper<std::uint32_t, check_null>(code);
+ } else {
+ return fixedLengthGetUntypedValueAndByteLengthHelper<std::uint32_t, check_null>(code);
+ }
+ }
+
/**
* @brief Get the value represented by the specified code as a TypedValue.
* @note This version is for codes of 8 bits or less. Also see
@@ -257,6 +266,39 @@ class CompressionDictionaryLite {
return retval;
}
+ template <typename CodeType, bool check_null = true>
+ inline std::pair<const void*, std::size_t> fixedLengthGetUntypedValueAndByteLengthHelper(
+ const CodeType code) const {
+ if (check_null && (code == getNullCode())) {
+ return std::make_pair(nullptr, 0);
+ }
+ DCHECK_LT(code, numberOfCodes());
+ return std::make_pair(static_cast<const char*>(dictionary_memory_)
+ + 2 * sizeof(std::uint32_t) // Header.
+ + code * type_fixed_byte_length_, // Index into value array.
+ type_fixed_byte_length_);
+ }
+
+ template <typename CodeType, bool check_null = true>
+ inline std::pair<const void*, std::size_t> variableLengthGetUntypedValueAndByteLengthHelper(
+ const CodeType code) const {
+ if (check_null && (code == getNullCode())) {
+ return std::make_pair(nullptr, 0);
+ }
+ DCHECK_LT(code, numberOfCodes());
+
+ const std::uint32_t value_offset = static_cast<const std::uint32_t*>(dictionary_memory_)[code + 2];
+ const void *data_ptr = variable_length_data_region_ + value_offset;
+ DCHECK_LT(data_ptr, static_cast<const char*>(dictionary_memory_) + dictionary_memory_size_);
+
+ std::size_t data_size = (code == *static_cast<const std::uint32_t*>(dictionary_memory_) - 1) ?
+ (static_cast<const char*>(dictionary_memory_)
+ + dictionary_memory_size_
+ - static_cast<const char*>(data_ptr))
+ : (static_cast<const std::uint32_t*>(dictionary_memory_)[code + 3] - value_offset);
+ return std::make_pair(data_ptr, data_size);
+ }
+
template <typename CodeType>
inline TypedValue fixedLengthGetTypedValueHelper(const CodeType code) const {
if (code == getNullCode()) {
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_execution/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_execution/CMakeLists.txt b/query_execution/CMakeLists.txt
index 74fcafb..ff0fe08 100644
--- a/query_execution/CMakeLists.txt
+++ b/query_execution/CMakeLists.txt
@@ -282,6 +282,7 @@ target_link_libraries(quickstep_queryexecution_Worker
quickstep_threading_Thread
quickstep_threading_ThreadIDBasedMap
quickstep_threading_ThreadUtil
+ quickstep_utility_EventProfiler
quickstep_utility_Macros
tmb)
target_link_libraries(quickstep_queryexecution_WorkerDirectory
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_execution/QueryContext.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryContext.cpp b/query_execution/QueryContext.cpp
index 2572e18..65405a6 100644
--- a/query_execution/QueryContext.cpp
+++ b/query_execution/QueryContext.cpp
@@ -61,15 +61,16 @@ QueryContext::QueryContext(const serialization::QueryContext &proto,
<< "Attempted to create QueryContext from an invalid proto description:\n"
<< proto.DebugString();
+ for (int i = 0; i < proto.bloom_filters_size(); ++i) {
+ bloom_filters_.emplace_back(new BloomFilter(proto.bloom_filters(i)));
+ }
+
for (int i = 0; i < proto.aggregation_states_size(); ++i) {
aggregation_states_.emplace_back(
AggregationOperationState::ReconstructFromProto(proto.aggregation_states(i),
database,
- storage_manager));
- }
-
- for (int i = 0; i < proto.bloom_filters_size(); ++i) {
- bloom_filters_.emplace_back(new BloomFilter(proto.bloom_filters(i)));
+ storage_manager,
+ bloom_filters_));
}
for (int i = 0; i < proto.generator_functions_size(); ++i) {
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_execution/Worker.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Worker.cpp b/query_execution/Worker.cpp
index 0b1efba..a1d3685 100644
--- a/query_execution/Worker.cpp
+++ b/query_execution/Worker.cpp
@@ -32,6 +32,7 @@
#include "relational_operators/WorkOrder.hpp"
#include "threading/ThreadIDBasedMap.hpp"
#include "threading/ThreadUtil.hpp"
+#include "utility/EventProfiler.hpp"
#include "glog/logging.h"
@@ -119,8 +120,12 @@ void Worker::executeWorkOrderHelper(const TaggedMessage &tagged_message,
const size_t query_id_for_workorder = worker_message.getWorkOrder()->getQueryID();
// Start measuring the execution time.
+ auto *container = relop_profiler.getContainer();
+ auto *line = container->getEventLine(worker_message.getRelationalOpIndex());
start = std::chrono::steady_clock::now();
+ line->emplace_back();
worker_message.getWorkOrder()->execute();
+ line->back().endEvent();
end = std::chrono::steady_clock::now();
delete worker_message.getWorkOrder();
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index c24ee89..aaad17c 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -79,6 +79,7 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
quickstep_queryoptimizer_QueryPlan
quickstep_queryoptimizer_costmodel_CostModel
quickstep_queryoptimizer_costmodel_SimpleCostModel
+ quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostModel
quickstep_queryoptimizer_expressions_AggregateFunction
quickstep_queryoptimizer_expressions_Alias
quickstep_queryoptimizer_expressions_AttributeReference
@@ -196,6 +197,7 @@ target_link_libraries(quickstep_queryoptimizer_PhysicalGenerator
quickstep_queryoptimizer_LogicalToPhysicalMapper
quickstep_queryoptimizer_logical_Logical
quickstep_queryoptimizer_physical_Physical
+ quickstep_queryoptimizer_rules_AttachBloomFilters
quickstep_queryoptimizer_rules_PruneColumns
quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization
quickstep_queryoptimizer_rules_SwapProbeBuild
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index a4b538e..6dbba51 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -59,6 +59,7 @@
#include "query_optimizer/QueryHandle.hpp"
#include "query_optimizer/QueryPlan.hpp"
#include "query_optimizer/cost_model/SimpleCostModel.hpp"
+#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
#include "query_optimizer/expressions/AggregateFunction.hpp"
#include "query_optimizer/expressions/Alias.hpp"
#include "query_optimizer/expressions/AttributeReference.hpp"
@@ -167,6 +168,8 @@ void ExecutionGenerator::generatePlan(const P::PhysicalPtr &physical_plan) {
cost_model_.reset(
new cost::SimpleCostModel(top_level_physical_plan_->shared_subplans()));
+ star_schema_cost_model_.reset(
+ new cost::StarSchemaSimpleCostModel(top_level_physical_plan_->shared_subplans()));
const CatalogRelation *result_relation = nullptr;
@@ -600,8 +603,10 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
std::vector<attribute_id> probe_attribute_ids;
std::vector<attribute_id> build_attribute_ids;
- std::vector<attribute_id> probe_original_attribute_ids;
- std::vector<attribute_id> build_original_attribute_ids;
+ const P::BloomFilterConfig &bloom_filter_config =
+ physical_plan->bloom_filter_config();
+ std::vector<attribute_id> probe_side_bloom_filter_attribute_ids;
+ std::vector<attribute_id> build_side_bloom_filter_attribute_ids;
const CatalogRelation *referenced_stored_probe_relation = nullptr;
const CatalogRelation *referenced_stored_build_relation = nullptr;
@@ -616,18 +621,6 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
const std::vector<E::AttributeReferencePtr> &left_join_attributes =
physical_plan->left_join_attributes();
for (const E::AttributeReferencePtr &left_join_attribute : left_join_attributes) {
- // Try to determine the original stored relation referenced in the Hash Join.
- referenced_stored_probe_relation =
- optimizer_context_->catalog_database()->getRelationByName(left_join_attribute->relation_name());
- if (referenced_stored_probe_relation == nullptr) {
- // Hash Join optimizations are not possible, if the referenced relation cannot be determined.
- skip_hash_join_optimization = true;
- } else {
- const attribute_id probe_operator_attribute_id =
- referenced_stored_probe_relation->getAttributeByName(left_join_attribute->attribute_name())->getID();
- probe_original_attribute_ids.emplace_back(probe_operator_attribute_id);
- }
-
const CatalogAttribute *probe_catalog_attribute
= attribute_substitution_map_[left_join_attribute->id()];
probe_attribute_ids.emplace_back(probe_catalog_attribute->getID());
@@ -640,18 +633,6 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
const std::vector<E::AttributeReferencePtr> &right_join_attributes =
physical_plan->right_join_attributes();
for (const E::AttributeReferencePtr &right_join_attribute : right_join_attributes) {
- // Try to determine the original stored relation referenced in the Hash Join.
- referenced_stored_build_relation =
- optimizer_context_->catalog_database()->getRelationByName(right_join_attribute->relation_name());
- if (referenced_stored_build_relation == nullptr) {
- // Hash Join optimizations are not possible, if the referenced relation cannot be determined.
- skip_hash_join_optimization = true;
- } else {
- const attribute_id build_operator_attribute_id =
- referenced_stored_build_relation->getAttributeByName(right_join_attribute->attribute_name())->getID();
- build_original_attribute_ids.emplace_back(build_operator_attribute_id);
- }
-
const CatalogAttribute *build_catalog_attribute
= attribute_substitution_map_[right_join_attribute->id()];
build_attribute_ids.emplace_back(build_catalog_attribute->getID());
@@ -661,6 +642,20 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
}
}
+ for (const auto &bf : bloom_filter_config.probe_side_bloom_filters) {
+ const CatalogAttribute *probe_bf_catalog_attribute
+ = attribute_substitution_map_[bf.attribute->id()];
+ probe_side_bloom_filter_attribute_ids.emplace_back(
+ probe_bf_catalog_attribute->getID());
+ }
+
+ for (const auto &bf : bloom_filter_config.build_side_bloom_filters) {
+ const CatalogAttribute *build_bf_catalog_attribute
+ = attribute_substitution_map_[bf.attribute->id()];
+ build_side_bloom_filter_attribute_ids.emplace_back(
+ build_bf_catalog_attribute->getID());
+ }
+
// Remember key types for call to SimplifyHashTableImplTypeProto() below.
std::vector<const Type*> key_types;
for (std::vector<E::AttributeReferencePtr>::size_type attr_idx = 0;
@@ -675,6 +670,8 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
key_types.push_back(&left_attribute_type);
}
+ std::size_t build_cardinality = cost_model_->estimateCardinality(build_physical);
+
// Convert the residual predicate proto.
QueryContext::predicate_id residual_predicate_index = QueryContext::kInvalidPredicateId;
if (physical_plan->residual_predicate()) {
@@ -835,9 +832,11 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
join_operator_index,
referenced_stored_build_relation,
referenced_stored_probe_relation,
- std::move(build_original_attribute_ids),
- std::move(probe_original_attribute_ids),
- join_hash_table_index);
+ bloom_filter_config,
+ std::move(build_side_bloom_filter_attribute_ids),
+ std::move(probe_side_bloom_filter_attribute_ids),
+ join_hash_table_index,
+ star_schema_cost_model_->estimateCardinality(build_physical));
}
}
@@ -1351,6 +1350,16 @@ void ExecutionGenerator::convertAggregate(
findRelationInfoOutputByPhysical(physical_plan->input());
aggr_state_proto->set_relation_id(input_relation_info->relation->getID());
+ const P::BloomFilterConfig &bloom_filter_config =
+ physical_plan->bloom_filter_config();
+ std::vector<attribute_id> bloom_filter_attribute_ids;
+
+ for (const auto &bf : bloom_filter_config.probe_side_bloom_filters) {
+ const CatalogAttribute *bf_catalog_attribute
+ = attribute_substitution_map_[bf.attribute->id()];
+ bloom_filter_attribute_ids.emplace_back(bf_catalog_attribute->getID());
+ }
+
std::vector<const Type*> group_by_types;
for (const E::NamedExpressionPtr &grouping_expression : physical_plan->grouping_expressions()) {
unique_ptr<const Scalar> execution_group_by_expression;
@@ -1465,6 +1474,13 @@ void ExecutionGenerator::convertAggregate(
std::forward_as_tuple(finalize_aggregation_operator_index, output_relation));
temporary_relation_info_vec_.emplace_back(finalize_aggregation_operator_index,
output_relation);
+
+ if (FLAGS_optimize_joins) {
+ execution_heuristics_->addAggregateInfo(aggregation_operator_index,
+ bloom_filter_config,
+ std::move(bloom_filter_attribute_ids),
+ aggr_state_index);
+ }
}
void ExecutionGenerator::convertSort(const P::SortPtr &physical_sort) {
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/ExecutionGenerator.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.hpp b/query_optimizer/ExecutionGenerator.hpp
index d5c7b06..8358233 100644
--- a/query_optimizer/ExecutionGenerator.hpp
+++ b/query_optimizer/ExecutionGenerator.hpp
@@ -37,7 +37,6 @@
#include "query_optimizer/QueryHandle.hpp"
#include "query_optimizer/QueryPlan.hpp"
#include "query_optimizer/cost_model/CostModel.hpp"
-#include "query_optimizer/cost_model/SimpleCostModel.hpp"
#include "query_optimizer/expressions/ExprId.hpp"
#include "query_optimizer/expressions/NamedExpression.hpp"
#include "query_optimizer/expressions/Predicate.hpp"
@@ -423,6 +422,7 @@ class ExecutionGenerator {
* @brief The cost model to use for creating the execution plan.
*/
std::unique_ptr<cost::CostModel> cost_model_;
+ std::unique_ptr<cost::CostModel> star_schema_cost_model_;
physical::TopLevelPlanPtr top_level_physical_plan_;
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/ExecutionHeuristics.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionHeuristics.cpp b/query_optimizer/ExecutionHeuristics.cpp
index 4fd7320..0bef716 100644
--- a/query_optimizer/ExecutionHeuristics.cpp
+++ b/query_optimizer/ExecutionHeuristics.cpp
@@ -27,6 +27,8 @@
#include "catalog/CatalogTypedefs.hpp"
#include "query_execution/QueryContext.pb.h"
#include "query_optimizer/QueryPlan.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/HashJoin.hpp"
#include "utility/Macros.hpp"
#include "glog/logging.h"
@@ -34,95 +36,106 @@
namespace quickstep {
namespace optimizer {
+namespace E = ::quickstep::optimizer::expressions;
+namespace P = ::quickstep::optimizer::physical;
+
+static const std::size_t kNumBitsPerByte = 8;
+DEFINE_double(bloom_num_bits_per_tuple, kNumBitsPerByte,
+ "Number of bits per tuple used to size the Bloom filter.");
+
+DEFINE_int32(bloom_num_hash_fns, 1,
+ "Number of hash functions used in the Bloom filter.");
+
void ExecutionHeuristics::optimizeExecutionPlan(QueryPlan *query_plan,
serialization::QueryContext *query_context_proto) {
- // Currently this only optimizes left deep joins using bloom filters.
- // It uses a simple algorithm to discover the left deep joins.
- // It starts with the first hash join in the plan and keeps on iterating
- // over the next hash joins, till a probe on a different relation id is found.
- // The set of hash joins found in this way forms a chain and can be recognized
- // as a left deep join. It becomes a candidate for optimization.
-
- // The optimization is done by modifying each of the build operators in the chain
- // to generate a bloom filter on the build key during their hash table creation.
- // The leaf-level probe operator is then modified to query all the bloom
- // filters generated from all the build operators in the chain. These
- // bloom filters are queried to test the membership of the probe key
- // just prior to probing the hash table.
-
- QueryPlan::DAGNodeIndex origin_node = 0;
- while (origin_node < hash_joins_.size() - 1) {
- std::vector<std::size_t> chained_nodes;
- chained_nodes.push_back(origin_node);
- for (std::size_t i = origin_node + 1; i < hash_joins_.size(); ++i) {
- const relation_id checked_relation_id = hash_joins_[origin_node].referenced_stored_probe_relation_->getID();
- const relation_id expected_relation_id = hash_joins_[i].referenced_stored_probe_relation_->getID();
- if (checked_relation_id == expected_relation_id) {
- chained_nodes.push_back(i);
- } else {
- break;
- }
+ std::map<std::pair<E::ExprId, P::PhysicalPtr>,
+ std::pair<QueryContext::bloom_filter_id, QueryPlan::DAGNodeIndex>> bloom_filter_map;
+ for (const auto &info : hash_joins_) {
+ auto *hash_table_proto =
+ query_context_proto->mutable_join_hash_tables(info.join_hash_table_id_);
+ const auto &bloom_filter_config = info.bloom_filter_config_;
+
+ for (std::size_t i = 0; i < info.build_side_bloom_filter_ids_.size(); ++i) {
+ const QueryContext::bloom_filter_id bloom_filter_id = query_context_proto->bloom_filters_size();
+ serialization::BloomFilter *bloom_filter_proto = query_context_proto->add_bloom_filters();
+ setBloomFilterProperties(bloom_filter_proto, info.estimated_build_relation_cardinality_);
+
+ const auto &build_side_bf =
+ bloom_filter_config.build_side_bloom_filters[i];
+ bloom_filter_map.emplace(
+ std::make_pair(build_side_bf.attribute->id(),
+ bloom_filter_config.builder),
+ std::make_pair(bloom_filter_id, info.build_operator_index_));
+
+ auto *build_side_bloom_filter = hash_table_proto->add_build_side_bloom_filters();
+ build_side_bloom_filter->set_bloom_filter_id(bloom_filter_id);
+ build_side_bloom_filter->set_attr_id(info.build_side_bloom_filter_ids_[i]);
+ std::cerr << "Build " << build_side_bf.attribute->toString()
+ << " @" << bloom_filter_config.builder << "\n";
}
+ }
- // Only chains of length greater than one are suitable candidates for semi-join optimization.
- if (chained_nodes.size() > 1) {
- std::unordered_map<QueryContext::bloom_filter_id, std::vector<attribute_id>> probe_bloom_filter_info;
- for (const std::size_t node : chained_nodes) {
- // Provision for a new bloom filter to be used by the build operator.
- const QueryContext::bloom_filter_id bloom_filter_id = query_context_proto->bloom_filters_size();
- serialization::BloomFilter *bloom_filter_proto = query_context_proto->add_bloom_filters();
-
- // Modify the bloom filter properties based on the statistics of the relation.
- setBloomFilterProperties(bloom_filter_proto, hash_joins_[node].referenced_stored_build_relation_);
-
- // Add build-side bloom filter information to the corresponding hash table proto.
- query_context_proto->mutable_join_hash_tables(hash_joins_[node].join_hash_table_id_)
- ->add_build_side_bloom_filter_id(bloom_filter_id);
-
- probe_bloom_filter_info.insert(std::make_pair(bloom_filter_id, hash_joins_[node].probe_attributes_));
- }
-
- // Add probe-side bloom filter information to the corresponding hash table proto for each build-side bloom filter.
- for (const std::pair<QueryContext::bloom_filter_id, std::vector<attribute_id>>
- &bloom_filter_info : probe_bloom_filter_info) {
- auto *probe_side_bloom_filter =
- query_context_proto->mutable_join_hash_tables(hash_joins_[origin_node].join_hash_table_id_)
- ->add_probe_side_bloom_filters();
- probe_side_bloom_filter->set_probe_side_bloom_filter_id(bloom_filter_info.first);
- for (const attribute_id &probe_attribute_id : bloom_filter_info.second) {
- probe_side_bloom_filter->add_probe_side_attr_ids(probe_attribute_id);
- }
- }
-
- // Add node dependencies from chained build nodes to origin node probe.
- for (std::size_t i = 1; i < chained_nodes.size(); ++i) { // Note: It starts from index 1.
- query_plan->addDirectDependency(hash_joins_[origin_node].join_operator_index_,
- hash_joins_[origin_node + i].build_operator_index_,
- true /* is_pipeline_breaker */);
- }
+ for (const auto &info : hash_joins_) {
+ auto *hash_table_proto =
+ query_context_proto->mutable_join_hash_tables(info.join_hash_table_id_);
+ const auto &bloom_filter_config = info.bloom_filter_config_;
+
+ for (std::size_t i = 0; i < info.probe_side_bloom_filter_ids_.size(); ++i) {
+ auto *probe_side_bloom_filter = hash_table_proto->add_probe_side_bloom_filters();
+ const auto &probe_side_bf =
+ bloom_filter_config.probe_side_bloom_filters[i];
+ std::cerr << "HashJoin probe " << probe_side_bf.attribute->toString()
+ << " @" << probe_side_bf.builder << "\n";
+
+ const auto &build_side_info =
+ bloom_filter_map.at(
+ std::make_pair(probe_side_bf.source_attribute->id(),
+ probe_side_bf.builder));
+ probe_side_bloom_filter->set_bloom_filter_id(build_side_info.first);
+ probe_side_bloom_filter->set_attr_id(info.probe_side_bloom_filter_ids_[i]);
+// std::cerr << "HashJoin probe attr_id = " << info.probe_side_bloom_filter_ids_[i] << "\n";
+
+ query_plan->addDirectDependency(info.join_operator_index_,
+ build_side_info.second,
+ true /* is_pipeline_breaker */);
}
+ }
- // Update the origin node.
- origin_node = chained_nodes.back() + 1;
+ for (const auto &info : aggregates_) {
+ auto *aggregate_proto =
+ query_context_proto->mutable_aggregation_states(info.aggregate_state_id_);
+ const auto &bloom_filter_config = info.bloom_filter_config_;
+
+ for (std::size_t i = 0; i < info.bloom_filter_ids_.size(); ++i) {
+ auto *bloom_filter = aggregate_proto->add_bloom_filters();
+ const auto &bf =
+ bloom_filter_config.probe_side_bloom_filters[i];
+ std::cerr << "Aggregate probe " << bf.attribute->toString()
+ << " @" << bf.builder << "\n";
+
+ const auto &build_side_info =
+ bloom_filter_map.at(
+ std::make_pair(bf.source_attribute->id(),
+ bf.builder));
+ bloom_filter->set_bloom_filter_id(build_side_info.first);
+ bloom_filter->set_attr_id(info.bloom_filter_ids_[i]);
+// std::cerr << "Aggregate probe attr_id = "
+// << info.bloom_filter_ids_[i] << "\n";
+
+ query_plan->addDirectDependency(info.aggregate_operator_index_,
+ build_side_info.second,
+ true /* is_pipeline_breaker */);
+ }
}
}
void ExecutionHeuristics::setBloomFilterProperties(serialization::BloomFilter *bloom_filter_proto,
- const CatalogRelation *relation) {
- const std::size_t cardinality = relation->estimateTupleCardinality();
- if (cardinality < kOneThousand) {
- bloom_filter_proto->set_bloom_filter_size(kOneThousand / kCompressionFactor);
- bloom_filter_proto->set_number_of_hashes(kVeryLowSparsityHash);
- } else if (cardinality < kTenThousand) {
- bloom_filter_proto->set_bloom_filter_size(kTenThousand / kCompressionFactor);
- bloom_filter_proto->set_number_of_hashes(kLowSparsityHash);
- } else if (cardinality < kHundredThousand) {
- bloom_filter_proto->set_bloom_filter_size(kHundredThousand / kCompressionFactor);
- bloom_filter_proto->set_number_of_hashes(kMediumSparsityHash);
- } else {
- bloom_filter_proto->set_bloom_filter_size(kMillion / kCompressionFactor);
- bloom_filter_proto->set_number_of_hashes(kHighSparsityHash);
- }
+ const std::size_t cardinality) {
+ bloom_filter_proto->set_bloom_filter_size(
+ BloomFilter::getNearestAllowedSize(
+ (FLAGS_bloom_num_bits_per_tuple * cardinality) / kNumBitsPerByte));
+// std::cerr << "bf size = " << bloom_filter_proto->bloom_filter_size() << "\n";
+ bloom_filter_proto->set_number_of_hashes(FLAGS_bloom_num_hash_fns);
}
} // namespace optimizer
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/ExecutionHeuristics.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionHeuristics.hpp b/query_optimizer/ExecutionHeuristics.hpp
index 8ad3b7a..9e5efc5 100644
--- a/query_optimizer/ExecutionHeuristics.hpp
+++ b/query_optimizer/ExecutionHeuristics.hpp
@@ -27,6 +27,7 @@
#include "query_execution/QueryContext.hpp"
#include "query_execution/QueryContext.pb.h"
#include "query_optimizer/QueryPlan.hpp"
+#include "query_optimizer/physical/HashJoin.hpp"
#include "utility/Macros.hpp"
#include "glog/logging.h"
@@ -51,7 +52,7 @@ class ExecutionHeuristics {
static const std::size_t kHundredThousand = 100000;
static const std::size_t kMillion = 1000000;
- static const std::size_t kCompressionFactor = 10;
+ static const std::size_t kCompressionFactor = 1;
static const std::size_t kVeryLowSparsityHash = 1;
static const std::size_t kLowSparsityHash = 2;
@@ -67,25 +68,48 @@ class ExecutionHeuristics {
const QueryPlan::DAGNodeIndex join_operator_index,
const CatalogRelation *referenced_stored_build_relation,
const CatalogRelation *referenced_stored_probe_relation,
- std::vector<attribute_id> &&build_attributes,
- std::vector<attribute_id> &&probe_attributes,
- const QueryContext::join_hash_table_id join_hash_table_id)
+ const physical::BloomFilterConfig &bloom_filter_config,
+ std::vector<attribute_id> &&build_side_bloom_filter_ids,
+ std::vector<attribute_id> &&probe_side_bloom_filter_ids,
+ const QueryContext::join_hash_table_id join_hash_table_id,
+ const std::size_t estimated_build_relation_cardinality)
: build_operator_index_(build_operator_index),
join_operator_index_(join_operator_index),
referenced_stored_build_relation_(referenced_stored_build_relation),
referenced_stored_probe_relation_(referenced_stored_probe_relation),
- build_attributes_(std::move(build_attributes)),
- probe_attributes_(std::move(probe_attributes)),
- join_hash_table_id_(join_hash_table_id) {
+ bloom_filter_config_(bloom_filter_config),
+ build_side_bloom_filter_ids_(std::move(build_side_bloom_filter_ids)),
+ probe_side_bloom_filter_ids_(std::move(probe_side_bloom_filter_ids)),
+ join_hash_table_id_(join_hash_table_id),
+ estimated_build_relation_cardinality_(estimated_build_relation_cardinality) {
}
const QueryPlan::DAGNodeIndex build_operator_index_;
const QueryPlan::DAGNodeIndex join_operator_index_;
const CatalogRelation *referenced_stored_build_relation_;
const CatalogRelation *referenced_stored_probe_relation_;
- const std::vector<attribute_id> build_attributes_;
- const std::vector<attribute_id> probe_attributes_;
+ const physical::BloomFilterConfig &bloom_filter_config_;
+ const std::vector<attribute_id> build_side_bloom_filter_ids_;
+ const std::vector<attribute_id> probe_side_bloom_filter_ids_;
const QueryContext::join_hash_table_id join_hash_table_id_;
+ const std::size_t estimated_build_relation_cardinality_;
+ };
+
+ struct AggregateInfo {
+ AggregateInfo(const QueryPlan::DAGNodeIndex aggregate_operator_index,
+ const physical::BloomFilterConfig &bloom_filter_config,
+ std::vector<attribute_id> &&bloom_filter_ids,
+ const QueryContext::aggregation_state_id aggregate_state_id)
+ : aggregate_operator_index_(aggregate_operator_index),
+ bloom_filter_config_(bloom_filter_config),
+ bloom_filter_ids_(bloom_filter_ids),
+ aggregate_state_id_(aggregate_state_id) {
+ }
+
+ const QueryPlan::DAGNodeIndex aggregate_operator_index_;
+ const physical::BloomFilterConfig &bloom_filter_config_;
+ const std::vector<attribute_id> bloom_filter_ids_;
+ const QueryContext::aggregation_state_id aggregate_state_id_;
};
@@ -111,16 +135,30 @@ class ExecutionHeuristics {
const QueryPlan::DAGNodeIndex join_operator_index,
const CatalogRelation *referenced_stored_build_relation,
const CatalogRelation *referenced_stored_probe_relation,
- std::vector<attribute_id> &&build_attributes,
- std::vector<attribute_id> &&probe_attributes,
- const QueryContext::join_hash_table_id join_hash_table_id) {
- hash_joins_.push_back(HashJoinInfo(build_operator_index,
- join_operator_index,
- referenced_stored_build_relation,
- referenced_stored_probe_relation,
- std::move(build_attributes),
- std::move(probe_attributes),
- join_hash_table_id));
+ const physical::BloomFilterConfig &bloom_filter_config,
+ std::vector<attribute_id> &&build_side_bloom_filter_ids,
+ std::vector<attribute_id> &&probe_side_bloom_filter_ids,
+ const QueryContext::join_hash_table_id join_hash_table_id,
+ const std::size_t estimated_build_relation_cardinality) {
+ hash_joins_.emplace_back(build_operator_index,
+ join_operator_index,
+ referenced_stored_build_relation,
+ referenced_stored_probe_relation,
+ bloom_filter_config,
+ std::move(build_side_bloom_filter_ids),
+ std::move(probe_side_bloom_filter_ids),
+ join_hash_table_id,
+ estimated_build_relation_cardinality);
+ }
+
+ inline void addAggregateInfo(const QueryPlan::DAGNodeIndex aggregate_operator_index,
+ const physical::BloomFilterConfig &bloom_filter_config,
+ std::vector<attribute_id> &&bloom_filter_ids,
+ const QueryContext::aggregation_state_id aggregate_state_id) {
+ aggregates_.emplace_back(aggregate_operator_index,
+ bloom_filter_config,
+ std::move(bloom_filter_ids),
+ aggregate_state_id);
}
/**
@@ -141,10 +179,11 @@ class ExecutionHeuristics {
* @param relation The catalog relation on which bloom filter is being built.
**/
void setBloomFilterProperties(serialization::BloomFilter *bloom_filter_proto,
- const CatalogRelation *relation);
+ const std::size_t cardinality);
private:
std::vector<HashJoinInfo> hash_joins_;
+ std::vector<AggregateInfo> aggregates_;
DISALLOW_COPY_AND_ASSIGN(ExecutionHeuristics);
};
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/PhysicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp
index 8f19702..9ee685d 100644
--- a/query_optimizer/PhysicalGenerator.cpp
+++ b/query_optimizer/PhysicalGenerator.cpp
@@ -26,6 +26,7 @@
#include "query_optimizer/Validator.hpp"
#include "query_optimizer/logical/Logical.hpp"
#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/rules/AttachBloomFilters.hpp"
#include "query_optimizer/rules/PruneColumns.hpp"
#include "query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp"
#include "query_optimizer/rules/SwapProbeBuild.hpp"
@@ -96,10 +97,12 @@ P::PhysicalPtr PhysicalGenerator::generateInitialPlan(
P::PhysicalPtr PhysicalGenerator::optimizePlan() {
std::vector<std::unique_ptr<Rule<P::Physical>>> rules;
if (FLAGS_reorder_hash_joins) {
+ rules.emplace_back(new PruneColumns());
rules.emplace_back(new StarSchemaHashJoinOrderOptimization());
}
rules.emplace_back(new PruneColumns());
- rules.emplace_back(new SwapProbeBuild());
+ // rules.emplace_back(new SwapProbeBuild());
+ rules.emplace_back(new AttachBloomFilters());
for (std::unique_ptr<Rule<P::Physical>> &rule : rules) {
physical_plan_ = rule->apply(physical_plan_);
@@ -110,7 +113,7 @@ P::PhysicalPtr PhysicalGenerator::optimizePlan() {
DVLOG(4) << "Optimized physical plan:\n" << physical_plan_->toString();
if (FLAGS_visualize_plan) {
- quickstep::PlanVisualizer plan_visualizer;
+ quickstep::PlanVisualizer plan_visualizer;
std::cerr << "\n" << plan_visualizer.visualize(physical_plan_) << "\n";
}
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/cost_model/SimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/SimpleCostModel.cpp b/query_optimizer/cost_model/SimpleCostModel.cpp
index f313c90..45e2f00 100644
--- a/query_optimizer/cost_model/SimpleCostModel.cpp
+++ b/query_optimizer/cost_model/SimpleCostModel.cpp
@@ -88,7 +88,7 @@ std::size_t SimpleCostModel::estimateCardinalityForTopLevelPlan(
std::size_t SimpleCostModel::estimateCardinalityForTableReference(
const P::TableReferencePtr &physical_plan) {
- return physical_plan->relation()->estimateTupleCardinality();
+ return physical_plan->relation()->getStatistics().getNumTuples();
}
std::size_t SimpleCostModel::estimateCardinalityForSelection(
@@ -119,7 +119,7 @@ std::size_t SimpleCostModel::estimateCardinalityForAggregate(
return 1;
}
return std::max(static_cast<std::size_t>(1),
- estimateCardinality(physical_plan->input()) / 10);
+ estimateCardinality(physical_plan->input()));
}
std::size_t SimpleCostModel::estimateCardinalityForWindowAggregate(
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
index 911a765..9eea27c 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
@@ -123,12 +123,26 @@ std::size_t StarSchemaSimpleCostModel::estimateCardinalityForTableGenerator(
std::size_t StarSchemaSimpleCostModel::estimateCardinalityForHashJoin(
const P::HashJoinPtr &physical_plan) {
- std::size_t left_cardinality = estimateCardinality(physical_plan->left());
- std::size_t right_cardinality = estimateCardinality(physical_plan->right());
- double left_selectivity = estimateSelectivity(physical_plan->left());
- double right_selectivity = estimateSelectivity(physical_plan->right());
- return std::max(static_cast<std::size_t>(left_cardinality * right_selectivity) + 1,
- static_cast<std::size_t>(right_cardinality * left_selectivity) + 1);
+ const P::PhysicalPtr &left_child = physical_plan->left();
+ const P::PhysicalPtr &right_child = physical_plan->right();
+
+ std::size_t left_cardinality = estimateCardinality(left_child);
+ std::size_t right_cardinality = estimateCardinality(right_child);
+
+ std::size_t estimated_cardinality = std::max(left_cardinality, right_cardinality);
+ if (left_child->impliesUniqueAttributes(physical_plan->left_join_attributes())) {
+ double left_selectivity = estimateSelectivity(left_child);
+ estimated_cardinality =
+ std::min(estimated_cardinality,
+ static_cast<std::size_t>(right_cardinality * left_selectivity));
+ }
+ if (right_child->impliesUniqueAttributes(physical_plan->right_join_attributes())) {
+ double right_selectivity = estimateSelectivity(right_child);
+ estimated_cardinality =
+ std::min(estimated_cardinality,
+ static_cast<std::size_t>(left_cardinality * right_selectivity));
+ }
+ return estimated_cardinality;
}
std::size_t StarSchemaSimpleCostModel::estimateCardinalityForNestedLoopsJoin(
@@ -143,7 +157,7 @@ std::size_t StarSchemaSimpleCostModel::estimateCardinalityForAggregate(
return 1;
}
return std::max(static_cast<std::size_t>(1),
- estimateCardinality(physical_plan->input()) / 10);
+ estimateCardinality(physical_plan->input()) / 100);
}
std::size_t StarSchemaSimpleCostModel::estimateCardinalityForWindowAggregate(
@@ -161,8 +175,14 @@ double StarSchemaSimpleCostModel::estimateSelectivity(
case P::PhysicalType::kHashJoin: {
const P::HashJoinPtr &hash_join =
std::static_pointer_cast<const P::HashJoin>(physical_plan);
- return std::min(estimateSelectivity(hash_join->left()),
- estimateSelectivity(hash_join->right()));
+ double left_selectivity = estimateSelectivity(hash_join->left());
+ double right_selectivity = estimateSelectivity(hash_join->right());
+ double min_sel = std::min(left_selectivity, right_selectivity);
+ double max_sel = std::max(left_selectivity, right_selectivity);
+ if (max_sel < 1) {
+ min_sel *= std::max(max_sel, 0.9);
+ }
+ return min_sel;
}
case P::PhysicalType::kNestedLoopsJoin: {
const P::NestedLoopsJoinPtr &nested_loop_join =
@@ -215,7 +235,7 @@ double StarSchemaSimpleCostModel::estimateSelectivityForPredicate(
case E::ExpressionType::kComparisonExpression: {
// Case 1 - Number of distinct values statistics available
// Case 1.1 - Equality comparison: 1.0 / num_distinct_values
- // Case 1.2 - Otherwise: 5.0 / num_distinct_values
+ // Case 1.2 - Otherwise: 0.5
// Case 2 - Number of distinct values statistics not available
// Case 2.1 - Equality comparison: 0.1
// Case 2.2 - Otherwise: 0.5
@@ -231,7 +251,7 @@ double StarSchemaSimpleCostModel::estimateSelectivityForPredicate(
double unit_selectivity = 1.0 / it->second;
return comparison_expression->isEqualityComparisonPredicate()
? unit_selectivity
- : std::min(0.5, unit_selectivity * 5.0);
+ : 0.5;
}
}
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/expressions/ExpressionUtil.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/ExpressionUtil.hpp b/query_optimizer/expressions/ExpressionUtil.hpp
index e9a4067..16c219e 100644
--- a/query_optimizer/expressions/ExpressionUtil.hpp
+++ b/query_optimizer/expressions/ExpressionUtil.hpp
@@ -103,12 +103,12 @@ bool ContainsExpression(
* contain the other operand).
* @return True if \p left is a subset of \p right.
*/
-template <class NamedExpressionType>
+template <class NamedExpressionType1, class NamedExpressionType2>
bool SubsetOfExpressions(
- const std::vector<std::shared_ptr<const NamedExpressionType>> &left,
- const std::vector<std::shared_ptr<const NamedExpressionType>> &right) {
+ const std::vector<std::shared_ptr<const NamedExpressionType1>> &left,
+ const std::vector<std::shared_ptr<const NamedExpressionType2>> &right) {
UnorderedNamedExpressionSet supset(right.begin(), right.end());
- for (const std::shared_ptr<const NamedExpressionType> &expr : left) {
+ for (const std::shared_ptr<const NamedExpressionType1> &expr : left) {
if (supset.find(expr) == supset.end()) {
return false;
}
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/physical/Aggregate.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Aggregate.cpp b/query_optimizer/physical/Aggregate.cpp
index 35476af..6d07a52 100644
--- a/query_optimizer/physical/Aggregate.cpp
+++ b/query_optimizer/physical/Aggregate.cpp
@@ -87,6 +87,11 @@ std::vector<E::AttributeReferencePtr> Aggregate::getReferencedAttributes()
return referenced_attributes;
}
+bool Aggregate::impliesUniqueAttributes(
+ const std::vector<expressions::AttributeReferencePtr> &attributes) const {
+ return E::SubsetOfExpressions(grouping_expressions_, attributes);
+}
+
void Aggregate::getFieldStringItems(
std::vector<std::string> *inline_field_names,
std::vector<std::string> *inline_field_values,
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/physical/Aggregate.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Aggregate.hpp b/query_optimizer/physical/Aggregate.hpp
index de36cc3..855d783 100644
--- a/query_optimizer/physical/Aggregate.hpp
+++ b/query_optimizer/physical/Aggregate.hpp
@@ -100,6 +100,13 @@ class Aggregate : public Physical {
return false;
}
+ bool impliesUniqueAttributes(
+ const std::vector<expressions::AttributeReferencePtr> &attributes) const override;
+
+ const BloomFilterConfig &bloom_filter_config() const {
+ return bloom_filter_config_;
+ }
+
/**
* @brief Creates an Aggregate physical node.
*
@@ -113,9 +120,14 @@ class Aggregate : public Physical {
PhysicalPtr input,
const std::vector<expressions::NamedExpressionPtr> &grouping_expressions,
const std::vector<expressions::AliasPtr> &aggregate_expressions,
- const expressions::PredicatePtr &filter_predicate) {
+ const expressions::PredicatePtr &filter_predicate,
+ const BloomFilterConfig bloom_filter_config = BloomFilterConfig()) {
return AggregatePtr(
- new Aggregate(input, grouping_expressions, aggregate_expressions, filter_predicate));
+ new Aggregate(input,
+ grouping_expressions,
+ aggregate_expressions,
+ filter_predicate,
+ bloom_filter_config));
}
protected:
@@ -132,11 +144,13 @@ class Aggregate : public Physical {
PhysicalPtr input,
const std::vector<expressions::NamedExpressionPtr> &grouping_expressions,
const std::vector<expressions::AliasPtr> &aggregate_expressions,
- const expressions::PredicatePtr &filter_predicate)
+ const expressions::PredicatePtr &filter_predicate,
+ const BloomFilterConfig &bloom_filter_config)
: input_(input),
grouping_expressions_(grouping_expressions),
aggregate_expressions_(aggregate_expressions),
- filter_predicate_(filter_predicate) {
+ filter_predicate_(filter_predicate),
+ bloom_filter_config_(bloom_filter_config) {
addChild(input_);
}
@@ -144,6 +158,7 @@ class Aggregate : public Physical {
std::vector<expressions::NamedExpressionPtr> grouping_expressions_;
std::vector<expressions::AliasPtr> aggregate_expressions_;
expressions::PredicatePtr filter_predicate_;
+ BloomFilterConfig bloom_filter_config_;
DISALLOW_COPY_AND_ASSIGN(Aggregate);
};
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/physical/HashJoin.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/HashJoin.cpp b/query_optimizer/physical/HashJoin.cpp
index e186072..883c87a 100644
--- a/query_optimizer/physical/HashJoin.cpp
+++ b/query_optimizer/physical/HashJoin.cpp
@@ -85,6 +85,15 @@ bool HashJoin::maybeCopyWithPrunedExpressions(
return false;
}
+bool HashJoin::impliesUniqueAttributes(
+ const std::vector<expressions::AttributeReferencePtr> &attributes) const {
+ return (left()->impliesUniqueAttributes(left_join_attributes_)
+ && right()->impliesUniqueAttributes(attributes))
+ || (right()->impliesUniqueAttributes(right_join_attributes_)
+ && left()->impliesUniqueAttributes(attributes));
+
+}
+
void HashJoin::getFieldStringItems(
std::vector<std::string> *inline_field_names,
std::vector<std::string> *inline_field_values,
@@ -106,6 +115,24 @@ void HashJoin::getFieldStringItems(
container_child_fields->push_back(CastSharedPtrVector<OptimizerTreeBase>(left_join_attributes_));
container_child_field_names->push_back("right_join_attributes");
container_child_fields->push_back(CastSharedPtrVector<OptimizerTreeBase>(right_join_attributes_));
+
+ if (!bloom_filter_config_.build_side_bloom_filters.empty()) {
+ container_child_field_names->push_back("build_side_bloom_filters");
+ container_child_fields->emplace_back();
+ auto &container = container_child_fields->back();
+ for (const auto& bf : bloom_filter_config_.build_side_bloom_filters) {
+ container.emplace_back(bf.attribute);
+ }
+ }
+
+ if (!bloom_filter_config_.probe_side_bloom_filters.empty()) {
+ container_child_field_names->push_back("probe_side_bloom_filters");
+ container_child_fields->emplace_back();
+ auto &container = container_child_fields->back();
+ for (const auto& bf : bloom_filter_config_.probe_side_bloom_filters) {
+ container.emplace_back(bf.attribute);
+ }
+ }
}
} // namespace physical
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/physical/HashJoin.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/HashJoin.hpp b/query_optimizer/physical/HashJoin.hpp
index c513f77..a830d0b 100644
--- a/query_optimizer/physical/HashJoin.hpp
+++ b/query_optimizer/physical/HashJoin.hpp
@@ -116,7 +116,8 @@ class HashJoin : public BinaryJoin {
right_join_attributes_,
residual_predicate_,
project_expressions(),
- join_type_);
+ join_type_,
+ bloom_filter_config_);
}
std::vector<expressions::AttributeReferencePtr> getReferencedAttributes() const override;
@@ -125,6 +126,13 @@ class HashJoin : public BinaryJoin {
const expressions::UnorderedNamedExpressionSet &referenced_expressions,
PhysicalPtr *output) const override;
+ bool impliesUniqueAttributes(
+ const std::vector<expressions::AttributeReferencePtr> &attributes) const override;
+
+ const BloomFilterConfig &bloom_filter_config() const {
+ return bloom_filter_config_;
+ }
+
/**
* @brief Creates a physical HashJoin. The left/right operand does not correspond to
* probe/build operand.
@@ -145,7 +153,8 @@ class HashJoin : public BinaryJoin {
const std::vector<expressions::AttributeReferencePtr> &right_join_attributes,
const expressions::PredicatePtr &residual_predicate,
const std::vector<expressions::NamedExpressionPtr> &project_expressions,
- const JoinType join_type) {
+ const JoinType join_type,
+ const BloomFilterConfig bloom_filter_config = BloomFilterConfig()) {
return HashJoinPtr(
new HashJoin(left,
right,
@@ -153,7 +162,8 @@ class HashJoin : public BinaryJoin {
right_join_attributes,
residual_predicate,
project_expressions,
- join_type));
+ join_type,
+ bloom_filter_config));
}
protected:
@@ -173,18 +183,21 @@ class HashJoin : public BinaryJoin {
const std::vector<expressions::AttributeReferencePtr> &right_join_attributes,
const expressions::PredicatePtr &residual_predicate,
const std::vector<expressions::NamedExpressionPtr> &project_expressions,
- const JoinType join_type)
+ const JoinType join_type,
+ const BloomFilterConfig &bloom_filter_config)
: BinaryJoin(left, right, project_expressions),
left_join_attributes_(left_join_attributes),
right_join_attributes_(right_join_attributes),
residual_predicate_(residual_predicate),
- join_type_(join_type) {
+ join_type_(join_type),
+ bloom_filter_config_(bloom_filter_config) {
}
std::vector<expressions::AttributeReferencePtr> left_join_attributes_;
std::vector<expressions::AttributeReferencePtr> right_join_attributes_;
expressions::PredicatePtr residual_predicate_;
JoinType join_type_;
+ BloomFilterConfig bloom_filter_config_;
DISALLOW_COPY_AND_ASSIGN(HashJoin);
};
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/physical/Physical.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Physical.hpp b/query_optimizer/physical/Physical.hpp
index 4bed593..f70d142 100644
--- a/query_optimizer/physical/Physical.hpp
+++ b/query_optimizer/physical/Physical.hpp
@@ -41,6 +41,56 @@ namespace physical {
class Physical;
typedef std::shared_ptr<const Physical> PhysicalPtr;
+struct BloomFilterConfig {
+ struct BuildSide {
+ BuildSide(const expressions::AttributeReferencePtr &attribute_in)
+ : attribute(attribute_in) {
+ }
+ expressions::AttributeReferencePtr attribute;
+ };
+ struct ProbeSide {
+ ProbeSide(const expressions::AttributeReferencePtr &attribute_in,
+ const expressions::AttributeReferencePtr &source_attribute_in,
+ const physical::PhysicalPtr &builder_in)
+ : attribute(attribute_in),
+ source_attribute(source_attribute_in),
+ builder(builder_in) {
+ }
+ expressions::AttributeReferencePtr attribute;
+ expressions::AttributeReferencePtr source_attribute;
+ PhysicalPtr builder;
+ };
+ BloomFilterConfig() {}
+ BloomFilterConfig(const PhysicalPtr &builder_in)
+ : builder(builder_in) {
+ }
+ BloomFilterConfig(const PhysicalPtr &builder_in,
+ const std::vector<BuildSide> &build_side_bloom_filters_in,
+ const std::vector<ProbeSide> &probe_side_bloom_filters_in)
+ : builder(builder_in),
+ build_side_bloom_filters(build_side_bloom_filters_in),
+ probe_side_bloom_filters(probe_side_bloom_filters_in) {
+ }
+ void addBuildSideBloomFilter(const expressions::AttributeReferencePtr &attribute_in) {
+ for (const auto &build_bf : build_side_bloom_filters) {
+ if (attribute_in == build_bf.attribute) {
+ return;
+ }
+ }
+ build_side_bloom_filters.emplace_back(attribute_in);
+ }
+ void addProbeSideBloomFilter(const expressions::AttributeReferencePtr &attribute_in,
+ const expressions::AttributeReferencePtr &source_attribute_in,
+ const physical::PhysicalPtr &builder_in) {
+ probe_side_bloom_filters.emplace_back(attribute_in,
+ source_attribute_in,
+ builder_in);
+ }
+ PhysicalPtr builder;
+ std::vector<BuildSide> build_side_bloom_filters;
+ std::vector<ProbeSide> probe_side_bloom_filters;
+};
+
/**
* @brief Base class for physical plan nodes.
*/
@@ -86,6 +136,11 @@ class Physical : public OptimizerTree<Physical> {
const expressions::UnorderedNamedExpressionSet &referenced_expressions,
PhysicalPtr *output) const = 0;
+ virtual bool impliesUniqueAttributes(
+ const std::vector<expressions::AttributeReferencePtr> &attributes) const {
+ return false;
+ }
+
protected:
/**
* @brief Constructor.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/physical/Selection.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Selection.cpp b/query_optimizer/physical/Selection.cpp
index 36ade04..73af500 100644
--- a/query_optimizer/physical/Selection.cpp
+++ b/query_optimizer/physical/Selection.cpp
@@ -82,6 +82,12 @@ bool Selection::maybeCopyWithPrunedExpressions(
return false;
}
+bool Selection::impliesUniqueAttributes(
+ const std::vector<expressions::AttributeReferencePtr> &attributes) const {
+ return input()->impliesUniqueAttributes(attributes);
+}
+
+
void Selection::getFieldStringItems(
std::vector<std::string> *inline_field_names,
std::vector<std::string> *inline_field_values,
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/physical/Selection.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Selection.hpp b/query_optimizer/physical/Selection.hpp
index b6874a1..f42fc71 100644
--- a/query_optimizer/physical/Selection.hpp
+++ b/query_optimizer/physical/Selection.hpp
@@ -86,6 +86,9 @@ class Selection : public Physical {
const expressions::UnorderedNamedExpressionSet &referenced_attributes,
PhysicalPtr *output) const override;
+ bool impliesUniqueAttributes(
+ const std::vector<expressions::AttributeReferencePtr> &attributes) const override;
+
/**
* @brief Creates a Selection.
*
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/physical/TableReference.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/TableReference.cpp b/query_optimizer/physical/TableReference.cpp
index bfd6464..399ee51 100644
--- a/query_optimizer/physical/TableReference.cpp
+++ b/query_optimizer/physical/TableReference.cpp
@@ -20,6 +20,7 @@
#include "query_optimizer/physical/TableReference.hpp"
#include <string>
+#include <set>
#include <vector>
#include "catalog/CatalogRelation.hpp"
@@ -32,6 +33,23 @@ namespace physical {
namespace E = ::quickstep::optimizer::expressions;
+bool TableReference::impliesUniqueAttributes(
+ const std::vector<expressions::AttributeReferencePtr> &attributes) const {
+ std::set<E::ExprId> attr_ids;
+ for (const auto &attr : attributes) {
+ attr_ids.emplace(attr->id());
+ }
+
+ std::set<attribute_id> rel_attr_ids;
+ for (std::size_t i = 0; i < attribute_list_.size(); ++i) {
+ if (attr_ids.find(attribute_list_[i]->id()) != attr_ids.end()) {
+ rel_attr_ids.emplace(i);
+ }
+ }
+
+ return relation_->getConstraints().impliesUniqueAttributes(rel_attr_ids);
+}
+
void TableReference::getFieldStringItems(
std::vector<std::string> *inline_field_names,
std::vector<std::string> *inline_field_values,
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/physical/TableReference.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/TableReference.hpp b/query_optimizer/physical/TableReference.hpp
index 638d73b..7643c07 100644
--- a/query_optimizer/physical/TableReference.hpp
+++ b/query_optimizer/physical/TableReference.hpp
@@ -90,6 +90,9 @@ class TableReference : public Physical {
return false;
}
+ bool impliesUniqueAttributes(
+ const std::vector<expressions::AttributeReferencePtr> &attributes) const override;
+
/**
* @brief Creates a TableReference.
*
[16/16] incubator-quickstep git commit: Initial work on better
estimation
Posted by ji...@apache.org.
Initial work on better estimation
Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/39f57499
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/39f57499
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/39f57499
Branch: refs/heads/LIP-for-tpch
Commit: 39f574999a280e0fbcb1792db84f584deca20675
Parents: 9b90665
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Thu Aug 11 15:20:53 2016 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Thu Aug 11 15:23:00 2016 -0500
----------------------------------------------------------------------
cli/QuickstepCli.cpp | 8 ++---
.../cost_model/StarSchemaSimpleCostModel.cpp | 36 ++++++++++++++++++++
.../cost_model/StarSchemaSimpleCostModel.hpp | 10 ++++++
utility/PlanVisualizer.cpp | 25 ++++++++++++--
4 files changed, 73 insertions(+), 6 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/39f57499/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index bf0e2c7..e7ffc0c 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -347,10 +347,10 @@ int main(int argc, char* argv[]) {
// addPrimaryKeyInfoForTPCHTables(query_processor->getDefaultDatabase());
// addPrimaryKeyInfoForSSBTables(query_processor->getDefaultDatabase());
- std::string proto_str;
- google::protobuf::TextFormat::PrintToString(
- query_processor->getDefaultDatabase()->getProto(), &proto_str);
- std::cerr << proto_str << "\n";
+// std::string proto_str;
+// google::protobuf::TextFormat::PrintToString(
+// query_processor->getDefaultDatabase()->getProto(), &proto_str);
+// std::cerr << proto_str << "\n";
// query_processor->markCatalogAltered();
// query_processor->saveCatalog();
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/39f57499/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
index 9eea27c..ba7a3c6 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
@@ -25,6 +25,8 @@
#include <vector>
#include "catalog/CatalogRelation.hpp"
+#include "catalog/CatalogRelationStatistics.hpp"
+#include "catalog/CatalogTypedefs.hpp"
#include "query_optimizer/expressions/AttributeReference.hpp"
#include "query_optimizer/expressions/ComparisonExpression.hpp"
#include "query_optimizer/expressions/ExprId.hpp"
@@ -283,6 +285,40 @@ double StarSchemaSimpleCostModel::estimateSelectivityForPredicate(
return 1.0;
}
+
+void StarSchemaSimpleCostModel::getStatistics(
+ const physical::PhysicalPtr &physical_plan,
+ const expressions::AttributeReferencePtr &attribute,
+ const CatalogRelationStatistics** stat,
+ attribute_id* attr_id) {
+ switch (physical_plan->getPhysicalType()) {
+ case P::PhysicalType::kTableReference: {
+ const P::TableReferencePtr table_reference =
+ std::static_pointer_cast<const P::TableReference>(physical_plan);
+ const CatalogRelation *catalog_relation = table_reference->relation();
+ const std::vector<E::AttributeReferencePtr> &attributes =
+ table_reference->attribute_list();
+ for (std::size_t i = 0; i < attributes.size(); ++i) {
+ if (attributes[i]->id() == attribute->id()) {
+ *stat = &catalog_relation->getStatistics();
+ *attr_id = i;
+ }
+ }
+ }
+ default:
+ break;
+ }
+
+ for (const auto &child : physical_plan->children()) {
+ for (const auto &attr : child->getOutputAttributes()) {
+ if (attr->id() == attribute->id()) {
+ getStatistics(child, attribute, stat, attr_id);
+ }
+ }
+ }
+}
+
+
} // namespace cost
} // namespace optimizer
} // namespace quickstep
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/39f57499/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
index 4314b92..fdd2b36 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
@@ -24,7 +24,9 @@
#include <unordered_map>
#include <vector>
+#include "catalog/CatalogTypedefs.hpp"
#include "query_optimizer/cost_model/CostModel.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
#include "query_optimizer/expressions/ExprId.hpp"
#include "query_optimizer/expressions/Predicate.hpp"
#include "query_optimizer/physical/Aggregate.hpp"
@@ -39,6 +41,9 @@
#include "utility/Macros.hpp"
namespace quickstep {
+
+class CatalogRelationStatistics;
+
namespace optimizer {
namespace cost {
@@ -75,6 +80,11 @@ class StarSchemaSimpleCostModel : public CostModel {
*/
double estimateSelectivity(const physical::PhysicalPtr &physical_plan);
+ void getStatistics(const physical::PhysicalPtr &physical_plan,
+ const expressions::AttributeReferencePtr &attribute,
+ const CatalogRelationStatistics** stat,
+ attribute_id* attr_id);
+
private:
std::size_t estimateCardinalityForTopLevelPlan(
const physical::TopLevelPlanPtr &physical_plan);
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/39f57499/utility/PlanVisualizer.cpp
----------------------------------------------------------------------
diff --git a/utility/PlanVisualizer.cpp b/utility/PlanVisualizer.cpp
index 9af00b4..2039a69 100644
--- a/utility/PlanVisualizer.cpp
+++ b/utility/PlanVisualizer.cpp
@@ -28,7 +28,8 @@
#include <vector>
#include "catalog/CatalogRelation.hpp"
-
+#include "catalog/CatalogRelationStatistics.hpp"
+#include "catalog/CatalogTypedefs.hpp"
#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
#include "query_optimizer/expressions/AttributeReference.hpp"
#include "query_optimizer/physical/Aggregate.hpp"
@@ -131,7 +132,27 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
for (const auto &attr : child->getOutputAttributes()) {
if (referenced_ids.find(attr->id()) != referenced_ids.end()) {
- edge_info.labels.emplace_back(attr->attribute_alias());
+ std::string attr_info = attr->attribute_alias();
+ const CatalogRelationStatistics *stat = nullptr;
+ attribute_id attr_id = 0;
+ cost_model_->getStatistics(child, attr, &stat, &attr_id);
+ if (stat != nullptr) {
+ if (stat->hasNumDistinctValues(attr_id)) {
+ attr_info.append(", # distinct = ");
+ attr_info.append(std::to_string(static_cast<std::size_t>(
+ stat->getNumDistinctValues(attr_id) * cost_model_->estimateSelectivity(child))));
+ }
+ const Type& attr_type = attr->getValueType();
+ if (stat->hasMinValue(attr_id)) {
+ attr_info.append(", min = ");
+ attr_info.append(attr_type.printValueToString(stat->getMinValue(attr_id)));
+ }
+ if (stat->hasMaxValue(attr_id)) {
+ attr_info.append(", max = ");
+ attr_info.append(attr_type.printValueToString(stat->getMaxValue(attr_id)));
+ }
+ }
+ edge_info.labels.emplace_back(attr_info);
}
}
}
[04/16] incubator-quickstep git commit: Deserialized Window Aggr
WorkOrder.
Posted by ji...@apache.org.
Deserialized Window Aggr WorkOrder.
Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/85e02de4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/85e02de4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/85e02de4
Branch: refs/heads/LIP-for-tpch
Commit: 85e02de49205409accfef3737dadfe95aad1f5c0
Parents: 658cb61
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Mon Aug 8 23:14:08 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Mon Aug 8 23:14:08 2016 -0700
----------------------------------------------------------------------
relational_operators/CMakeLists.txt | 1 +
relational_operators/WorkOrderFactory.cpp | 25 +++++++++++++++++++++++++
2 files changed, 26 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/85e02de4/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index 9696392..43a42f9 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -467,6 +467,7 @@ target_link_libraries(quickstep_relationaloperators_WorkOrderFactory
quickstep_relationaloperators_TableGeneratorOperator
quickstep_relationaloperators_TextScanOperator
quickstep_relationaloperators_UpdateOperator
+ quickstep_relationaloperators_WindowAggregationOperator
quickstep_relationaloperators_WorkOrder_proto
quickstep_storage_StorageBlockInfo
quickstep_utility_Macros
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/85e02de4/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index f920cac..721d735 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -46,6 +46,7 @@
#include "relational_operators/TableGeneratorOperator.hpp"
#include "relational_operators/TextScanOperator.hpp"
#include "relational_operators/UpdateOperator.hpp"
+#include "relational_operators/WindowAggregationOperator.hpp"
#include "relational_operators/WorkOrder.pb.h"
#include "storage/StorageBlockInfo.hpp"
@@ -419,6 +420,22 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
shiftboss_client_id,
bus);
}
+ case serialization::WINDOW_AGGREGATION: {
+ LOG(INFO) << "Creating WindowAggregationWorkOrder";
+ vector<block_id> blocks;
+ for (int i = 0; i < proto.ExtensionSize(serialization::WindowAggregationWorkOrder::block_ids); ++i) {
+ blocks.push_back(
+ proto.GetExtension(serialization::WindowAggregationWorkOrder::block_ids, i));
+ }
+
+ return new WindowAggregationWorkOrder(
+ proto.query_id(),
+ query_context->getWindowAggregationState(
+ proto.GetExtension(serialization::WindowAggregationWorkOrder::window_aggr_state_index)),
+ move(blocks),
+ query_context->getInsertDestination(
+ proto.GetExtension(serialization::FinalizeAggregationWorkOrder::insert_destination_index)));
+ }
default:
LOG(FATAL) << "Unknown WorkOrder Type in WorkOrderFactory::ReconstructFromProto";
}
@@ -697,6 +714,14 @@ bool WorkOrderFactory::ProtoIsValid(const serialization::WorkOrder &proto,
proto.HasExtension(serialization::UpdateWorkOrder::operator_index) &&
proto.HasExtension(serialization::UpdateWorkOrder::block_id);
}
+ case serialization::WINDOW_AGGREGATION: {
+ return proto.HasExtension(serialization::WindowAggregationWorkOrder::window_aggr_state_index) &&
+ query_context.isValidWindowAggregationStateId(
+ proto.GetExtension(serialization::WindowAggregationWorkOrder::window_aggr_state_index)) &&
+ proto.HasExtension(serialization::WindowAggregationWorkOrder::insert_destination_index) &&
+ query_context.isValidInsertDestinationId(
+ proto.GetExtension(serialization::WindowAggregationWorkOrder::insert_destination_index));
+ }
default:
return false;
}
[14/16] incubator-quickstep git commit: MinMaxStatistics
Posted by ji...@apache.org.
MinMaxStatistics
Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/9b906657
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/9b906657
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/9b906657
Branch: refs/heads/LIP-for-tpch
Commit: 9b90665796962baa9a80ef5240db99d6b38a9016
Parents: ebdc041
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Thu Aug 11 03:21:30 2016 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Thu Aug 11 15:23:00 2016 -0500
----------------------------------------------------------------------
catalog/Catalog.proto | 10 +--
catalog/CatalogRelationStatistics.cpp | 50 ++++++++++++---
catalog/CatalogRelationStatistics.hpp | 93 +++++++++++++++++++++++-----
cli/CommandExecutor.cpp | 98 ++++++++++++++++++++++--------
cli/QuickstepCli.cpp | 8 +--
5 files changed, 201 insertions(+), 58 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9b906657/catalog/Catalog.proto
----------------------------------------------------------------------
diff --git a/catalog/Catalog.proto b/catalog/Catalog.proto
index a4cc3ce..6423a7c 100644
--- a/catalog/Catalog.proto
+++ b/catalog/Catalog.proto
@@ -85,13 +85,15 @@ message CatalogRelationConstraints {
}
message CatalogRelationStatistics {
- optional fixed64 num_tuples = 1;
+ optional TypedValue num_tuples = 1;
- message NumDistinctValuesEntry {
+ message ColumnStats {
required int32 attr_id = 1;
- required fixed64 num_distinct_values = 2;
+ optional TypedValue num_distinct_values = 2;
+ optional TypedValue min_value = 3;
+ optional TypedValue max_value = 4;
}
- repeated NumDistinctValuesEntry num_distinct_values_map = 2;
+ repeated ColumnStats column_stats = 2;
}
message CatalogRelationSchema {
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9b906657/catalog/CatalogRelationStatistics.cpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationStatistics.cpp b/catalog/CatalogRelationStatistics.cpp
index 6a51570..04939a5 100644
--- a/catalog/CatalogRelationStatistics.cpp
+++ b/catalog/CatalogRelationStatistics.cpp
@@ -20,30 +20,60 @@
#include "catalog/CatalogRelationStatistics.hpp"
#include "catalog/Catalog.pb.h"
+#include "types/NullType.hpp"
namespace quickstep {
+const TypedValue CatalogRelationStatistics::kNullValue =
+ NullType::InstanceNullable().makeNullValue();
+
CatalogRelationStatistics::CatalogRelationStatistics(
const serialization::CatalogRelationStatistics &proto) {
if (proto.has_num_tuples()) {
- num_tuples_ = proto.num_tuples();
+ num_tuples_ = TypedValue::ReconstructFromProto(proto.num_tuples());
+ } else {
+ num_tuples_ = kNullValue;
}
- for (int i = 0; i < proto.num_distinct_values_map_size(); ++i) {
- const auto &entry = proto.num_distinct_values_map(i);
- num_distinct_values_map_.emplace(entry.attr_id(),
- entry.num_distinct_values());
+
+ for (int i = 0; i < proto.column_stats_size(); ++i) {
+ const auto &stat_proto = proto.column_stats(i);
+ auto &stat = column_stats_[stat_proto.attr_id()];
+ if (stat_proto.has_num_distinct_values()) {
+ stat.num_distinct_values =
+ TypedValue::ReconstructFromProto(stat_proto.num_distinct_values());
+ }
+ if (stat_proto.has_min_value()) {
+ stat.min_value =
+ TypedValue::ReconstructFromProto(stat_proto.min_value());
+ }
+ if (stat_proto.has_max_value()) {
+ stat.max_value =
+ TypedValue::ReconstructFromProto(stat_proto.max_value());
+ }
}
}
serialization::CatalogRelationStatistics CatalogRelationStatistics::getProto() const {
serialization::CatalogRelationStatistics proto;
- if (num_tuples_ != 0) {
- proto.set_num_tuples(num_tuples_);
+ if (!num_tuples_.isNull()) {
+ proto.mutable_num_tuples()->CopyFrom(num_tuples_.getProto());
}
- for (const auto &pair : num_distinct_values_map_) {
- auto entry = proto.add_num_distinct_values_map();
+ for (const auto &pair : column_stats_) {
+ auto entry = proto.add_column_stats();
entry->set_attr_id(pair.first);
- entry->set_num_distinct_values(pair.second);
+ const auto &stat = pair.second;
+ if (!stat.num_distinct_values.isNull()) {
+ entry->mutable_num_distinct_values()->CopyFrom(
+ stat.num_distinct_values.getProto());
+ }
+ if (!stat.min_value.isNull()) {
+ entry->mutable_min_value()->CopyFrom(
+ stat.min_value.getProto());
+ }
+ if (!stat.max_value.isNull()) {
+ entry->mutable_max_value()->CopyFrom(
+ stat.max_value.getProto());
+ }
}
return proto;
}
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9b906657/catalog/CatalogRelationStatistics.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationStatistics.hpp b/catalog/CatalogRelationStatistics.hpp
index f2056f3..532ea46 100644
--- a/catalog/CatalogRelationStatistics.hpp
+++ b/catalog/CatalogRelationStatistics.hpp
@@ -26,8 +26,13 @@
#include "catalog/Catalog.pb.h"
#include "catalog/CatalogTypedefs.hpp"
+#include "types/LongType.hpp"
+#include "types/NullType.hpp"
+#include "types/TypedValue.hpp"
#include "utility/Macros.hpp"
+#include "glog/logging.h"
+
namespace quickstep {
/** \addtogroup Catalog
@@ -44,7 +49,7 @@ class CatalogRelationStatistics {
* @brief Constructor.
**/
CatalogRelationStatistics()
- : num_tuples_(0) {}
+ : num_tuples_(kNullValue) {}
/**
* @brief Reconstruct a CatalogRelationStatistics object from its serialized
@@ -68,8 +73,12 @@ class CatalogRelationStatistics {
*
* @param num_tuples The number of tuples statistic.
*/
- void setNumTuples(std::size_t num_tuples) {
- num_tuples_ = num_tuples;
+ void setNumTuples(const std::size_t num_tuples) {
+ num_tuples_ = LongType::InstanceNonNullable().makeValue(&num_tuples);
+ }
+
+ bool hasNumTuples() const {
+ return !num_tuples_.isNull();
}
/**
@@ -78,7 +87,8 @@ class CatalogRelationStatistics {
* @return The number of tuples. Returns 0 if the statistic is not set.
*/
std::size_t getNumTuples() const {
- return num_tuples_;
+ DCHECK(hasNumTuples());
+ return num_tuples_.getLiteral<std::int64_t>();
}
/**
@@ -87,8 +97,15 @@ class CatalogRelationStatistics {
* @param attr_id The id of the column.
* @param num_distinct_values The number of distinct values statistic.
*/
- void setNumDistinctValues(attribute_id attr_id, std::size_t num_distinct_values) {
- num_distinct_values_map_[attr_id] = num_distinct_values;
+ void setNumDistinctValues(const attribute_id attr_id,
+ const std::size_t num_distinct_values) {
+ column_stats_[attr_id].num_distinct_values =
+ LongType::InstanceNonNullable().makeValue(&num_distinct_values);
+ }
+
+ bool hasNumDistinctValues(const attribute_id attr_id) const {
+ const auto stats = getColumnStats(attr_id);
+ return (stats != nullptr && !stats->num_distinct_values.isNull());
}
/**
@@ -98,21 +115,65 @@ class CatalogRelationStatistics {
* @return The number of distinct values statistic for the column. Returns 0
* if the statistic is not set.
*/
- std::size_t getNumDistinctValues(attribute_id attr_id) const {
- const auto it = num_distinct_values_map_.find(attr_id);
- if (it == num_distinct_values_map_.end()) {
- return static_cast<std::size_t>(0);
- } else {
- return it->second;
- }
+ std::size_t getNumDistinctValues(const attribute_id attr_id) const {
+ DCHECK(hasNumDistinctValues(attr_id));
+ return column_stats_.at(attr_id).num_distinct_values.getLiteral<std::int64_t>();
+ }
+
+ void setMinValue(const attribute_id attr_id,
+ const TypedValue &min_value) {
+ column_stats_[attr_id].min_value = min_value;
+ }
+
+ bool hasMinValue(const attribute_id attr_id) const {
+ const auto stats = getColumnStats(attr_id);
+ return (stats != nullptr && !stats->min_value.isNull());
+ }
+
+ const TypedValue& getMinValue(const attribute_id attr_id) const {
+ DCHECK(hasMinValue(attr_id));
+ return column_stats_.at(attr_id).min_value;
+ }
+
+ void setMaxValue(const attribute_id attr_id,
+ const TypedValue &max_value) {
+ column_stats_[attr_id].max_value = max_value;
+ }
+
+ bool hasMaxValue(const attribute_id attr_id) const {
+ const auto stats = getColumnStats(attr_id);
+ return (stats != nullptr && !stats->max_value.isNull());
+ }
+
+ const TypedValue& getMaxValue(const attribute_id attr_id) const {
+ DCHECK(hasMaxValue(attr_id));
+ return column_stats_.at(attr_id).max_value;
}
private:
+ struct ColumnStats {
+ ColumnStats()
+ : num_distinct_values(kNullValue),
+ min_value(kNullValue),
+ max_value(kNullValue) {
+ }
+ TypedValue num_distinct_values;
+ TypedValue min_value;
+ TypedValue max_value;
+ };
+
+ inline const ColumnStats* getColumnStats(const attribute_id attr_id) const {
+ const auto it = column_stats_.find(attr_id);
+ return (it == column_stats_.end() ? nullptr : &it->second);
+ }
+
+ static const TypedValue kNullValue;
+
// Total number of tuples in the relation.
- std::size_t num_tuples_;
+ TypedValue num_tuples_;
- // Number of distinct values for each column.
- std::unordered_map<attribute_id, std::size_t> num_distinct_values_map_;
+ // Statistics for each column.
+ std::unordered_map<attribute_id, ColumnStats> column_stats_;
DISALLOW_COPY_AND_ASSIGN(CatalogRelationStatistics);
};
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9b906657/cli/CommandExecutor.cpp
----------------------------------------------------------------------
diff --git a/cli/CommandExecutor.cpp b/cli/CommandExecutor.cpp
index f27f1ef..feafec1 100644
--- a/cli/CommandExecutor.cpp
+++ b/cli/CommandExecutor.cpp
@@ -196,10 +196,7 @@ void executeDescribeTable(
}
}
-/**
- * @brief A helper function that executes a SQL query to obtain a scalar result.
- */
-inline TypedValue executeQueryForSingleResult(
+inline std::vector<TypedValue> executeQueryForSingleRow(
const tmb::client_id main_thread_client_id,
const tmb::client_id foreman_client_id,
const std::string &query_string,
@@ -232,22 +229,29 @@ inline TypedValue executeQueryForSingleResult(
const CatalogRelation *query_result_relation = query_handle->getQueryResultRelation();
DCHECK(query_result_relation != nullptr);
- TypedValue value;
+ std::vector<TypedValue> values;
{
std::vector<block_id> blocks = query_result_relation->getBlocksSnapshot();
DCHECK_EQ(1u, blocks.size());
+
BlockReference block = storage_manager->getBlock(blocks[0], *query_result_relation);
const TupleStorageSubBlock &tuple_store = block->getTupleStorageSubBlock();
DCHECK_EQ(1, tuple_store.numTuples());
- DCHECK_EQ(1u, tuple_store.getRelation().size());
+ const std::size_t num_columns = tuple_store.getRelation().size();
if (tuple_store.isPacked()) {
- value = tuple_store.getAttributeValueTyped(0, 0);
+ for (std::size_t i = 0; i < num_columns; ++i) {
+ values.emplace_back(tuple_store.getAttributeValueTyped(0, i));
+ values[i].ensureNotReference();
+ }
} else {
std::unique_ptr<TupleIdSequence> existence_map(tuple_store.getExistenceMap());
- value = tuple_store.getAttributeValueTyped(*existence_map->begin(), 0);
+ for (std::size_t i = 0; i < num_columns; ++i) {
+ values.emplace_back(
+ tuple_store.getAttributeValueTyped(*existence_map->begin(), i));
+ values[i].ensureNotReference();
+ }
}
- value.ensureNotReference();
}
// Drop the result relation.
@@ -255,7 +259,31 @@ inline TypedValue executeQueryForSingleResult(
query_processor->getDefaultDatabase(),
query_processor->getStorageManager());
- return value;
+ return values;
+}
+
+/**
+ * @brief A helper function that executes a SQL query to obtain a scalar result.
+ */
+inline TypedValue executeQueryForSingleResult(
+ const tmb::client_id main_thread_client_id,
+ const tmb::client_id foreman_client_id,
+ const std::string &query_string,
+ tmb::MessageBus *bus,
+ StorageManager *storage_manager,
+ QueryProcessor *query_processor,
+ SqlParserWrapper *parser_wrapper) {
+ std::vector<TypedValue> results =
+ executeQueryForSingleRow(
+ main_thread_client_id,
+ foreman_client_id,
+ query_string,
+ bus,
+ storage_manager,
+ query_processor,
+ parser_wrapper);
+ DCHECK_EQ(1u, results.size());
+ return results[0];
}
void executeAnalyze(const PtrVector<ParseString> *arguments,
@@ -292,25 +320,47 @@ void executeAnalyze(const PtrVector<ParseString> *arguments,
// Get the number of distinct values for each column.
for (const CatalogAttribute &attribute : relation) {
+ const Type &attr_type = attribute.getType();
+ bool is_min_max_applicable =
+ (attr_type.getSuperTypeID() == Type::SuperTypeID::kNumeric);
+
std::string query_string = "SELECT COUNT(DISTINCT ";
query_string.append(attribute.getName());
- query_string.append(") FROM ");
+ query_string.append(")");
+ if (is_min_max_applicable) {
+ query_string.append(", MIN(");
+ query_string.append(attribute.getName());
+ query_string.append("), MAX(");
+ query_string.append(attribute.getName());
+ query_string.append(")");
+ }
+ query_string.append(" FROM ");
query_string.append(relation.getName());
query_string.append(";");
- TypedValue num_distinct_values =
- executeQueryForSingleResult(main_thread_client_id,
- foreman_client_id,
- query_string,
- bus,
- storage_manager,
- query_processor,
- parser_wrapper.get());
-
- DCHECK(num_distinct_values.getTypeID() == TypeID::kLong);
- mutable_relation->getStatisticsMutable()->setNumDistinctValues(
- attribute.getID(),
- num_distinct_values.getLiteral<std::int64_t>());
+ std::vector<TypedValue> results =
+ executeQueryForSingleRow(main_thread_client_id,
+ foreman_client_id,
+ query_string,
+ bus,
+ storage_manager,
+ query_processor,
+ parser_wrapper.get());
+
+ auto *stat = mutable_relation->getStatisticsMutable();
+ const attribute_id attr_id = attribute.getID();
+
+ DCHECK(results[0].getTypeID() == TypeID::kLong);
+ stat->setNumDistinctValues(attr_id,
+ results[0].getLiteral<std::int64_t>());
+
+ if (is_min_max_applicable) {
+ DCHECK_GE(results.size(), 3u);
+ DCHECK(results[1].getTypeID() == attr_type.getTypeID());
+ DCHECK(results[2].getTypeID() == attr_type.getTypeID());
+ stat->setMinValue(attr_id, results[1]);
+ stat->setMaxValue(attr_id, results[2]);
+ }
}
// Get the number of tuples for the relation.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9b906657/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index e7ffc0c..bf0e2c7 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -347,10 +347,10 @@ int main(int argc, char* argv[]) {
// addPrimaryKeyInfoForTPCHTables(query_processor->getDefaultDatabase());
// addPrimaryKeyInfoForSSBTables(query_processor->getDefaultDatabase());
-// std::string proto_str;
-// google::protobuf::TextFormat::PrintToString(
-// query_processor->getDefaultDatabase()->getProto(), &proto_str);
-// std::cerr << proto_str << "\n";
+ std::string proto_str;
+ google::protobuf::TextFormat::PrintToString(
+ query_processor->getDefaultDatabase()->getProto(), &proto_str);
+ std::cerr << proto_str << "\n";
// query_processor->markCatalogAltered();
// query_processor->saveCatalog();
[12/16] incubator-quickstep git commit: Updates
Posted by ji...@apache.org.
Updates
Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/c588775b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/c588775b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/c588775b
Branch: refs/heads/LIP-for-tpch
Commit: c588775bcb71eb23bba8cda313e4dbc83b23c78c
Parents: b7150fb
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Wed Aug 3 20:56:18 2016 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Thu Aug 11 15:23:00 2016 -0500
----------------------------------------------------------------------
cli/QuickstepCli.cpp | 5 -----
query_optimizer/ExecutionGenerator.cpp | 2 --
2 files changed, 7 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c588775b/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index a3dfae4..e7ffc0c 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -516,11 +516,6 @@ int main(int argc, char* argv[]) {
main_thread_client_id, &bus);
end = std::chrono::steady_clock::now();
- if (quickstep::FLAGS_visualize_dag) {
- quickstep::DAGVisualizer visualizer(*query_handle->getQueryPlanMutable());
- std::cerr << "\n" << visualizer.toDOT() << "\n";
- }
-
const CatalogRelation *query_result_relation = query_handle->getQueryResultRelation();
if (query_result_relation) {
PrintToScreen::PrintRelation(*query_result_relation,
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c588775b/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 6dbba51..a194d46 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -670,8 +670,6 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
key_types.push_back(&left_attribute_type);
}
- std::size_t build_cardinality = cost_model_->estimateCardinality(build_physical);
-
// Convert the residual predicate proto.
QueryContext::predicate_id residual_predicate_index = QueryContext::kInvalidPredicateId;
if (physical_plan->residual_predicate()) {
[13/16] incubator-quickstep git commit: Updates to build bloom filters
Posted by ji...@apache.org.
Updates to build bloom filters
Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/6f3129f7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/6f3129f7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/6f3129f7
Branch: refs/heads/LIP-for-tpch
Commit: 6f3129f723f8b76c818368c0f7c30393db6565af
Parents: ef4cd1e
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Mon Aug 8 12:08:32 2016 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Thu Aug 11 15:23:00 2016 -0500
----------------------------------------------------------------------
query_execution/QueryManagerBase.cpp | 2 +-
query_optimizer/ExecutionGenerator.cpp | 26 ++++---
query_optimizer/ExecutionHeuristics.cpp | 4 +-
query_optimizer/cost_model/SimpleCostModel.cpp | 2 +-
query_optimizer/rules/AttachBloomFilters.cpp | 41 ++++++-----
.../StarSchemaHashJoinOrderOptimization.cpp | 6 +-
relational_operators/BuildHashOperator.cpp | 12 +++-
relational_operators/BuildHashOperator.hpp | 4 ++
relational_operators/CreateIndexOperator.cpp | 2 +-
relational_operators/CreateIndexOperator.hpp | 2 +-
relational_operators/CreateTableOperator.cpp | 2 +-
relational_operators/CreateTableOperator.hpp | 2 +-
relational_operators/DropTableOperator.cpp | 2 +-
relational_operators/DropTableOperator.hpp | 2 +-
relational_operators/RelationalOperator.hpp | 4 +-
storage/HashTable.hpp | 76 ++++++++++++++------
storage/StorageBlock.cpp | 12 ++--
storage/StorageBlock.hpp | 2 +-
utility/DAG.hpp | 4 +-
19 files changed, 129 insertions(+), 78 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/query_execution/QueryManagerBase.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerBase.cpp b/query_execution/QueryManagerBase.cpp
index 8e37da8..d49ee91 100644
--- a/query_execution/QueryManagerBase.cpp
+++ b/query_execution/QueryManagerBase.cpp
@@ -209,7 +209,7 @@ void QueryManagerBase::markOperatorFinished(const dag_node_index index) {
query_exec_state_->setExecutionFinished(index);
RelationalOperator *op = query_dag_->getNodePayloadMutable(index);
- op->updateCatalogOnCompletion();
+ op->actionOnCompletion();
const relation_id output_rel = op->getOutputRelationID();
for (const pair<dag_node_index, bool> &dependent_link : query_dag_->getDependents(index)) {
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 21c6e30..02deb3a 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -631,8 +631,6 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
bool any_probe_attributes_nullable = false;
bool any_build_attributes_nullable = false;
- bool skip_hash_join_optimization = false;
-
const std::vector<E::AttributeReferencePtr> &left_join_attributes =
physical_plan->left_join_attributes();
for (const E::AttributeReferencePtr &left_join_attribute : left_join_attributes) {
@@ -840,17 +838,15 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
temporary_relation_info_vec_.emplace_back(join_operator_index, output_relation);
// Add heuristics for the Hash Join, if enabled.
- if (FLAGS_optimize_joins && !skip_hash_join_optimization) {
- execution_heuristics_->addHashJoinInfo(build_operator_index,
- join_operator_index,
- referenced_stored_build_relation,
- referenced_stored_probe_relation,
- bloom_filter_config,
- std::move(build_side_bloom_filter_attribute_ids),
- std::move(probe_side_bloom_filter_attribute_ids),
- join_hash_table_index,
- star_schema_cost_model_->estimateCardinality(build_physical));
- }
+ execution_heuristics_->addHashJoinInfo(build_operator_index,
+ join_operator_index,
+ referenced_stored_build_relation,
+ referenced_stored_probe_relation,
+ bloom_filter_config,
+ std::move(build_side_bloom_filter_attribute_ids),
+ std::move(probe_side_bloom_filter_attribute_ids),
+ join_hash_table_index,
+ star_schema_cost_model_->estimateCardinality(build_physical));
}
void ExecutionGenerator::convertNestedLoopsJoin(
@@ -1443,7 +1439,9 @@ void ExecutionGenerator::convertAggregate(
aggr_state_proto->mutable_predicate()->CopyFrom(predicate->getProto());
}
- aggr_state_proto->set_estimated_num_entries(cost_model_->estimateCardinality(physical_plan));
+// aggr_state_proto->set_estimated_num_entries(cost_model_->estimateCardinality(physical_plan));
+ aggr_state_proto->set_estimated_num_entries(
+ star_schema_cost_model_->estimateCardinality(physical_plan) * 10);
const QueryPlan::DAGNodeIndex aggregation_operator_index =
execution_plan_->addRelationalOperator(
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/query_optimizer/ExecutionHeuristics.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionHeuristics.cpp b/query_optimizer/ExecutionHeuristics.cpp
index d5d7640..81e7362 100644
--- a/query_optimizer/ExecutionHeuristics.cpp
+++ b/query_optimizer/ExecutionHeuristics.cpp
@@ -70,8 +70,8 @@ void ExecutionHeuristics::optimizeExecutionPlan(QueryPlan *query_plan,
auto *build_side_bloom_filter = hash_table_proto->add_build_side_bloom_filters();
build_side_bloom_filter->set_bloom_filter_id(bloom_filter_id);
build_side_bloom_filter->set_attr_id(info.build_side_bloom_filter_ids_[i]);
- std::cerr << "Build " << build_side_bf.attribute->toString()
- << " @" << bloom_filter_config.builder << "\n";
+// std::cerr << "Build " << build_side_bf.attribute->toString()
+// << " @" << bloom_filter_config.builder << "\n";
}
}
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/query_optimizer/cost_model/SimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/SimpleCostModel.cpp b/query_optimizer/cost_model/SimpleCostModel.cpp
index 45e2f00..f3d4fee 100644
--- a/query_optimizer/cost_model/SimpleCostModel.cpp
+++ b/query_optimizer/cost_model/SimpleCostModel.cpp
@@ -119,7 +119,7 @@ std::size_t SimpleCostModel::estimateCardinalityForAggregate(
return 1;
}
return std::max(static_cast<std::size_t>(1),
- estimateCardinality(physical_plan->input()));
+ estimateCardinality(physical_plan->input()) / 10);
}
std::size_t SimpleCostModel::estimateCardinalityForWindowAggregate(
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/query_optimizer/rules/AttachBloomFilters.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/AttachBloomFilters.cpp b/query_optimizer/rules/AttachBloomFilters.cpp
index 10ed512..898d831 100644
--- a/query_optimizer/rules/AttachBloomFilters.cpp
+++ b/query_optimizer/rules/AttachBloomFilters.cpp
@@ -143,12 +143,11 @@ void AttachBloomFilters::visitConsumer(const P::PhysicalPtr &node) {
// Bloom filters from parent
const auto &parent_bloom_filters = consumers_[node];
if (!parent_bloom_filters.empty()) {
-// if (node->getPhysicalType() == P::PhysicalType::kHashJoin) {
-// const P::HashJoinPtr hash_join =
-// std::static_pointer_cast<const P::HashJoin>(node);
+// P::HashJoinPtr hash_join;
+// if (P::SomeHashJoin::MatchesWithConditionalCast(node, &hash_join) &&
+// hash_join->join_type() == P::HashJoin::JoinType::kInnerJoin) {
// const std::vector<const std::vector<E::AttributeReferencePtr>*> join_attributes =
// { &hash_join->left_join_attributes(), &hash_join->right_join_attributes() };
-//
// for (std::size_t i = 0; i < 2; ++i) {
// const auto child = hash_join->children()[i];
// std::unordered_set<E::ExprId> child_output_attribute_ids;
@@ -188,7 +187,7 @@ void AttachBloomFilters::visitConsumer(const P::PhysicalPtr &node) {
// }
// consumers_.emplace(child, std::move(bloom_filters));
// }
-// }
+// } else {
for (const auto &child : node->children()) {
std::unordered_set<E::ExprId> child_output_attribute_ids;
for (const auto &attr : child->getOutputAttributes()) {
@@ -209,6 +208,7 @@ void AttachBloomFilters::visitConsumer(const P::PhysicalPtr &node) {
}
consumers_.emplace(child, std::move(bloom_filters));
}
+// }
}
// Bloom filters from build side to probe side via HashJoin
@@ -260,14 +260,18 @@ void AttachBloomFilters::decideAttach(
}
P::PhysicalPtr consumer_child = nullptr;
- if (node->getPhysicalType() == P::PhysicalType::kHashJoin) {
- consumer_child = std::static_pointer_cast<const P::HashJoin>(node)->left();
- }
- if (node->getPhysicalType() == P::PhysicalType::kAggregate) {
- consumer_child = std::static_pointer_cast<const P::Aggregate>(node)->input();
- }
- if (node->getPhysicalType() == P::PhysicalType::kSelection) {
- consumer_child = std::static_pointer_cast<const P::Selection>(node)->input();
+ switch (node->getPhysicalType()) {
+ case P::PhysicalType::kHashJoin:
+ consumer_child = std::static_pointer_cast<const P::HashJoin>(node)->left();
+ break;
+ case P::PhysicalType::kAggregate:
+ consumer_child = std::static_pointer_cast<const P::Aggregate>(node)->input();
+ break;
+ case P::PhysicalType::kSelection:
+ consumer_child = std::static_pointer_cast<const P::Selection>(node)->input();
+ break;
+ default:
+ break;
}
if (consumer_child != nullptr) {
@@ -320,7 +324,7 @@ P::PhysicalPtr AttachBloomFilters::performAttach(const physical::PhysicalPtr &no
const auto attach_it = attaches_.find(node);
if (attach_it != attaches_.end()) {
// for (const auto& item : attach_it->second.probe_side_bloom_filters) {
-// std::cout << "Attach probe from " << item.builder
+// std::cerr << "Attach probe from " << item.builder
// << " to " << node << "\n";
// }
@@ -342,14 +346,14 @@ P::PhysicalPtr AttachBloomFilters::performAttach(const physical::PhysicalPtr &no
const auto attach_it = attaches_.find(node);
if (attach_it != attaches_.end()) {
// for (const auto& item : attach_it->second.probe_side_bloom_filters) {
-// std::cout << "Attach probe from " << item.builder
+// std::cerr << "Attach probe from " << item.builder
// << " to " << node << "\n";
// }
const P::AggregatePtr aggregate =
std::static_pointer_cast<const P::Aggregate>(node);
return P::Aggregate::Create(
- aggregate->input(),
+ new_children[0],
aggregate->grouping_expressions(),
aggregate->aggregate_expressions(),
aggregate->filter_predicate(),
@@ -361,14 +365,14 @@ P::PhysicalPtr AttachBloomFilters::performAttach(const physical::PhysicalPtr &no
const auto attach_it = attaches_.find(node);
if (attach_it != attaches_.end()) {
// for (const auto& item : attach_it->second.probe_side_bloom_filters) {
-// std::cout << "Attach probe from " << item.builder
+// std::cerr << "Attach probe from " << item.builder
// << " to " << node << "\n";
// }
const P::SelectionPtr selection =
std::static_pointer_cast<const P::Selection>(node);
return P::Selection::Create(
- selection->input(),
+ new_children[0],
selection->project_expressions(),
selection->filter_predicate(),
attach_it->second);
@@ -378,7 +382,6 @@ P::PhysicalPtr AttachBloomFilters::performAttach(const physical::PhysicalPtr &no
if (has_changed) {
return node->copyWithNewChildren(new_children);
}
-
return node;
}
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
index 9e8d794..1e38f63 100644
--- a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
+++ b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
@@ -254,9 +254,9 @@ physical::PhysicalPtr StarSchemaHashJoinOrderOptimization::generatePlan(
TableInfo *selected_probe_table_info = best_join->probe;
TableInfo *selected_build_table_info = best_join->build;
- std::cerr << "card: " << selected_probe_table_info->estimated_cardinality << "\n";
- std::cerr << "card: " << selected_build_table_info->estimated_cardinality << "\n";
- std::cerr << "--------\n";
+// std::cerr << "card: " << selected_probe_table_info->estimated_cardinality << "\n";
+// std::cerr << "card: " << selected_build_table_info->estimated_cardinality << "\n";
+// std::cerr << "--------\n";
if (!best_join->build_side_unique &&
selected_probe_table_info->estimated_cardinality < selected_build_table_info->estimated_cardinality) {
std::swap(selected_probe_table_info, selected_build_table_info);
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/relational_operators/BuildHashOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.cpp b/relational_operators/BuildHashOperator.cpp
index 465621c..b4e20e4 100644
--- a/relational_operators/BuildHashOperator.cpp
+++ b/relational_operators/BuildHashOperator.cpp
@@ -68,7 +68,10 @@ bool BuildHashOperator::getAllWorkOrders(
tmb::MessageBus *bus) {
DCHECK(query_context != nullptr);
- JoinHashTable *hash_table = query_context->getJoinHashTable(hash_table_index_);
+ if (hash_table_ == nullptr) {
+ hash_table_ = query_context->getJoinHashTable(hash_table_index_);
+ }
+
if (input_relation_is_stored_) {
if (!started_) {
for (const block_id input_block_id : input_relation_block_ids_) {
@@ -78,7 +81,7 @@ bool BuildHashOperator::getAllWorkOrders(
join_key_attributes_,
any_join_key_attributes_nullable_,
input_block_id,
- hash_table,
+ hash_table_,
storage_manager),
op_index_);
}
@@ -94,7 +97,7 @@ bool BuildHashOperator::getAllWorkOrders(
join_key_attributes_,
any_join_key_attributes_nullable_,
input_relation_block_ids_[num_workorders_generated_],
- hash_table,
+ hash_table_,
storage_manager),
op_index_);
++num_workorders_generated_;
@@ -140,6 +143,9 @@ serialization::WorkOrder* BuildHashOperator::createWorkOrderProto(const block_id
return proto;
}
+void BuildHashOperator::actionOnCompletion() {
+ hash_table_->finalizeBuildSideThreadLocalBloomFilters();
+}
void BuildHashWorkOrder::execute() {
BlockReference block(
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/relational_operators/BuildHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.hpp b/relational_operators/BuildHashOperator.hpp
index 4a80a8a..15b23f5 100644
--- a/relational_operators/BuildHashOperator.hpp
+++ b/relational_operators/BuildHashOperator.hpp
@@ -89,6 +89,7 @@ class BuildHashOperator : public RelationalOperator {
join_key_attributes_(join_key_attributes),
any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
hash_table_index_(hash_table_index),
+ hash_table_(nullptr),
input_relation_block_ids_(input_relation_is_stored ? input_relation.getBlocksSnapshot()
: std::vector<block_id>()),
num_workorders_generated_(0),
@@ -124,6 +125,8 @@ class BuildHashOperator : public RelationalOperator {
partially_filled_blocks->end());
}
+ void actionOnCompletion() override;
+
private:
/**
* @brief Create Work Order proto.
@@ -137,6 +140,7 @@ class BuildHashOperator : public RelationalOperator {
const std::vector<attribute_id> join_key_attributes_;
const bool any_join_key_attributes_nullable_;
const QueryContext::join_hash_table_id hash_table_index_;
+ JoinHashTable *hash_table_;
std::vector<block_id> input_relation_block_ids_;
std::vector<block_id>::size_type num_workorders_generated_;
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/relational_operators/CreateIndexOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateIndexOperator.cpp b/relational_operators/CreateIndexOperator.cpp
index ab3624c..dd311bf 100644
--- a/relational_operators/CreateIndexOperator.cpp
+++ b/relational_operators/CreateIndexOperator.cpp
@@ -33,7 +33,7 @@ bool CreateIndexOperator::getAllWorkOrders(WorkOrdersContainer *container,
return true;
}
-void CreateIndexOperator::updateCatalogOnCompletion() {
+void CreateIndexOperator::actionOnCompletion() {
relation_->addIndex(index_name_, std::move(index_description_));
}
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/relational_operators/CreateIndexOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateIndexOperator.hpp b/relational_operators/CreateIndexOperator.hpp
index fa992c9..1f6775a 100644
--- a/relational_operators/CreateIndexOperator.hpp
+++ b/relational_operators/CreateIndexOperator.hpp
@@ -90,7 +90,7 @@ class CreateIndexOperator : public RelationalOperator {
return true;
}
- void updateCatalogOnCompletion() override;
+ void actionOnCompletion() override;
private:
CatalogRelation *relation_;
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/relational_operators/CreateTableOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateTableOperator.cpp b/relational_operators/CreateTableOperator.cpp
index 261bec1..4cc615b 100644
--- a/relational_operators/CreateTableOperator.cpp
+++ b/relational_operators/CreateTableOperator.cpp
@@ -36,7 +36,7 @@ bool CreateTableOperator::getAllWorkOrders(
return true;
}
-void CreateTableOperator::updateCatalogOnCompletion() {
+void CreateTableOperator::actionOnCompletion() {
database_->addRelation(relation_.release());
}
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/relational_operators/CreateTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateTableOperator.hpp b/relational_operators/CreateTableOperator.hpp
index 3a2e29b..9b889ca 100644
--- a/relational_operators/CreateTableOperator.hpp
+++ b/relational_operators/CreateTableOperator.hpp
@@ -89,7 +89,7 @@ class CreateTableOperator : public RelationalOperator {
return true;
}
- void updateCatalogOnCompletion() override;
+ void actionOnCompletion() override;
private:
std::unique_ptr<CatalogRelation> relation_;
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/relational_operators/DropTableOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DropTableOperator.cpp b/relational_operators/DropTableOperator.cpp
index 5cd5ebc..a5e3d7c 100644
--- a/relational_operators/DropTableOperator.cpp
+++ b/relational_operators/DropTableOperator.cpp
@@ -80,7 +80,7 @@ bool DropTableOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *containe
return work_generated_;
}
-void DropTableOperator::updateCatalogOnCompletion() {
+void DropTableOperator::actionOnCompletion() {
const relation_id rel_id = relation_.getID();
if (only_drop_blocks_) {
database_->getRelationByIdMutable(rel_id)->clearBlocks();
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/relational_operators/DropTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DropTableOperator.hpp b/relational_operators/DropTableOperator.hpp
index e713bd6..534a266 100644
--- a/relational_operators/DropTableOperator.hpp
+++ b/relational_operators/DropTableOperator.hpp
@@ -89,7 +89,7 @@ class DropTableOperator : public RelationalOperator {
bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
- void updateCatalogOnCompletion() override;
+ void actionOnCompletion() override;
private:
const CatalogRelation &relation_;
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/relational_operators/RelationalOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RelationalOperator.hpp b/relational_operators/RelationalOperator.hpp
index f0303e5..ac5dd54 100644
--- a/relational_operators/RelationalOperator.hpp
+++ b/relational_operators/RelationalOperator.hpp
@@ -114,11 +114,11 @@ class RelationalOperator {
virtual bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) = 0;
/**
- * @brief Update Catalog upon the completion of this RelationalOperator, if
+ * @brief Perform action upon the completion of this RelationalOperator, if
* necessary.
*
**/
- virtual void updateCatalogOnCompletion() {
+ virtual void actionOnCompletion() {
}
/**
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/storage/HashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTable.hpp b/storage/HashTable.hpp
index 3538181..2ef9359 100644
--- a/storage/HashTable.hpp
+++ b/storage/HashTable.hpp
@@ -23,7 +23,9 @@
#include <atomic>
#include <cstddef>
#include <cstdlib>
+#include <map>
#include <memory>
+#include <thread>
#include <type_traits>
#include <vector>
@@ -37,6 +39,7 @@
#include "storage/ValueAccessor.hpp"
#include "storage/ValueAccessorUtil.hpp"
#include "threading/SpinSharedMutex.hpp"
+#include "threading/SpinMutex.hpp"
#include "types/Type.hpp"
#include "types/TypedValue.hpp"
#include "utility/BloomFilter.hpp"
@@ -1028,6 +1031,17 @@ class HashTable : public HashTableBase<resizable,
build_attribute_ids_.push_back(build_attribute_id);
}
+ inline void finalizeBuildSideThreadLocalBloomFilters() {
+ if (has_build_side_bloom_filter_) {
+ for (const auto &thread_local_bf_pair : thread_local_bloom_filters_) {
+ for (std::size_t i = 0; i < build_bloom_filters_.size(); ++i) {
+ build_bloom_filters_[i]->bitwiseOr(
+ thread_local_bf_pair.second[i].get());
+ }
+ }
+ }
+ }
+
/**
* @brief This function adds a pointer to the list of bloom filters to be
* used during the probe phase of this hash table.
@@ -1338,6 +1352,8 @@ class HashTable : public HashTableBase<resizable,
bool has_build_side_bloom_filter_ = false;
bool has_probe_side_bloom_filter_ = false;
std::vector<BloomFilter *> build_bloom_filters_;
+ std::map<std::thread::id, std::vector<std::unique_ptr<BloomFilter>>> thread_local_bloom_filters_;
+ SpinMutex bloom_filter_mutex_;
std::vector<attribute_id> build_attribute_ids_;
std::vector<const BloomFilter*> probe_bloom_filters_;
std::vector<attribute_id> probe_attribute_ids_;
@@ -1487,22 +1503,19 @@ HashTablePutResult HashTable<ValueT, resizable, serializable, force_key_copy, al
}
}
+ BloomFilter *thread_local_bloom_filter = nullptr;
if (has_build_side_bloom_filter_) {
- for (std::size_t i = 0; i < build_bloom_filters_.size(); ++i) {
- auto *build_bloom_filter = build_bloom_filters_[i];
- std::unique_ptr<BloomFilter> thread_local_bloom_filter(
- new BloomFilter(build_bloom_filter->getNumberOfHashes(),
- build_bloom_filter->getBitArraySize()));
- const auto &build_attr = build_attribute_ids_[i];
- const std::size_t attr_size =
- accessor->template getUntypedValueAndByteLengthAtAbsolutePosition<false>(0, build_attr).second;
- while (accessor->next()) {
- thread_local_bloom_filter->insertUnSafe(
- static_cast<const std::uint8_t *>(accessor->getUntypedValue(build_attr)),
- attr_size);
- }
- build_bloom_filter->bitwiseOr(thread_local_bloom_filter.get());
- accessor->beginIteration();
+ const auto tid = std::this_thread::get_id();
+ SpinMutexLock lock(bloom_filter_mutex_);
+ auto bf_it = thread_local_bloom_filters_.find(tid);
+ if (bf_it == thread_local_bloom_filters_.end()) {
+ auto &bf_vector = thread_local_bloom_filters_[tid];
+ bf_vector.emplace_back(
+ std::make_unique<BloomFilter>(build_bloom_filters_[0]->getNumberOfHashes(),
+ build_bloom_filters_[0]->getBitArraySize()));
+ thread_local_bloom_filter = bf_vector[0].get();
+ } else {
+ thread_local_bloom_filter = bf_it->second[0].get();
}
}
@@ -1521,6 +1534,11 @@ HashTablePutResult HashTable<ValueT, resizable, serializable, force_key_copy, al
variable_size,
(*functor)(*accessor),
using_prealloc ? &prealloc_state : nullptr);
+ // Insert into bloom filter, if enabled.
+ if (has_build_side_bloom_filter_) {
+ thread_local_bloom_filter->insertUnSafe(static_cast<const std::uint8_t *>(key.getDataPtr()),
+ key.getDataSize());
+ }
if (result == HashTablePutResult::kDuplicateKey) {
DEBUG_ASSERT(!using_prealloc);
return result;
@@ -1546,6 +1564,11 @@ HashTablePutResult HashTable<ValueT, resizable, serializable, force_key_copy, al
variable_size,
(*functor)(*accessor),
using_prealloc ? &prealloc_state : nullptr);
+ // Insert into bloom filter, if enabled.
+ if (has_build_side_bloom_filter_) {
+ thread_local_bloom_filter->insertUnSafe(static_cast<const std::uint8_t *>(key.getDataPtr()),
+ key.getDataSize());
+ }
if (result != HashTablePutResult::kOK) {
return result;
}
@@ -1618,12 +1641,26 @@ HashTablePutResult HashTable<ValueT, resizable, serializable, force_key_copy, al
}
if (has_build_side_bloom_filter_) {
+ const auto tid = std::this_thread::get_id();
+ std::vector<std::unique_ptr<BloomFilter>> *thread_local_bf_vector;
+ {
+ SpinMutexLock lock(bloom_filter_mutex_);
+ auto bf_it = thread_local_bloom_filters_.find(tid);
+ if (bf_it == thread_local_bloom_filters_.end()) {
+ thread_local_bf_vector = &thread_local_bloom_filters_[tid];
+ for (const auto &build_side_bf : build_bloom_filters_) {
+ thread_local_bf_vector->emplace_back(
+ std::make_unique<BloomFilter>(build_side_bf->getNumberOfHashes(),
+ build_side_bf->getBitArraySize()));
+ }
+ } else {
+ thread_local_bf_vector = &bf_it->second;
+ }
+ }
+
for (std::size_t i = 0; i < build_bloom_filters_.size(); ++i) {
- auto *build_bloom_filter = build_bloom_filters_[i];
- std::unique_ptr<BloomFilter> thread_local_bloom_filter(
- new BloomFilter(build_bloom_filter->getNumberOfHashes(),
- build_bloom_filter->getBitArraySize()));
const auto &build_attr = build_attribute_ids_[i];
+ BloomFilter *thread_local_bloom_filter = (*thread_local_bf_vector)[i].get();
const std::size_t attr_size =
accessor->template getUntypedValueAndByteLengthAtAbsolutePosition<false>(0, build_attr).second;
while (accessor->next()) {
@@ -1631,7 +1668,6 @@ HashTablePutResult HashTable<ValueT, resizable, serializable, force_key_copy, al
static_cast<const std::uint8_t *>(accessor->getUntypedValue(build_attr)),
attr_size);
}
- build_bloom_filter->bitwiseOr(thread_local_bloom_filter.get());
accessor->beginIteration();
}
}
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/storage/StorageBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.cpp b/storage/StorageBlock.cpp
index 74ff5b6..7bbba9a 100644
--- a/storage/StorageBlock.cpp
+++ b/storage/StorageBlock.cpp
@@ -1326,12 +1326,16 @@ bool StorageBlock::rebuildIndexes(bool short_circuit) {
}
TupleIdSequence* StorageBlock::getMatchesForPredicate(const Predicate *predicate,
- const TupleIdSequence *sequence) const {
+ const TupleIdSequence *filter) const {
if (predicate == nullptr) {
- return tuple_store_->getExistenceMap();
+ TupleIdSequence *sequence = tuple_store_->getExistenceMap();
+ if (filter != nullptr) {
+ sequence->intersectWith(*filter);
+ }
+ return sequence;
}
- std::unique_ptr<ValueAccessor> value_accessor(tuple_store_->createValueAccessor(sequence));
+ std::unique_ptr<ValueAccessor> value_accessor(tuple_store_->createValueAccessor());
std::unique_ptr<TupleIdSequence> existence_map;
if (!tuple_store_->isPacked()) {
existence_map.reset(tuple_store_->getExistenceMap());
@@ -1341,7 +1345,7 @@ TupleIdSequence* StorageBlock::getMatchesForPredicate(const Predicate *predicate
indices_consistent_);
return predicate->getAllMatches(value_accessor.get(),
&sub_blocks_ref,
- nullptr,
+ filter,
existence_map.get());
}
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/storage/StorageBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.hpp b/storage/StorageBlock.hpp
index 5cca51c..4284ea1 100644
--- a/storage/StorageBlock.hpp
+++ b/storage/StorageBlock.hpp
@@ -593,7 +593,7 @@ class StorageBlock : public StorageBlockBase {
const std::size_t getNumTuples() const;
TupleIdSequence* getMatchesForPredicate(const Predicate *predicate,
- const TupleIdSequence *sequence = nullptr) const;
+ const TupleIdSequence *filter = nullptr) const;
private:
static TupleStorageSubBlock* CreateTupleStorageSubBlock(
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/utility/DAG.hpp
----------------------------------------------------------------------
diff --git a/utility/DAG.hpp b/utility/DAG.hpp
index a1f2619..1d1caa1 100644
--- a/utility/DAG.hpp
+++ b/utility/DAG.hpp
@@ -293,8 +293,8 @@ class DAG {
* node at node_index.
**/
inline void addDependent(const size_type_nodes node_index, const LinkMetadataT &link_metadata) {
- DCHECK(dependents_with_metadata_.find(node_index) == dependents_with_metadata_.end());
- dependents_with_metadata_.emplace(node_index, link_metadata);
+ // DCHECK(dependents_with_metadata_.find(node_index) == dependents_with_metadata_.end());
+ dependents_with_metadata_[node_index] = link_metadata;
}
/**
[11/16] incubator-quickstep git commit: Minor updates to mutex
Posted by ji...@apache.org.
Minor updates to mutex
Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/ebdc041f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/ebdc041f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/ebdc041f
Branch: refs/heads/LIP-for-tpch
Commit: ebdc041f5be5fab9c8dd702a50811326d4178bac
Parents: 6f3129f
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Mon Aug 8 12:57:04 2016 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Thu Aug 11 15:23:00 2016 -0500
----------------------------------------------------------------------
storage/HashTable.hpp | 8 ++++----
1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ebdc041f/storage/HashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTable.hpp b/storage/HashTable.hpp
index 2ef9359..1caa183 100644
--- a/storage/HashTable.hpp
+++ b/storage/HashTable.hpp
@@ -38,8 +38,8 @@
#include "storage/TupleReference.hpp"
#include "storage/ValueAccessor.hpp"
#include "storage/ValueAccessorUtil.hpp"
+#include "threading/Mutex.hpp"
#include "threading/SpinSharedMutex.hpp"
-#include "threading/SpinMutex.hpp"
#include "types/Type.hpp"
#include "types/TypedValue.hpp"
#include "utility/BloomFilter.hpp"
@@ -1353,7 +1353,7 @@ class HashTable : public HashTableBase<resizable,
bool has_probe_side_bloom_filter_ = false;
std::vector<BloomFilter *> build_bloom_filters_;
std::map<std::thread::id, std::vector<std::unique_ptr<BloomFilter>>> thread_local_bloom_filters_;
- SpinMutex bloom_filter_mutex_;
+ Mutex bloom_filter_mutex_;
std::vector<attribute_id> build_attribute_ids_;
std::vector<const BloomFilter*> probe_bloom_filters_;
std::vector<attribute_id> probe_attribute_ids_;
@@ -1506,7 +1506,7 @@ HashTablePutResult HashTable<ValueT, resizable, serializable, force_key_copy, al
BloomFilter *thread_local_bloom_filter = nullptr;
if (has_build_side_bloom_filter_) {
const auto tid = std::this_thread::get_id();
- SpinMutexLock lock(bloom_filter_mutex_);
+ MutexLock lock(bloom_filter_mutex_);
auto bf_it = thread_local_bloom_filters_.find(tid);
if (bf_it == thread_local_bloom_filters_.end()) {
auto &bf_vector = thread_local_bloom_filters_[tid];
@@ -1644,7 +1644,7 @@ HashTablePutResult HashTable<ValueT, resizable, serializable, force_key_copy, al
const auto tid = std::this_thread::get_id();
std::vector<std::unique_ptr<BloomFilter>> *thread_local_bf_vector;
{
- SpinMutexLock lock(bloom_filter_mutex_);
+ MutexLock lock(bloom_filter_mutex_);
auto bf_it = thread_local_bloom_filters_.find(tid);
if (bf_it == thread_local_bloom_filters_.end()) {
thread_local_bf_vector = &thread_local_bloom_filters_[tid];
[06/16] incubator-quickstep git commit: Removed an unused message
type.
Posted by ji...@apache.org.
Removed an unused message type.
Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/61689962
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/61689962
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/61689962
Branch: refs/heads/LIP-for-tpch
Commit: 6168996216af8278d5c789c67aa4ec8325fab483
Parents: 2c0ce6a
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Mon Aug 8 15:32:34 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Tue Aug 9 10:44:50 2016 -0700
----------------------------------------------------------------------
query_execution/ForemanSingleNode.cpp | 4 +---
query_execution/PolicyEnforcerBase.cpp | 13 -------------
query_execution/QueryExecutionMessages.proto | 5 -----
query_execution/QueryExecutionTypedefs.hpp | 1 -
query_execution/Shiftboss.cpp | 1 -
query_execution/Shiftboss.hpp | 1 -
query_execution/Worker.hpp | 1 -
7 files changed, 1 insertion(+), 25 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/61689962/query_execution/ForemanSingleNode.cpp
----------------------------------------------------------------------
diff --git a/query_execution/ForemanSingleNode.cpp b/query_execution/ForemanSingleNode.cpp
index 23db379..d064a6f 100644
--- a/query_execution/ForemanSingleNode.cpp
+++ b/query_execution/ForemanSingleNode.cpp
@@ -87,7 +87,6 @@ ForemanSingleNode::ForemanSingleNode(
kPoisonMessage,
kRebuildWorkOrderCompleteMessage,
kWorkOrderFeedbackMessage,
- kWorkOrdersAvailableMessage,
kWorkOrderCompleteMessage};
for (const auto message_type : receiver_message_types) {
@@ -122,8 +121,7 @@ void ForemanSingleNode::run() {
case kDataPipelineMessage:
case kRebuildWorkOrderCompleteMessage:
case kWorkOrderCompleteMessage:
- case kWorkOrderFeedbackMessage:
- case kWorkOrdersAvailableMessage: {
+ case kWorkOrderFeedbackMessage: {
policy_enforcer_->processMessage(tagged_message);
break;
}
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/61689962/query_execution/PolicyEnforcerBase.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerBase.cpp b/query_execution/PolicyEnforcerBase.cpp
index 78f7b44..4174bd6 100644
--- a/query_execution/PolicyEnforcerBase.cpp
+++ b/query_execution/PolicyEnforcerBase.cpp
@@ -107,19 +107,6 @@ void PolicyEnforcerBase::processMessage(const TaggedMessage &tagged_message) {
op_index, proto.block_id(), proto.relation_id());
break;
}
- case kWorkOrdersAvailableMessage: {
- serialization::WorkOrdersAvailableMessage proto;
- CHECK(proto.ParseFromArray(tagged_message.message(),
- tagged_message.message_bytes()));
- query_id = proto.query_id();
- DCHECK(admitted_queries_.find(query_id) != admitted_queries_.end());
-
- op_index = proto.operator_index();
-
- // Check if new work orders are available.
- admitted_queries_[query_id]->fetchNormalWorkOrders(op_index);
- break;
- }
case kWorkOrderFeedbackMessage: {
WorkOrder::FeedbackMessage msg(
const_cast<void *>(tagged_message.message()),
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/61689962/query_execution/QueryExecutionMessages.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionMessages.proto b/query_execution/QueryExecutionMessages.proto
index 20b684e..060efa1 100644
--- a/query_execution/QueryExecutionMessages.proto
+++ b/query_execution/QueryExecutionMessages.proto
@@ -74,11 +74,6 @@ message DataPipelineMessage {
required uint64 query_id = 4;
}
-message WorkOrdersAvailableMessage {
- required uint64 operator_index = 1;
- required uint64 query_id = 2;
-}
-
// Distributed version related messages.
message ShiftbossRegistrationMessage {
// The total Work Order processing capacity in Shiftboss, which equals to the
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/61689962/query_execution/QueryExecutionTypedefs.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionTypedefs.hpp b/query_execution/QueryExecutionTypedefs.hpp
index d154d84..33a93b0 100644
--- a/query_execution/QueryExecutionTypedefs.hpp
+++ b/query_execution/QueryExecutionTypedefs.hpp
@@ -69,7 +69,6 @@ enum QueryExecutionMessageType : message_type_id {
kWorkOrderCompleteMessage, // From Worker to Foreman.
kCatalogRelationNewBlockMessage, // From InsertDestination to Foreman.
kDataPipelineMessage, // From InsertDestination or some WorkOrders to Foreman.
- kWorkOrdersAvailableMessage, // From some WorkOrders to Foreman.
kWorkOrderFeedbackMessage, // From some WorkOrders to Foreman on behalf of
// their corresponding RelationalOperators.
kRebuildWorkOrderMessage, // From Foreman to Worker.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/61689962/query_execution/Shiftboss.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Shiftboss.cpp b/query_execution/Shiftboss.cpp
index 24c91fe..bd83dd4 100644
--- a/query_execution/Shiftboss.cpp
+++ b/query_execution/Shiftboss.cpp
@@ -152,7 +152,6 @@ void Shiftboss::run() {
case kWorkOrderCompleteMessage: // Fall through.
case kRebuildWorkOrderCompleteMessage:
case kDataPipelineMessage:
- case kWorkOrdersAvailableMessage:
case kWorkOrderFeedbackMessage: {
LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
<< "') forwarded typed '" << annotated_message.tagged_message.message_type()
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/61689962/query_execution/Shiftboss.hpp
----------------------------------------------------------------------
diff --git a/query_execution/Shiftboss.hpp b/query_execution/Shiftboss.hpp
index 9464a4d..30a8d1a 100644
--- a/query_execution/Shiftboss.hpp
+++ b/query_execution/Shiftboss.hpp
@@ -103,7 +103,6 @@ class Shiftboss : public Thread {
// Message sent to Foreman.
bus_->RegisterClientAsSender(shiftboss_client_id_, kCatalogRelationNewBlockMessage);
bus_->RegisterClientAsSender(shiftboss_client_id_, kDataPipelineMessage);
- bus_->RegisterClientAsSender(shiftboss_client_id_, kWorkOrdersAvailableMessage);
bus_->RegisterClientAsSender(shiftboss_client_id_, kWorkOrderFeedbackMessage);
// Forward the following message types from Foreman to Workers.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/61689962/query_execution/Worker.hpp
----------------------------------------------------------------------
diff --git a/query_execution/Worker.hpp b/query_execution/Worker.hpp
index 44a7447..aa39bb3 100644
--- a/query_execution/Worker.hpp
+++ b/query_execution/Worker.hpp
@@ -68,7 +68,6 @@ class Worker : public Thread {
kRebuildWorkOrderCompleteMessage);
bus_->RegisterClientAsSender(worker_client_id_, kCatalogRelationNewBlockMessage);
bus_->RegisterClientAsSender(worker_client_id_, kDataPipelineMessage);
- bus_->RegisterClientAsSender(worker_client_id_, kWorkOrdersAvailableMessage);
bus_->RegisterClientAsSender(worker_client_id_, kWorkOrderFeedbackMessage);
bus_->RegisterClientAsReceiver(worker_client_id_, kWorkOrderMessage);
[05/16] incubator-quickstep git commit: Fixed bugs in creating
WorkOrderProtos.
Posted by ji...@apache.org.
Fixed bugs in creating WorkOrderProtos.
Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/2c0ce6a3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/2c0ce6a3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/2c0ce6a3
Branch: refs/heads/LIP-for-tpch
Commit: 2c0ce6a3bcf2ec40b0c32d077552cda3e225f787
Parents: 85e02de
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Mon Aug 8 18:42:32 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Tue Aug 9 10:42:25 2016 -0700
----------------------------------------------------------------------
relational_operators/HashJoinOperator.cpp | 1 +
relational_operators/WorkOrderFactory.cpp | 4 +---
2 files changed, 2 insertions(+), 3 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2c0ce6a3/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index 7851f41..779c0fe 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -393,6 +393,7 @@ bool HashJoinOperator::getAllOuterJoinWorkOrderProtos(WorkOrderProtosContainer *
serialization::WorkOrder* HashJoinOperator::createOuterJoinWorkOrderProto(const block_id block) {
serialization::WorkOrder *proto = new serialization::WorkOrder;
proto->set_work_order_type(serialization::HASH_JOIN);
+ proto->set_query_id(query_id_);
proto->SetExtension(serialization::HashJoinWorkOrder::hash_join_work_order_type,
serialization::HashJoinWorkOrder::HASH_OUTER_JOIN);
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2c0ce6a3/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index 721d735..7d7af59 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -533,13 +533,11 @@ bool WorkOrderFactory::ProtoIsValid(const serialization::WorkOrder &proto,
return false;
}
- const CatalogRelationSchema &build_relation = catalog_database.getRelationSchemaById(build_relation_id);
const CatalogRelationSchema &probe_relation = catalog_database.getRelationSchemaById(probe_relation_id);
for (int i = 0; i < proto.ExtensionSize(serialization::HashJoinWorkOrder::join_key_attributes); ++i) {
const attribute_id attr_id =
proto.GetExtension(serialization::HashJoinWorkOrder::join_key_attributes, i);
- if (!build_relation.hasAttributeWithId(attr_id) ||
- !probe_relation.hasAttributeWithId(attr_id)) {
+ if (!probe_relation.hasAttributeWithId(attr_id)) {
return false;
}
}
[09/16] incubator-quickstep git commit: Initial commit
Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/rules/AttachBloomFilters.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/AttachBloomFilters.cpp b/query_optimizer/rules/AttachBloomFilters.cpp
new file mode 100644
index 0000000..03a42a0
--- /dev/null
+++ b/query_optimizer/rules/AttachBloomFilters.cpp
@@ -0,0 +1,308 @@
+/**
+ * Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ * University of Wisconsin\u2014Madison.
+ *
+ * Licensed 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 "query_optimizer/rules/AttachBloomFilters.hpp"
+
+#include <memory>
+#include <set>
+#include <unordered_set>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/NamedExpression.hpp"
+#include "query_optimizer/expressions/PatternMatcher.hpp"
+#include "query_optimizer/physical/HashJoin.hpp"
+#include "query_optimizer/physical/PatternMatcher.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/PhysicalType.hpp"
+#include "query_optimizer/physical/TopLevelPlan.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+
+namespace E = ::quickstep::optimizer::expressions;
+namespace P = ::quickstep::optimizer::physical;
+
+P::PhysicalPtr AttachBloomFilters::apply(const P::PhysicalPtr &input) {
+ DCHECK(input->getPhysicalType() == P::PhysicalType::kTopLevelPlan);
+ cost_model_.reset(
+ new cost::StarSchemaSimpleCostModel(
+ std::static_pointer_cast<const P::TopLevelPlan>(input)->shared_subplans()));
+
+ visitProducer(input, 0);
+ visitConsumer(input);
+
+// for (const auto &info_vec_pair : consumers_) {
+// std::cerr << "--------\n"
+// << "Node " << info_vec_pair.first->getName()
+// << " " << info_vec_pair.first << "\n";
+//
+// for (const auto &info : info_vec_pair.second) {
+// std::cerr << info.attribute->attribute_alias();
+// if (info.attribute->id() != info.source_attribute->id()) {
+// std::cerr << "{FROM " << info.source_attribute->attribute_alias() << "}";
+// }
+// if (info.from_sibling) {
+// std::cerr << " sibling";
+// }
+// std::cerr << " @" << info.source << "[" << info.depth << "]"
+// << ": " << info.selectivity << "\n";
+// }
+// std::cerr << "********\n";
+// }
+
+ return visitAndAttach(input);
+}
+
+void AttachBloomFilters::visitProducer(const P::PhysicalPtr &node, const int depth) {
+ for (const P::PhysicalPtr &child : node->children()) {
+ visitProducer(child, depth+1);
+ }
+
+ std::vector<BloomFilterInfo> bloom_filters;
+
+ if (node->getPhysicalType() == P::PhysicalType::kHashJoin) {
+ const P::HashJoinPtr &hash_join =
+ std::static_pointer_cast<const P::HashJoin>(node);
+ const P::PhysicalPtr &build_node = hash_join->right();
+ double selectivity = cost_model_->estimateSelectivity(build_node);
+ if (selectivity < 1.0) {
+ auto &build_node_info = producers_[build_node];
+ for (const auto &attr : hash_join->right_join_attributes()) {
+ build_node_info.emplace_back(node, attr, depth, selectivity, false);
+ }
+ }
+ }
+
+ const std::vector<E::AttributeReferencePtr> output_attributes(
+ node->getOutputAttributes());
+ std::unordered_set<E::ExprId> output_attribute_ids;
+ for (const auto &attr : output_attributes) {
+ output_attribute_ids.emplace(attr->id());
+ }
+
+ // First check inherited bloom filters
+ std::vector<const BloomFilterInfo*> candidates;
+ switch (node->getPhysicalType()) {
+ case P::PhysicalType::kAggregate:
+ case P::PhysicalType::kSelection:
+ case P::PhysicalType::kHashJoin: {
+ for (const P::PhysicalPtr &child : node->children()) {
+ for (const BloomFilterInfo &info : producers_[child]) {
+ candidates.emplace_back(&info);
+ }
+ }
+ }
+ default:
+ break;
+ }
+
+ for (const BloomFilterInfo *info : candidates) {
+ if (output_attribute_ids.find(info->attribute->id()) != output_attribute_ids.end()) {
+ bloom_filters.emplace_back(
+ info->source, info->attribute, info->depth, info->selectivity, false);
+ }
+ }
+
+ // Self-produced bloom filters
+// double selectivity = cost_model_->estimateSelectivity(node);
+// if (selectivity < 1.0) {
+// for (const auto &attr : output_attributes) {
+// bloom_filters.emplace_back(node, attr, depth, selectivity, false);
+// }
+// }
+
+ producers_.emplace(node, std::move(bloom_filters));
+}
+
+void AttachBloomFilters::visitConsumer(const P::PhysicalPtr &node) {
+ std::vector<BloomFilterInfo> bloom_filters;
+
+ // Bloom filters from parent
+ const auto &parent_bloom_filters = consumers_[node];
+ if (!parent_bloom_filters.empty()) {
+ for (const auto &child : node->children()) {
+ std::unordered_set<E::ExprId> child_output_attribute_ids;
+ for (const auto &attr : child->getOutputAttributes()) {
+ child_output_attribute_ids.emplace(attr->id());
+ }
+
+ std::vector<BloomFilterInfo> bloom_filters;
+ for (const auto &info : parent_bloom_filters) {
+ if (child_output_attribute_ids.find(info.attribute->id())
+ != child_output_attribute_ids.end()) {
+ bloom_filters.emplace_back(info.source,
+ info.attribute,
+ info.depth,
+ info.selectivity,
+ false,
+ info.source_attribute);
+ }
+ }
+ consumers_.emplace(child, std::move(bloom_filters));
+ }
+ }
+
+ // Bloom filters from build side to probe side via HashJoin
+ if (node->getPhysicalType() == P::PhysicalType::kHashJoin) {
+ const P::HashJoinPtr hash_join =
+ std::static_pointer_cast<const P::HashJoin>(node);
+ if (hash_join->join_type() == P::HashJoin::JoinType::kInnerJoin ||
+ hash_join->join_type() == P::HashJoin::JoinType::kLeftSemiJoin) {
+ const P::PhysicalPtr &producer_child = hash_join->right();
+ const P::PhysicalPtr &consumer_child = hash_join->left();
+ std::unordered_map<E::ExprId, E::AttributeReferencePtr> join_attribute_pairs;
+ for (std::size_t i = 0; i < hash_join->left_join_attributes().size(); ++i) {
+ const E::AttributeReferencePtr probe_join_attribute =
+ hash_join->left_join_attributes()[i];
+ const E::AttributeReferencePtr build_join_attribute =
+ hash_join->right_join_attributes()[i];
+ join_attribute_pairs.emplace(build_join_attribute->id(),
+ probe_join_attribute);
+ }
+
+ auto &consumer_bloom_filters = consumers_[consumer_child];
+ for (const auto &info : producers_[producer_child]) {
+ const auto pair_it = join_attribute_pairs.find(info.attribute->id());
+ if (pair_it != join_attribute_pairs.end()) {
+ consumer_bloom_filters.emplace_back(info.source,
+ pair_it->second,
+ info.depth,
+ info.selectivity,
+ true,
+ info.attribute);
+ }
+ }
+ }
+ }
+
+ P::PhysicalPtr consumer_child = nullptr;
+ if (node->getPhysicalType() == P::PhysicalType::kHashJoin) {
+ consumer_child = std::static_pointer_cast<const P::HashJoin>(node)->left();
+ }
+ if (node->getPhysicalType() == P::PhysicalType::kAggregate) {
+ consumer_child = std::static_pointer_cast<const P::Aggregate>(node)->input();
+ }
+
+ if (consumer_child != nullptr) {
+ // Decide attaches
+ auto &consumer_bloom_filters = consumers_[consumer_child];
+ if (cost_model_->estimateCardinality(consumer_child) > 10000000 &&
+ !consumer_bloom_filters.empty()) {
+ std::map<E::AttributeReferencePtr, const BloomFilterInfo*> filters;
+ for (const auto &info : consumer_bloom_filters) {
+ auto it = filters.find(info.attribute);
+ if (it == filters.end()) {
+ filters.emplace(info.attribute, &info);
+ } else {
+ if (BloomFilterInfo::isBetterThan(&info, it->second)) {
+ it->second = &info;
+ }
+ }
+ }
+
+ auto &probe_attaches = getBloomFilterConfig(node);
+ for (const auto &pair : filters) {
+ auto &build_attaches = getBloomFilterConfig(pair.second->source);
+ build_attaches.addBuildSideBloomFilter(
+ pair.second->source_attribute);
+ probe_attaches.addProbeSideBloomFilter(
+ pair.first,
+ pair.second->source_attribute,
+ pair.second->source);
+ }
+ }
+ }
+
+ for (const auto &child : node->children()) {
+ visitConsumer(child);
+ }
+}
+
+P::PhysicalPtr AttachBloomFilters::visitAndAttach(const physical::PhysicalPtr &node) {
+ std::vector<P::PhysicalPtr> new_children;
+ bool has_changed = false;
+ for (const auto &child : node->children()) {
+ P::PhysicalPtr new_child = visitAndAttach(child);
+ if (new_child != child) {
+ has_changed = true;
+ }
+ new_children.emplace_back(new_child);
+ }
+
+ if (node->getPhysicalType() == P::PhysicalType::kHashJoin) {
+ const auto attach_it = attaches_.find(node);
+ if (attach_it != attaches_.end()) {
+// for (const auto& item : attach_it->second.probe_side_bloom_filters) {
+// std::cout << "Attach probe from " << item.builder
+// << " to " << node << "\n";
+// }
+
+ const P::HashJoinPtr hash_join =
+ std::static_pointer_cast<const P::HashJoin>(node);
+ return P::HashJoin::Create(
+ new_children[0],
+ new_children[1],
+ hash_join->left_join_attributes(),
+ hash_join->right_join_attributes(),
+ hash_join->residual_predicate(),
+ hash_join->project_expressions(),
+ hash_join->join_type(),
+ attach_it->second);
+ }
+ }
+
+ if (node->getPhysicalType() == P::PhysicalType::kAggregate) {
+ const auto attach_it = attaches_.find(node);
+ if (attach_it != attaches_.end()) {
+// for (const auto& item : attach_it->second.probe_side_bloom_filters) {
+// std::cout << "Attach probe from " << item.builder
+// << " to " << node << "\n";
+// }
+
+ const P::AggregatePtr aggregate =
+ std::static_pointer_cast<const P::Aggregate>(node);
+ return P::Aggregate::Create(
+ aggregate->input(),
+ aggregate->grouping_expressions(),
+ aggregate->aggregate_expressions(),
+ aggregate->filter_predicate(),
+ attach_it->second);
+ }
+ }
+
+ if (has_changed) {
+ return node->copyWithNewChildren(new_children);
+ }
+
+ return node;
+}
+
+P::BloomFilterConfig& AttachBloomFilters::getBloomFilterConfig(const physical::PhysicalPtr &node) {
+ if (attaches_.find(node) == attaches_.end()) {
+ attaches_.emplace(node, node);
+ }
+ return attaches_[node];
+}
+
+} // namespace optimizer
+} // namespace quickstep
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/rules/AttachBloomFilters.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/AttachBloomFilters.hpp b/query_optimizer/rules/AttachBloomFilters.hpp
new file mode 100644
index 0000000..e4437f7
--- /dev/null
+++ b/query_optimizer/rules/AttachBloomFilters.hpp
@@ -0,0 +1,118 @@
+/**
+ * Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ * University of Wisconsin\u2014Madison.
+ *
+ * Licensed 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.
+ **/
+
+#ifndef QUICKSTEP_QUERY_OPTIMIZER_RULES_ATTACH_BLOOM_FILTERS_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_RULES_ATTACH_BLOOM_FILTERS_HPP_
+
+#include <algorithm>
+#include <cstddef>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <unordered_set>
+#include <utility>
+#include <vector>
+
+#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
+#include "query_optimizer/expressions/ExprId.hpp"
+#include "query_optimizer/expressions/NamedExpression.hpp"
+#include "query_optimizer/expressions/Predicate.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/HashJoin.hpp"
+#include "query_optimizer/rules/Rule.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+namespace optimizer {
+
+/** \addtogroup OptimizerRules
+ * @{
+ */
+
+/**
+ * @brief TODO
+ */
+class AttachBloomFilters : public Rule<physical::Physical> {
+ public:
+ AttachBloomFilters() {}
+
+ ~AttachBloomFilters() override {}
+
+ std::string getName() const override {
+ return "AttachBloomFilters";
+ }
+
+ physical::PhysicalPtr apply(const physical::PhysicalPtr &input) override;
+
+ private:
+ struct BloomFilterInfo {
+ BloomFilterInfo(const physical::PhysicalPtr &source_in,
+ const expressions::AttributeReferencePtr &attribute_in,
+ const int depth_in,
+ const double selectivity_in,
+ const bool from_sibling_in,
+ const expressions::AttributeReferencePtr &source_attribute_in = nullptr)
+ : source(source_in),
+ attribute(attribute_in),
+ depth(depth_in),
+ selectivity(selectivity_in),
+ from_sibling(from_sibling_in),
+ source_attribute(
+ source_attribute_in == nullptr
+ ? attribute_in
+ : source_attribute_in) {
+
+ }
+ static bool isBetterThan(const BloomFilterInfo *a,
+ const BloomFilterInfo *b) {
+ if (a->selectivity == b->selectivity) {
+ return a->depth > b->depth;
+ } else {
+ return a->selectivity < b->selectivity;
+ }
+ }
+ physical::PhysicalPtr source;
+ expressions::AttributeReferencePtr attribute;
+ int depth;
+ double selectivity;
+ bool from_sibling;
+ expressions::AttributeReferencePtr source_attribute;
+ };
+
+ void visitProducer(const physical::PhysicalPtr &node, const int depth);
+
+ void visitConsumer(const physical::PhysicalPtr &node);
+
+ physical::PhysicalPtr visitAndAttach(const physical::PhysicalPtr &node);
+
+ physical::BloomFilterConfig &getBloomFilterConfig(const physical::PhysicalPtr &node);
+
+ std::unique_ptr<cost::StarSchemaSimpleCostModel> cost_model_;
+
+ std::map<physical::PhysicalPtr, std::vector<BloomFilterInfo>> producers_;
+ std::map<physical::PhysicalPtr, std::vector<BloomFilterInfo>> consumers_;
+ std::map<physical::PhysicalPtr, physical::BloomFilterConfig> attaches_;
+
+ DISALLOW_COPY_AND_ASSIGN(AttachBloomFilters);
+};
+
+/** @} */
+
+} // namespace optimizer
+} // namespace quickstep
+
+#endif /* QUICKSTEP_QUERY_OPTIMIZER_RULES_ATTACH_BLOOM_FILTERS_HPP_ */
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/rules/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CMakeLists.txt b/query_optimizer/rules/CMakeLists.txt
index d9709ce..54b1e59 100644
--- a/query_optimizer/rules/CMakeLists.txt
+++ b/query_optimizer/rules/CMakeLists.txt
@@ -18,6 +18,7 @@
add_subdirectory(tests)
# Declare micro-libs:
+add_library(quickstep_queryoptimizer_rules_AttachBloomFilters AttachBloomFilters.cpp AttachBloomFilters.hpp)
add_library(quickstep_queryoptimizer_rules_BottomUpRule ../../empty_src.cpp BottomUpRule.hpp)
add_library(quickstep_queryoptimizer_rules_CollapseProject CollapseProject.cpp CollapseProject.hpp)
add_library(quickstep_queryoptimizer_rules_GenerateJoins GenerateJoins.cpp GenerateJoins.hpp)
@@ -36,6 +37,20 @@ add_library(quickstep_queryoptimizer_rules_UnnestSubqueries UnnestSubqueries.cpp
# Link dependencies:
+target_link_libraries(quickstep_queryoptimizer_rules_AttachBloomFilters
+ quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostModel
+ quickstep_queryoptimizer_expressions_AttributeReference
+ quickstep_queryoptimizer_expressions_ExprId
+ quickstep_queryoptimizer_expressions_NamedExpression
+ quickstep_queryoptimizer_expressions_PatternMatcher
+ quickstep_queryoptimizer_expressions_Predicate
+ quickstep_queryoptimizer_physical_HashJoin
+ quickstep_queryoptimizer_physical_PatternMatcher
+ quickstep_queryoptimizer_physical_Physical
+ quickstep_queryoptimizer_physical_PhysicalType
+ quickstep_queryoptimizer_physical_TopLevelPlan
+ quickstep_queryoptimizer_rules_Rule
+ quickstep_utility_Macros)
target_link_libraries(quickstep_queryoptimizer_rules_BottomUpRule
glog
quickstep_queryoptimizer_rules_Rule
@@ -127,6 +142,7 @@ target_link_libraries(quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOpti
quickstep_queryoptimizer_physical_PhysicalType
quickstep_queryoptimizer_physical_TopLevelPlan
quickstep_queryoptimizer_rules_Rule
+ quickstep_utility_DisjointTreeForest
quickstep_utility_Macros)
target_link_libraries(quickstep_queryoptimizer_rules_SwapProbeBuild
quickstep_queryoptimizer_costmodel_SimpleCostModel
@@ -187,6 +203,7 @@ target_link_libraries(quickstep_queryoptimizer_rules_UpdateExpression
# Module all-in-one library:
add_library(quickstep_queryoptimizer_rules ../../empty_src.cpp OptimizerRulesModule.hpp)
target_link_libraries(quickstep_queryoptimizer_rules
+ quickstep_queryoptimizer_rules_AttachBloomFilters
quickstep_queryoptimizer_rules_BottomUpRule
quickstep_queryoptimizer_rules_CollapseProject
quickstep_queryoptimizer_rules_GenerateJoins
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
index 946d316..15a7154 100644
--- a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
+++ b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
@@ -33,6 +33,7 @@
#include "query_optimizer/physical/Physical.hpp"
#include "query_optimizer/physical/PhysicalType.hpp"
#include "query_optimizer/physical/TopLevelPlan.hpp"
+#include "utility/DisjointTreeForest.hpp"
#include "glog/logging.h"
@@ -74,6 +75,9 @@ P::PhysicalPtr StarSchemaHashJoinOrderOptimization::applyInternal(const P::Physi
JoinGroupInfo *join_group = nullptr;
if (parent_join_group == nullptr || !is_valid_cascading_hash_join) {
new_join_group.reset(new JoinGroupInfo());
+ for (const auto &attr : input->getReferencedAttributes()) {
+ new_join_group->referenced_attributes.emplace(attr->id());
+ }
join_group = new_join_group.get();
} else {
join_group = parent_join_group;
@@ -146,7 +150,10 @@ physical::PhysicalPtr StarSchemaHashJoinOrderOptimization::generatePlan(
i,
tables[i],
cost_model_->estimateCardinality(tables[i]),
- cost_model_->estimateSelectivity(tables[i]));
+ cost_model_->estimateSelectivity(tables[i]),
+ CountSharedAttributes(join_group.referenced_attributes,
+ tables[i]->getOutputAttributes()),
+ tables[i]->getPhysicalType() == physical::PhysicalType::kAggregate);
}
// Auxiliary mapping info.
@@ -163,9 +170,19 @@ physical::PhysicalPtr StarSchemaHashJoinOrderOptimization::generatePlan(
}
}
- // Create a join graph where tables are vertices, and add an edge between vertices
- // t1 and t2 for each join predicate t1.x = t2.y
- std::vector<std::unordered_set<std::size_t>> join_graph(table_info_storage.size());
+ std::set<TableInfo*> remaining_tables;
+ for (auto &table_info : table_info_storage) {
+ remaining_tables.emplace(&table_info);
+ }
+
+ DisjointTreeForest<E::ExprId> join_attribute_forest;
+ for (const auto &attr_id_pair : join_group.join_attribute_pairs) {
+ join_attribute_forest.makeSet(attr_id_pair.first);
+ join_attribute_forest.makeSet(attr_id_pair.second);
+ join_attribute_forest.merge(attr_id_pair.first, attr_id_pair.second);
+ }
+
+ std::map<std::size_t, std::map<std::size_t, E::ExprId>> join_attribute_groups;
for (const auto &attr_id_pair : join_group.join_attribute_pairs) {
DCHECK(attribute_id_to_table_info_index_map.find(attr_id_pair.first)
!= attribute_id_to_table_info_index_map.end());
@@ -178,128 +195,169 @@ physical::PhysicalPtr StarSchemaHashJoinOrderOptimization::generatePlan(
attribute_id_to_table_info_index_map[attr_id_pair.second];
DCHECK_NE(first_table_idx, second_table_idx);
- table_info_storage[first_table_idx].join_attribute_pairs.emplace(
- attr_id_pair.first, attr_id_pair.second);
- table_info_storage[second_table_idx].join_attribute_pairs.emplace(
- attr_id_pair.second, attr_id_pair.first);
-
- join_graph[first_table_idx].emplace(second_table_idx);
- join_graph[second_table_idx].emplace(first_table_idx);
- }
-
- std::set<TableInfo*, TableInfoPtrLessComparator> table_info_ordered_by_priority;
- for (std::size_t i = 0; i < table_info_storage.size(); ++i) {
- table_info_ordered_by_priority.emplace(&table_info_storage[i]);
+ DCHECK_EQ(join_attribute_forest.find(attr_id_pair.first),
+ join_attribute_forest.find(attr_id_pair.second));
+ const std::size_t attr_group_id = join_attribute_forest.find(attr_id_pair.first);
+ auto &attr_group = join_attribute_groups[attr_group_id];
+ attr_group.emplace(first_table_idx, attr_id_pair.first);
+ attr_group.emplace(second_table_idx, attr_id_pair.second);
}
- // Contruct hash join tree.
while (true) {
- TableInfo *first_table_info = *table_info_ordered_by_priority.begin();
- table_info_ordered_by_priority.erase(
- table_info_ordered_by_priority.begin());
- const std::size_t first_table_info_id = first_table_info->table_info_id;
-
- TableInfo *second_table_info = nullptr;
- std::set<TableInfo*, TableInfoPtrLessComparator>::iterator second_table_info_it;
- for (auto candidate_table_info_it = table_info_ordered_by_priority.begin();
- candidate_table_info_it != table_info_ordered_by_priority.end();
- ++candidate_table_info_it) {
- TableInfo *candidate_table_info = *candidate_table_info_it;
- const std::size_t candidate_table_info_id = candidate_table_info->table_info_id;
-
- if (join_graph[first_table_info_id].find(candidate_table_info_id)
- == join_graph[first_table_info_id].end() &&
- join_graph[candidate_table_info_id].find(first_table_info_id)
- == join_graph[candidate_table_info_id].end()) {
- continue;
- } else if (second_table_info == nullptr) {
- second_table_info = candidate_table_info;
- second_table_info_it = candidate_table_info_it;
- }
-
- bool is_likely_many_to_many_join = false;
- for (const auto join_attr_pair : first_table_info->join_attribute_pairs) {
- if (candidate_table_info->joined_attribute_set.find(join_attr_pair.second)
- != candidate_table_info->joined_attribute_set.end()) {
- is_likely_many_to_many_join = true;
- break;
- }
- }
- for (const auto join_attr_pair : candidate_table_info->join_attribute_pairs) {
- if (first_table_info->joined_attribute_set.find(join_attr_pair.second)
- != first_table_info->joined_attribute_set.end()) {
- is_likely_many_to_many_join = true;
- break;
+ // TODO(jianqiao): design better data structure to improve efficiency here.
+ std::unique_ptr<JoinPair> best_join = nullptr;
+ for (TableInfo *probe_table_info : remaining_tables) {
+ for (TableInfo *build_table_info : remaining_tables) {
+ if (probe_table_info != build_table_info) {
+ std::vector<E::AttributeReferencePtr> build_attrs;
+ const std::size_t probe_table_id = probe_table_info->table_info_id;
+ const std::size_t build_table_id = build_table_info->table_info_id;
+ for (const auto &attr_group_pair : join_attribute_groups) {
+ const auto &attr_group = attr_group_pair.second;
+ auto probe_it = attr_group.find(probe_table_id);
+ auto build_it = attr_group.find(build_table_id);
+ if (probe_it != attr_group.end() && build_it != attr_group.end()) {
+ build_attrs.emplace_back(
+ attribute_id_to_reference_map.at(build_it->second));
+ }
+ }
+ if (!build_attrs.empty()
+ && build_table_info->table->impliesUniqueAttributes(build_attrs)) {
+ std::unique_ptr<JoinPair> new_join(
+ new JoinPair(probe_table_info, build_table_info));
+ if (best_join == nullptr || new_join->isBetterThan(*best_join)) {
+// if (best_join != nullptr) {
+// std::cerr << "(" << best_join->probe->estimated_selectivity
+// << ", " << best_join->probe->estimated_cardinality << ")"
+// << " -- "
+// << "(" << best_join->build->estimated_selectivity
+// << ", " << best_join->build->estimated_cardinality << ")"
+// << "\n";
+// std::cerr << "REPLACED WITH\n";
+// }
+// std::cerr << "(" << new_join->probe->estimated_selectivity
+// << ", " << new_join->probe->estimated_cardinality << ")"
+// << " -- "
+// << "(" << new_join->build->estimated_selectivity
+// << ", " << new_join->build->estimated_cardinality << ")"
+// << "\n****\n";
+ best_join.reset(new_join.release());
+ }
+ }
}
}
- if (!is_likely_many_to_many_join) {
- second_table_info = candidate_table_info;
- second_table_info_it = candidate_table_info_it;
- break;
- }
}
- DCHECK(second_table_info != nullptr);
- table_info_ordered_by_priority.erase(second_table_info_it);
- const P::PhysicalPtr &left_child = first_table_info->table;
- const P::PhysicalPtr &right_child = second_table_info->table;
+ TableInfo *selected_probe_table_info = nullptr;
+ TableInfo *selected_build_table_info = nullptr;
+
+ if (best_join != nullptr) {
+ selected_probe_table_info = best_join->probe;
+ selected_build_table_info = best_join->build;
+ }
+
+ // TODO(jianqiao): Handle the case when there is no primary key-foreign key information available.
+ CHECK(selected_probe_table_info != nullptr);
+ CHECK(selected_build_table_info != nullptr);
+
+// std::cerr << selected_probe_table_info->estimated_selectivity
+// << " -- "
+// << selected_build_table_info->estimated_selectivity
+// << "\n";
+
+// std::cerr << selected_probe_table_info->estimated_num_output_attributes
+// << " -- "
+// << selected_build_table_info->estimated_num_output_attributes
+// << "\n";
+
+ remaining_tables.erase(selected_probe_table_info);
+ remaining_tables.erase(selected_build_table_info);
+
+ const P::PhysicalPtr &probe_child = selected_probe_table_info->table;
+ const P::PhysicalPtr &build_child = selected_build_table_info->table;
std::vector<E::NamedExpressionPtr> output_attributes;
- for (const E::AttributeReferencePtr &left_attr : left_child->getOutputAttributes()) {
- output_attributes.emplace_back(left_attr);
+ for (const E::AttributeReferencePtr &probe_attr : probe_child->getOutputAttributes()) {
+ output_attributes.emplace_back(probe_attr);
}
- for (const E::AttributeReferencePtr &right_attr : right_child->getOutputAttributes()) {
- output_attributes.emplace_back(right_attr);
+ for (const E::AttributeReferencePtr &build_attr : build_child->getOutputAttributes()) {
+ output_attributes.emplace_back(build_attr);
}
- std::vector<E::AttributeReferencePtr> left_join_attributes;
- std::vector<E::AttributeReferencePtr> right_join_attributes;
- std::unordered_set<expressions::ExprId> new_joined_attribute_set;
- for (const auto &join_attr_pair : first_table_info->join_attribute_pairs) {
- if (second_table_info->join_attribute_pairs.find(join_attr_pair.second)
- != second_table_info->join_attribute_pairs.end()) {
- left_join_attributes.emplace_back(
- attribute_id_to_reference_map[join_attr_pair.first]);
- right_join_attributes.emplace_back(
- attribute_id_to_reference_map[join_attr_pair.second]);
-
- new_joined_attribute_set.emplace(join_attr_pair.first);
- new_joined_attribute_set.emplace(join_attr_pair.second);
+ std::vector<E::AttributeReferencePtr> probe_attributes;
+ std::vector<E::AttributeReferencePtr> build_attributes;
+ const std::size_t probe_table_id = selected_probe_table_info->table_info_id;
+ const std::size_t build_table_id = selected_build_table_info->table_info_id;
+ for (const auto &attr_group_pair : join_attribute_groups) {
+ const auto &attr_group = attr_group_pair.second;
+ auto probe_it = attr_group.find(probe_table_id);
+ auto build_it = attr_group.find(build_table_id);
+ if (probe_it != attr_group.end() && build_it != attr_group.end()) {
+ probe_attributes.emplace_back(
+ attribute_id_to_reference_map.at(probe_it->second));
+ build_attributes.emplace_back(
+ attribute_id_to_reference_map.at(build_it->second));
}
}
- DCHECK_GE(left_join_attributes.size(), static_cast<std::size_t>(1));
- if (table_info_ordered_by_priority.size() > 0) {
+ if (remaining_tables.size() > 0) {
P::PhysicalPtr output =
- P::HashJoin::Create(left_child,
- right_child,
- left_join_attributes,
- right_join_attributes,
+ P::HashJoin::Create(probe_child,
+ build_child,
+ probe_attributes,
+ build_attributes,
nullptr,
output_attributes,
P::HashJoin::JoinType::kInnerJoin);
- second_table_info->table = output;
+// P::PhysicalPtr output;
+// if (selected_build_table_info->estimated_num_output_attributes >= 4 &&
+// selected_probe_table_info->estimated_num_output_attributes < 4) {
+// output = P::HashJoin::Create(build_child,
+// probe_child,
+// build_attributes,
+// probe_attributes,
+// nullptr,
+// output_attributes,
+// P::HashJoin::JoinType::kInnerJoin);
+// } else {
+// output = P::HashJoin::Create(probe_child,
+// build_child,
+// probe_attributes,
+// build_attributes,
+// nullptr,
+// output_attributes,
+// P::HashJoin::JoinType::kInnerJoin);
+// }
+
+ selected_probe_table_info->table = output;
// TODO(jianqiao): Cache the estimated cardinality for each plan in cost
// model to avoid duplicated estimation.
- second_table_info->estimated_cardinality = cost_model_->estimateCardinality(output);
-
- second_table_info->join_attribute_pairs.insert(first_table_info->join_attribute_pairs.begin(),
- first_table_info->join_attribute_pairs.end());
- second_table_info->joined_attribute_set.insert(first_table_info->joined_attribute_set.begin(),
- first_table_info->joined_attribute_set.end());
- second_table_info->joined_attribute_set.insert(new_joined_attribute_set.begin(),
- new_joined_attribute_set.end());
- table_info_ordered_by_priority.emplace(second_table_info);
-
- join_graph[second_table_info->table_info_id].insert(join_graph[first_table_info_id].begin(),
- join_graph[first_table_info_id].end());
-
+ selected_probe_table_info->estimated_cardinality = cost_model_->estimateCardinality(output);
+ selected_probe_table_info->estimated_selectivity = cost_model_->estimateSelectivity(output);
+
+ selected_probe_table_info->estimated_num_output_attributes =
+ CountSharedAttributes(join_group.referenced_attributes,
+ output->getOutputAttributes());
+ selected_probe_table_info->is_aggregation = false;
+
+ remaining_tables.emplace(selected_probe_table_info);
+
+ // Update join attribute groups.
+ for (auto &attr_group_pair : join_attribute_groups) {
+ auto &attr_group = attr_group_pair.second;
+ auto build_it = attr_group.find(build_table_id);
+ if (build_it != attr_group.end()) {
+ const E::ExprId attr_id = build_it->second;
+ attr_group.erase(build_it);
+ attr_group.emplace(probe_table_id, attr_id);
+ }
+ }
} else {
- return P::HashJoin::Create(left_child,
- right_child,
- left_join_attributes,
- right_join_attributes,
+ return P::HashJoin::Create(probe_child,
+ build_child,
+ probe_attributes,
+ build_attributes,
residual_predicate,
project_expressions,
P::HashJoin::JoinType::kInnerJoin);
@@ -307,5 +365,18 @@ physical::PhysicalPtr StarSchemaHashJoinOrderOptimization::generatePlan(
}
}
+std::size_t StarSchemaHashJoinOrderOptimization::CountSharedAttributes(
+ const std::unordered_set<expressions::ExprId> &attr_set1,
+ const std::vector<expressions::AttributeReferencePtr> &attr_set2) {
+ std::size_t cnt = 0;
+ for (const auto &attr : attr_set2) {
+ if (attr_set1.find(attr->id()) != attr_set1.end()) {
+ ++cnt;
+ }
+ }
+ return cnt;
+}
+
+
} // namespace optimizer
} // namespace quickstep
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
index 4d6765c..eb21d03 100644
--- a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
+++ b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
@@ -64,6 +64,7 @@ class StarSchemaHashJoinOrderOptimization : public Rule<physical::Physical> {
* @brief A group of tables to form a hash join tree.
*/
struct JoinGroupInfo {
+ std::unordered_set<expressions::ExprId> referenced_attributes;
std::vector<physical::PhysicalPtr> tables;
std::vector<std::pair<expressions::ExprId, expressions::ExprId>> join_attribute_pairs;
};
@@ -72,49 +73,84 @@ class StarSchemaHashJoinOrderOptimization : public Rule<physical::Physical> {
* @brief Auxiliary information of a table for the optimizer.
*/
struct TableInfo {
- TableInfo(const std::size_t in_table_info_id,
- const physical::PhysicalPtr &in_table,
- const std::size_t in_estimated_cardinality,
- const double in_estimated_selectivity)
- : table_info_id(in_table_info_id),
- table(in_table),
- estimated_cardinality(in_estimated_cardinality),
- estimated_selectivity(in_estimated_selectivity) {
+ TableInfo(const std::size_t table_info_id_in,
+ const physical::PhysicalPtr &table_in,
+ const std::size_t estimated_cardinality_in,
+ const double estimated_selectivity_in,
+ const std::size_t estimated_num_output_attributes_in,
+ const bool is_aggregation_in)
+ : table_info_id(table_info_id_in),
+ table(table_in),
+ estimated_cardinality(estimated_cardinality_in),
+ estimated_selectivity(estimated_selectivity_in),
+ estimated_num_output_attributes(estimated_num_output_attributes_in),
+ is_aggregation(is_aggregation_in) {
}
const std::size_t table_info_id;
physical::PhysicalPtr table;
std::size_t estimated_cardinality;
double estimated_selectivity;
- std::unordered_multimap<expressions::ExprId, expressions::ExprId> join_attribute_pairs;
- std::unordered_set<expressions::ExprId> joined_attribute_set;
+ std::size_t estimated_num_output_attributes;
+ bool is_aggregation;
};
- /**
- * @brief Comparator that compares the join priorities between two tables.
- */
- struct TableInfoPtrLessComparator {
- inline bool operator() (const TableInfo *lhs, const TableInfo *rhs) {
- bool swapped = false;
- if (lhs->estimated_cardinality > rhs->estimated_cardinality) {
- std::swap(lhs, rhs);
- swapped = true;
+ struct JoinPair {
+ JoinPair(TableInfo *probe_in, TableInfo *build_in)
+ : probe(probe_in), build(build_in) {
+ }
+
+ inline bool isBetterThan(const JoinPair &rhs) const {
+ const auto &lhs = *this;
+ const bool lhs_has_large_output =
+ lhs.build->estimated_num_output_attributes
+ + lhs.probe->estimated_num_output_attributes > 5;
+ const bool rhs_has_large_output =
+ rhs.build->estimated_num_output_attributes
+ + rhs.probe->estimated_num_output_attributes > 5;
+ if (lhs_has_large_output || rhs_has_large_output) {
+ if (lhs_has_large_output != rhs_has_large_output) {
+ return rhs_has_large_output;
+ }
+ double lhs_selectivity =
+ lhs.build->estimated_selectivity * lhs.probe->estimated_selectivity;
+ double rhs_selectivity =
+ rhs.build->estimated_selectivity * rhs.probe->estimated_selectivity;
+ if (lhs_selectivity != rhs_selectivity) {
+ return lhs_selectivity < rhs_selectivity;
+ }
}
- if (lhs->estimated_selectivity < rhs->estimated_selectivity) {
- return !swapped;
- } else if (lhs->estimated_cardinality < 1000u &&
- rhs->estimated_cardinality > 10000u &&
- lhs->estimated_selectivity < rhs->estimated_selectivity * 1.5) {
- return !swapped;
- } else if (lhs->estimated_selectivity > rhs->estimated_selectivity) {
- return swapped;
- } else if (lhs->estimated_cardinality != rhs->estimated_cardinality) {
- return !swapped;
+ const bool lhs_has_small_build =
+ !lhs_has_large_output && lhs.build->estimated_cardinality < 0x100;
+ const bool rhs_has_small_build =
+ !rhs_has_large_output && rhs.build->estimated_cardinality < 0x100;
+ if (lhs_has_small_build != rhs_has_small_build) {
+ return lhs_has_small_build;
+ }
+
+ if (lhs.probe->is_aggregation != rhs.probe->is_aggregation) {
+ return lhs.probe->is_aggregation;
+ }
+
+ if (lhs.probe->estimated_cardinality != rhs.probe->estimated_cardinality) {
+ return lhs.probe->estimated_cardinality < rhs.probe->estimated_cardinality;
+ }
+ if (lhs.build->estimated_selectivity != rhs.build->estimated_selectivity) {
+ return lhs.build->estimated_selectivity < rhs.build->estimated_selectivity;
+ }
+ if (lhs.build->estimated_cardinality != rhs.build->estimated_cardinality) {
+ return lhs.build->estimated_cardinality < rhs.build->estimated_cardinality;
+ }
+ if (lhs.probe->table != rhs.probe->table) {
+ return lhs.probe->table < rhs.probe->table;
} else {
- return swapped ^ (lhs->table < rhs->table);
+ return lhs.build->table < rhs.build->table;
}
}
+
+ TableInfo *probe;
+ TableInfo *build;
};
physical::PhysicalPtr applyInternal(const physical::PhysicalPtr &input,
@@ -125,6 +161,10 @@ class StarSchemaHashJoinOrderOptimization : public Rule<physical::Physical> {
const expressions::PredicatePtr &residual_predicate,
const std::vector<expressions::NamedExpressionPtr> &project_expressions);
+ static std::size_t CountSharedAttributes(
+ const std::unordered_set<expressions::ExprId> &attr_set1,
+ const std::vector<expressions::AttributeReferencePtr> &attr_set2);
+
std::unique_ptr<cost::StarSchemaSimpleCostModel> cost_model_;
DISALLOW_COPY_AND_ASSIGN(StarSchemaHashJoinOrderOptimization);
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/ExecutionHeuristics_unittest.cpp b/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
index 73b3e84..efea423 100644
--- a/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
+++ b/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
@@ -72,7 +72,8 @@ class ExecutionHeuristicsTest : public ::testing::Test {
probe_relation,
std::move(build_attribute_ids),
std::move(probe_attribute_ids),
- join_hash_table_id);
+ join_hash_table_id,
+ build_relation->estimateTupleCardinality());
}
QueryPlan::DAGNodeIndex createDummyBuildHashOperator(QueryPlan *query_plan,
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index 779c0fe..9b573ac 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -59,6 +59,11 @@ using std::vector;
namespace quickstep {
+DEFINE_int64(bloom_adapter_batch_size, 64,
+ "Number of tuples to probe in bulk in Bloom filter adapter.");
+DEFINE_bool(adapt_bloom_filters, true,
+ "Whether to adaptively adjust the ordering of bloom filters.");
+
namespace {
// Functor passed to HashTable::getAllFromValueAccessor() to collect matching
@@ -75,6 +80,11 @@ class MapBasedJoinedTupleCollector {
joined_tuples_[tref.block].emplace_back(tref.tuple, accessor.getCurrentPosition());
}
+ inline void operator()(const tuple_id probe_tid,
+ const TupleReference &build_tref) {
+ joined_tuples_[build_tref.block].emplace_back(build_tref.tuple, probe_tid);
+ }
+
// Get a mutable pointer to the collected map of joined tuple ID pairs. The
// key is inner block_id, values are vectors of joined tuple ID pairs with
// tuple ID from the inner block on the left and the outer block on the
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index fa393b6..30571a1 100644
--- a/relational_operators/HashJoinOperator.hpp
+++ b/relational_operators/HashJoinOperator.hpp
@@ -307,8 +307,9 @@ class HashInnerJoinWorkOrder : public WorkOrder {
const std::vector<std::unique_ptr<const Scalar>> &selection,
const JoinHashTable &hash_table,
InsertDestination *output_destination,
- StorageManager *storage_manager)
- : WorkOrder(query_id),
+ StorageManager *storage_manager,
+ const int op_index = -1)
+ : WorkOrder(query_id, op_index),
build_relation_(build_relation),
probe_relation_(probe_relation),
join_key_attributes_(join_key_attributes),
@@ -354,8 +355,9 @@ class HashInnerJoinWorkOrder : public WorkOrder {
const std::vector<std::unique_ptr<const Scalar>> &selection,
const JoinHashTable &hash_table,
InsertDestination *output_destination,
- StorageManager *storage_manager)
- : WorkOrder(query_id),
+ StorageManager *storage_manager,
+ const int op_index = -1)
+ : WorkOrder(query_id, op_index),
build_relation_(build_relation),
probe_relation_(probe_relation),
join_key_attributes_(std::move(join_key_attributes)),
@@ -435,8 +437,9 @@ class HashSemiJoinWorkOrder : public WorkOrder {
const std::vector<std::unique_ptr<const Scalar>> &selection,
const JoinHashTable &hash_table,
InsertDestination *output_destination,
- StorageManager *storage_manager)
- : WorkOrder(query_id),
+ StorageManager *storage_manager,
+ const int op_index = -1)
+ : WorkOrder(query_id, op_index),
build_relation_(build_relation),
probe_relation_(probe_relation),
join_key_attributes_(join_key_attributes),
@@ -482,8 +485,9 @@ class HashSemiJoinWorkOrder : public WorkOrder {
const std::vector<std::unique_ptr<const Scalar>> &selection,
const JoinHashTable &hash_table,
InsertDestination *output_destination,
- StorageManager *storage_manager)
- : WorkOrder(query_id),
+ StorageManager *storage_manager,
+ const int op_index = -1)
+ : WorkOrder(query_id, op_index),
build_relation_(build_relation),
probe_relation_(probe_relation),
join_key_attributes_(std::move(join_key_attributes)),
@@ -559,8 +563,9 @@ class HashAntiJoinWorkOrder : public WorkOrder {
const std::vector<std::unique_ptr<const Scalar>> &selection,
const JoinHashTable &hash_table,
InsertDestination *output_destination,
- StorageManager *storage_manager)
- : WorkOrder(query_id),
+ StorageManager *storage_manager,
+ const int op_index = -1)
+ : WorkOrder(query_id, op_index),
build_relation_(build_relation),
probe_relation_(probe_relation),
join_key_attributes_(join_key_attributes),
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/relational_operators/WorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.hpp b/relational_operators/WorkOrder.hpp
index c1b9b68..2eeca06 100644
--- a/relational_operators/WorkOrder.hpp
+++ b/relational_operators/WorkOrder.hpp
@@ -300,16 +300,23 @@ class WorkOrder {
return query_id_;
}
+ inline const int getOperatorIndex() const {
+ return op_index_;
+ }
+
protected:
/**
* @brief Constructor.
*
* @param query_id The ID of the query to which this WorkOrder belongs.
**/
- explicit WorkOrder(const std::size_t query_id)
- : query_id_(query_id) {}
+ explicit WorkOrder(const std::size_t query_id,
+ const int op_index = -1)
+ : query_id_(query_id),
+ op_index_(op_index) {}
const std::size_t query_id_;
+ const int op_index_;
// A vector of preferred NUMA node IDs where this workorder should be executed.
// These node IDs typically indicate the NUMA node IDs of the input(s) of the
// workorder. Derived classes should ensure that there are no duplicate entries
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index 3f6e23a..d225258 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -46,10 +46,13 @@
#include "storage/StorageBlock.hpp"
#include "storage/StorageBlockInfo.hpp"
#include "storage/StorageManager.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorUtil.hpp"
#include "types/TypedValue.hpp"
#include "types/containers/ColumnVector.hpp"
#include "types/containers/ColumnVectorsValueAccessor.hpp"
#include "types/containers/Tuple.hpp"
+#include "utility/BloomFilterAdapter.hpp"
#include "glog/logging.h"
@@ -57,6 +60,8 @@ using std::unique_ptr;
namespace quickstep {
+DECLARE_int64(bloom_adapter_batch_size);
+
AggregationOperationState::AggregationOperationState(
const CatalogRelationSchema &input_relation,
const std::vector<const AggregateFunction*> &aggregate_functions,
@@ -64,12 +69,16 @@ AggregationOperationState::AggregationOperationState(
std::vector<bool> &&is_distinct,
std::vector<std::unique_ptr<const Scalar>> &&group_by,
const Predicate *predicate,
+ std::vector<const BloomFilter *> &&bloom_filters,
+ std::vector<attribute_id> &&bloom_filter_attribute_ids,
const std::size_t estimated_num_entries,
const HashTableImplType hash_table_impl_type,
const std::vector<HashTableImplType> &distinctify_hash_table_impl_types,
StorageManager *storage_manager)
: input_relation_(input_relation),
predicate_(predicate),
+ bloom_filters_(std::move(bloom_filters)),
+ bloom_filter_attribute_ids_(std::move(bloom_filter_attribute_ids)),
group_by_list_(std::move(group_by)),
arguments_(std::move(arguments)),
is_distinct_(std::move(is_distinct)),
@@ -183,7 +192,8 @@ AggregationOperationState::AggregationOperationState(
AggregationOperationState* AggregationOperationState::ReconstructFromProto(
const serialization::AggregationOperationState &proto,
const CatalogDatabaseLite &database,
- StorageManager *storage_manager) {
+ StorageManager *storage_manager,
+ const std::vector<std::unique_ptr<BloomFilter>> &bloom_filters) {
DCHECK(ProtoIsValid(proto, database));
// Rebuild contructor arguments from their representation in 'proto'.
@@ -232,12 +242,24 @@ AggregationOperationState* AggregationOperationState::ReconstructFromProto(
database));
}
+ std::vector<const BloomFilter*> bloom_filter_vector;
+ std::vector<attribute_id> bloom_filter_attribute_ids;
+ for (int i = 0; i < proto.bloom_filters_size(); ++i) {
+ // Add the pointer to the probe bloom filter within the list of probe bloom filters to use.
+ const auto bloom_filter_proto = proto.bloom_filters(i);
+ bloom_filter_vector.emplace_back(
+ bloom_filters[bloom_filter_proto.bloom_filter_id()].get());
+ bloom_filter_attribute_ids.emplace_back(bloom_filter_proto.attr_id());
+ }
+
return new AggregationOperationState(database.getRelationSchemaById(proto.relation_id()),
aggregate_functions,
std::move(arguments),
std::move(is_distinct),
std::move(group_by_expressions),
predicate.release(),
+ std::move(bloom_filter_vector),
+ std::move(bloom_filter_attribute_ids),
proto.estimated_num_entries(),
HashTableImplTypeFromProto(proto.hash_table_impl_type()),
distinctify_hash_table_impl_types,
@@ -340,6 +362,10 @@ void AggregationOperationState::aggregateBlockSingleState(const block_id input_b
// tuples so that it can be reused across multiple aggregates (i.e. we only
// pay the cost of evaluating the predicate once).
std::unique_ptr<TupleIdSequence> reuse_matches;
+ if (predicate_) {
+ reuse_matches.reset(block->getMatchesForPredicate(predicate_.get()));
+ }
+
for (std::size_t agg_idx = 0;
agg_idx < handles_.size();
++agg_idx) {
@@ -358,7 +384,6 @@ void AggregationOperationState::aggregateBlockSingleState(const block_id input_b
arguments_[agg_idx],
local_arguments_as_attributes,
{}, /* group_by */
- predicate_.get(),
distinctify_hashtables_[agg_idx].get(),
&reuse_matches,
nullptr /* reuse_group_by_vectors */);
@@ -369,7 +394,6 @@ void AggregationOperationState::aggregateBlockSingleState(const block_id input_b
block->aggregate(*handles_[agg_idx],
arguments_[agg_idx],
local_arguments_as_attributes,
- predicate_.get(),
&reuse_matches));
}
}
@@ -391,6 +415,72 @@ void AggregationOperationState::aggregateBlockHashTable(const block_id input_blo
// GROUP BY expressions once).
std::vector<std::unique_ptr<ColumnVector>> reuse_group_by_vectors;
+ if (predicate_) {
+ reuse_matches.reset(block->getMatchesForPredicate(predicate_.get()));
+ }
+
+ if (bloom_filters_.size() > 0) {
+ const std::size_t num_tuples = block->getNumTuples();
+// std::cerr << "Before: " << num_tuples << " -- "
+// << (reuse_matches ? reuse_matches->numTuples() : num_tuples)
+// << "\n";
+ std::unique_ptr<ValueAccessor> accessor;
+ if (reuse_matches) {
+ accessor.reset(
+ block->getTupleStorageSubBlock().createValueAccessor(reuse_matches.get()));
+ } else {
+ accessor.reset(
+ block->getTupleStorageSubBlock().createValueAccessor());
+ }
+ InvokeOnAnyValueAccessor(
+ accessor.get(),
+ [&](auto *accessor) -> void { // NOLINT(build/c++11)
+ std::unique_ptr<TupleIdSequence> filtered(new TupleIdSequence(num_tuples));
+
+ std::vector<std::size_t> attr_size_vector;
+ attr_size_vector.reserve(bloom_filter_attribute_ids_.size());
+ for (const auto &attr : bloom_filter_attribute_ids_) {
+ auto val_and_size =
+ accessor->template getUntypedValueAndByteLengthAtAbsolutePosition<false>(0, attr);
+ attr_size_vector.emplace_back(val_and_size.second);
+ }
+
+ std::unique_ptr<BloomFilterAdapter> bloom_filter_adapter;
+ bloom_filter_adapter.reset(new BloomFilterAdapter(
+ bloom_filters_, bloom_filter_attribute_ids_, attr_size_vector));
+
+ std::uint32_t batch_size_try = FLAGS_bloom_adapter_batch_size;
+ std::uint32_t num_tuples_left = accessor->getNumTuples();
+ std::vector<tuple_id> batch(num_tuples_left);
+
+ do {
+ std::uint32_t batch_size =
+ batch_size_try < num_tuples_left ? batch_size_try : num_tuples_left;
+ for (std::size_t i = 0; i < batch_size; ++i) {
+ accessor->next();
+ batch.push_back(accessor->getCurrentPosition());
+ }
+
+ std::size_t num_hits =
+ bloom_filter_adapter->bulkProbe<true>(accessor, batch, batch_size);
+ for (std::size_t t = 0; t < num_hits; ++t){
+ filtered->set(batch[t], true);
+ }
+
+ batch.clear();
+ num_tuples_left -= batch_size;
+ batch_size_try = batch_size * 2;
+ } while (num_tuples_left > 0);
+
+ if (reuse_matches) {
+ reuse_matches->intersectWith(*filtered);
+ } else {
+ reuse_matches.reset(filtered.release());
+ }
+ });
+// std::cerr << "After: " << reuse_matches->numTuples() << "\n";
+ }
+
for (std::size_t agg_idx = 0;
agg_idx < handles_.size();
++agg_idx) {
@@ -402,7 +492,6 @@ void AggregationOperationState::aggregateBlockHashTable(const block_id input_blo
arguments_[agg_idx],
nullptr, /* arguments_as_attributes */
group_by_list_,
- predicate_.get(),
distinctify_hashtables_[agg_idx].get(),
&reuse_matches,
&reuse_group_by_vectors);
@@ -416,7 +505,6 @@ void AggregationOperationState::aggregateBlockHashTable(const block_id input_blo
block->aggregateGroupBy(*handles_[agg_idx],
arguments_[agg_idx],
group_by_list_,
- predicate_.get(),
agg_hash_table,
&reuse_matches,
&reuse_group_by_vectors);
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/AggregationOperationState.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.hpp b/storage/AggregationOperationState.hpp
index ecd116b..30b8f76 100644
--- a/storage/AggregationOperationState.hpp
+++ b/storage/AggregationOperationState.hpp
@@ -33,6 +33,7 @@
#include "storage/HashTableBase.hpp"
#include "storage/HashTablePool.hpp"
#include "storage/StorageBlockInfo.hpp"
+#include "utility/BloomFilter.hpp"
#include "utility/Macros.hpp"
namespace quickstep {
@@ -108,6 +109,8 @@ class AggregationOperationState {
std::vector<bool> &&is_distinct,
std::vector<std::unique_ptr<const Scalar>> &&group_by,
const Predicate *predicate,
+ std::vector<const BloomFilter *> &&bloom_filters,
+ std::vector<attribute_id> &&bloom_filter_attribute_ids,
const std::size_t estimated_num_entries,
const HashTableImplType hash_table_impl_type,
const std::vector<HashTableImplType> &distinctify_hash_table_impl_types,
@@ -131,7 +134,8 @@ class AggregationOperationState {
static AggregationOperationState* ReconstructFromProto(
const serialization::AggregationOperationState &proto,
const CatalogDatabaseLite &database,
- StorageManager *storage_manager);
+ StorageManager *storage_manager,
+ const std::vector<std::unique_ptr<BloomFilter>> &bloom_filters);
/**
* @brief Check whether a serialization::AggregationOperationState is
@@ -181,6 +185,10 @@ class AggregationOperationState {
// filter predicate (if any), and the list of GROUP BY expressions (if any).
const CatalogRelationSchema &input_relation_;
std::unique_ptr<const Predicate> predicate_;
+
+ std::vector<const BloomFilter*> bloom_filters_;
+ std::vector<attribute_id> bloom_filter_attribute_ids_;
+
std::vector<std::unique_ptr<const Scalar>> group_by_list_;
// Each individual aggregate in this operation has an AggregationHandle and
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/AggregationOperationState.proto
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.proto b/storage/AggregationOperationState.proto
index 7521d73..04b661c 100644
--- a/storage/AggregationOperationState.proto
+++ b/storage/AggregationOperationState.proto
@@ -42,4 +42,10 @@ message AggregationOperationState {
// Each DISTINCT aggregation has its distinctify hash table impl type.
repeated HashTableImplType distinctify_hash_table_impl_types = 7;
+
+ message BloomFilter {
+ required uint32 bloom_filter_id = 1;
+ required uint32 attr_id = 2;
+ }
+ repeated BloomFilter bloom_filters = 8;
}
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/BasicColumnStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/BasicColumnStoreValueAccessor.hpp b/storage/BasicColumnStoreValueAccessor.hpp
index 0560d99..f4c6cd9 100644
--- a/storage/BasicColumnStoreValueAccessor.hpp
+++ b/storage/BasicColumnStoreValueAccessor.hpp
@@ -20,6 +20,8 @@
#ifndef QUICKSTEP_STORAGE_BASIC_COLUMN_STORE_VALUE_ACCESSOR_HPP_
#define QUICKSTEP_STORAGE_BASIC_COLUMN_STORE_VALUE_ACCESSOR_HPP_
+#include <cstddef>
+#include <utility>
#include <vector>
#include "catalog/CatalogRelationSchema.hpp"
@@ -45,7 +47,8 @@ class BasicColumnStoreValueAccessorHelper {
: relation_(relation),
num_tuples_(num_tuples),
column_stripes_(column_stripes),
- column_null_bitmaps_(column_null_bitmaps) {
+ column_null_bitmaps_(column_null_bitmaps),
+ attr_max_lengths_(relation.getMaximumAttributeByteLengths()) {
}
inline tuple_id numPackedTuples() const {
@@ -63,9 +66,23 @@ class BasicColumnStoreValueAccessorHelper {
return nullptr;
}
- // TODO(chasseur): Consider cacheing the byte lengths of attributes.
- return static_cast<const char*>(column_stripes_[attr])
- + (tuple * relation_.getAttributeById(attr)->getType().maximumByteLength());
+ return static_cast<const char*>(column_stripes_[attr]) + (tuple * attr_max_lengths_[attr]);
+ }
+
+ template <bool check_null>
+ inline std::pair<const void*, std::size_t> getAttributeValueAndByteLength(const tuple_id tuple,
+ const attribute_id attr) const {
+ DEBUG_ASSERT(tuple < num_tuples_);
+ DEBUG_ASSERT(relation_.hasAttributeWithId(attr));
+ if (check_null
+ && (!column_null_bitmaps_.elementIsNull(attr))
+ && column_null_bitmaps_[attr].getBit(tuple)) {
+ return std::make_pair(nullptr, 0);
+ }
+
+ const std::size_t attr_length = attr_max_lengths_[attr];
+ return std::make_pair(static_cast<const char*>(column_stripes_[attr]) + (tuple * attr_length),
+ attr_length);
}
inline TypedValue getAttributeValueTyped(const tuple_id tuple,
@@ -82,6 +99,7 @@ class BasicColumnStoreValueAccessorHelper {
const tuple_id num_tuples_;
const std::vector<void*> &column_stripes_;
const PtrVector<BitVector<false>, true> &column_null_bitmaps_;
+ const std::vector<std::size_t> &attr_max_lengths_;
DISALLOW_COPY_AND_ASSIGN(BasicColumnStoreValueAccessorHelper);
};
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/BloomFilterIndexSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/BloomFilterIndexSubBlock.cpp b/storage/BloomFilterIndexSubBlock.cpp
index 4351c05..6ba466e 100644
--- a/storage/BloomFilterIndexSubBlock.cpp
+++ b/storage/BloomFilterIndexSubBlock.cpp
@@ -57,7 +57,6 @@ BloomFilterIndexSubBlock::BloomFilterIndexSubBlock(const TupleStorageSubBlock &t
sub_block_memory_size),
is_initialized_(false),
is_consistent_(false),
- random_seed_(kBloomFilterSeed),
bit_array_size_in_bytes_(description.GetExtension(
BloomFilterIndexSubBlockDescription::bloom_filter_size)) {
CHECK(DescriptionIsValid(relation_, description_))
@@ -76,8 +75,7 @@ BloomFilterIndexSubBlock::BloomFilterIndexSubBlock(const TupleStorageSubBlock &t
const std::uint32_t salt_count = description.GetExtension(BloomFilterIndexSubBlockDescription::number_of_hashes);
// Initialize the bloom_filter_ data structure to operate on bit_array.
- bloom_filter_.reset(new BloomFilter(random_seed_,
- salt_count,
+ bloom_filter_.reset(new BloomFilter(salt_count,
bit_array_size_in_bytes_,
bit_array_.get(),
is_bloom_filter_initialized));
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/BloomFilterIndexSubBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/BloomFilterIndexSubBlock.hpp b/storage/BloomFilterIndexSubBlock.hpp
index 09d6225..4ffbc2c 100644
--- a/storage/BloomFilterIndexSubBlock.hpp
+++ b/storage/BloomFilterIndexSubBlock.hpp
@@ -67,11 +67,6 @@ class BloomFilterIndexSubBlock : public IndexSubBlock {
kSelectivityNone
};
- /**
- * @brief A random seed to initialize the bloom filter hash functions.
- **/
- static const std::uint64_t kBloomFilterSeed = 0xA5A5A5A55A5A5A5AULL;
-
BloomFilterIndexSubBlock(const TupleStorageSubBlock &tuple_store,
const IndexSubBlockDescription &description,
const bool new_block,
@@ -181,7 +176,6 @@ class BloomFilterIndexSubBlock : public IndexSubBlock {
private:
bool is_initialized_;
bool is_consistent_;
- const std::uint64_t random_seed_;
const std::uint64_t bit_array_size_in_bytes_;
std::vector<attribute_id> indexed_attribute_ids_;
std::unique_ptr<unsigned char> bit_array_;
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index 65a7975..8fbe7de 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -679,6 +679,8 @@ target_link_libraries(quickstep_storage_HashTable
quickstep_types_Type
quickstep_types_TypedValue
quickstep_utility_BloomFilter
+ quickstep_utility_BloomFilterAdapter
+ quickstep_utility_EventProfiler
quickstep_utility_HashPair
quickstep_utility_Macros)
target_link_libraries(quickstep_storage_HashTableBase
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/CompressedColumnStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/CompressedColumnStoreValueAccessor.hpp b/storage/CompressedColumnStoreValueAccessor.hpp
index 25e5eed..31ddef5 100644
--- a/storage/CompressedColumnStoreValueAccessor.hpp
+++ b/storage/CompressedColumnStoreValueAccessor.hpp
@@ -54,6 +54,7 @@ class CompressedColumnStoreValueAccessorHelper {
const PtrVector<BitVector<false>, true> &uncompressed_column_null_bitmaps)
: relation_(relation),
num_tuples_(num_tuples),
+ attr_max_lengths_(relation.getMaximumAttributeByteLengths()),
compression_info_(compression_info),
dictionary_coded_attributes_(dictionary_coded_attributes),
truncated_attributes_(truncated_attributes),
@@ -86,6 +87,26 @@ class CompressedColumnStoreValueAccessorHelper {
}
}
+ template <bool check_null>
+ inline std::pair<const void*, std::size_t> getAttributeValueAndByteLength(const tuple_id tuple,
+ const attribute_id attr) const {
+ if (dictionary_coded_attributes_[attr]) {
+ return dictionaries_.atUnchecked(attr).getUntypedValueAndByteLengthForCode<check_null>(
+ getCode(tuple, attr));
+ } else if (truncated_attributes_[attr]) {
+ if (truncated_attribute_is_int_[attr]) {
+ int_buffer_ = getCode(tuple, attr);
+ return std::make_pair(&int_buffer_, sizeof(int_buffer_));
+ } else {
+ long_buffer_ = getCode(tuple, attr);
+ return std::make_pair(&long_buffer_, sizeof(long_buffer_));
+ }
+ } else {
+ return std::make_pair(getAttributePtr<check_null>(tuple, attr),
+ attr_max_lengths_[attr]);
+ }
+ }
+
inline TypedValue getAttributeValueTyped(const tuple_id tuple,
const attribute_id attr) const {
if (dictionary_coded_attributes_[attr]) {
@@ -140,6 +161,7 @@ class CompressedColumnStoreValueAccessorHelper {
const CatalogRelationSchema &relation_;
const tuple_id num_tuples_;
+ const std::vector<std::size_t> &attr_max_lengths_;
const CompressedBlockInfo &compression_info_;
const std::vector<bool> &dictionary_coded_attributes_;
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/CompressedPackedRowStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/CompressedPackedRowStoreValueAccessor.hpp b/storage/CompressedPackedRowStoreValueAccessor.hpp
index 8858175..66b2f5f 100644
--- a/storage/CompressedPackedRowStoreValueAccessor.hpp
+++ b/storage/CompressedPackedRowStoreValueAccessor.hpp
@@ -60,6 +60,7 @@ class CompressedPackedRowStoreValueAccessorHelper {
num_tuples_(num_tuples),
tuple_length_bytes_(tuple_length_bytes),
attribute_offsets_(attribute_offsets),
+ attr_max_lengths_(relation.getMaximumAttributeByteLengths()),
compression_info_(compression_info),
dictionary_coded_attributes_(dictionary_coded_attributes),
truncated_attributes_(truncated_attributes),
@@ -94,6 +95,26 @@ class CompressedPackedRowStoreValueAccessorHelper {
}
}
+ template <bool check_null>
+ inline std::pair<const void*, std::size_t> getAttributeValueAndByteLength(const tuple_id tuple,
+ const attribute_id attr) const {
+ if (dictionary_coded_attributes_[attr]) {
+ return dictionaries_.atUnchecked(attr).getUntypedValueAndByteLengthForCode<check_null>(
+ getCode(tuple, attr));
+ } else if (truncated_attributes_[attr]) {
+ if (truncated_attribute_is_int_[attr]) {
+ int_buffer_ = getCode(tuple, attr);
+ return std::make_pair(&int_buffer_, sizeof(int_buffer_));
+ } else {
+ long_buffer_ = getCode(tuple, attr);
+ return std::make_pair(&long_buffer_, sizeof(long_buffer_));
+ }
+ } else {
+ return std::make_pair(getAttributePtr<check_null>(tuple, attr),
+ attr_max_lengths_[attr]);
+ }
+ }
+
inline TypedValue getAttributeValueTyped(const tuple_id tuple,
const attribute_id attr) const {
if (dictionary_coded_attributes_[attr]) {
@@ -152,6 +173,7 @@ class CompressedPackedRowStoreValueAccessorHelper {
const tuple_id num_tuples_;
const std::size_t tuple_length_bytes_;
const std::vector<std::size_t> &attribute_offsets_;
+ const std::vector<std::size_t> &attr_max_lengths_;
const CompressedBlockInfo &compression_info_;
const std::vector<bool> &dictionary_coded_attributes_;
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/HashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTable.hpp b/storage/HashTable.hpp
index 9fa41a2..3538181 100644
--- a/storage/HashTable.hpp
+++ b/storage/HashTable.hpp
@@ -23,6 +23,7 @@
#include <atomic>
#include <cstddef>
#include <cstdlib>
+#include <memory>
#include <type_traits>
#include <vector>
@@ -39,11 +40,14 @@
#include "types/Type.hpp"
#include "types/TypedValue.hpp"
#include "utility/BloomFilter.hpp"
+#include "utility/BloomFilterAdapter.hpp"
#include "utility/HashPair.hpp"
#include "utility/Macros.hpp"
namespace quickstep {
+DECLARE_int64(bloom_adapter_batch_size);
+
/** \addtogroup Storage
* @{
*/
@@ -1016,8 +1020,12 @@ class HashTable : public HashTableBase<resizable,
*
* @param bloom_filter The pointer to the bloom filter.
**/
- inline void setBuildSideBloomFilter(BloomFilter *bloom_filter) {
- build_bloom_filter_ = bloom_filter;
+ inline void addBuildSideBloomFilter(BloomFilter *bloom_filter) {
+ build_bloom_filters_.emplace_back(bloom_filter);
+ }
+
+ inline void addBuildSideAttributeId(const attribute_id build_attribute_id) {
+ build_attribute_ids_.push_back(build_attribute_id);
}
/**
@@ -1042,8 +1050,8 @@ class HashTable : public HashTableBase<resizable,
* @param probe_attribute_ids The vector of attribute ids to use for probing
* the bloom filter.
**/
- inline void addProbeSideAttributeIds(std::vector<attribute_id> &&probe_attribute_ids) {
- probe_attribute_ids_.push_back(probe_attribute_ids);
+ inline void addProbeSideAttributeId(const attribute_id probe_attribute_id) {
+ probe_attribute_ids_.push_back(probe_attribute_id);
}
protected:
@@ -1329,9 +1337,10 @@ class HashTable : public HashTableBase<resizable,
// Data structures used for bloom filter optimized semi-joins.
bool has_build_side_bloom_filter_ = false;
bool has_probe_side_bloom_filter_ = false;
- BloomFilter *build_bloom_filter_;
+ std::vector<BloomFilter *> build_bloom_filters_;
+ std::vector<attribute_id> build_attribute_ids_;
std::vector<const BloomFilter*> probe_bloom_filters_;
- std::vector<std::vector<attribute_id>> probe_attribute_ids_;
+ std::vector<attribute_id> probe_attribute_ids_;
DISALLOW_COPY_AND_ASSIGN(HashTable);
};
@@ -1477,12 +1486,26 @@ HashTablePutResult HashTable<ValueT, resizable, serializable, force_key_copy, al
&prealloc_state);
}
}
- std::unique_ptr<BloomFilter> thread_local_bloom_filter;
+
if (has_build_side_bloom_filter_) {
- thread_local_bloom_filter.reset(new BloomFilter(build_bloom_filter_->getRandomSeed(),
- build_bloom_filter_->getNumberOfHashes(),
- build_bloom_filter_->getBitArraySize()));
+ for (std::size_t i = 0; i < build_bloom_filters_.size(); ++i) {
+ auto *build_bloom_filter = build_bloom_filters_[i];
+ std::unique_ptr<BloomFilter> thread_local_bloom_filter(
+ new BloomFilter(build_bloom_filter->getNumberOfHashes(),
+ build_bloom_filter->getBitArraySize()));
+ const auto &build_attr = build_attribute_ids_[i];
+ const std::size_t attr_size =
+ accessor->template getUntypedValueAndByteLengthAtAbsolutePosition<false>(0, build_attr).second;
+ while (accessor->next()) {
+ thread_local_bloom_filter->insertUnSafe(
+ static_cast<const std::uint8_t *>(accessor->getUntypedValue(build_attr)),
+ attr_size);
+ }
+ build_bloom_filter->bitwiseOr(thread_local_bloom_filter.get());
+ accessor->beginIteration();
+ }
}
+
if (resizable) {
while (result == HashTablePutResult::kOutOfSpace) {
{
@@ -1498,11 +1521,6 @@ HashTablePutResult HashTable<ValueT, resizable, serializable, force_key_copy, al
variable_size,
(*functor)(*accessor),
using_prealloc ? &prealloc_state : nullptr);
- // Insert into bloom filter, if enabled.
- if (has_build_side_bloom_filter_) {
- thread_local_bloom_filter->insertUnSafe(static_cast<const std::uint8_t *>(key.getDataPtr()),
- key.getDataSize());
- }
if (result == HashTablePutResult::kDuplicateKey) {
DEBUG_ASSERT(!using_prealloc);
return result;
@@ -1528,20 +1546,11 @@ HashTablePutResult HashTable<ValueT, resizable, serializable, force_key_copy, al
variable_size,
(*functor)(*accessor),
using_prealloc ? &prealloc_state : nullptr);
- // Insert into bloom filter, if enabled.
- if (has_build_side_bloom_filter_) {
- thread_local_bloom_filter->insertUnSafe(static_cast<const std::uint8_t *>(key.getDataPtr()),
- key.getDataSize());
- }
if (result != HashTablePutResult::kOK) {
return result;
}
}
}
- // Update the build side bloom filter with thread local copy, if available.
- if (has_build_side_bloom_filter_) {
- build_bloom_filter_->bitwiseOr(thread_local_bloom_filter.get());
- }
return HashTablePutResult::kOK;
});
@@ -1607,6 +1616,26 @@ HashTablePutResult HashTable<ValueT, resizable, serializable, force_key_copy, al
&prealloc_state);
}
}
+
+ if (has_build_side_bloom_filter_) {
+ for (std::size_t i = 0; i < build_bloom_filters_.size(); ++i) {
+ auto *build_bloom_filter = build_bloom_filters_[i];
+ std::unique_ptr<BloomFilter> thread_local_bloom_filter(
+ new BloomFilter(build_bloom_filter->getNumberOfHashes(),
+ build_bloom_filter->getBitArraySize()));
+ const auto &build_attr = build_attribute_ids_[i];
+ const std::size_t attr_size =
+ accessor->template getUntypedValueAndByteLengthAtAbsolutePosition<false>(0, build_attr).second;
+ while (accessor->next()) {
+ thread_local_bloom_filter->insertUnSafe(
+ static_cast<const std::uint8_t *>(accessor->getUntypedValue(build_attr)),
+ attr_size);
+ }
+ build_bloom_filter->bitwiseOr(thread_local_bloom_filter.get());
+ accessor->beginIteration();
+ }
+ }
+
if (resizable) {
while (result == HashTablePutResult::kOutOfSpace) {
{
@@ -2229,6 +2258,7 @@ inline std::size_t HashTable<ValueT, resizable, serializable, force_key_copy, al
}
}
+
template <typename ValueT,
bool resizable,
bool serializable,
@@ -2246,42 +2276,85 @@ void HashTable<ValueT, resizable, serializable, force_key_copy, allow_duplicate_
InvokeOnAnyValueAccessor(
accessor,
[&](auto *accessor) -> void { // NOLINT(build/c++11)
- while (accessor->next()) {
- // Probe any bloom filters, if enabled.
- if (has_probe_side_bloom_filter_) {
- DCHECK_EQ(probe_bloom_filters_.size(), probe_attribute_ids_.size());
- // Check if the key is contained in the BloomFilters or not.
- bool bloom_miss = false;
- for (std::size_t i = 0; i < probe_bloom_filters_.size() && !bloom_miss; ++i) {
- const BloomFilter *bloom_filter = probe_bloom_filters_[i];
- for (const attribute_id &attr_id : probe_attribute_ids_[i]) {
- TypedValue bloom_key = accessor->getTypedValue(attr_id);
- if (!bloom_filter->contains(static_cast<const std::uint8_t*>(bloom_key.getDataPtr()),
- bloom_key.getDataSize())) {
- bloom_miss = true;
+ std::unique_ptr<BloomFilterAdapter> bloom_filter_adapter;
+ if (has_probe_side_bloom_filter_) {
+ // Find (and cache) the size of each attribute in the probe lists.
+ // NOTE(nav): This code uses the accessor to get the size,
+ // and hence only works if there's at least one tuple.
+ std::vector<std::size_t> attr_size_vector;
+ attr_size_vector.reserve(probe_attribute_ids_.size());
+ for (const auto &probe_attr : probe_attribute_ids_) {
+ auto val_and_size =
+ accessor->template getUntypedValueAndByteLengthAtAbsolutePosition<false>(0, probe_attr);
+// std::cerr << "BF attr size = " << val_and_size.second << "\n";
+ attr_size_vector.emplace_back(val_and_size.second);
+ }
+
+ bloom_filter_adapter.reset(new BloomFilterAdapter(
+ probe_bloom_filters_, probe_attribute_ids_, attr_size_vector));
+
+ // We want to have large batch sizes for cache efficiency while probeing,
+ // but small batch sizes to ensure that the adaptation logic kicks in
+ // (and does early). We use exponentially increasing batch sizes to
+ // achieve a balance between the two.
+ //
+ // We also keep track of num_tuples_left in the block, to ensure that
+ // we don't reserve an unnecessarily large vector.
+ std::uint32_t batch_size_try = FLAGS_bloom_adapter_batch_size;
+ std::uint32_t num_tuples_left = accessor->getNumTuples();
+ std::vector<tuple_id> batch(num_tuples_left);
+
+ do {
+ std::uint32_t batch_size =
+ batch_size_try < num_tuples_left ? batch_size_try : num_tuples_left;
+ for (std::size_t i = 0; i < batch_size; ++i) {
+ accessor->next();
+ batch.push_back(accessor->getCurrentPosition());
+ }
+
+ std::size_t num_hits =
+ bloom_filter_adapter->bulkProbe<true>(accessor, batch, batch_size);
+
+ for (std::size_t t = 0; t < num_hits; ++t){
+ tuple_id probe_tid = batch[t];
+ TypedValue key = accessor->getTypedValueAtAbsolutePosition(key_attr_id, probe_tid);
+ if (check_for_null_keys && key.isNull()) {
+ continue;
+ }
+ const std::size_t true_hash = use_scalar_literal_hash_template ? key.getHashScalarLiteral()
+ : key.getHash();
+ const std::size_t adjusted_hash = adjust_hashes_template ? this->AdjustHash(true_hash)
+ : true_hash;
+ std::size_t entry_num = 0;
+ const ValueT *value;
+ while (this->getNextEntryForKey(key, adjusted_hash, &value, &entry_num)) {
+ (*functor)(probe_tid, *value);
+ if (!allow_duplicate_keys)
break;
- }
}
}
- if (bloom_miss) {
- continue; // On a bloom filter miss, probing the hash table can be skipped.
- }
- }
+ batch.clear();
+ num_tuples_left -= batch_size;
+ batch_size_try = batch_size * 2;
+ } while (!accessor->iterationFinished());
+ }
- TypedValue key = accessor->getTypedValue(key_attr_id);
- if (check_for_null_keys && key.isNull()) {
- continue;
- }
- const std::size_t true_hash = use_scalar_literal_hash_template ? key.getHashScalarLiteral()
- : key.getHash();
- const std::size_t adjusted_hash = adjust_hashes_template ? this->AdjustHash(true_hash)
- : true_hash;
- std::size_t entry_num = 0;
- const ValueT *value;
- while (this->getNextEntryForKey(key, adjusted_hash, &value, &entry_num)) {
- (*functor)(*accessor, *value);
- if (!allow_duplicate_keys) {
- break;
+ else { // no Bloom filters to probe
+ while(accessor->next()) {
+ TypedValue key = accessor->getTypedValue(key_attr_id);
+ if (check_for_null_keys && key.isNull()) {
+ continue;
+ }
+ const std::size_t true_hash = use_scalar_literal_hash_template ? key.getHashScalarLiteral()
+ : key.getHash();
+ const std::size_t adjusted_hash = adjust_hashes_template ? this->AdjustHash(true_hash)
+ : true_hash;
+ std::size_t entry_num = 0;
+ const ValueT *value;
+ while (this->getNextEntryForKey(key, adjusted_hash, &value, &entry_num)) {
+ (*functor)(*accessor, *value);
+ if (!allow_duplicate_keys)
+ break;
}
}
}
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/HashTable.proto
----------------------------------------------------------------------
diff --git a/storage/HashTable.proto b/storage/HashTable.proto
index ade30d8..209144d 100644
--- a/storage/HashTable.proto
+++ b/storage/HashTable.proto
@@ -34,10 +34,10 @@ message HashTable {
required HashTableImplType hash_table_impl_type = 1;
repeated Type key_types = 2;
required uint64 estimated_num_entries = 3;
- repeated uint32 build_side_bloom_filter_id = 4;
- message ProbeSideBloomFilter {
- required uint32 probe_side_bloom_filter_id = 1;
- repeated uint32 probe_side_attr_ids = 2;
+ message BloomFilter {
+ required uint32 bloom_filter_id = 1;
+ required uint32 attr_id = 2;
}
- repeated ProbeSideBloomFilter probe_side_bloom_filters = 6;
+ repeated BloomFilter probe_side_bloom_filters = 4;
+ repeated BloomFilter build_side_bloom_filters = 5;
}
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/HashTableFactory.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTableFactory.hpp b/storage/HashTableFactory.hpp
index 40b39de..1b60c53 100644
--- a/storage/HashTableFactory.hpp
+++ b/storage/HashTableFactory.hpp
@@ -321,9 +321,15 @@ class HashTableFactory {
// individual implementations of the hash table constructors.
// Check if there are any build side bloom filter defined on the hash table.
- if (proto.build_side_bloom_filter_id_size() > 0) {
+ if (proto.build_side_bloom_filters_size() > 0) {
hash_table->enableBuildSideBloomFilter();
- hash_table->setBuildSideBloomFilter(bloom_filters[proto.build_side_bloom_filter_id(0)].get());
+ for (int j = 0; j < proto.build_side_bloom_filters_size(); ++j) {
+ const auto build_side_bloom_filter = proto.build_side_bloom_filters(j);
+ hash_table->addBuildSideBloomFilter(
+ bloom_filters[build_side_bloom_filter.bloom_filter_id()].get());
+
+ hash_table->addBuildSideAttributeId(build_side_bloom_filter.attr_id());
+ }
}
// Check if there are any probe side bloom filters defined on the hash table.
@@ -333,15 +339,10 @@ class HashTableFactory {
for (int j = 0; j < proto.probe_side_bloom_filters_size(); ++j) {
// Add the pointer to the probe bloom filter within the list of probe bloom filters to use.
const auto probe_side_bloom_filter = proto.probe_side_bloom_filters(j);
- hash_table->addProbeSideBloomFilter(bloom_filters[probe_side_bloom_filter.probe_side_bloom_filter_id()].get());
-
- // Add the attribute ids corresponding to this probe bloom filter.
- std::vector<attribute_id> probe_attribute_ids;
- for (int k = 0; k < probe_side_bloom_filter.probe_side_attr_ids_size(); ++k) {
- const attribute_id probe_attribute_id = probe_side_bloom_filter.probe_side_attr_ids(k);
- probe_attribute_ids.push_back(probe_attribute_id);
- }
- hash_table->addProbeSideAttributeIds(std::move(probe_attribute_ids));
+ hash_table->addProbeSideBloomFilter(
+ bloom_filters[probe_side_bloom_filter.bloom_filter_id()].get());
+
+ hash_table->addProbeSideAttributeId(probe_side_bloom_filter.attr_id());
}
}
[07/16] incubator-quickstep git commit: Logged all sent messages
using glog.
Posted by ji...@apache.org.
Logged all sent messages using glog.
Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/d9135a8a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/d9135a8a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/d9135a8a
Branch: refs/heads/LIP-for-tpch
Commit: d9135a8a2d11a1eabf6705c88391c498f4be38bb
Parents: 6168996
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Mon Aug 8 22:49:59 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Tue Aug 9 19:48:01 2016 -0700
----------------------------------------------------------------------
query_execution/ForemanSingleNode.cpp | 14 ++--
query_execution/PolicyEnforcerDistributed.cpp | 38 ++++------
query_execution/QueryExecutionUtil.hpp | 6 +-
query_execution/Shiftboss.cpp | 86 +++++++++-------------
query_execution/Worker.cpp | 6 +-
relational_operators/DeleteOperator.cpp | 10 +--
relational_operators/RebuildWorkOrder.hpp | 7 +-
relational_operators/UpdateOperator.cpp | 10 +--
relational_operators/WorkOrder.hpp | 7 +-
storage/InsertDestination.cpp | 18 +++--
storage/InsertDestination.hpp | 10 +--
11 files changed, 92 insertions(+), 120 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/query_execution/ForemanSingleNode.cpp
----------------------------------------------------------------------
diff --git a/query_execution/ForemanSingleNode.cpp b/query_execution/ForemanSingleNode.cpp
index d064a6f..7596b00 100644
--- a/query_execution/ForemanSingleNode.cpp
+++ b/query_execution/ForemanSingleNode.cpp
@@ -168,16 +168,15 @@ void ForemanSingleNode::run() {
// Signal the main thread that there are no queries to be executed.
// Currently the message doesn't have any real content.
TaggedMessage completion_tagged_message(kWorkloadCompletionMessage);
+ DLOG(INFO) << "ForemanSingleNode sent WorkloadCompletionMessage (typed '" << kWorkloadCompletionMessage
+ << "') to CLI with TMB client ID " << main_thread_client_id_;
const tmb::MessageBus::SendStatus send_status =
QueryExecutionUtil::SendTMBMessage(
bus_,
foreman_client_id_,
main_thread_client_id_,
move(completion_tagged_message));
- CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
- << "Message could not be sent from Foreman with TMB client ID "
- << foreman_client_id_ << " to main thread with TMB client ID"
- << main_thread_client_id_;
+ CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
}
}
}
@@ -225,15 +224,14 @@ void ForemanSingleNode::sendWorkerMessage(const size_t worker_thread_index,
}
TaggedMessage worker_tagged_message(&message, sizeof(message), type);
+ DLOG(INFO) << "ForemanSingleNode sent WorkOrderMessage (typed '" << type
+ << "') to Worker with TMB client ID " << worker_directory_->getClientID(worker_thread_index);
const tmb::MessageBus::SendStatus send_status =
QueryExecutionUtil::SendTMBMessage(bus_,
foreman_client_id_,
worker_directory_->getClientID(worker_thread_index),
move(worker_tagged_message));
- CHECK(send_status == tmb::MessageBus::SendStatus::kOK) <<
- "Message could not be sent from Foreman with TMB client ID "
- << foreman_client_id_ << " to Foreman with TMB client ID "
- << worker_directory_->getClientID(worker_thread_index);
+ CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
}
const std::vector<WorkOrderTimeEntry>& ForemanSingleNode
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/query_execution/PolicyEnforcerDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerDistributed.cpp b/query_execution/PolicyEnforcerDistributed.cpp
index 6d0de47..c76a9e1 100644
--- a/query_execution/PolicyEnforcerDistributed.cpp
+++ b/query_execution/PolicyEnforcerDistributed.cpp
@@ -170,25 +170,22 @@ void PolicyEnforcerDistributed::initiateQueryInShiftboss(QueryHandle *query_hand
kQueryInitiateMessage);
free(proto_bytes);
- LOG(INFO) << "PolicyEnforcerDistributed sent QueryInitiateMessage (typed '" << kQueryInitiateMessage
- << "') to Shiftboss 0";
-
// TODO(zuyu): Multiple Shiftbosses support.
+ DLOG(INFO) << "PolicyEnforcerDistributed sent QueryInitiateMessage (typed '" << kQueryInitiateMessage
+ << "') to Shiftboss with TMB client ID " << shiftboss_directory_->getClientId(0);
const tmb::MessageBus::SendStatus send_status =
QueryExecutionUtil::SendTMBMessage(bus_,
foreman_client_id_,
shiftboss_directory_->getClientId(0),
move(message));
- CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
- << "Message could not be sent from Foreman with TMB client ID " << foreman_client_id_
- << " to Shiftboss with TMB client ID " << shiftboss_directory_->getClientId(0);
+ CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
// Wait Shiftboss for QueryInitiateResponseMessage.
const tmb::AnnotatedMessage annotated_message = bus_->Receive(foreman_client_id_, 0, true);
const TaggedMessage &tagged_message = annotated_message.tagged_message;
DCHECK_EQ(kQueryInitiateResponseMessage, tagged_message.message_type());
- LOG(INFO) << "PolicyEnforcerDistributed received typed '" << tagged_message.message_type()
- << "' message from client " << annotated_message.sender;
+ DLOG(INFO) << "PolicyEnforcerDistributed received typed '" << tagged_message.message_type()
+ << "' message from client " << annotated_message.sender;
S::QueryInitiateResponseMessage proto_response;
CHECK(proto_response.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
@@ -215,30 +212,27 @@ void PolicyEnforcerDistributed::onQueryCompletion(QueryManagerBase *query_manage
kQueryTeardownMessage);
// TODO(zuyu): Support multiple shiftbosses.
- LOG(INFO) << "PolicyEnforcerDistributed sent QueryTeardownMessage (typed '" << kQueryTeardownMessage
- << "') to Shiftboss 0";
+ DLOG(INFO) << "PolicyEnforcerDistributed sent QueryTeardownMessage (typed '" << kQueryTeardownMessage
+ << "') to Shiftboss with TMB client ID " << shiftboss_directory_->getClientId(0);
tmb::MessageBus::SendStatus send_status =
QueryExecutionUtil::SendTMBMessage(bus_,
foreman_client_id_,
shiftboss_directory_->getClientId(0),
move(message));
- CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
- << "Message could not be sent from Foreman with TMB client ID " << foreman_client_id_
- << " to Shiftboss";
+ CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
TaggedMessage cli_message(kQueryExecutionSuccessMessage);
// Notify the CLI query execution successfully.
- LOG(INFO) << "PolicyEnforcerDistributed sent QueryExecutionSuccessMessage (typed '" << kQueryExecutionSuccessMessage
- << "') to CLI with TMB client id " << cli_id;
+ DLOG(INFO) << "PolicyEnforcerDistributed sent QueryExecutionSuccessMessage (typed '"
+ << kQueryExecutionSuccessMessage
+ << "') to CLI with TMB client id " << cli_id;
send_status =
QueryExecutionUtil::SendTMBMessage(bus_,
foreman_client_id_,
cli_id,
move(cli_message));
- CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
- << "Message could not be sent from Foreman with TMB client ID " << foreman_client_id_
- << " to CLI with TMB client ID " << cli_id;
+ CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
return;
}
@@ -263,17 +257,15 @@ void PolicyEnforcerDistributed::onQueryCompletion(QueryManagerBase *query_manage
kSaveQueryResultMessage);
free(proto_bytes);
- LOG(INFO) << "PolicyEnforcerDistributed sent SaveQueryResultMessage (typed '" << kSaveQueryResultMessage
- << "') to Shiftboss 0";
// TODO(zuyu): Support multiple shiftbosses.
+ DLOG(INFO) << "PolicyEnforcerDistributed sent SaveQueryResultMessage (typed '" << kSaveQueryResultMessage
+ << "') to Shiftboss with TMB client ID " << shiftboss_directory_->getClientId(0);
const tmb::MessageBus::SendStatus send_status =
QueryExecutionUtil::SendTMBMessage(bus_,
foreman_client_id_,
shiftboss_directory_->getClientId(0),
move(message));
- CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
- << "Message could not be sent from Foreman with TMB client ID " << foreman_client_id_
- << " to Shiftboss with TMB client ID " << shiftboss_directory_->getClientId(0);
+ CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
}
} // namespace quickstep
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/query_execution/QueryExecutionUtil.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionUtil.hpp b/query_execution/QueryExecutionUtil.hpp
index 906fb6b..feb4cc0 100644
--- a/query_execution/QueryExecutionUtil.hpp
+++ b/query_execution/QueryExecutionUtil.hpp
@@ -128,11 +128,11 @@ class QueryExecutionUtil {
address.All(true);
TaggedMessage poison_tagged_message(kPoisonMessage);
+ DLOG(INFO) << "TMB client ID " << sender_id
+ << " broadcast PoisonMessage (typed '" << kPoisonMessage << "') to all";
const tmb::MessageBus::SendStatus send_status = bus->Send(
sender_id, address, style, std::move(poison_tagged_message));
- CHECK(send_status == tmb::MessageBus::SendStatus::kOK) <<
- "Broadcast poison message from sender with TMB client ID " << sender_id
- << " failed";
+ CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
}
private:
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/query_execution/Shiftboss.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Shiftboss.cpp b/query_execution/Shiftboss.cpp
index bd83dd4..ddfd47f 100644
--- a/query_execution/Shiftboss.cpp
+++ b/query_execution/Shiftboss.cpp
@@ -73,9 +73,9 @@ void Shiftboss::run() {
// Receive() is a blocking call, causing this thread to sleep until next
// message is received.
AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
- LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
- << "') received the typed '" << annotated_message.tagged_message.message_type()
- << "' message from client " << annotated_message.sender;
+ DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+ << "') received the typed '" << annotated_message.tagged_message.message_type()
+ << "' message from client " << annotated_message.sender;
switch (annotated_message.tagged_message.message_type()) {
case kShiftbossRegistrationResponseMessage: {
foreman_client_id_ = annotated_message.sender;
@@ -121,18 +121,16 @@ void Shiftboss::run() {
kWorkOrderMessage);
const size_t worker_index = getSchedulableWorker();
- LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
- << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
- << "') from Foreman to worker " << worker_index;
+ DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+ << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
+ << "') from Foreman to worker " << worker_index;
const MessageBus::SendStatus send_status =
QueryExecutionUtil::SendTMBMessage(bus_,
shiftboss_client_id_,
workers_->getClientID(worker_index),
move(worker_tagged_message));
- CHECK(send_status == MessageBus::SendStatus::kOK)
- << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
- << " to Worker with TMB client ID " << workers_->getClientID(worker_index);
+ CHECK(send_status == MessageBus::SendStatus::kOK);
break;
}
case kInitiateRebuildMessage: {
@@ -153,9 +151,10 @@ void Shiftboss::run() {
case kRebuildWorkOrderCompleteMessage:
case kDataPipelineMessage:
case kWorkOrderFeedbackMessage: {
- LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
- << "') forwarded typed '" << annotated_message.tagged_message.message_type()
- << "' message from worker (client " << annotated_message.sender << ") to Foreman";
+ DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+ << "') forwarded typed '" << annotated_message.tagged_message.message_type()
+ << "' message from Worker with TMB client ID '" << annotated_message.sender
+ << "' to Foreman with TMB client ID " << foreman_client_id_;
DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
const MessageBus::SendStatus send_status =
@@ -163,9 +162,7 @@ void Shiftboss::run() {
shiftboss_client_id_,
foreman_client_id_,
move(annotated_message.tagged_message));
- CHECK(send_status == MessageBus::SendStatus::kOK)
- << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
- << " to Foreman with TMB client ID " << foreman_client_id_;
+ CHECK(send_status == MessageBus::SendStatus::kOK);
break;
}
case kSaveQueryResultMessage: {
@@ -190,23 +187,21 @@ void Shiftboss::run() {
kSaveQueryResultResponseMessage);
free(proto_response_bytes);
- LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
- << "') sent SaveQueryResultResponseMessage (typed '" << kSaveQueryResultResponseMessage
- << "') to Foreman";
+ DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+ << "') sent SaveQueryResultResponseMessage (typed '" << kSaveQueryResultResponseMessage
+ << "') to Foreman with TMB client ID " << foreman_client_id_;
const MessageBus::SendStatus send_status =
QueryExecutionUtil::SendTMBMessage(bus_,
shiftboss_client_id_,
foreman_client_id_,
move(message_response));
- CHECK(send_status == MessageBus::SendStatus::kOK)
- << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
- << " to Foreman with TMB client ID " << foreman_client_id_;
+ CHECK(send_status == MessageBus::SendStatus::kOK);
break;
}
case kPoisonMessage: {
- LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
- << "') forwarded PoisonMessage (typed '" << kPoisonMessage
- << "') from Foreman to all workers";
+ DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+ << "') forwarded PoisonMessage (typed '" << kPoisonMessage
+ << "') from Foreman to all workers";
tmb::MessageStyle broadcast_style;
broadcast_style.Broadcast(true);
@@ -216,9 +211,7 @@ void Shiftboss::run() {
worker_addresses_,
broadcast_style,
move(annotated_message.tagged_message));
- CHECK(send_status == MessageBus::SendStatus::kOK)
- << "Message could not be broadcast from Shiftboss with TMB client ID " << shiftboss_client_id_
- << " to All workers";
+ CHECK(send_status == MessageBus::SendStatus::kOK);
return;
}
default: {
@@ -245,10 +238,6 @@ size_t Shiftboss::getSchedulableWorker() {
}
void Shiftboss::registerWithForeman() {
- LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
- << "') sent ShiftbossRegistrationMessage (typed '" << kShiftbossRegistrationMessage
- << "') to all";
-
tmb::Address all_addresses;
all_addresses.All(true);
@@ -266,6 +255,9 @@ void Shiftboss::registerWithForeman() {
kShiftbossRegistrationMessage);
free(proto_bytes);
+ DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+ << "') sent ShiftbossRegistrationMessage (typed '" << kShiftbossRegistrationMessage
+ << "') to all";
tmb::MessageBus::SendStatus send_status =
bus_->Send(shiftboss_client_id_, all_addresses, style, move(message));
DCHECK(send_status == tmb::MessageBus::SendStatus::kOK);
@@ -285,10 +277,6 @@ void Shiftboss::processQueryInitiateMessage(
bus_));
query_contexts_.emplace(query_id, move(query_context));
- LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
- << "') sent QueryInitiateResponseMessage (typed '" << kQueryInitiateResponseMessage
- << "') to Foreman";
-
serialization::QueryInitiateResponseMessage proto;
proto.set_query_id(query_id);
@@ -301,14 +289,15 @@ void Shiftboss::processQueryInitiateMessage(
kQueryInitiateResponseMessage);
free(proto_bytes);
+ DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+ << "') sent QueryInitiateResponseMessage (typed '" << kQueryInitiateResponseMessage
+ << "') to Foreman with TMB client ID " << foreman_client_id_;
const MessageBus::SendStatus send_status =
QueryExecutionUtil::SendTMBMessage(bus_,
shiftboss_client_id_,
foreman_client_id_,
move(message_response));
- CHECK(send_status == MessageBus::SendStatus::kOK)
- << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
- << " to Foreman with TMB client ID " << foreman_client_id_;
+ CHECK(send_status == MessageBus::SendStatus::kOK);
}
void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
@@ -324,10 +313,6 @@ void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
vector<MutableBlockReference> partially_filled_block_refs;
insert_destination->getPartiallyFilledBlocks(&partially_filled_block_refs);
- LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
- << "') sent InitiateRebuildResponseMessage (typed '" << kInitiateRebuildResponseMessage
- << "') to Foreman";
-
serialization::InitiateRebuildResponseMessage proto;
proto.set_query_id(query_id);
proto.set_operator_index(op_index);
@@ -343,14 +328,15 @@ void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
kInitiateRebuildResponseMessage);
free(proto_bytes);
+ DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+ << "') sent InitiateRebuildResponseMessage (typed '" << kInitiateRebuildResponseMessage
+ << "') to Foreman with TMB client ID " << foreman_client_id_;
const MessageBus::SendStatus send_status =
QueryExecutionUtil::SendTMBMessage(bus_,
shiftboss_client_id_,
foreman_client_id_,
move(message_response));
- CHECK(send_status == MessageBus::SendStatus::kOK)
- << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
- << " to Foreman with TMB client ID " << foreman_client_id_;
+ CHECK(send_status == MessageBus::SendStatus::kOK);
for (size_t i = 0; i < partially_filled_block_refs.size(); ++i) {
// NOTE(zuyu): Worker releases the memory after the execution of
@@ -371,18 +357,16 @@ void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
kRebuildWorkOrderMessage);
const size_t worker_index = getSchedulableWorker();
- LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
- << "') sent RebuildWorkOrderMessage (typed '" << kRebuildWorkOrderMessage
- << "') to worker " << worker_index;
+ DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+ << "') sent RebuildWorkOrderMessage (typed '" << kRebuildWorkOrderMessage
+ << "') to worker " << worker_index;
const MessageBus::SendStatus send_status =
QueryExecutionUtil::SendTMBMessage(bus_,
shiftboss_client_id_,
workers_->getClientID(worker_index),
move(worker_tagged_message));
- CHECK(send_status == MessageBus::SendStatus::kOK)
- << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
- << " to Worker with TMB client ID " << workers_->getClientID(worker_index);
+ CHECK(send_status == MessageBus::SendStatus::kOK);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/query_execution/Worker.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Worker.cpp b/query_execution/Worker.cpp
index d497be6..0b1efba 100644
--- a/query_execution/Worker.cpp
+++ b/query_execution/Worker.cpp
@@ -101,12 +101,12 @@ void Worker::sendWorkOrderCompleteMessage(const tmb::client_id receiver,
static_cast<const void *>(proto_bytes), proto_length, message_type);
std::free(proto_bytes);
+ DLOG(INFO) << "Worker sent WorkOrderCompleteMessage (typed '" << message_type
+ << "') to Scheduler with TMB client ID " << receiver;
const tmb::MessageBus::SendStatus send_status =
QueryExecutionUtil::SendTMBMessage(
bus_, worker_client_id_, receiver, std::move(tagged_message));
- CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
- << "Message could not be sent from worker with TMB client ID "
- << worker_client_id_ << " to Foreman with TMB client ID " << receiver;
+ CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
}
template <typename CompletionMessageProtoT>
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/relational_operators/DeleteOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.cpp b/relational_operators/DeleteOperator.cpp
index 8197aef..24da9bf 100644
--- a/relational_operators/DeleteOperator.cpp
+++ b/relational_operators/DeleteOperator.cpp
@@ -146,17 +146,15 @@ void DeleteWorkOrder::execute() {
kDataPipelineMessage);
std::free(proto_bytes);
- const tmb::client_id worker_thread_client_id = ClientIDMap::Instance()->getValue();
+ DLOG(INFO) << "DeleteWorkOrder sent DataPipelineMessage (typed '" << kDataPipelineMessage
+ << "') to Scheduler with TMB client ID " << scheduler_client_id_;
const tmb::MessageBus::SendStatus send_status =
QueryExecutionUtil::SendTMBMessage(
bus_,
- worker_thread_client_id,
+ ClientIDMap::Instance()->getValue(),
scheduler_client_id_,
std::move(tagged_message));
- CHECK(send_status == tmb::MessageBus::SendStatus::kOK) << "Message could not"
- " be sent from thread with TMB client ID " <<
- worker_thread_client_id << " to Foreman with TMB client ID "
- << scheduler_client_id_;
+ CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
}
} // namespace quickstep
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/relational_operators/RebuildWorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RebuildWorkOrder.hpp b/relational_operators/RebuildWorkOrder.hpp
index fe4be68..2cef1f1 100644
--- a/relational_operators/RebuildWorkOrder.hpp
+++ b/relational_operators/RebuildWorkOrder.hpp
@@ -101,15 +101,14 @@ class RebuildWorkOrder : public WorkOrder {
// Refer to InsertDestination::sendBlockFilledMessage for the rationale
// behind using the ClientIDMap map.
+ DLOG(INFO) << "RebuildWorkOrder sent DataPipelineMessage (typed '" << kDataPipelineMessage
+ << "') to Scheduler with TMB client ID " << scheduler_client_id_;
const tmb::MessageBus::SendStatus send_status =
QueryExecutionUtil::SendTMBMessage(bus_,
ClientIDMap::Instance()->getValue(),
scheduler_client_id_,
std::move(tagged_message));
- CHECK(send_status == tmb::MessageBus::SendStatus::kOK) << "Message could "
- " not be sent from thread with TMB client ID " <<
- ClientIDMap::Instance()->getValue() << " to Foreman with TMB client ID "
- << scheduler_client_id_;
+ CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
}
private:
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/relational_operators/UpdateOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/UpdateOperator.cpp b/relational_operators/UpdateOperator.cpp
index bc29365..143c741 100644
--- a/relational_operators/UpdateOperator.cpp
+++ b/relational_operators/UpdateOperator.cpp
@@ -128,17 +128,15 @@ void UpdateWorkOrder::execute() {
kDataPipelineMessage);
std::free(proto_bytes);
- const tmb::client_id worker_thread_client_id = ClientIDMap::Instance()->getValue();
+ DLOG(INFO) << "UpdateWorkOrder sent DataPipelineMessage (typed '" << kDataPipelineMessage
+ << "') to Scheduler with TMB client ID " << scheduler_client_id_;
const tmb::MessageBus::SendStatus send_status =
QueryExecutionUtil::SendTMBMessage(
bus_,
- worker_thread_client_id,
+ ClientIDMap::Instance()->getValue(),
scheduler_client_id_,
std::move(tagged_message));
- CHECK(send_status == tmb::MessageBus::SendStatus::kOK) << "Message could not"
- " be sent from thread with TMB client ID " <<
- worker_thread_client_id << " to Foreman with TMB client ID "
- << scheduler_client_id_;
+ CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
}
} // namespace quickstep
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/relational_operators/WorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.hpp b/relational_operators/WorkOrder.hpp
index 3cbab94..c1b9b68 100644
--- a/relational_operators/WorkOrder.hpp
+++ b/relational_operators/WorkOrder.hpp
@@ -44,7 +44,6 @@ namespace quickstep {
* @{
*/
-
/**
* @brief A single unit of work in a query plan, produced by a
* RelationalOperator. Where possible, WorkOrders should be of
@@ -284,14 +283,14 @@ class WorkOrder {
tmb::MessageStyle single_receiver_style;
DCHECK(bus != nullptr);
+ DLOG(INFO) << "WorkOrder sent WorkOrderFeedbackMessage (typed '" << kWorkOrderFeedbackMessage
+ << "') to Scheduler with TMB client ID " << receiver_id;
const tmb::MessageBus::SendStatus send_status =
bus->Send(sender_id,
receiver_address,
single_receiver_style,
std::move(msg));
- CHECK(send_status == tmb::MessageBus::SendStatus::kOK) << "Message could"
- " not be sent from thread with TMB client ID " << sender_id << " to"
- " receiver thread with TMB client ID " << receiver_id;
+ CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
}
/**
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/storage/InsertDestination.cpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.cpp b/storage/InsertDestination.cpp
index 9897aed..5e83453 100644
--- a/storage/InsertDestination.cpp
+++ b/storage/InsertDestination.cpp
@@ -282,13 +282,15 @@ MutableBlockReference AlwaysCreateBlockInsertDestination::createNewBlock() {
kCatalogRelationNewBlockMessage);
free(proto_bytes);
+ DLOG(INFO) << "AlwaysCreateBlockInsertDestination sent CatalogRelationNewBlockMessage (typed '"
+ << kCatalogRelationNewBlockMessage
+ << "') to Scheduler with TMB client ID " << scheduler_client_id_;
const tmb::MessageBus::SendStatus send_status =
QueryExecutionUtil::SendTMBMessage(bus_,
thread_id_map_.getValue(),
scheduler_client_id_,
move(tagged_msg));
- CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
- << "CatalogRelationNewBlockMessage could not be sent from InsertDestination to Foreman.";
+ CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
return storage_manager_->getBlockMutable(new_id, relation_);
}
@@ -330,13 +332,15 @@ MutableBlockReference BlockPoolInsertDestination::createNewBlock() {
kCatalogRelationNewBlockMessage);
free(proto_bytes);
+ DLOG(INFO) << "BlockPoolInsertDestination sent CatalogRelationNewBlockMessage (typed '"
+ << kCatalogRelationNewBlockMessage
+ << "') to Scheduler with TMB client ID " << scheduler_client_id_;
const tmb::MessageBus::SendStatus send_status =
QueryExecutionUtil::SendTMBMessage(bus_,
thread_id_map_.getValue(),
scheduler_client_id_,
move(tagged_msg));
- CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
- << "CatalogRelationNewBlockMessage could not be sent from InsertDestination to Foreman.";
+ CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
return storage_manager_->getBlockMutable(new_id, relation_);
}
@@ -445,13 +449,15 @@ MutableBlockReference PartitionAwareInsertDestination::createNewBlockInPartition
kCatalogRelationNewBlockMessage);
free(proto_bytes);
+ DLOG(INFO) << "PartitionAwareInsertDestination sent CatalogRelationNewBlockMessage (typed '"
+ << kCatalogRelationNewBlockMessage
+ << "') to Scheduler with TMB client ID " << scheduler_client_id_;
const tmb::MessageBus::SendStatus send_status =
QueryExecutionUtil::SendTMBMessage(bus_,
thread_id_map_.getValue(),
scheduler_client_id_,
move(tagged_msg));
- CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
- << "CatalogRelationNewBlockMessage could not be sent from InsertDestination to Foreman.";
+ CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
return storage_manager_->getBlockMutable(new_id, relation_);
}
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/storage/InsertDestination.hpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.hpp b/storage/InsertDestination.hpp
index 3dae9a0..408e76b 100644
--- a/storage/InsertDestination.hpp
+++ b/storage/InsertDestination.hpp
@@ -249,16 +249,14 @@ class InsertDestination : public InsertDestinationInterface {
// option 3.
DCHECK(bus_ != nullptr);
- const tmb::client_id worker_thread_client_id = thread_id_map_.getValue();
+ DLOG(INFO) << "InsertDestination sent DataPipelineMessage (typed '" << kDataPipelineMessage
+ << "') to Scheduler with TMB client ID " << scheduler_client_id_;
const tmb::MessageBus::SendStatus send_status =
QueryExecutionUtil::SendTMBMessage(bus_,
- worker_thread_client_id,
+ thread_id_map_.getValue(),
scheduler_client_id_,
std::move(tagged_message));
- CHECK(send_status == tmb::MessageBus::SendStatus::kOK) <<
- "Message could not be sent from thread with TMB client ID "
- << worker_thread_client_id << " to Scheduler with TMB client"
- " ID " << scheduler_client_id_;
+ CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
}
inline const std::size_t getQueryID() const {
[02/16] incubator-quickstep git commit: Added PolicyEnforcer
implementation for the distributed version.
Posted by ji...@apache.org.
Added PolicyEnforcer implementation for the distributed version.
Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/3c2749ea
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/3c2749ea
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/3c2749ea
Branch: refs/heads/LIP-for-tpch
Commit: 3c2749eafcff25283915bdef21822056d36f5281
Parents: e443b2b
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Thu Aug 4 11:45:51 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Mon Aug 8 10:35:45 2016 -0700
----------------------------------------------------------------------
query_execution/CMakeLists.txt | 24 ++
query_execution/PolicyEnforcerBase.cpp | 2 +
query_execution/PolicyEnforcerBase.hpp | 7 +
query_execution/PolicyEnforcerDistributed.cpp | 279 +++++++++++++++++++++
query_execution/PolicyEnforcerDistributed.hpp | 113 +++++++++
query_execution/QueryExecutionMessages.proto | 16 +-
query_execution/QueryExecutionTypedefs.hpp | 5 +
query_execution/QueryManagerBase.cpp | 3 +-
query_execution/QueryManagerBase.hpp | 11 +-
9 files changed, 456 insertions(+), 4 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_execution/CMakeLists.txt b/query_execution/CMakeLists.txt
index 4b180e3..74fcafb 100644
--- a/query_execution/CMakeLists.txt
+++ b/query_execution/CMakeLists.txt
@@ -35,6 +35,9 @@ endif()
add_library(quickstep_queryexecution_ForemanBase ../empty_src.cpp ForemanBase.hpp)
add_library(quickstep_queryexecution_ForemanSingleNode ForemanSingleNode.cpp ForemanSingleNode.hpp)
add_library(quickstep_queryexecution_PolicyEnforcerBase PolicyEnforcerBase.cpp PolicyEnforcerBase.hpp)
+if (ENABLE_DISTRIBUTED)
+ add_library(quickstep_queryexecution_PolicyEnforcerDistributed PolicyEnforcerDistributed.cpp PolicyEnforcerDistributed.hpp)
+endif(ENABLE_DISTRIBUTED)
add_library(quickstep_queryexecution_PolicyEnforcerSingleNode PolicyEnforcerSingleNode.cpp PolicyEnforcerSingleNode.hpp)
add_library(quickstep_queryexecution_QueryContext QueryContext.cpp QueryContext.hpp)
add_library(quickstep_queryexecution_QueryContext_proto
@@ -110,6 +113,26 @@ target_link_libraries(quickstep_queryexecution_PolicyEnforcerBase
quickstep_storage_StorageBlockInfo
quickstep_utility_Macros
tmb)
+if (ENABLE_DISTRIBUTED)
+ target_link_libraries(quickstep_queryexecution_PolicyEnforcerDistributed
+ glog
+ quickstep_catalog_CatalogRelation
+ quickstep_catalog_Catalog_proto
+ quickstep_queryexecution_PolicyEnforcerBase
+ quickstep_queryexecution_QueryContext_proto
+ quickstep_queryexecution_QueryExecutionMessages_proto
+ quickstep_queryexecution_QueryExecutionState
+ quickstep_queryexecution_QueryExecutionTypedefs
+ quickstep_queryexecution_QueryExecutionUtil
+ quickstep_queryexecution_QueryManagerBase
+ quickstep_queryexecution_QueryManagerDistributed
+ quickstep_queryexecution_ShiftbossDirectory
+ quickstep_queryoptimizer_QueryHandle
+ quickstep_storage_StorageBlockInfo
+ quickstep_utility_Macros
+ tmb
+ ${GFLAGS_LIB_NAME})
+endif(ENABLE_DISTRIBUTED)
target_link_libraries(quickstep_queryexecution_PolicyEnforcerSingleNode
glog
quickstep_catalog_CatalogTypedefs
@@ -293,6 +316,7 @@ target_link_libraries(quickstep_queryexecution
if (ENABLE_DISTRIBUTED)
target_link_libraries(quickstep_queryexecution
quickstep_queryexecution_BlockLocator
+ quickstep_queryexecution_PolicyEnforcerDistributed
quickstep_queryexecution_QueryManagerDistributed
quickstep_queryexecution_Shiftboss
quickstep_queryexecution_ShiftbossDirectory)
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/PolicyEnforcerBase.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerBase.cpp b/query_execution/PolicyEnforcerBase.cpp
index bf6edf9..78f7b44 100644
--- a/query_execution/PolicyEnforcerBase.cpp
+++ b/query_execution/PolicyEnforcerBase.cpp
@@ -136,6 +136,8 @@ void PolicyEnforcerBase::processMessage(const TaggedMessage &tagged_message) {
}
if (admitted_queries_[query_id]->queryStatus(op_index) ==
QueryManagerBase::QueryStatusCode::kQueryExecuted) {
+ onQueryCompletion(admitted_queries_[query_id].get());
+
removeQuery(query_id);
if (!waiting_queries_.empty()) {
// Admit the earliest waiting query.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/PolicyEnforcerBase.hpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerBase.hpp b/query_execution/PolicyEnforcerBase.hpp
index c75a531..e95799e 100644
--- a/query_execution/PolicyEnforcerBase.hpp
+++ b/query_execution/PolicyEnforcerBase.hpp
@@ -140,6 +140,13 @@ class PolicyEnforcerBase {
static constexpr std::size_t kMaxConcurrentQueries = 1;
/**
+ * @brief Add custom actions upon the completion of a query.
+ *
+ * @param query_manager The query manager.
+ **/
+ virtual void onQueryCompletion(QueryManagerBase *query_manager) {}
+
+ /**
* @brief Record the execution time for a finished WorkOrder.
*
* TODO(harshad) - Extend the functionality to rebuild work orders.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/PolicyEnforcerDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerDistributed.cpp b/query_execution/PolicyEnforcerDistributed.cpp
new file mode 100644
index 0000000..6d0de47
--- /dev/null
+++ b/query_execution/PolicyEnforcerDistributed.cpp
@@ -0,0 +1,279 @@
+/**
+ * Licensed 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 "query_execution/PolicyEnforcerDistributed.hpp"
+
+#include <cstddef>
+#include <cstdlib>
+#include <memory>
+#include <queue>
+#include <utility>
+#include <unordered_map>
+#include <vector>
+
+#include "catalog/Catalog.pb.h"
+#include "catalog/CatalogRelation.hpp"
+#include "query_execution/QueryContext.pb.h"
+#include "query_execution/QueryExecutionMessages.pb.h"
+#include "query_execution/QueryExecutionState.hpp"
+#include "query_execution/QueryExecutionTypedefs.hpp"
+#include "query_execution/QueryExecutionUtil.hpp"
+#include "query_execution/QueryManagerBase.hpp"
+#include "query_execution/QueryManagerDistributed.hpp"
+#include "query_optimizer/QueryHandle.hpp"
+#include "storage/StorageBlockInfo.hpp"
+
+#include "gflags/gflags.h"
+#include "glog/logging.h"
+
+#include "tmb/id_typedefs.h"
+#include "tmb/message_bus.h"
+#include "tmb/tagged_message.h"
+
+using std::free;
+using std::malloc;
+using std::move;
+using std::size_t;
+using std::unique_ptr;
+using std::vector;
+
+using tmb::TaggedMessage;
+
+namespace quickstep {
+
+namespace S = serialization;
+
+DEFINE_uint64(max_msgs_per_dispatch_round, 20, "Maximum number of messages that"
+ " can be allocated in a single round of dispatch of messages to"
+ " the workers.");
+
+void PolicyEnforcerDistributed::getWorkOrderMessages(
+ vector<unique_ptr<S::WorkOrderMessage>> *work_order_messages) {
+ // Iterate over admitted queries until either there are no more
+ // messages available, or the maximum number of messages have
+ // been collected.
+ DCHECK(work_order_messages->empty());
+ // TODO(harshad) - Make this function generic enough so that it
+ // works well when multiple queries are getting executed.
+ if (admitted_queries_.empty()) {
+ LOG(WARNING) << "Requesting WorkerMessages when no query is running";
+ return;
+ }
+
+ const std::size_t per_query_share =
+ FLAGS_max_msgs_per_dispatch_round / admitted_queries_.size();
+ DCHECK_GT(per_query_share, 0u);
+
+ vector<std::size_t> finished_queries_ids;
+
+ for (const auto &admitted_query_info : admitted_queries_) {
+ QueryManagerBase *curr_query_manager = admitted_query_info.second.get();
+ DCHECK(curr_query_manager != nullptr);
+ std::size_t messages_collected_curr_query = 0;
+ while (messages_collected_curr_query < per_query_share) {
+ S::WorkOrderMessage *next_work_order_message =
+ static_cast<QueryManagerDistributed*>(curr_query_manager)->getNextWorkOrderMessage(0);
+ if (next_work_order_message != nullptr) {
+ ++messages_collected_curr_query;
+ work_order_messages->push_back(unique_ptr<S::WorkOrderMessage>(next_work_order_message));
+ } else {
+ // No more work ordes from the current query at this time.
+ // Check if the query's execution is over.
+ if (curr_query_manager->getQueryExecutionState().hasQueryExecutionFinished()) {
+ // If the query has been executed, remove it.
+ finished_queries_ids.push_back(admitted_query_info.first);
+ }
+ break;
+ }
+ }
+ }
+ for (const std::size_t finished_qid : finished_queries_ids) {
+ onQueryCompletion(admitted_queries_[finished_qid].get());
+ removeQuery(finished_qid);
+ }
+}
+
+bool PolicyEnforcerDistributed::admitQuery(QueryHandle *query_handle) {
+ if (admitted_queries_.size() < PolicyEnforcerBase::kMaxConcurrentQueries) {
+ // Ok to admit the query.
+ const std::size_t query_id = query_handle->query_id();
+ if (admitted_queries_.find(query_id) == admitted_queries_.end()) {
+ // NOTE(zuyu): Should call before constructing a 'QueryManager'.
+ // Otherwise, an InitiateRebuildMessage may be sent before 'QueryContext'
+ // initializes.
+ initiateQueryInShiftboss(query_handle);
+
+ // Query with the same ID not present, ok to admit.
+ admitted_queries_[query_id].reset(
+ new QueryManagerDistributed(query_handle, shiftboss_directory_, foreman_client_id_, bus_));
+ return true;
+ } else {
+ LOG(ERROR) << "Query with the same ID " << query_id << " exists";
+ return false;
+ }
+ } else {
+ // This query will have to wait.
+ waiting_queries_.push(query_handle);
+ return false;
+ }
+}
+
+void PolicyEnforcerDistributed::processInitiateRebuildResponseMessage(const tmb::TaggedMessage &tagged_message) {
+ S::InitiateRebuildResponseMessage proto;
+ CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
+
+ const std::size_t query_id = proto.query_id();
+ DCHECK(admitted_queries_.find(query_id) != admitted_queries_.end());
+
+ QueryManagerDistributed *query_manager = static_cast<QueryManagerDistributed*>(admitted_queries_[query_id].get());
+
+ const std::size_t num_rebuild_work_orders = proto.num_rebuild_work_orders();
+ query_manager->processInitiateRebuildResponseMessage(proto.operator_index(), num_rebuild_work_orders);
+ shiftboss_directory_->addNumQueuedWorkOrders(proto.shiftboss_index(), num_rebuild_work_orders);
+
+ if (query_manager->getQueryExecutionState().hasQueryExecutionFinished()) {
+ onQueryCompletion(query_manager);
+
+ removeQuery(query_id);
+ if (!waiting_queries_.empty()) {
+ // Admit the earliest waiting query.
+ QueryHandle *new_query = waiting_queries_.front();
+ waiting_queries_.pop();
+ admitQuery(new_query);
+ }
+ }
+}
+
+void PolicyEnforcerDistributed::initiateQueryInShiftboss(QueryHandle *query_handle) {
+ S::QueryInitiateMessage proto;
+ proto.set_query_id(query_handle->query_id());
+ proto.mutable_catalog_database_cache()->MergeFrom(query_handle->getCatalogDatabaseCacheProto());
+ proto.mutable_query_context()->MergeFrom(query_handle->getQueryContextProto());
+
+ const size_t proto_length = proto.ByteSize();
+ char *proto_bytes = static_cast<char*>(malloc(proto_length));
+ CHECK(proto.SerializeToArray(proto_bytes, proto_length));
+
+ TaggedMessage message(static_cast<const void*>(proto_bytes),
+ proto_length,
+ kQueryInitiateMessage);
+ free(proto_bytes);
+
+ LOG(INFO) << "PolicyEnforcerDistributed sent QueryInitiateMessage (typed '" << kQueryInitiateMessage
+ << "') to Shiftboss 0";
+
+ // TODO(zuyu): Multiple Shiftbosses support.
+ const tmb::MessageBus::SendStatus send_status =
+ QueryExecutionUtil::SendTMBMessage(bus_,
+ foreman_client_id_,
+ shiftboss_directory_->getClientId(0),
+ move(message));
+ CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
+ << "Message could not be sent from Foreman with TMB client ID " << foreman_client_id_
+ << " to Shiftboss with TMB client ID " << shiftboss_directory_->getClientId(0);
+
+ // Wait Shiftboss for QueryInitiateResponseMessage.
+ const tmb::AnnotatedMessage annotated_message = bus_->Receive(foreman_client_id_, 0, true);
+ const TaggedMessage &tagged_message = annotated_message.tagged_message;
+ DCHECK_EQ(kQueryInitiateResponseMessage, tagged_message.message_type());
+ LOG(INFO) << "PolicyEnforcerDistributed received typed '" << tagged_message.message_type()
+ << "' message from client " << annotated_message.sender;
+
+ S::QueryInitiateResponseMessage proto_response;
+ CHECK(proto_response.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
+}
+
+void PolicyEnforcerDistributed::onQueryCompletion(QueryManagerBase *query_manager) {
+ const QueryHandle *query_handle = query_manager->query_handle();
+
+ const CatalogRelation *query_result = query_handle->getQueryResultRelation();
+ const tmb::client_id cli_id = query_handle->getClientId();
+ const std::size_t query_id = query_handle->query_id();
+
+ if (query_result == nullptr) {
+ // Clean up query execution states, i.e., QueryContext, in Shiftboss.
+ serialization::QueryTeardownMessage proto;
+ proto.set_query_id(query_id);
+
+ const size_t proto_length = proto.ByteSize();
+ char *proto_bytes = static_cast<char*>(malloc(proto_length));
+ CHECK(proto.SerializeToArray(proto_bytes, proto_length));
+
+ TaggedMessage message(static_cast<const void*>(proto_bytes),
+ proto_length,
+ kQueryTeardownMessage);
+
+ // TODO(zuyu): Support multiple shiftbosses.
+ LOG(INFO) << "PolicyEnforcerDistributed sent QueryTeardownMessage (typed '" << kQueryTeardownMessage
+ << "') to Shiftboss 0";
+ tmb::MessageBus::SendStatus send_status =
+ QueryExecutionUtil::SendTMBMessage(bus_,
+ foreman_client_id_,
+ shiftboss_directory_->getClientId(0),
+ move(message));
+ CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
+ << "Message could not be sent from Foreman with TMB client ID " << foreman_client_id_
+ << " to Shiftboss";
+
+ TaggedMessage cli_message(kQueryExecutionSuccessMessage);
+
+ // Notify the CLI query execution successfully.
+ LOG(INFO) << "PolicyEnforcerDistributed sent QueryExecutionSuccessMessage (typed '" << kQueryExecutionSuccessMessage
+ << "') to CLI with TMB client id " << cli_id;
+ send_status =
+ QueryExecutionUtil::SendTMBMessage(bus_,
+ foreman_client_id_,
+ cli_id,
+ move(cli_message));
+ CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
+ << "Message could not be sent from Foreman with TMB client ID " << foreman_client_id_
+ << " to CLI with TMB client ID " << cli_id;
+ return;
+ }
+
+ // NOTE(zuyu): SaveQueryResultMessage implicitly triggers QueryTeardown in Shiftboss.
+ S::SaveQueryResultMessage proto;
+ proto.set_query_id(query_id);
+ proto.set_relation_id(query_result->getID());
+
+ const vector<block_id> blocks(query_result->getBlocksSnapshot());
+ for (const block_id block : blocks) {
+ proto.add_blocks(block);
+ }
+
+ proto.set_cli_id(cli_id);
+
+ const size_t proto_length = proto.ByteSize();
+ char *proto_bytes = static_cast<char*>(malloc(proto_length));
+ CHECK(proto.SerializeToArray(proto_bytes, proto_length));
+
+ TaggedMessage message(static_cast<const void*>(proto_bytes),
+ proto_length,
+ kSaveQueryResultMessage);
+ free(proto_bytes);
+
+ LOG(INFO) << "PolicyEnforcerDistributed sent SaveQueryResultMessage (typed '" << kSaveQueryResultMessage
+ << "') to Shiftboss 0";
+ // TODO(zuyu): Support multiple shiftbosses.
+ const tmb::MessageBus::SendStatus send_status =
+ QueryExecutionUtil::SendTMBMessage(bus_,
+ foreman_client_id_,
+ shiftboss_directory_->getClientId(0),
+ move(message));
+ CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
+ << "Message could not be sent from Foreman with TMB client ID " << foreman_client_id_
+ << " to Shiftboss with TMB client ID " << shiftboss_directory_->getClientId(0);
+}
+
+} // namespace quickstep
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/PolicyEnforcerDistributed.hpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerDistributed.hpp b/query_execution/PolicyEnforcerDistributed.hpp
new file mode 100644
index 0000000..16ebe36
--- /dev/null
+++ b/query_execution/PolicyEnforcerDistributed.hpp
@@ -0,0 +1,113 @@
+/**
+ * Licensed 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.
+ **/
+
+#ifndef QUICKSTEP_QUERY_EXECUTION_POLICY_ENFORCER_DISTRIBUTED_HPP_
+#define QUICKSTEP_QUERY_EXECUTION_POLICY_ENFORCER_DISTRIBUTED_HPP_
+
+#include <cstddef>
+#include <memory>
+#include <vector>
+
+#include "query_execution/PolicyEnforcerBase.hpp"
+#include "query_execution/ShiftbossDirectory.hpp"
+#include "utility/Macros.hpp"
+
+#include "tmb/id_typedefs.h"
+
+namespace tmb {
+class MessageBus;
+class TaggedMessage;
+}
+
+namespace quickstep {
+
+class CatalogDatabaseLite;
+class QueryHandle;
+class QueryManagerBase;
+
+namespace serialization { class WorkOrderMessage; }
+
+/** \addtogroup QueryExecution
+ * @{
+ */
+
+/**
+ * @brief A class that ensures that a high level policy is maintained
+ * in sharing resources among concurrent queries.
+ **/
+class PolicyEnforcerDistributed final : public PolicyEnforcerBase {
+ public:
+ /**
+ * @brief Constructor.
+ *
+ * @param foreman_client_id The TMB client ID of the Foreman.
+ * @param catalog_database The CatalogDatabase used.
+ * @param bus The TMB.
+ **/
+ PolicyEnforcerDistributed(const tmb::client_id foreman_client_id,
+ CatalogDatabaseLite *catalog_database,
+ ShiftbossDirectory *shiftboss_directory,
+ tmb::MessageBus *bus,
+ const bool profile_individual_workorders = false)
+ : PolicyEnforcerBase(catalog_database, profile_individual_workorders),
+ foreman_client_id_(foreman_client_id),
+ shiftboss_directory_(shiftboss_directory),
+ bus_(bus) {}
+
+ /**
+ * @brief Destructor.
+ **/
+ ~PolicyEnforcerDistributed() override {}
+
+ bool admitQuery(QueryHandle *query_handle) override;
+
+ /**
+ * @brief Get work order messages to be dispatched. These messages come from
+ * the active queries.
+ *
+ * @param work_order_messages The work order messages to be dispatched.
+ **/
+ void getWorkOrderMessages(
+ std::vector<std::unique_ptr<serialization::WorkOrderMessage>> *work_order_messages);
+
+ /**
+ * @brief Process the initiate rebuild work order response message.
+ *
+ * @param tagged_message The message.
+ **/
+ void processInitiateRebuildResponseMessage(const tmb::TaggedMessage &tagged_message);
+
+ private:
+ void decrementNumQueuedWorkOrders(const std::size_t shiftboss_index) override {
+ shiftboss_directory_->decrementNumQueuedWorkOrders(shiftboss_index);
+ }
+
+ void onQueryCompletion(QueryManagerBase *query_manager) override;
+
+ void initiateQueryInShiftboss(QueryHandle *query_handle);
+
+ const tmb::client_id foreman_client_id_;
+
+ ShiftbossDirectory *shiftboss_directory_;
+
+ tmb::MessageBus *bus_;
+
+ DISALLOW_COPY_AND_ASSIGN(PolicyEnforcerDistributed);
+};
+
+/** @} */
+
+} // namespace quickstep
+
+#endif // QUICKSTEP_QUERY_EXECUTION_POLICY_ENFORCER_DISTRIBUTED_HPP_
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/QueryExecutionMessages.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionMessages.proto b/query_execution/QueryExecutionMessages.proto
index f680d35..20b684e 100644
--- a/query_execution/QueryExecutionMessages.proto
+++ b/query_execution/QueryExecutionMessages.proto
@@ -120,13 +120,25 @@ message InitiateRebuildResponseMessage {
required uint64 shiftboss_index = 4;
}
+message QueryTeardownMessage {
+ required uint64 query_id = 1;
+}
+
message SaveQueryResultMessage {
- required int32 relation_id = 1;
- repeated fixed64 blocks = 2 [packed=true];
+ required uint64 query_id = 1;
+ required int32 relation_id = 2;
+ repeated fixed64 blocks = 3 [packed=true];
+
+ required uint32 cli_id = 4; // tmb::client_id.
}
message SaveQueryResultResponseMessage {
required int32 relation_id = 1;
+ required uint32 cli_id = 2; // tmb::client_id.
+}
+
+message QueryExecutionSuccessMessage {
+ optional CatalogRelationSchema result_relation = 1;
}
// BlockLocator related messages.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/QueryExecutionTypedefs.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionTypedefs.hpp b/query_execution/QueryExecutionTypedefs.hpp
index 4643096..d154d84 100644
--- a/query_execution/QueryExecutionTypedefs.hpp
+++ b/query_execution/QueryExecutionTypedefs.hpp
@@ -86,9 +86,14 @@ enum QueryExecutionMessageType : message_type_id {
kInitiateRebuildMessage, // From Foreman to Shiftboss.
kInitiateRebuildResponseMessage, // From Shiftboss to Foreman.
+ kQueryTeardownMessage, // From Foreman to Shiftboss.
+
kSaveQueryResultMessage, // From Foreman to Shiftboss.
kSaveQueryResultResponseMessage, // From Shiftboss to Foreman.
+ // From Foreman to CLI.
+ kQueryExecutionSuccessMessage,
+
// BlockLocator related messages, sorted in a life cycle of StorageManager
// with a unique block domain.
kBlockDomainRegistrationMessage, // From Worker to BlockLocator.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/QueryManagerBase.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerBase.cpp b/query_execution/QueryManagerBase.cpp
index c60e323..8e37da8 100644
--- a/query_execution/QueryManagerBase.cpp
+++ b/query_execution/QueryManagerBase.cpp
@@ -37,7 +37,8 @@ using std::pair;
namespace quickstep {
QueryManagerBase::QueryManagerBase(QueryHandle *query_handle)
- : query_id_(DCHECK_NOTNULL(query_handle)->query_id()),
+ : query_handle_(DCHECK_NOTNULL(query_handle)),
+ query_id_(query_handle->query_id()),
query_dag_(DCHECK_NOTNULL(
DCHECK_NOTNULL(query_handle->getQueryPlanMutable())->getQueryPlanDAGMutable())),
num_operators_in_dag_(query_dag_->size()),
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/QueryManagerBase.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerBase.hpp b/query_execution/QueryManagerBase.hpp
index 782b8ed..a274742 100644
--- a/query_execution/QueryManagerBase.hpp
+++ b/query_execution/QueryManagerBase.hpp
@@ -74,6 +74,13 @@ class QueryManagerBase {
virtual ~QueryManagerBase() {}
/**
+ * @brief Get the query handle.
+ **/
+ const QueryHandle* query_handle() const {
+ return query_handle_;
+ }
+
+ /**
* @brief Get the QueryExecutionState for this query.
**/
inline const QueryExecutionState& getQueryExecutionState() const {
@@ -252,9 +259,11 @@ class QueryManagerBase {
return query_exec_state_->hasRebuildInitiated(index);
}
+ const QueryHandle *query_handle_;
+
const std::size_t query_id_;
- DAG<RelationalOperator, bool> *query_dag_;
+ DAG<RelationalOperator, bool> *query_dag_; // Owned by 'query_handle_'.
const dag_node_index num_operators_in_dag_;
// For all nodes, store their receiving dependents.
[03/16] incubator-quickstep git commit: Added guards for gnu_source
definition in CMake.
Posted by ji...@apache.org.
Added guards for gnu_source definition in CMake.
Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/658cb617
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/658cb617
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/658cb617
Branch: refs/heads/LIP-for-tpch
Commit: 658cb61760ab6bc98ead1faf231b7b7f27a0fa1e
Parents: 3c2749e
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Mon Aug 8 12:19:00 2016 -0500
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Mon Aug 8 10:42:51 2016 -0700
----------------------------------------------------------------------
storage/CMakeLists.txt | 2 ++
1 file changed, 2 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/658cb617/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index c4b8f70..65a7975 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -50,7 +50,9 @@ endif()
# See if mmap can be used to allocate Linux hugetlb pages.
include(CheckCXXSourceCompiles)
CHECK_CXX_SOURCE_COMPILES("
+ #ifndef _GNU_SOURCE
#define _GNU_SOURCE
+ #endif
#include <sys/mman.h>
int main() {