[Feature] support asof join (BE Part) (#63236)
Signed-off-by: stephen <stephen5217@163.com>
This commit is contained in:
parent
23965e357e
commit
7d81781448
|
|
@ -99,6 +99,15 @@ Status HashJoinNode::init(const TPlanNode& tnode, RuntimeState* state) {
|
|||
}
|
||||
}
|
||||
|
||||
if (tnode.hash_join_node.__isset.asof_join_condition) {
|
||||
auto asof_join_condition = tnode.hash_join_node.asof_join_condition;
|
||||
RETURN_IF_ERROR(
|
||||
Expr::create_expr_tree(_pool, asof_join_condition.left, &_asof_join_condition_probe_expr_ctx, state));
|
||||
RETURN_IF_ERROR(
|
||||
Expr::create_expr_tree(_pool, asof_join_condition.right, &_asof_join_condition_build_expr_ctx, state));
|
||||
_asof_join_condition_op = tnode.hash_join_node.asof_join_condition.opcode;
|
||||
}
|
||||
|
||||
if (tnode.hash_join_node.__isset.partition_exprs) {
|
||||
// the same column can appear more than once in either lateral side of eq_join_conjuncts, but multiple
|
||||
// occurrences are accounted for once when determining local shuffle partition_exprs for bucket shuffle join.
|
||||
|
|
@ -196,6 +205,13 @@ Status HashJoinNode::prepare(RuntimeState* state) {
|
|||
RETURN_IF_ERROR(Expr::prepare(_build_expr_ctxs, state));
|
||||
RETURN_IF_ERROR(Expr::prepare(_probe_expr_ctxs, state));
|
||||
RETURN_IF_ERROR(Expr::prepare(_other_join_conjunct_ctxs, state));
|
||||
if (_asof_join_condition_build_expr_ctx != nullptr) {
|
||||
RETURN_IF_ERROR(_asof_join_condition_build_expr_ctx->prepare(state));
|
||||
}
|
||||
|
||||
if (_asof_join_condition_probe_expr_ctx != nullptr) {
|
||||
RETURN_IF_ERROR(_asof_join_condition_probe_expr_ctx->prepare(state));
|
||||
}
|
||||
|
||||
HashTableParam param;
|
||||
_init_hash_table_param(¶m, state);
|
||||
|
|
@ -234,6 +250,22 @@ void HashJoinNode::_init_hash_table_param(HashTableParam* param, RuntimeState* r
|
|||
expr_context->root()->get_slot_ids(&expr_slots);
|
||||
predicate_slots.insert(expr_slots.begin(), expr_slots.end());
|
||||
}
|
||||
|
||||
if (_asof_join_condition_build_expr_ctx && _asof_join_condition_probe_expr_ctx) {
|
||||
std::vector<SlotId> build_slots, probe_slots;
|
||||
_asof_join_condition_probe_expr_ctx->root()->get_slot_ids(&probe_slots);
|
||||
_asof_join_condition_build_expr_ctx->root()->get_slot_ids(&build_slots);
|
||||
|
||||
DCHECK_EQ(probe_slots.size(), 1);
|
||||
DCHECK_EQ(build_slots.size(), 1);
|
||||
|
||||
LogicalType probe_type = _asof_join_condition_probe_expr_ctx->root()->type().type;
|
||||
LogicalType build_type = _asof_join_condition_build_expr_ctx->root()->type().type;
|
||||
SlotId build_slot = build_slots[0], probe_slot = probe_slots[0];
|
||||
param->asof_join_condition_desc = {probe_slot, probe_type, build_slot, build_type, _asof_join_condition_op};
|
||||
predicate_slots.insert({build_slot, probe_slot});
|
||||
}
|
||||
|
||||
param->predicate_slots = std::move(predicate_slots);
|
||||
|
||||
for (auto i = 0; i < _build_expr_ctxs.size(); i++) {
|
||||
|
|
@ -255,6 +287,13 @@ Status HashJoinNode::open(RuntimeState* state) {
|
|||
RETURN_IF_ERROR(Expr::open(_build_expr_ctxs, state));
|
||||
RETURN_IF_ERROR(Expr::open(_probe_expr_ctxs, state));
|
||||
RETURN_IF_ERROR(Expr::open(_other_join_conjunct_ctxs, state));
|
||||
if (_asof_join_condition_build_expr_ctx != nullptr) {
|
||||
RETURN_IF_ERROR(_asof_join_condition_build_expr_ctx->open(state));
|
||||
}
|
||||
|
||||
if (_asof_join_condition_probe_expr_ctx != nullptr) {
|
||||
RETURN_IF_ERROR(_asof_join_condition_probe_expr_ctx->open(state));
|
||||
}
|
||||
|
||||
{
|
||||
build_timer.stop();
|
||||
|
|
@ -445,6 +484,12 @@ void HashJoinNode::close(RuntimeState* state) {
|
|||
Expr::close(_build_expr_ctxs, state);
|
||||
Expr::close(_probe_expr_ctxs, state);
|
||||
Expr::close(_other_join_conjunct_ctxs, state);
|
||||
if (_asof_join_condition_build_expr_ctx != nullptr) {
|
||||
_asof_join_condition_build_expr_ctx->close(state);
|
||||
}
|
||||
if (_asof_join_condition_probe_expr_ctx != nullptr) {
|
||||
_asof_join_condition_probe_expr_ctx->close(state);
|
||||
}
|
||||
|
||||
_ht.close();
|
||||
|
||||
|
|
@ -492,7 +537,9 @@ pipeline::OpFactories HashJoinNode::_decompose_to_pipeline(pipeline::PipelineBui
|
|||
_other_join_conjunct_ctxs, _conjunct_ctxs, child(1)->row_desc(), child(0)->row_desc(),
|
||||
child(1)->type(), child(0)->type(), child(1)->conjunct_ctxs().empty(), _build_runtime_filters,
|
||||
_output_slots, _output_slots, context->degree_of_parallelism(), _distribution_mode,
|
||||
_enable_late_materialization, _enable_partition_hash_join, _is_skew_join, _common_expr_ctxs);
|
||||
_enable_late_materialization, _enable_partition_hash_join, _is_skew_join, _common_expr_ctxs,
|
||||
_asof_join_condition_op, _asof_join_condition_probe_expr_ctx,
|
||||
_asof_join_condition_build_expr_ctx);
|
||||
auto hash_joiner_factory = std::make_shared<starrocks::pipeline::HashJoinerFactory>(param);
|
||||
|
||||
// Create a shared RefCountedRuntimeFilterCollector
|
||||
|
|
@ -582,10 +629,12 @@ pipeline::OpFactories HashJoinNode::_decompose_to_pipeline(pipeline::PipelineBui
|
|||
}
|
||||
|
||||
// Use ChunkAccumulateOperator, when any following condition occurs:
|
||||
// - not left outer join,
|
||||
// - not left/asof left outer join,
|
||||
// - left outer join, with conjuncts or runtime filters.
|
||||
bool need_accumulate_chunk = _join_type != TJoinOp::LEFT_OUTER_JOIN || !_conjunct_ctxs.empty() ||
|
||||
!_other_join_conjunct_ctxs.empty() || lhs_operators.back()->has_runtime_filters();
|
||||
bool need_accumulate_chunk =
|
||||
(_join_type != TJoinOp::LEFT_OUTER_JOIN && _join_type != TJoinOp::ASOF_LEFT_OUTER_JOIN) ||
|
||||
!_conjunct_ctxs.empty() || !_other_join_conjunct_ctxs.empty() ||
|
||||
lhs_operators.back()->has_runtime_filters();
|
||||
if (need_accumulate_chunk) {
|
||||
may_add_chunk_accumulate_operator(lhs_operators, context, id());
|
||||
}
|
||||
|
|
@ -906,6 +955,7 @@ Status HashJoinNode::_process_other_conjunct(ChunkPtr* chunk) {
|
|||
switch (_join_type) {
|
||||
case TJoinOp::LEFT_OUTER_JOIN:
|
||||
case TJoinOp::FULL_OUTER_JOIN:
|
||||
case TJoinOp::ASOF_LEFT_OUTER_JOIN:
|
||||
return _process_outer_join_with_other_conjunct(chunk, _output_probe_column_count, _output_build_column_count);
|
||||
case TJoinOp::RIGHT_OUTER_JOIN:
|
||||
case TJoinOp::LEFT_SEMI_JOIN:
|
||||
|
|
|
|||
|
|
@ -113,6 +113,10 @@ private:
|
|||
TJoinDistributionMode::type _distribution_mode = TJoinDistributionMode::NONE;
|
||||
std::set<SlotId> _output_slots;
|
||||
|
||||
ExprContext* _asof_join_condition_build_expr_ctx = nullptr;
|
||||
ExprContext* _asof_join_condition_probe_expr_ctx = nullptr;
|
||||
TExprOpcode::type _asof_join_condition_op = TExprOpcode::INVALID_OPCODE;
|
||||
|
||||
bool _is_push_down = false;
|
||||
bool _enable_late_materialization = false;
|
||||
|
||||
|
|
|
|||
|
|
@ -85,7 +85,10 @@ HashJoiner::HashJoiner(const HashJoinerParam& param)
|
|||
_build_runtime_filters(param._build_runtime_filters.begin(), param._build_runtime_filters.end()),
|
||||
_enable_late_materialization(param._enable_late_materialization),
|
||||
_max_dop(param._max_dop),
|
||||
_is_skew_join(param._is_skew_join) {
|
||||
_is_skew_join(param._is_skew_join),
|
||||
_asof_join_condition_op(param._asof_join_condition_op),
|
||||
_asof_join_condition_probe_expr_ctx(param._asof_join_condition_probe_expr_ctx),
|
||||
_asof_join_condition_build_expr_ctx(param._asof_join_condition_build_expr_ctx) {
|
||||
_is_push_down = param._hash_join_node.is_push_down;
|
||||
if (_join_type == TJoinOp::LEFT_ANTI_JOIN && param._hash_join_node.is_rewritten_from_not_in) {
|
||||
_join_type = TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN;
|
||||
|
|
@ -175,6 +178,22 @@ void HashJoiner::_init_hash_table_param(HashTableParam* param, RuntimeState* sta
|
|||
expr_context->root()->get_slot_ids(&expr_slots);
|
||||
predicate_slots.insert(expr_slots.begin(), expr_slots.end());
|
||||
}
|
||||
|
||||
if (_asof_join_condition_build_expr_ctx && _asof_join_condition_probe_expr_ctx) {
|
||||
std::vector<SlotId> build_slots, probe_slots;
|
||||
_asof_join_condition_probe_expr_ctx->root()->get_slot_ids(&probe_slots);
|
||||
_asof_join_condition_build_expr_ctx->root()->get_slot_ids(&build_slots);
|
||||
|
||||
DCHECK_EQ(probe_slots.size(), 1);
|
||||
DCHECK_EQ(build_slots.size(), 1);
|
||||
|
||||
LogicalType probe_type = _asof_join_condition_probe_expr_ctx->root()->type().type;
|
||||
LogicalType build_type = _asof_join_condition_build_expr_ctx->root()->type().type;
|
||||
SlotId build_slot = build_slots[0], probe_slot = probe_slots[0];
|
||||
param->asof_join_condition_desc = {probe_slot, probe_type, build_slot, build_type, _asof_join_condition_op};
|
||||
predicate_slots.insert({build_slot, probe_slot});
|
||||
}
|
||||
|
||||
param->predicate_slots = std::move(predicate_slots);
|
||||
|
||||
for (auto i = 0; i < _build_expr_ctxs.size(); i++) {
|
||||
|
|
@ -186,6 +205,7 @@ void HashJoiner::_init_hash_table_param(HashTableParam* param, RuntimeState* sta
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
Status HashJoiner::append_chunk_to_ht(RuntimeState* state, const ChunkPtr& chunk) {
|
||||
if (_phase != HashJoinPhase::BUILD) {
|
||||
return Status::OK();
|
||||
|
|
@ -506,6 +526,7 @@ Status HashJoiner::_process_other_conjunct(ChunkPtr* chunk, JoinHashTable& hash_
|
|||
switch (_join_type) {
|
||||
case TJoinOp::LEFT_OUTER_JOIN:
|
||||
case TJoinOp::FULL_OUTER_JOIN:
|
||||
case TJoinOp::ASOF_LEFT_OUTER_JOIN:
|
||||
return _process_outer_join_with_other_conjunct(chunk, _output_probe_column_count, _output_build_column_count,
|
||||
hash_table);
|
||||
case TJoinOp::RIGHT_OUTER_JOIN:
|
||||
|
|
|
|||
|
|
@ -73,7 +73,8 @@ struct HashJoinerParam {
|
|||
std::set<SlotId> build_output_slots, std::set<SlotId> probe_output_slots, size_t max_dop,
|
||||
const TJoinDistributionMode::type distribution_mode, bool enable_late_materialization,
|
||||
bool enable_partition_hash_join, bool is_skew_join,
|
||||
const std::map<SlotId, ExprContext*>& common_expr_ctxs)
|
||||
const std::map<SlotId, ExprContext*>& common_expr_ctxs, TExprOpcode::type asof_join_condition_op,
|
||||
ExprContext* asof_join_condition_probe_expr_ctx, ExprContext* asof_join_condition_build_expr_ctx)
|
||||
: _pool(pool),
|
||||
_hash_join_node(hash_join_node),
|
||||
_is_null_safes(std::move(is_null_safes)),
|
||||
|
|
@ -94,7 +95,10 @@ struct HashJoinerParam {
|
|||
_enable_late_materialization(enable_late_materialization),
|
||||
_enable_partition_hash_join(enable_partition_hash_join),
|
||||
_is_skew_join(is_skew_join),
|
||||
_common_expr_ctxs(common_expr_ctxs) {}
|
||||
_common_expr_ctxs(common_expr_ctxs),
|
||||
_asof_join_condition_op(asof_join_condition_op),
|
||||
_asof_join_condition_probe_expr_ctx(asof_join_condition_probe_expr_ctx),
|
||||
_asof_join_condition_build_expr_ctx(asof_join_condition_build_expr_ctx) {}
|
||||
|
||||
HashJoinerParam(HashJoinerParam&&) = default;
|
||||
HashJoinerParam(HashJoinerParam&) = default;
|
||||
|
|
@ -123,6 +127,9 @@ struct HashJoinerParam {
|
|||
const bool _enable_partition_hash_join;
|
||||
const bool _is_skew_join;
|
||||
const std::map<SlotId, ExprContext*> _common_expr_ctxs;
|
||||
TExprOpcode::type _asof_join_condition_op;
|
||||
ExprContext* _asof_join_condition_probe_expr_ctx;
|
||||
ExprContext* _asof_join_condition_build_expr_ctx;
|
||||
};
|
||||
|
||||
inline bool could_short_circuit(TJoinOp::type join_type) {
|
||||
|
|
@ -496,6 +503,10 @@ private:
|
|||
size_t _max_dop = 0;
|
||||
|
||||
bool _is_skew_join = false;
|
||||
|
||||
TExprOpcode::type _asof_join_condition_op = TExprOpcode::INVALID_OPCODE;
|
||||
ExprContext* _asof_join_condition_probe_expr_ctx = nullptr;
|
||||
ExprContext* _asof_join_condition_build_expr_ctx = nullptr;
|
||||
};
|
||||
|
||||
} // namespace starrocks
|
||||
|
|
|
|||
|
|
@ -52,6 +52,10 @@ private:
|
|||
template <LogicalType LT>
|
||||
static std::pair<bool, JoinHashMapMethodUnaryType> _try_use_linear_chained(RuntimeState* state,
|
||||
JoinHashTableItems* table_items);
|
||||
|
||||
// Helper method to get fallback hash map method type based on join type
|
||||
template <LogicalType LT>
|
||||
static std::pair<bool, JoinHashMapMethodUnaryType> _get_fallback_method(bool is_asof_join_type);
|
||||
};
|
||||
|
||||
std::tuple<JoinKeyConstructorUnaryType, JoinHashMapMethodUnaryType>
|
||||
|
|
@ -161,16 +165,25 @@ JoinHashMapMethodUnaryType JoinHashMapSelector::_determine_hash_map_method(
|
|||
return hash_map_type;
|
||||
}
|
||||
|
||||
return JoinHashMapMethodTypeTraits<JoinHashMapMethodType::BUCKET_CHAINED, LT>::unary_type;
|
||||
return _get_fallback_method<LT>(is_asof_join(table_items->join_type)).second;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
template <LogicalType LT>
|
||||
std::pair<bool, JoinHashMapMethodUnaryType> JoinHashMapSelector::_get_fallback_method(bool is_asof_join_type) {
|
||||
return {false, is_asof_join_type
|
||||
? JoinHashMapMethodTypeTraits<JoinHashMapMethodType::LINEAR_CHAINED_ASOF, LT>::unary_type
|
||||
: JoinHashMapMethodTypeTraits<JoinHashMapMethodType::BUCKET_CHAINED, LT>::unary_type};
|
||||
}
|
||||
|
||||
template <LogicalType LT>
|
||||
std::pair<bool, JoinHashMapMethodUnaryType> JoinHashMapSelector::_try_use_range_direct_mapping(
|
||||
RuntimeState* state, JoinHashTableItems* table_items) {
|
||||
bool is_asof_join_type = is_asof_join(table_items->join_type);
|
||||
|
||||
if (!state->enable_hash_join_range_direct_mapping_opt()) {
|
||||
return {false, JoinHashMapMethodUnaryType::BUCKET_CHAINED_INT};
|
||||
return _get_fallback_method<LT>(is_asof_join_type);
|
||||
}
|
||||
|
||||
using KeyConstructor = typename JoinKeyConstructorTypeTraits<JoinKeyConstructorType::ONE_KEY, LT>::BuildType;
|
||||
|
|
@ -181,12 +194,12 @@ std::pair<bool, JoinHashMapMethodUnaryType> JoinHashMapSelector::_try_use_range_
|
|||
|
||||
// `max_value - min_value + 1` will be overflow.
|
||||
if (min_value == std::numeric_limits<int64_t>::min() && max_value == std::numeric_limits<int64_t>::max()) {
|
||||
return {false, JoinHashMapMethodUnaryType::BUCKET_CHAINED_INT};
|
||||
return _get_fallback_method<LT>(is_asof_join_type);
|
||||
}
|
||||
|
||||
const uint64_t value_interval = static_cast<uint64_t>(max_value) - min_value + 1;
|
||||
if (value_interval >= std::numeric_limits<uint32_t>::max()) {
|
||||
return {false, JoinHashMapMethodUnaryType::BUCKET_CHAINED_INT};
|
||||
return _get_fallback_method<LT>(is_asof_join_type);
|
||||
}
|
||||
|
||||
table_items->min_value = min_value;
|
||||
|
|
@ -226,19 +239,20 @@ std::pair<bool, JoinHashMapMethodUnaryType> JoinHashMapSelector::_try_use_range_
|
|||
}
|
||||
}
|
||||
|
||||
return {false, JoinHashMapMethodUnaryType::BUCKET_CHAINED_INT};
|
||||
return _get_fallback_method<LT>(is_asof_join_type);
|
||||
}
|
||||
|
||||
template <LogicalType LT>
|
||||
std::pair<bool, JoinHashMapMethodUnaryType> JoinHashMapSelector::_try_use_linear_chained(
|
||||
RuntimeState* state, JoinHashTableItems* table_items) {
|
||||
bool is_asof_join_type = is_asof_join(table_items->join_type);
|
||||
if (!state->enable_hash_join_linear_chained_opt()) {
|
||||
return {false, JoinHashMapMethodTypeTraits<JoinHashMapMethodType::BUCKET_CHAINED, LT>::unary_type};
|
||||
return _get_fallback_method<LT>(is_asof_join_type);
|
||||
}
|
||||
|
||||
const uint64_t bucket_size = JoinHashMapHelper::calc_bucket_size(table_items->row_count + 1);
|
||||
if (bucket_size > LinearChainedJoinHashMap<LT>::max_supported_bucket_size()) {
|
||||
return {false, JoinHashMapMethodTypeTraits<JoinHashMapMethodType::BUCKET_CHAINED, LT>::unary_type};
|
||||
return _get_fallback_method<LT>(is_asof_join_type);
|
||||
}
|
||||
|
||||
const bool is_left_anti_join_without_other_conjunct =
|
||||
|
|
@ -347,6 +361,7 @@ void JoinHashTable::create(const HashTableParam& param) {
|
|||
_table_items->with_other_conjunct = param.with_other_conjunct;
|
||||
_table_items->join_type = param.join_type;
|
||||
_table_items->enable_late_materialization = param.enable_late_materialization;
|
||||
_table_items->asof_join_condition_desc = param.asof_join_condition_desc;
|
||||
|
||||
if (_table_items->join_type == TJoinOp::RIGHT_SEMI_JOIN || _table_items->join_type == TJoinOp::RIGHT_ANTI_JOIN ||
|
||||
_table_items->join_type == TJoinOp::RIGHT_OUTER_JOIN) {
|
||||
|
|
@ -354,12 +369,21 @@ void JoinHashTable::create(const HashTableParam& param) {
|
|||
} else if (_table_items->join_type == TJoinOp::LEFT_SEMI_JOIN ||
|
||||
_table_items->join_type == TJoinOp::LEFT_ANTI_JOIN ||
|
||||
_table_items->join_type == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN ||
|
||||
_table_items->join_type == TJoinOp::LEFT_OUTER_JOIN) {
|
||||
_table_items->join_type == TJoinOp::LEFT_OUTER_JOIN ||
|
||||
_table_items->join_type == TJoinOp::ASOF_LEFT_OUTER_JOIN) {
|
||||
_table_items->right_to_nullable = true;
|
||||
} else if (_table_items->join_type == TJoinOp::FULL_OUTER_JOIN) {
|
||||
_table_items->left_to_nullable = true;
|
||||
_table_items->right_to_nullable = true;
|
||||
}
|
||||
|
||||
if (is_asof_join(_table_items->join_type)) {
|
||||
auto variant_index = get_asof_variant_index(_table_items->asof_join_condition_desc.build_logical_type,
|
||||
_table_items->asof_join_condition_desc.condition_op);
|
||||
DCHECK_LT(variant_index, 12) << "Invalid variant index";
|
||||
_table_items->asof_index_vector = create_asof_index_vector(variant_index);
|
||||
}
|
||||
|
||||
_table_items->join_keys = param.join_keys;
|
||||
|
||||
_init_probe_column(param);
|
||||
|
|
@ -694,6 +718,7 @@ void JoinHashTable::remove_duplicate_index(Filter* filter) {
|
|||
if (_is_empty_map) {
|
||||
switch (_table_items->join_type) {
|
||||
case TJoinOp::LEFT_OUTER_JOIN:
|
||||
case TJoinOp::ASOF_LEFT_OUTER_JOIN:
|
||||
case TJoinOp::LEFT_ANTI_JOIN:
|
||||
case TJoinOp::FULL_OUTER_JOIN:
|
||||
case TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN: {
|
||||
|
|
@ -712,6 +737,7 @@ void JoinHashTable::remove_duplicate_index(Filter* filter) {
|
|||
DCHECK_LT(0, _table_items->row_count);
|
||||
switch (_table_items->join_type) {
|
||||
case TJoinOp::LEFT_OUTER_JOIN:
|
||||
case TJoinOp::ASOF_LEFT_OUTER_JOIN:
|
||||
_remove_duplicate_index_for_left_outer_join(filter);
|
||||
break;
|
||||
case TJoinOp::LEFT_SEMI_JOIN:
|
||||
|
|
|
|||
|
|
@ -52,7 +52,8 @@ public:
|
|||
case TJoinOp::FULL_OUTER_JOIN:
|
||||
case TJoinOp::LEFT_ANTI_JOIN:
|
||||
case TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN:
|
||||
case TJoinOp::LEFT_OUTER_JOIN: {
|
||||
case TJoinOp::LEFT_OUTER_JOIN:
|
||||
case TJoinOp::ASOF_LEFT_OUTER_JOIN: {
|
||||
_probe_state->count = (*probe_chunk)->num_rows();
|
||||
_probe_output<false>(probe_chunk, chunk);
|
||||
_build_output<false>(chunk);
|
||||
|
|
@ -320,6 +321,21 @@ private:
|
|||
void _probe_from_ht_for_left_outer_left_anti_full_outer_join_with_other_conjunct(
|
||||
RuntimeState* state, const ImmBuffer<CppType> build_data, const ImmBuffer<CppType> probe_data);
|
||||
|
||||
// for AsOf inner join
|
||||
template <bool first_probe, bool is_collision_free_and_unique>
|
||||
void _probe_from_ht_for_asof_inner_join(RuntimeState* state, const ImmBuffer<CppType> build_data,
|
||||
const ImmBuffer<CppType> probe_data);
|
||||
// for asof left outer join
|
||||
template <bool first_probe, bool is_collision_free_and_unique>
|
||||
void _probe_from_ht_for_asof_left_outer_join(RuntimeState* state, const ImmBuffer<CppType> build_data,
|
||||
const ImmBuffer<CppType> probe_data);
|
||||
|
||||
// for asof left outer join with other conjunct
|
||||
template <bool first_probe, bool is_collision_free_and_unique>
|
||||
void _probe_from_ht_for_asof_left_outer_join_with_other_conjunct(RuntimeState* state,
|
||||
const ImmBuffer<CppType> build_data,
|
||||
const ImmBuffer<CppType> probe_data);
|
||||
|
||||
JoinHashTableItems* _table_items = nullptr;
|
||||
HashTableProbeState* _probe_state = nullptr;
|
||||
};
|
||||
|
|
@ -438,6 +454,7 @@ private:
|
|||
JoinHashMapForNonSmallKey(BUCKET_CHAINED), //
|
||||
JoinHashMapForNonSmallKey(LINEAR_CHAINED), //
|
||||
JoinHashMapForNonSmallKey(LINEAR_CHAINED_SET), //
|
||||
JoinHashMapForNonSmallKey(LINEAR_CHAINED_ASOF), //
|
||||
JoinHashMapForIntBigintKey(RANGE_DIRECT_MAPPING), //
|
||||
JoinHashMapForIntBigintKey(RANGE_DIRECT_MAPPING_SET), //
|
||||
JoinHashMapForIntBigintKey(DENSE_RANGE_DIRECT_MAPPING) //
|
||||
|
|
|
|||
|
|
@ -15,7 +15,7 @@
|
|||
#pragma once
|
||||
|
||||
#include "column/column.h"
|
||||
#include "simd/gather.h"
|
||||
#include "exec/join/join_hash_table_descriptor.h"
|
||||
#include "simd/simd.h"
|
||||
#include "util/runtime_profile.h"
|
||||
|
||||
|
|
@ -389,6 +389,14 @@ void JoinHashMap<LT, CT, MT>::_search_ht(RuntimeState* state, ChunkPtr* probe_ch
|
|||
_probe_state->probe_index.resize(state->chunk_size() + 8);
|
||||
_probe_state->build_index.resize(state->chunk_size() + 8);
|
||||
}
|
||||
|
||||
if (is_asof_join(_table_items->join_type)) {
|
||||
_probe_state->asof_temporal_condition_column =
|
||||
(*probe_chunk)->get_column_by_slot_id(_table_items->asof_join_condition_desc.probe_slot_id);
|
||||
// Disable coroutines for ASOF joins unconditionally
|
||||
_probe_state->active_coroutines = 0;
|
||||
}
|
||||
|
||||
if (!_probe_state->has_remain) {
|
||||
_probe_state->probe_row_count = (*probe_chunk)->num_rows();
|
||||
_probe_state->active_coroutines = state->query_options().interleaving_group_size;
|
||||
|
|
@ -509,6 +517,13 @@ void JoinHashMap<LT, CT, MT>::_search_ht_impl(RuntimeState* state, const ImmBuff
|
|||
case TJoinOp::FULL_OUTER_JOIN:
|
||||
DO_PROBE(_probe_from_ht_for_full_outer_join);
|
||||
break;
|
||||
case TJoinOp::ASOF_INNER_JOIN:
|
||||
_probe_from_ht_for_asof_inner_join<first_probe, is_collision_free_and_unique>(state, build_data, data);
|
||||
break;
|
||||
case TJoinOp::ASOF_LEFT_OUTER_JOIN:
|
||||
_probe_from_ht_for_asof_left_outer_join_with_other_conjunct<first_probe, is_collision_free_and_unique>(
|
||||
state, build_data, data);
|
||||
break;
|
||||
default:
|
||||
DO_PROBE(_probe_from_ht);
|
||||
break;
|
||||
|
|
@ -539,6 +554,13 @@ void JoinHashMap<LT, CT, MT>::_search_ht_impl(RuntimeState* state, const ImmBuff
|
|||
is_collision_free_and_unique>(
|
||||
state, build_data, data);
|
||||
break;
|
||||
case TJoinOp::ASOF_INNER_JOIN:
|
||||
_probe_from_ht_for_asof_inner_join<first_probe, is_collision_free_and_unique>(state, build_data, data);
|
||||
break;
|
||||
case TJoinOp::ASOF_LEFT_OUTER_JOIN:
|
||||
_probe_from_ht_for_asof_left_outer_join_with_other_conjunct<first_probe, is_collision_free_and_unique>(
|
||||
state, build_data, data);
|
||||
break;
|
||||
default:
|
||||
// can't reach here
|
||||
_probe_from_ht<first_probe, is_collision_free_and_unique>(state, build_data, data);
|
||||
|
|
@ -651,12 +673,13 @@ void JoinHashMap<LT, CT, MT>::_search_ht_impl(RuntimeState* state, const ImmBuff
|
|||
} \
|
||||
}
|
||||
|
||||
#define PROBE_OVER() \
|
||||
_probe_state->has_remain = false; \
|
||||
_probe_state->cur_probe_index = 0; \
|
||||
_probe_state->cur_build_index = 0; \
|
||||
_probe_state->count = match_count; \
|
||||
_probe_state->cur_row_match_count = 0;
|
||||
#define PROBE_OVER() \
|
||||
_probe_state->has_remain = false; \
|
||||
_probe_state->cur_probe_index = 0; \
|
||||
_probe_state->cur_build_index = 0; \
|
||||
_probe_state->count = match_count; \
|
||||
_probe_state->cur_row_match_count = 0; \
|
||||
_probe_state->asof_temporal_condition_column = nullptr;
|
||||
|
||||
#define MATCH_RIGHT_TABLE_ROWS() \
|
||||
_probe_state->probe_index[match_count] = i; \
|
||||
|
|
@ -819,6 +842,92 @@ HashTableProbeState::ProbeCoroutine JoinHashMap<LT, CT, MT>::_probe_from_ht(Runt
|
|||
PROBE_OVER()
|
||||
}
|
||||
|
||||
template <LogicalType LT, JoinKeyConstructorType CT, JoinHashMapMethodType MT>
|
||||
template <bool first_probe, bool is_collision_free_and_unique>
|
||||
void JoinHashMap<LT, CT, MT>::_probe_from_ht_for_asof_inner_join(RuntimeState* state,
|
||||
const ImmBuffer<CppType> build_data,
|
||||
const ImmBuffer<CppType> probe_data) {
|
||||
_probe_state->match_flag = JoinMatchFlag::NORMAL;
|
||||
size_t match_count = 0;
|
||||
constexpr bool one_to_many = false;
|
||||
size_t i = _probe_state->cur_probe_index;
|
||||
|
||||
if constexpr (!first_probe) {
|
||||
_probe_state->probe_index[0] = _probe_state->cur_probe_index;
|
||||
_probe_state->build_index[0] = _probe_state->cur_build_index;
|
||||
match_count = 1;
|
||||
if (_probe_state->next[i] == 0) {
|
||||
i++;
|
||||
}
|
||||
}
|
||||
|
||||
[[maybe_unused]] size_t probe_cont = 0;
|
||||
|
||||
if constexpr (first_probe) {
|
||||
memset(_probe_state->probe_match_filter.data(), 0, _probe_state->probe_row_count * sizeof(uint8_t));
|
||||
}
|
||||
|
||||
const size_t probe_row_count = _probe_state->probe_row_count;
|
||||
const auto* probe_buckets = _probe_state->next.data();
|
||||
|
||||
LogicalType asof_temporal_probe_type = _table_items->asof_join_condition_desc.probe_logical_type;
|
||||
TExprOpcode::type opcode = _table_items->asof_join_condition_desc.condition_op;
|
||||
|
||||
auto process_probe_rows = [&]<LogicalType ASOF_LT, TExprOpcode::type OpCode>() {
|
||||
using AsofTemporalCppType = RunTimeCppType<ASOF_LT>;
|
||||
|
||||
const auto* asof_temporal_data_column =
|
||||
ColumnHelper::get_data_column_by_type<ASOF_LT>(_probe_state->asof_temporal_condition_column.get());
|
||||
const NullColumn* asof_temporal_col_nulls =
|
||||
ColumnHelper::get_null_column(_probe_state->asof_temporal_condition_column);
|
||||
const AsofTemporalCppType* asof_temporal_probe_values = asof_temporal_data_column->immutable_data().data();
|
||||
|
||||
if (!_probe_state->asof_temporal_condition_column || _probe_state->asof_temporal_condition_column->empty()) {
|
||||
PROBE_OVER();
|
||||
return;
|
||||
}
|
||||
|
||||
constexpr size_t variant_index = get_asof_variant_index(ASOF_LT, OpCode);
|
||||
auto& asof_index_vector = get_asof_index_vector_static<variant_index>(_table_items);
|
||||
|
||||
for (; i < probe_row_count; i++) {
|
||||
uint32_t build_index = probe_buckets[i];
|
||||
|
||||
if (build_index == 0) {
|
||||
continue;
|
||||
}
|
||||
|
||||
if (asof_temporal_col_nulls && const_cast<NullColumn*>(asof_temporal_col_nulls)->get_data()[i] != 0) {
|
||||
continue;
|
||||
}
|
||||
|
||||
DCHECK_LT(i, asof_temporal_data_column->size());
|
||||
AsofTemporalCppType probe_temporal_value = asof_temporal_probe_values[i];
|
||||
|
||||
uint32_t matched_build_row_index = asof_index_vector[build_index]->find_asof_match(probe_temporal_value);
|
||||
if (matched_build_row_index != 0) {
|
||||
_probe_state->probe_index[match_count] = i;
|
||||
_probe_state->build_index[match_count] = matched_build_row_index;
|
||||
match_count++;
|
||||
|
||||
if constexpr (first_probe) {
|
||||
_probe_state->probe_match_filter[i] = 1;
|
||||
}
|
||||
|
||||
probe_cont++;
|
||||
}
|
||||
}
|
||||
|
||||
if constexpr (first_probe) {
|
||||
CHECK_MATCH()
|
||||
}
|
||||
|
||||
PROBE_OVER();
|
||||
};
|
||||
|
||||
AsofJoinProbeDispatcher::dispatch(asof_temporal_probe_type, opcode, process_probe_rows);
|
||||
}
|
||||
|
||||
template <LogicalType LT, JoinKeyConstructorType CT, JoinHashMapMethodType MT>
|
||||
HashTableProbeState::ProbeCoroutine JoinHashMap<LT, CT, MT>::_probe_from_ht_for_left_outer_join(
|
||||
RuntimeState* state, const ImmBuffer<CppType> build_data, const ImmBuffer<CppType> probe_data) {
|
||||
|
|
@ -938,6 +1047,101 @@ void JoinHashMap<LT, CT, MT>::_probe_from_ht_for_left_outer_join(RuntimeState* s
|
|||
}
|
||||
PROBE_OVER()
|
||||
}
|
||||
|
||||
template <LogicalType LT, JoinKeyConstructorType CT, JoinHashMapMethodType MT>
|
||||
template <bool first_probe, bool is_collision_free_and_unique>
|
||||
void JoinHashMap<LT, CT, MT>::_probe_from_ht_for_asof_left_outer_join(RuntimeState* state,
|
||||
const ImmBuffer<CppType> build_data,
|
||||
const ImmBuffer<CppType> probe_data) {
|
||||
_probe_state->match_flag = JoinMatchFlag::NORMAL;
|
||||
size_t match_count = 0;
|
||||
constexpr bool one_to_many = false;
|
||||
size_t i = _probe_state->cur_probe_index;
|
||||
|
||||
if constexpr (!first_probe) {
|
||||
_probe_state->probe_index[0] = _probe_state->cur_probe_index;
|
||||
_probe_state->build_index[0] = _probe_state->cur_build_index;
|
||||
match_count = 1;
|
||||
if (_probe_state->next[i] == 0) {
|
||||
i++;
|
||||
_probe_state->cur_row_match_count = 0;
|
||||
}
|
||||
}
|
||||
|
||||
[[maybe_unused]] size_t probe_cont = 0;
|
||||
|
||||
if constexpr (first_probe) {
|
||||
memset(_probe_state->probe_match_filter.data(), 0, _probe_state->probe_row_count * sizeof(uint8_t));
|
||||
}
|
||||
|
||||
uint32_t cur_row_match_count = _probe_state->cur_row_match_count;
|
||||
const size_t probe_row_count = _probe_state->probe_row_count;
|
||||
const auto* probe_buckets = _probe_state->next.data();
|
||||
|
||||
LogicalType asof_temporal_probe_type = _table_items->asof_join_condition_desc.probe_logical_type;
|
||||
TExprOpcode::type opcode = _table_items->asof_join_condition_desc.condition_op;
|
||||
|
||||
auto process_probe_rows = [&]<LogicalType ASOF_LT, TExprOpcode::type OpCode>() {
|
||||
using AsofTemporalCppType = RunTimeCppType<ASOF_LT>;
|
||||
constexpr size_t variant_index = get_asof_variant_index(ASOF_LT, OpCode);
|
||||
auto& asof_index_vector = get_asof_index_vector_static<variant_index>(_table_items);
|
||||
|
||||
const auto* asof_temporal_data_column =
|
||||
ColumnHelper::get_data_column_by_type<ASOF_LT>(_probe_state->asof_temporal_condition_column.get());
|
||||
const NullColumn* asof_temporal_col_nulls =
|
||||
ColumnHelper::get_null_column(_probe_state->asof_temporal_condition_column);
|
||||
const AsofTemporalCppType* asof_temporal_probe_values = asof_temporal_data_column->immutable_data().data();
|
||||
|
||||
if (!_probe_state->asof_temporal_condition_column || _probe_state->asof_temporal_condition_column->empty()) {
|
||||
LOG(WARNING) << "ASOF LEFT OUTER: No valid asof column";
|
||||
PROBE_OVER();
|
||||
return;
|
||||
}
|
||||
|
||||
for (; i < probe_row_count; i++) {
|
||||
uint32_t build_index = probe_buckets[i];
|
||||
if (build_index == 0 ||
|
||||
(asof_temporal_col_nulls && const_cast<NullColumn*>(asof_temporal_col_nulls)->get_data()[i] != 0)) {
|
||||
_probe_state->probe_index[match_count] = i;
|
||||
_probe_state->build_index[match_count] = 0;
|
||||
match_count++;
|
||||
RETURN_IF_CHUNK_FULL2()
|
||||
continue;
|
||||
}
|
||||
|
||||
DCHECK_LT(i, asof_temporal_data_column->size());
|
||||
AsofTemporalCppType probe_temporal_value = asof_temporal_probe_values[i];
|
||||
uint32_t matched_build_row_index = asof_index_vector[build_index]->find_asof_match(probe_temporal_value);
|
||||
|
||||
if (matched_build_row_index != 0) {
|
||||
_probe_state->probe_index[match_count] = i;
|
||||
_probe_state->build_index[match_count] = matched_build_row_index;
|
||||
match_count++;
|
||||
cur_row_match_count++;
|
||||
RETURN_IF_CHUNK_FULL2()
|
||||
} else {
|
||||
_probe_state->probe_index[match_count] = i;
|
||||
_probe_state->build_index[match_count] = 0;
|
||||
match_count++;
|
||||
|
||||
RETURN_IF_CHUNK_FULL2()
|
||||
}
|
||||
|
||||
cur_row_match_count = 0;
|
||||
}
|
||||
|
||||
_probe_state->cur_row_match_count = cur_row_match_count;
|
||||
|
||||
if constexpr (first_probe) {
|
||||
CHECK_MATCH()
|
||||
}
|
||||
|
||||
PROBE_OVER();
|
||||
};
|
||||
|
||||
AsofJoinProbeDispatcher::dispatch(asof_temporal_probe_type, opcode, process_probe_rows);
|
||||
}
|
||||
|
||||
template <LogicalType LT, JoinKeyConstructorType CT, JoinHashMapMethodType MT>
|
||||
HashTableProbeState::ProbeCoroutine JoinHashMap<LT, CT, MT>::_probe_from_ht_for_left_semi_join(
|
||||
RuntimeState* state, const ImmBuffer<CppType> build_data, const ImmBuffer<CppType> probe_data) {
|
||||
|
|
@ -1652,6 +1856,101 @@ void JoinHashMap<LT, CT, MT>::_probe_from_ht_for_left_outer_left_anti_full_outer
|
|||
PROBE_OVER()
|
||||
}
|
||||
|
||||
template <LogicalType LT, JoinKeyConstructorType CT, JoinHashMapMethodType MT>
|
||||
template <bool first_probe, bool is_collision_free_and_unique>
|
||||
void JoinHashMap<LT, CT, MT>::_probe_from_ht_for_asof_left_outer_join_with_other_conjunct(
|
||||
RuntimeState* state, const ImmBuffer<CppType> build_data, const ImmBuffer<CppType> probe_data) {
|
||||
_probe_state->match_flag = JoinMatchFlag::NORMAL;
|
||||
size_t match_count = 0;
|
||||
constexpr bool one_to_many = false;
|
||||
size_t i = _probe_state->cur_probe_index;
|
||||
|
||||
if constexpr (!first_probe) {
|
||||
_probe_state->probe_index[0] = _probe_state->cur_probe_index;
|
||||
_probe_state->build_index[0] = _probe_state->cur_build_index;
|
||||
match_count = 1;
|
||||
if (_probe_state->next[i] == 0) {
|
||||
i++;
|
||||
_probe_state->cur_row_match_count = 0;
|
||||
}
|
||||
} else {
|
||||
_probe_state->cur_row_match_count = 0;
|
||||
for (size_t j = 0; j < state->chunk_size(); j++) {
|
||||
_probe_state->probe_match_index[j] = 0;
|
||||
}
|
||||
}
|
||||
|
||||
const size_t probe_row_count = _probe_state->probe_row_count;
|
||||
const auto* probe_buckets = _probe_state->next.data();
|
||||
|
||||
LogicalType asof_temporal_probe_type = _table_items->asof_join_condition_desc.probe_logical_type;
|
||||
TExprOpcode::type opcode = _table_items->asof_join_condition_desc.condition_op;
|
||||
|
||||
auto process_probe_rows = [&]<LogicalType ASOF_LT, TExprOpcode::type OpCode>() {
|
||||
using AsofTemporalCppType = RunTimeCppType<ASOF_LT>;
|
||||
constexpr size_t variant_index = get_asof_variant_index(ASOF_LT, OpCode);
|
||||
auto& asof_index_vector = get_asof_index_vector_static<variant_index>(_table_items);
|
||||
|
||||
const auto* asof_temporal_data_column =
|
||||
ColumnHelper::get_data_column_by_type<ASOF_LT>(_probe_state->asof_temporal_condition_column.get());
|
||||
const NullColumn* asof_temporal_col_nulls =
|
||||
ColumnHelper::get_null_column(_probe_state->asof_temporal_condition_column);
|
||||
const AsofTemporalCppType* asof_temporal_probe_values = asof_temporal_data_column->immutable_data().data();
|
||||
|
||||
if (!_probe_state->asof_temporal_condition_column || _probe_state->asof_temporal_condition_column->empty()) {
|
||||
LOG(WARNING) << "ASOF LEFT OUTER WITH OTHER CONJUNCT: No valid asof column";
|
||||
for (; i < probe_row_count; i++) {
|
||||
_probe_state->probe_index[match_count] = i;
|
||||
_probe_state->build_index[match_count] = 0;
|
||||
match_count++;
|
||||
}
|
||||
|
||||
PROBE_OVER();
|
||||
return;
|
||||
}
|
||||
|
||||
for (; i < probe_row_count; i++) {
|
||||
uint32_t build_index = probe_buckets[i];
|
||||
if (build_index == 0 ||
|
||||
(asof_temporal_col_nulls && const_cast<NullColumn*>(asof_temporal_col_nulls)->get_data()[i] != 0)) {
|
||||
_probe_state->probe_index[match_count] = i;
|
||||
_probe_state->build_index[match_count] = 0;
|
||||
match_count++;
|
||||
RETURN_IF_CHUNK_FULL()
|
||||
continue;
|
||||
}
|
||||
|
||||
DCHECK_LT(i, asof_temporal_data_column->size());
|
||||
AsofTemporalCppType probe_temporal_value = asof_temporal_probe_values[i];
|
||||
uint32_t matched_build_row_index = asof_index_vector[build_index]->find_asof_match(probe_temporal_value);
|
||||
|
||||
if (matched_build_row_index != 0) {
|
||||
_probe_state->probe_index[match_count] = i;
|
||||
_probe_state->build_index[match_count] = matched_build_row_index;
|
||||
_probe_state->probe_match_index[i]++;
|
||||
_probe_state->cur_row_match_count++;
|
||||
match_count++;
|
||||
RETURN_IF_CHUNK_FULL()
|
||||
} else {
|
||||
_probe_state->probe_index[match_count] = i;
|
||||
_probe_state->build_index[match_count] = 0;
|
||||
match_count++;
|
||||
RETURN_IF_CHUNK_FULL()
|
||||
}
|
||||
|
||||
_probe_state->cur_row_match_count = 0;
|
||||
}
|
||||
|
||||
if constexpr (first_probe) {
|
||||
CHECK_MATCH()
|
||||
}
|
||||
|
||||
PROBE_OVER();
|
||||
};
|
||||
|
||||
AsofJoinProbeDispatcher::dispatch(asof_temporal_probe_type, opcode, process_probe_rows);
|
||||
}
|
||||
|
||||
// ------------------------------------------------------------------------------------
|
||||
// JoinHashTable
|
||||
// ------------------------------------------------------------------------------------
|
||||
|
|
|
|||
|
|
@ -91,6 +91,15 @@ public:
|
|||
}
|
||||
}
|
||||
|
||||
template <typename CppType>
|
||||
static std::pair<uint32_t, uint8_t> calc_bucket_num_and_fp(const CppType& value, uint32_t bucket_size,
|
||||
uint32_t num_log_buckets) {
|
||||
static constexpr uint64_t FP_BITS = 7;
|
||||
using HashFunc = JoinKeyHash<CppType>;
|
||||
const uint64_t hash = HashFunc()(value, bucket_size << FP_BITS, num_log_buckets + FP_BITS);
|
||||
return {hash >> FP_BITS, (hash & 0x7F) | 0x80};
|
||||
}
|
||||
|
||||
static Slice get_hash_key(const Columns& key_columns, size_t row_idx, uint8_t* buffer) {
|
||||
size_t byte_size = 0;
|
||||
for (const auto& key_column : key_columns) {
|
||||
|
|
|
|||
|
|
@ -150,6 +150,72 @@ private:
|
|||
template <LogicalType LT>
|
||||
using LinearChainedJoinHashSet = LinearChainedJoinHashMap<LT, false>;
|
||||
|
||||
// The `LinearChainedAsofJoinHashMap` is specifically designed for ASOF JOIN operations on time-series data.
|
||||
// It uses linear probing with separated fingerprint storage and maintains ASOF temporal indexes for efficient
|
||||
// time-based matching.
|
||||
// - `first` stores the build index for each distinct equi-join key.
|
||||
// - `fps` stores the separated fingerprints for fast collision detection.
|
||||
// - `asof_index_vector` maintains sorted temporal indexes for ASOF matching logic.
|
||||
//
|
||||
// Fingerprint Design
|
||||
// - Uses 7-bit fingerprints stored separately in the `fps` array, with the highest bit always set to 1
|
||||
// to ensure non-zero values (distinguishable from empty buckets).
|
||||
// - Fingerprints range from 0x80 to 0xFF (128 possible values), providing 1/128 ≈ 0.78% collision rate.
|
||||
// - The fingerprint is computed using an extended hash space: `bucket_size << 7` and `num_log_buckets + 7`.
|
||||
//
|
||||
// ASOF Temporal Processing
|
||||
// - Each equi-join bucket maintains its own ASOF index containing temporal column values and row indexes.
|
||||
// - ASOF indexes are sorted by temporal values to enable efficient binary search during probe phase.
|
||||
// - Supports various ASOF operations: LT, LE, GT, GE for different temporal matching requirements.
|
||||
//
|
||||
// Memory Layout Comparison with LinearChainedJoinHashMap:
|
||||
// - LinearChainedJoinHashMap: `first[bucket] = (8-bit FP << 24) | 24-bit build_index` (packed)
|
||||
// - LinearChainedAsofJoinHashMap: `first[bucket] = 32-bit build_index`, `fps[bucket] = 8-bit FP` (separated)
|
||||
// - Trade-off: Uses 25% more memory per bucket but supports unlimited bucket sizes and easier ASOF processing.
|
||||
//
|
||||
// Insert and Probe for ASOF JOIN
|
||||
// - During insertion, linear probing locates the appropriate bucket for the equi-join key.
|
||||
// - For each equi-join bucket, temporal values are added to the corresponding ASOF index.
|
||||
// - During probing, equi-join keys are matched first, then ASOF binary search finds temporal matches.
|
||||
// - No bucket size limitation (unlike LinearChainedJoinHashMap's 16M bucket limit).
|
||||
//
|
||||
// The following diagram illustrates the structure of `LinearChainedAsofJoinHashMap`:
|
||||
//
|
||||
// build keys + temporal first fps asof_index_vector
|
||||
// ┌───────┐ ┌───┐ ┌─────────────────────┐
|
||||
// │ index │ │FP │ │ sorted temporal │
|
||||
// ├───────┤ ├───┤ │ [(t1,idx1),(t2,idx2)│
|
||||
// ┌────┐ ┌─►│ │ │ │ ┌───►│ (t3,idx3),(t4,idx4)]│
|
||||
// ┌──────┐ │ │ │ ├───────┤ ├───┤ │ ├─────────────────────┤
|
||||
// │ key ├──►│hash├─┘ │ │ │ │ │ │ │
|
||||
// └──────┘ │ │ ┌─►├───────┤ ├───┤ │ │ │
|
||||
// └────┘ │ │ index ├──►│FP ├─┘ ├─────────────────────┤
|
||||
// │ ├───────┤ ├───┤ │ sorted temporal │
|
||||
// │ │ │ │ │ │ [(t5,idx5),(t6,idx6)│
|
||||
// │ ├───────┤ ├───┤ ┌───►│ (t7,idx7)] │
|
||||
// └─►│ index ├──►│FP ├─┘ └─────────────────────┘
|
||||
// └───────┘ └───┘ ASOF binary search
|
||||
// No size for temporal matching
|
||||
// limitation
|
||||
template <LogicalType LT>
|
||||
class LinearChainedAsofJoinHashMap {
|
||||
public:
|
||||
using CppType = typename RunTimeTypeTraits<LT>::CppType;
|
||||
using ColumnType = typename RunTimeTypeTraits<LT>::ColumnType;
|
||||
|
||||
static constexpr bool AreKeysInChainIdentical = true;
|
||||
|
||||
static void build_prepare(RuntimeState* state, JoinHashTableItems* table_items);
|
||||
static void construct_hash_table(JoinHashTableItems* table_items, const ImmBuffer<CppType>& keys,
|
||||
const std::optional<ImmBuffer<uint8_t>> is_nulls);
|
||||
|
||||
static void lookup_init(const JoinHashTableItems& table_items, HashTableProbeState* probe_state,
|
||||
const ImmBuffer<CppType>& build_keys, const ImmBuffer<CppType>& probe_keys,
|
||||
const std::optional<ImmBuffer<uint8_t>> is_nulls);
|
||||
|
||||
static bool equal(const CppType& x, const CppType& y) { return true; }
|
||||
};
|
||||
|
||||
// The bucket-chained linked list formed by first` and `next` is the same as that of `BucketChainedJoinHashMap`.
|
||||
//
|
||||
// `DirectMappingJoinHashMap` maps to a position in `first` using `key-MIN_VALUE`.
|
||||
|
|
|
|||
|
|
@ -16,6 +16,7 @@
|
|||
|
||||
#include "exec/join/join_hash_map_helper.h"
|
||||
#include "exec/join/join_hash_map_method.h"
|
||||
#include "exec/join/join_hash_table_descriptor.h"
|
||||
#include "simd/gather.h"
|
||||
|
||||
namespace starrocks {
|
||||
|
|
@ -129,80 +130,110 @@ void LinearChainedJoinHashMap<LT, NeedBuildChained>::build_prepare(RuntimeState*
|
|||
table_items->log_bucket_size = __builtin_ctz(table_items->bucket_size);
|
||||
table_items->first.resize(table_items->bucket_size, 0);
|
||||
table_items->next.resize(table_items->row_count + 1, 0);
|
||||
if (is_asof_join(table_items->join_type)) {
|
||||
table_items->resize_asof_index_vector(table_items->row_count + 1);
|
||||
}
|
||||
}
|
||||
|
||||
template <LogicalType LT, bool NeedBuildChained>
|
||||
void LinearChainedJoinHashMap<LT, NeedBuildChained>::construct_hash_table(
|
||||
JoinHashTableItems* table_items, const ImmBuffer<CppType>& keys,
|
||||
const std::optional<ImmBuffer<uint8_t>> is_nulls) {
|
||||
auto process = [&]<bool IsNullable>() {
|
||||
const auto num_rows = 1 + table_items->row_count;
|
||||
const uint32_t bucket_size_mask = table_items->bucket_size - 1;
|
||||
const auto num_rows = 1 + table_items->row_count;
|
||||
const uint32_t bucket_size_mask = table_items->bucket_size - 1;
|
||||
auto* __restrict next = table_items->next.data();
|
||||
auto* __restrict first = table_items->first.data();
|
||||
const uint8_t* __restrict equi_join_key_nulls = is_nulls.has_value() ? is_nulls->data() : nullptr;
|
||||
|
||||
auto* __restrict next = table_items->next.data();
|
||||
auto* __restrict first = table_items->first.data();
|
||||
const uint8_t* __restrict is_nulls_data = IsNullable ? is_nulls->data() : nullptr;
|
||||
|
||||
auto need_calc_bucket_num = [&](const uint32_t index) {
|
||||
// Only check `is_nulls_data[i]` for the nullable slice type. The hash calculation overhead for
|
||||
// fixed-size types is small, and thus we do not check it to allow vectorization of the hash calculation.
|
||||
if constexpr (!IsNullable || !std::is_same_v<CppType, Slice>) {
|
||||
return true;
|
||||
} else {
|
||||
return is_nulls_data[index] == 0;
|
||||
}
|
||||
};
|
||||
auto is_null = [&](const uint32_t index) {
|
||||
if constexpr (!IsNullable) {
|
||||
return false;
|
||||
} else {
|
||||
return is_nulls_data[index] != 0;
|
||||
}
|
||||
};
|
||||
|
||||
for (uint32_t i = 1; i < num_rows; i++) {
|
||||
// Use `next` stores `bucket_num` temporarily.
|
||||
if (need_calc_bucket_num(i)) {
|
||||
next[i] = JoinHashMapHelper::calc_bucket_num<CppType>(keys[i], table_items->bucket_size << FP_BITS,
|
||||
table_items->log_bucket_size + FP_BITS);
|
||||
}
|
||||
auto linear_probe = [&]<bool BuildChained, bool ReturnAnchor>(const ImmBuffer<CppType>& keys_ref, uint32_t i,
|
||||
auto&& is_null_pred) -> uint32_t {
|
||||
if (i + 16 < num_rows && !is_null_pred(i + 16)) {
|
||||
__builtin_prefetch(first + _get_bucket_num_from_hash(next[i + 16]));
|
||||
}
|
||||
|
||||
for (uint32_t i = 1; i < num_rows; i++) {
|
||||
if (i + 16 < num_rows && !is_null(i + 16)) {
|
||||
__builtin_prefetch(first + _get_bucket_num_from_hash(next[i + 16]));
|
||||
const uint32_t hash = next[i];
|
||||
const uint32_t fp = _get_fp_from_hash(hash);
|
||||
uint32_t bucket_num = _get_bucket_num_from_hash(hash);
|
||||
|
||||
uint32_t probe_times = 1;
|
||||
while (true) {
|
||||
if (first[bucket_num] == 0) {
|
||||
if constexpr (BuildChained) {
|
||||
next[i] = 0;
|
||||
}
|
||||
first[bucket_num] = _combine_data_fp(i, fp);
|
||||
break;
|
||||
}
|
||||
|
||||
if (is_null(i)) {
|
||||
if (fp == _extract_fp(first[bucket_num]) && keys_ref[i] == keys_ref[_extract_data(first[bucket_num])]) {
|
||||
if constexpr (BuildChained) {
|
||||
next[i] = _extract_data(first[bucket_num]);
|
||||
first[bucket_num] = _combine_data_fp(i, fp);
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
||||
bucket_num = (bucket_num + probe_times) & bucket_size_mask;
|
||||
probe_times++;
|
||||
}
|
||||
|
||||
if constexpr (ReturnAnchor) {
|
||||
return _extract_data(first[bucket_num]);
|
||||
} else {
|
||||
return 0u;
|
||||
}
|
||||
};
|
||||
|
||||
auto compute_hash_values = [&]<bool HasEquiJoinKeyNulls, bool HasAsofTemporalNulls>(
|
||||
const uint8_t* asof_temporal_nulls) {
|
||||
for (uint32_t i = 1; i < num_rows; i++) {
|
||||
if constexpr (HasAsofTemporalNulls) {
|
||||
if (asof_temporal_nulls[i] != 0) continue;
|
||||
}
|
||||
if constexpr (std::is_same_v<CppType, Slice> && HasEquiJoinKeyNulls) {
|
||||
if (equi_join_key_nulls[i] != 0) continue;
|
||||
}
|
||||
|
||||
// Only check `is_nulls_data[i]` for the nullable slice type. The hash calculation overhead for
|
||||
// fixed-size types is small, and thus we do not check it to allow vectorization of the hash calculation.
|
||||
next[i] = JoinHashMapHelper::calc_bucket_num<CppType>(keys[i], table_items->bucket_size << FP_BITS,
|
||||
table_items->log_bucket_size + FP_BITS);
|
||||
}
|
||||
};
|
||||
|
||||
auto dispatch_hash_computation = [&](const uint8_t* asof_temporal_nulls) {
|
||||
if (equi_join_key_nulls == nullptr) {
|
||||
if (asof_temporal_nulls == nullptr) {
|
||||
compute_hash_values.template operator()<false, false>(nullptr);
|
||||
} else {
|
||||
compute_hash_values.template operator()<false, true>(asof_temporal_nulls);
|
||||
}
|
||||
} else {
|
||||
if (asof_temporal_nulls == nullptr) {
|
||||
compute_hash_values.template operator()<true, false>(nullptr);
|
||||
} else {
|
||||
compute_hash_values.template operator()<true, true>(asof_temporal_nulls);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
auto build_hash_table_without_temporal_index = [&]<bool HasEquiJoinKeyNulls>() {
|
||||
dispatch_hash_computation(nullptr);
|
||||
|
||||
auto is_null_row = [&](const uint32_t index) {
|
||||
if constexpr (!HasEquiJoinKeyNulls) {
|
||||
return false;
|
||||
} else {
|
||||
return equi_join_key_nulls[index] != 0;
|
||||
}
|
||||
};
|
||||
|
||||
for (uint32_t i = 1; i < num_rows; i++) {
|
||||
if (is_null_row(i)) {
|
||||
next[i] = 0;
|
||||
continue;
|
||||
}
|
||||
|
||||
const uint32_t hash = next[i];
|
||||
const uint32_t fp = _get_fp_from_hash(hash);
|
||||
uint32_t bucket_num = _get_bucket_num_from_hash(hash);
|
||||
|
||||
uint32_t probe_times = 1;
|
||||
while (true) {
|
||||
if (first[bucket_num] == 0) {
|
||||
if constexpr (NeedBuildChained) {
|
||||
next[i] = 0;
|
||||
}
|
||||
first[bucket_num] = _combine_data_fp(i, fp);
|
||||
break;
|
||||
}
|
||||
|
||||
if (fp == _extract_fp(first[bucket_num]) && keys[i] == keys[_extract_data(first[bucket_num])]) {
|
||||
if constexpr (NeedBuildChained) {
|
||||
next[i] = _extract_data(first[bucket_num]);
|
||||
first[bucket_num] = _combine_data_fp(i, fp);
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
||||
bucket_num = (bucket_num + probe_times) & bucket_size_mask;
|
||||
probe_times++;
|
||||
}
|
||||
(void)linear_probe.template operator()<NeedBuildChained, false>(keys, i, is_null_row);
|
||||
}
|
||||
|
||||
if constexpr (!NeedBuildChained) {
|
||||
|
|
@ -210,11 +241,46 @@ void LinearChainedJoinHashMap<LT, NeedBuildChained>::construct_hash_table(
|
|||
}
|
||||
};
|
||||
|
||||
if (!is_nulls.has_value()) {
|
||||
process.template operator()<false>();
|
||||
} else {
|
||||
process.template operator()<true>();
|
||||
if (!is_asof_join(table_items->join_type)) {
|
||||
if (!is_nulls.has_value()) {
|
||||
build_hash_table_without_temporal_index.template operator()<false>();
|
||||
} else {
|
||||
build_hash_table_without_temporal_index.template operator()<true>();
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
auto build_hash_table_with_temporal_index = [&]() {
|
||||
const ColumnPtr& asof_temporal_column =
|
||||
table_items->build_chunk->get_column_by_slot_id(table_items->asof_join_condition_desc.build_slot_id);
|
||||
const NullColumn* asof_temporal_col_nulls = ColumnHelper::get_null_column(asof_temporal_column);
|
||||
const uint8_t* __restrict asof_temporal_nulls = nullptr;
|
||||
if (asof_temporal_col_nulls != nullptr) {
|
||||
auto* mutable_null_column = const_cast<NullColumn*>(asof_temporal_col_nulls);
|
||||
asof_temporal_nulls = mutable_null_column->get_data().data();
|
||||
}
|
||||
|
||||
dispatch_hash_computation(asof_temporal_nulls);
|
||||
|
||||
if (equi_join_key_nulls == nullptr) {
|
||||
auto equi_join_bucket_locator = [&](JoinHashTableItems*, const ImmBuffer<CppType>& keys_ref, uint32_t i) {
|
||||
return linear_probe.template operator()<false, true>(keys_ref, i, [](uint32_t) { return false; });
|
||||
};
|
||||
AsofJoinDispatcher::dispatch_and_process(
|
||||
table_items, keys, is_nulls.has_value() ? &is_nulls.value() : nullptr, equi_join_bucket_locator);
|
||||
} else {
|
||||
auto is_null_predicate = [&](const uint32_t index) { return equi_join_key_nulls[index] != 0; };
|
||||
auto equi_join_bucket_locator = [&, is_null_predicate](JoinHashTableItems*,
|
||||
const ImmBuffer<CppType>& keys_ref, uint32_t i) {
|
||||
return linear_probe.template operator()<false, true>(keys_ref, i, is_null_predicate);
|
||||
};
|
||||
AsofJoinDispatcher::dispatch_and_process(
|
||||
table_items, keys, is_nulls.has_value() ? &is_nulls.value() : nullptr, equi_join_bucket_locator);
|
||||
}
|
||||
table_items->finalize_asof_index_vector();
|
||||
};
|
||||
|
||||
build_hash_table_with_temporal_index();
|
||||
}
|
||||
|
||||
template <LogicalType LT, bool NeedBuildChained>
|
||||
|
|
@ -301,6 +367,173 @@ void LinearChainedJoinHashMap<LT, NeedBuildChained>::lookup_init(const JoinHashT
|
|||
}
|
||||
}
|
||||
|
||||
// ------------------------------------------------------------------------------------
|
||||
// LinearChainedAsofJoinHashMap
|
||||
// ------------------------------------------------------------------------------------
|
||||
|
||||
template <LogicalType LT>
|
||||
void LinearChainedAsofJoinHashMap<LT>::build_prepare(RuntimeState* state, JoinHashTableItems* table_items) {
|
||||
table_items->bucket_size = JoinHashMapHelper::calc_bucket_size(table_items->row_count + 1);
|
||||
table_items->log_bucket_size = __builtin_ctz(table_items->bucket_size);
|
||||
table_items->first.resize(table_items->bucket_size, 0);
|
||||
table_items->fps.resize(table_items->bucket_size, 0);
|
||||
table_items->next.resize(table_items->row_count + 1, 0);
|
||||
table_items->resize_asof_index_vector(table_items->row_count + 1);
|
||||
}
|
||||
|
||||
template <LogicalType LT>
|
||||
void LinearChainedAsofJoinHashMap<LT>::construct_hash_table(JoinHashTableItems* table_items,
|
||||
const ImmBuffer<CppType>& keys,
|
||||
const std::optional<ImmBuffer<uint8_t>> is_nulls) {
|
||||
const uint32_t num_rows = table_items->row_count + 1;
|
||||
auto* __restrict temp_bucket_numbers = table_items->next.data();
|
||||
std::vector<uint8_t> temp_fingerprints(num_rows);
|
||||
|
||||
const uint8_t* __restrict equi_join_key_nulls = is_nulls.has_value() ? is_nulls->data() : nullptr;
|
||||
const ColumnPtr& asof_temporal_column =
|
||||
table_items->build_chunk->get_column_by_slot_id(table_items->asof_join_condition_desc.build_slot_id);
|
||||
const NullColumn* asof_temporal_col_nulls = ColumnHelper::get_null_column(asof_temporal_column);
|
||||
const uint8_t* __restrict asof_temporal_nulls =
|
||||
asof_temporal_col_nulls ? const_cast<NullColumn*>(asof_temporal_col_nulls)->get_data().data() : nullptr;
|
||||
|
||||
static constexpr uint32_t BATCH_SIZE = 4096;
|
||||
auto compute_batch_hash_values = [&]<bool HasEquiJoinKeyNulls, bool HasAsofTemporalNulls>() {
|
||||
for (uint32_t i = 1; i < num_rows; i += BATCH_SIZE) {
|
||||
const uint32_t batch_count = std::min<uint32_t>(BATCH_SIZE, num_rows - i);
|
||||
auto* bucket_buffer = temp_bucket_numbers + i;
|
||||
auto* fingerprint_buffer = temp_fingerprints.data() + i;
|
||||
for (uint32_t j = 0; j < batch_count; j++) {
|
||||
const uint32_t row_index = i + j;
|
||||
if constexpr (HasEquiJoinKeyNulls && std::is_same_v<CppType, Slice>) {
|
||||
if (equi_join_key_nulls[row_index] != 0) continue;
|
||||
}
|
||||
if constexpr (HasAsofTemporalNulls) {
|
||||
if (asof_temporal_nulls[row_index] != 0) continue;
|
||||
}
|
||||
std::tie(bucket_buffer[j], fingerprint_buffer[j]) = JoinHashMapHelper::calc_bucket_num_and_fp<CppType>(
|
||||
keys[row_index], table_items->bucket_size, table_items->log_bucket_size);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
if (equi_join_key_nulls == nullptr) {
|
||||
if (asof_temporal_nulls == nullptr) {
|
||||
compute_batch_hash_values.template operator()<false, false>();
|
||||
} else {
|
||||
compute_batch_hash_values.template operator()<false, true>();
|
||||
}
|
||||
} else {
|
||||
if (asof_temporal_nulls == nullptr) {
|
||||
compute_batch_hash_values.template operator()<true, false>();
|
||||
} else {
|
||||
compute_batch_hash_values.template operator()<true, true>();
|
||||
}
|
||||
}
|
||||
|
||||
const uint32_t bucket_mask = table_items->bucket_size - 1;
|
||||
auto* __restrict bucket_first_indices = table_items->first.data();
|
||||
auto* __restrict bucket_fingerprints = table_items->fps.data();
|
||||
|
||||
auto equi_join_bucket_locator = [&](JoinHashTableItems*, const ImmBuffer<CppType>&,
|
||||
uint32_t row_index) -> uint32_t {
|
||||
uint32_t bucket_number = temp_bucket_numbers[row_index];
|
||||
const uint8_t fingerprint = temp_fingerprints[row_index];
|
||||
uint32_t probe_attempts = 1;
|
||||
while (true) {
|
||||
if (bucket_fingerprints[bucket_number] == 0) {
|
||||
bucket_first_indices[bucket_number] = row_index;
|
||||
bucket_fingerprints[bucket_number] = fingerprint;
|
||||
break;
|
||||
}
|
||||
if (fingerprint == bucket_fingerprints[bucket_number] &&
|
||||
keys[row_index] == keys[bucket_first_indices[bucket_number]]) {
|
||||
break;
|
||||
}
|
||||
bucket_number = (bucket_number + probe_attempts) & bucket_mask;
|
||||
probe_attempts++;
|
||||
}
|
||||
return bucket_first_indices[bucket_number];
|
||||
};
|
||||
|
||||
AsofJoinDispatcher::dispatch_and_process(table_items, keys, is_nulls.has_value() ? &is_nulls.value() : nullptr,
|
||||
equi_join_bucket_locator);
|
||||
table_items->finalize_asof_index_vector();
|
||||
}
|
||||
|
||||
template <LogicalType LT>
|
||||
void LinearChainedAsofJoinHashMap<LT>::lookup_init(const JoinHashTableItems& table_items,
|
||||
HashTableProbeState* probe_state,
|
||||
const ImmBuffer<CppType>& build_keys,
|
||||
const ImmBuffer<CppType>& probe_keys,
|
||||
const std::optional<ImmBuffer<uint8_t>> is_nulls) {
|
||||
auto process = [&]<bool IsNullable>() {
|
||||
const uint32_t bucket_size_mask = table_items.bucket_size - 1;
|
||||
const uint32_t row_count = probe_state->probe_row_count;
|
||||
|
||||
const auto* firsts = table_items.first.data();
|
||||
const auto* fps = table_items.fps.data();
|
||||
auto* bucket_nums = probe_state->buckets.data();
|
||||
auto* nexts = probe_state->next.data();
|
||||
const uint8_t* is_nulls_data = IsNullable && is_nulls.has_value() ? is_nulls->data() : nullptr;
|
||||
|
||||
auto need_calc_bucket_num = [&](const uint32_t index) {
|
||||
if constexpr (!IsNullable || !std::is_same_v<CppType, Slice>) {
|
||||
// Only check `is_nulls_data[i]` for the nullable slice type. The hash calculation overhead for
|
||||
// fixed-size types is small, and thus we do not check it to allow vectorization of the hash calculation.
|
||||
return true;
|
||||
} else {
|
||||
return is_nulls_data[index] == 0;
|
||||
}
|
||||
};
|
||||
auto is_null = [&](const uint32_t index) {
|
||||
if constexpr (!IsNullable) {
|
||||
return false;
|
||||
} else {
|
||||
return is_nulls_data[index] != 0;
|
||||
}
|
||||
};
|
||||
|
||||
for (uint32_t i = 0; i < row_count; i++) {
|
||||
if (need_calc_bucket_num(i)) {
|
||||
std::tie(bucket_nums[i], nexts[i]) = JoinHashMapHelper::calc_bucket_num_and_fp<CppType>(
|
||||
probe_keys[i], table_items.bucket_size, table_items.log_bucket_size);
|
||||
}
|
||||
}
|
||||
|
||||
for (uint32_t i = 0; i < row_count; i++) {
|
||||
if (is_null(i)) {
|
||||
nexts[i] = 0;
|
||||
continue;
|
||||
}
|
||||
|
||||
const uint8_t fp = nexts[i];
|
||||
uint32_t bucket_num = bucket_nums[i];
|
||||
|
||||
uint32_t probe_times = 1;
|
||||
while (true) {
|
||||
if (fps[bucket_num] == 0) {
|
||||
nexts[i] = 0;
|
||||
break;
|
||||
}
|
||||
|
||||
if (fp == fps[bucket_num] && probe_keys[i] == build_keys[firsts[bucket_num]]) {
|
||||
nexts[i] = firsts[bucket_num];
|
||||
break;
|
||||
}
|
||||
|
||||
bucket_num = (bucket_num + probe_times) & bucket_size_mask;
|
||||
probe_times++;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
if (!is_nulls.has_value()) {
|
||||
process.template operator()<false>();
|
||||
} else {
|
||||
process.template operator()<true>();
|
||||
}
|
||||
}
|
||||
|
||||
// ------------------------------------------------------------------------------------
|
||||
// DirectMappingJoinHashMap
|
||||
// ------------------------------------------------------------------------------------
|
||||
|
|
@ -313,6 +546,9 @@ void DirectMappingJoinHashMap<LT>::build_prepare(RuntimeState* state, JoinHashTa
|
|||
table_items->log_bucket_size = __builtin_ctz(table_items->bucket_size);
|
||||
table_items->first.resize(table_items->bucket_size, 0);
|
||||
table_items->next.resize(table_items->row_count + 1, 0);
|
||||
if (is_asof_join(table_items->join_type)) {
|
||||
table_items->resize_asof_index_vector(table_items->row_count + 1);
|
||||
}
|
||||
}
|
||||
|
||||
template <LogicalType LT>
|
||||
|
|
@ -321,20 +557,36 @@ void DirectMappingJoinHashMap<LT>::construct_hash_table(JoinHashTableItems* tabl
|
|||
static constexpr CppType MIN_VALUE = RunTimeTypeLimits<LT>::min_value();
|
||||
|
||||
const auto num_rows = 1 + table_items->row_count;
|
||||
if (!is_nulls.has_value()) {
|
||||
for (uint32_t i = 1; i < num_rows; i++) {
|
||||
const size_t bucket_num = keys[i] - MIN_VALUE;
|
||||
table_items->next[i] = table_items->first[bucket_num];
|
||||
table_items->first[bucket_num] = i;
|
||||
}
|
||||
if (is_asof_join(table_items->join_type)) {
|
||||
auto equi_join_bucket_locator = [](JoinHashTableItems* table_items, const ImmBuffer<CppType>& keys,
|
||||
uint32_t row_index) -> uint32_t {
|
||||
const size_t bucket_num = keys[row_index] - MIN_VALUE;
|
||||
if (table_items->first[bucket_num] == 0) {
|
||||
table_items->first[bucket_num] = row_index;
|
||||
}
|
||||
return table_items->first[bucket_num];
|
||||
};
|
||||
|
||||
AsofJoinDispatcher::dispatch_and_process(table_items, keys, is_nulls.has_value() ? &is_nulls.value() : nullptr,
|
||||
equi_join_bucket_locator);
|
||||
|
||||
table_items->finalize_asof_index_vector();
|
||||
} else {
|
||||
const auto* is_nulls_data = is_nulls->data();
|
||||
for (uint32_t i = 1; i < num_rows; i++) {
|
||||
if (is_nulls_data[i] == 0) {
|
||||
if (!is_nulls.has_value()) {
|
||||
for (uint32_t i = 1; i < num_rows; i++) {
|
||||
const size_t bucket_num = keys[i] - MIN_VALUE;
|
||||
table_items->next[i] = table_items->first[bucket_num];
|
||||
table_items->first[bucket_num] = i;
|
||||
}
|
||||
} else {
|
||||
const auto* is_nulls_data = is_nulls->data();
|
||||
for (uint32_t i = 1; i < num_rows; i++) {
|
||||
if (is_nulls_data[i] == 0) {
|
||||
const size_t bucket_num = keys[i] - MIN_VALUE;
|
||||
table_items->next[i] = table_items->first[bucket_num];
|
||||
table_items->first[bucket_num] = i;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -375,6 +627,9 @@ void RangeDirectMappingJoinHashMap<LT>::build_prepare(RuntimeState* state, JoinH
|
|||
table_items->bucket_size = value_interval;
|
||||
table_items->first.resize(table_items->bucket_size, 0);
|
||||
table_items->next.resize(table_items->row_count + 1, 0);
|
||||
if (is_asof_join(table_items->join_type)) {
|
||||
table_items->resize_asof_index_vector(table_items->row_count + 1);
|
||||
}
|
||||
}
|
||||
|
||||
template <LogicalType LT>
|
||||
|
|
@ -383,20 +638,36 @@ void RangeDirectMappingJoinHashMap<LT>::construct_hash_table(JoinHashTableItems*
|
|||
const std::optional<ImmBuffer<uint8_t>> is_nulls) {
|
||||
const uint64_t min_value = table_items->min_value;
|
||||
const auto num_rows = 1 + table_items->row_count;
|
||||
if (!is_nulls.has_value()) {
|
||||
for (uint32_t i = 1; i < num_rows; i++) {
|
||||
const size_t bucket_num = keys[i] - min_value;
|
||||
table_items->next[i] = table_items->first[bucket_num];
|
||||
table_items->first[bucket_num] = i;
|
||||
}
|
||||
|
||||
if (is_asof_join(table_items->join_type)) {
|
||||
auto equi_join_bucket_locator = [min_value](JoinHashTableItems* ti, const ImmBuffer<CppType>& k,
|
||||
uint32_t row_index) -> uint32_t {
|
||||
const uint64_t index = static_cast<uint64_t>(k[row_index] - min_value);
|
||||
if (ti->first[index] == 0) {
|
||||
ti->first[index] = row_index;
|
||||
}
|
||||
return ti->first[index];
|
||||
};
|
||||
|
||||
AsofJoinDispatcher::dispatch_and_process(table_items, keys, is_nulls.has_value() ? &is_nulls.value() : nullptr,
|
||||
equi_join_bucket_locator);
|
||||
table_items->finalize_asof_index_vector();
|
||||
} else {
|
||||
const auto* is_nulls_data = is_nulls->data();
|
||||
for (uint32_t i = 1; i < num_rows; i++) {
|
||||
if (is_nulls_data[i] == 0) {
|
||||
if (!is_nulls.has_value()) {
|
||||
for (uint32_t i = 1; i < num_rows; i++) {
|
||||
const size_t bucket_num = keys[i] - min_value;
|
||||
table_items->next[i] = table_items->first[bucket_num];
|
||||
table_items->first[bucket_num] = i;
|
||||
}
|
||||
} else {
|
||||
const auto* is_nulls_data = is_nulls->data();
|
||||
for (uint32_t i = 1; i < num_rows; i++) {
|
||||
if (is_nulls_data[i] == 0) {
|
||||
const size_t bucket_num = keys[i] - min_value;
|
||||
table_items->next[i] = table_items->first[bucket_num];
|
||||
table_items->first[bucket_num] = i;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -517,6 +788,9 @@ void DenseRangeDirectMappingJoinHashMap<LT>::build_prepare(RuntimeState* state,
|
|||
table_items->dense_groups.resize((value_interval + 31) / 32);
|
||||
table_items->first.resize(table_items->row_count + 1, 0);
|
||||
table_items->next.resize(table_items->row_count + 1, 0);
|
||||
if (is_asof_join(table_items->join_type)) {
|
||||
table_items->resize_asof_index_vector(table_items->row_count + 1);
|
||||
}
|
||||
}
|
||||
|
||||
template <LogicalType LT>
|
||||
|
|
@ -524,56 +798,107 @@ void DenseRangeDirectMappingJoinHashMap<LT>::construct_hash_table(JoinHashTableI
|
|||
const ImmBuffer<CppType>& keys,
|
||||
const std::optional<ImmBuffer<uint8_t>> is_nulls) {
|
||||
const uint64_t min_value = table_items->min_value;
|
||||
const auto num_rows = 1 + table_items->row_count;
|
||||
const uint32_t num_rows = table_items->row_count + 1;
|
||||
|
||||
const uint8_t* is_nulls_data = !is_nulls.has_value() ? nullptr : is_nulls->data();
|
||||
auto is_null = [&]<bool Nullable>(const uint32_t index) {
|
||||
if constexpr (Nullable) {
|
||||
return is_nulls_data[index] != 0;
|
||||
} else {
|
||||
return false;
|
||||
const bool is_asof_join_type = is_asof_join(table_items->join_type);
|
||||
|
||||
const uint8_t* equi_join_key_nulls_data = is_nulls ? is_nulls->data() : nullptr;
|
||||
const bool has_equi_join_key_nulls = (equi_join_key_nulls_data != nullptr);
|
||||
|
||||
const uint8_t* asof_temporal_null_data = nullptr;
|
||||
bool has_asof_temporal_nulls = false;
|
||||
|
||||
if (is_asof_join_type) {
|
||||
const ColumnPtr& asof_temporal_col =
|
||||
table_items->build_chunk->get_column_by_slot_id(table_items->asof_join_condition_desc.build_slot_id);
|
||||
const NullColumn* asof_temporal_col_nulls = ColumnHelper::get_null_column(asof_temporal_col);
|
||||
if (asof_temporal_col_nulls != nullptr) {
|
||||
has_asof_temporal_nulls = true;
|
||||
auto* mutable_null_column = const_cast<NullColumn*>(asof_temporal_col_nulls);
|
||||
asof_temporal_null_data = mutable_null_column->get_data().data();
|
||||
}
|
||||
}
|
||||
|
||||
auto get_dense_slot = [min_value](JoinHashTableItems* table_items, const ImmBuffer<CppType>& keys,
|
||||
uint32_t row) ALWAYS_INLINE {
|
||||
const uint32_t bucket_num = keys[row] - min_value;
|
||||
const uint32_t group_index = bucket_num / 32;
|
||||
const uint32_t index_in_group = bucket_num % 32;
|
||||
|
||||
// Keep the low `index_in_group`-th bits of the bitset to count the number of ones from 0 to index_in_group-1.
|
||||
const uint32_t cur_bitset = table_items->dense_groups[group_index].bitset & ((1u << index_in_group) - 1);
|
||||
const uint32_t offset_in_group = BitUtil::count_one_bits(cur_bitset);
|
||||
return table_items->dense_groups[group_index].start_index + offset_in_group;
|
||||
};
|
||||
|
||||
// Initialize `bitset` of each group.
|
||||
auto init_group_bitsets = [&]<bool HasNullableKey, bool HasAsofTemporalNulls>() {
|
||||
for (uint32_t row = 1; row < num_rows; ++row) {
|
||||
if constexpr (HasNullableKey) {
|
||||
if (equi_join_key_nulls_data[row] != 0) continue;
|
||||
}
|
||||
if constexpr (HasAsofTemporalNulls) {
|
||||
if (asof_temporal_null_data[row] != 0) continue;
|
||||
}
|
||||
const uint32_t bucket_num = keys[row] - min_value;
|
||||
const uint32_t group_index = bucket_num / 32;
|
||||
const uint32_t index_in_group = bucket_num % 32;
|
||||
table_items->dense_groups[group_index].bitset |= (1u << index_in_group);
|
||||
}
|
||||
};
|
||||
|
||||
auto process = [&]<bool Nullable>() {
|
||||
// Initialize `bitset` of each group.
|
||||
for (uint32_t i = 1; i < num_rows; i++) {
|
||||
if (!is_null.template operator()<Nullable>(i)) {
|
||||
const uint32_t bucket_num = keys[i] - min_value;
|
||||
const uint32_t group_index = bucket_num / 32;
|
||||
const uint32_t index_in_group = bucket_num % 32;
|
||||
table_items->dense_groups[group_index].bitset |= 1 << index_in_group;
|
||||
}
|
||||
}
|
||||
|
||||
// Calculate `start_index` of each group.
|
||||
for (uint32_t start_index = 0; auto& group : table_items->dense_groups) {
|
||||
group.start_index = start_index;
|
||||
start_index += BitUtil::count_one_bits(group.bitset);
|
||||
}
|
||||
|
||||
auto build_hash_chains = [&]<bool HasNullableKey>() {
|
||||
// Initialize `first` and `next` arrays by `bitset` and `start_index` of each group.
|
||||
for (size_t i = 1; i < num_rows; i++) {
|
||||
if (!is_null.template operator()<Nullable>(i)) {
|
||||
const uint32_t bucket_num = keys[i] - min_value;
|
||||
const uint32_t group_index = bucket_num / 32;
|
||||
const uint32_t index_in_group = bucket_num % 32;
|
||||
|
||||
// Keep the low `index_in_group`-th bits of the bitset to count the number of ones from 0 to index_in_group-1.
|
||||
const uint32_t cur_bitset = table_items->dense_groups[group_index].bitset & ((1 << index_in_group) - 1);
|
||||
const uint32_t offset_in_group = BitUtil::count_one_bits(cur_bitset);
|
||||
const uint32_t index = table_items->dense_groups[group_index].start_index + offset_in_group;
|
||||
|
||||
table_items->next[i] = table_items->first[index];
|
||||
table_items->first[index] = i;
|
||||
for (uint32_t row = 1; row < num_rows; ++row) {
|
||||
if constexpr (HasNullableKey) {
|
||||
if (equi_join_key_nulls_data[row] != 0) continue;
|
||||
}
|
||||
const uint32_t index = get_dense_slot(table_items, keys, row);
|
||||
|
||||
table_items->next[row] = table_items->first[index];
|
||||
table_items->first[index] = row;
|
||||
}
|
||||
};
|
||||
|
||||
if (!is_nulls.has_value()) {
|
||||
process.template operator()<false>();
|
||||
auto dispatch_bitset_init = [&]<bool HasNullableKey>() {
|
||||
if (has_asof_temporal_nulls) {
|
||||
init_group_bitsets.template operator()<HasNullableKey, true>();
|
||||
} else {
|
||||
init_group_bitsets.template operator()<HasNullableKey, false>();
|
||||
}
|
||||
};
|
||||
|
||||
if (has_equi_join_key_nulls) {
|
||||
dispatch_bitset_init.template operator()<true>();
|
||||
} else {
|
||||
process.template operator()<true>();
|
||||
dispatch_bitset_init.template operator()<false>();
|
||||
}
|
||||
|
||||
// Calculate `start_index` of each group.
|
||||
for (uint32_t start_index = 0; auto& group : table_items->dense_groups) {
|
||||
group.start_index = start_index;
|
||||
start_index += BitUtil::count_one_bits(group.bitset);
|
||||
}
|
||||
|
||||
if (is_asof_join_type) {
|
||||
auto equi_join_bucket_locator = [get_dense_slot](JoinHashTableItems* ti, const ImmBuffer<CppType>& k,
|
||||
uint32_t row) -> uint32_t {
|
||||
const uint32_t index = get_dense_slot(ti, k, row);
|
||||
if (ti->first[index] == 0) {
|
||||
ti->first[index] = row;
|
||||
}
|
||||
return ti->first[index];
|
||||
};
|
||||
|
||||
AsofJoinDispatcher::dispatch_and_process(table_items, keys, is_nulls.has_value() ? &is_nulls.value() : nullptr,
|
||||
equi_join_bucket_locator);
|
||||
table_items->finalize_asof_index_vector();
|
||||
} else {
|
||||
if (!has_equi_join_key_nulls) {
|
||||
build_hash_chains.template operator()<false>();
|
||||
} else {
|
||||
build_hash_chains.template operator()<true>();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -14,6 +14,9 @@
|
|||
|
||||
#include "join_hash_table_descriptor.h"
|
||||
|
||||
#include "exec/sorting/sort_helper.h"
|
||||
#include "util/orlp/pdqsort.h"
|
||||
|
||||
namespace starrocks {
|
||||
|
||||
// if the same hash values are clustered, after the first probe, all related hash buckets are cached, without too many
|
||||
|
|
@ -62,4 +65,82 @@ void HashTableProbeState::consider_probe_time_locality() {
|
|||
++probe_chunks;
|
||||
}
|
||||
|
||||
template <typename CppType, TExprOpcode::type OpCode>
|
||||
void AsofIndex<CppType, OpCode>::sort() {
|
||||
auto comparator = [](const Entry& lhs, const Entry& rhs) {
|
||||
if constexpr (is_descending) {
|
||||
return SorterComparator<CppType>::compare(lhs.asof_value, rhs.asof_value) > 0;
|
||||
} else {
|
||||
return SorterComparator<CppType>::compare(lhs.asof_value, rhs.asof_value) < 0;
|
||||
}
|
||||
};
|
||||
|
||||
::pdqsort(_entries.begin(), _entries.end(), comparator);
|
||||
}
|
||||
|
||||
template <typename CppType, TExprOpcode::type OpCode>
|
||||
uint32_t AsofIndex<CppType, OpCode>::find_asof_match(CppType probe_value) const {
|
||||
if (_entries.empty()) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
size_t size = _entries.size();
|
||||
size_t low = 0;
|
||||
|
||||
#pragma GCC unroll 3
|
||||
while (size >= 8) {
|
||||
_bound_search_iteration(probe_value, low, size);
|
||||
}
|
||||
|
||||
while (size > 0) {
|
||||
_bound_search_iteration(probe_value, low, size);
|
||||
}
|
||||
|
||||
uint32_t result = (low < _entries.size()) ? _entries[low].row_index : 0;
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
template <typename CppType, TExprOpcode::type OpCode>
|
||||
void AsofIndex<CppType, OpCode>::_bound_search_iteration(CppType probe_value, size_t& low, size_t& size) const {
|
||||
size_t half = size / 2;
|
||||
size_t other_half = size - half;
|
||||
size_t probe_pos = low + half;
|
||||
size_t other_low = low + other_half;
|
||||
const CppType& entry_value = _entries[probe_pos].asof_value;
|
||||
|
||||
size = half;
|
||||
|
||||
bool condition_result;
|
||||
if constexpr (is_descending) {
|
||||
if constexpr (is_strict) {
|
||||
condition_result = (SorterComparator<CppType>::compare(probe_value, entry_value) <= 0);
|
||||
low = condition_result ? other_low : low;
|
||||
} else {
|
||||
condition_result = (SorterComparator<CppType>::compare(probe_value, entry_value) < 0);
|
||||
low = condition_result ? other_low : low;
|
||||
}
|
||||
} else {
|
||||
if constexpr (is_strict) {
|
||||
condition_result = (SorterComparator<CppType>::compare(probe_value, entry_value) >= 0);
|
||||
low = condition_result ? other_low : low;
|
||||
} else {
|
||||
condition_result = (SorterComparator<CppType>::compare(probe_value, entry_value) > 0);
|
||||
low = condition_result ? other_low : low;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#define INSTANTIATE_ASOF_INDEX(CppType) \
|
||||
template class AsofIndex<CppType, TExprOpcode::LT>; \
|
||||
template class AsofIndex<CppType, TExprOpcode::LE>; \
|
||||
template class AsofIndex<CppType, TExprOpcode::GT>; \
|
||||
template class AsofIndex<CppType, TExprOpcode::GE>;
|
||||
|
||||
INSTANTIATE_ASOF_INDEX(int64_t)
|
||||
INSTANTIATE_ASOF_INDEX(DateValue)
|
||||
INSTANTIATE_ASOF_INDEX(TimestampValue)
|
||||
|
||||
#undef INSTANTIATE_ASOF_INDEX
|
||||
|
||||
} // namespace starrocks
|
||||
|
|
@ -20,14 +20,16 @@
|
|||
|
||||
#include <coroutine>
|
||||
#include <cstdint>
|
||||
#include <optional>
|
||||
#include <set>
|
||||
#include <variant>
|
||||
|
||||
#include "column/chunk.h"
|
||||
#include "column/column_hash.h"
|
||||
#include "column/column_helper.h"
|
||||
#include "column/vectorized_fwd.h"
|
||||
#include "common/statusor.h"
|
||||
#include "exec/sorting/sort_helper.h"
|
||||
#include "simd/simd.h"
|
||||
#include "util/phmap/phmap.h"
|
||||
#include "util/runtime_profile.h"
|
||||
|
||||
namespace starrocks {
|
||||
|
|
@ -42,12 +44,67 @@ struct JoinKeyDesc {
|
|||
ColumnRef* col_ref = nullptr;
|
||||
};
|
||||
|
||||
struct AsofJoinConditionDesc {
|
||||
SlotId probe_slot_id;
|
||||
LogicalType probe_logical_type;
|
||||
SlotId build_slot_id;
|
||||
LogicalType build_logical_type;
|
||||
TExprOpcode::type condition_op = TExprOpcode::INVALID_OPCODE;
|
||||
};
|
||||
|
||||
struct HashTableSlotDescriptor {
|
||||
SlotDescriptor* slot;
|
||||
bool need_output;
|
||||
bool need_lazy_materialize = false;
|
||||
};
|
||||
|
||||
template <typename CppType, TExprOpcode::type OpCode>
|
||||
class AsofIndex {
|
||||
public:
|
||||
struct Entry {
|
||||
CppType asof_value;
|
||||
uint32_t row_index;
|
||||
|
||||
Entry() = default;
|
||||
Entry(CppType value, uint32_t index) : asof_value(value), row_index(index) {}
|
||||
};
|
||||
|
||||
private:
|
||||
using Entries = std::vector<Entry>;
|
||||
|
||||
static constexpr bool is_descending = (OpCode == TExprOpcode::GE || OpCode == TExprOpcode::GT);
|
||||
static constexpr bool is_strict = (OpCode == TExprOpcode::LT || OpCode == TExprOpcode::GT);
|
||||
|
||||
Entries _entries;
|
||||
|
||||
public:
|
||||
void add_row(CppType asof_value, uint32_t row_index) { _entries.emplace_back(asof_value, row_index); }
|
||||
|
||||
void sort();
|
||||
|
||||
uint32_t find_asof_match(CppType probe_value) const;
|
||||
|
||||
size_t size() const { return _entries.size(); }
|
||||
bool empty() const { return _entries.empty(); }
|
||||
void clear() { _entries.clear(); }
|
||||
|
||||
private:
|
||||
void _bound_search_iteration(CppType probe_value, size_t& low, size_t& size) const;
|
||||
};
|
||||
|
||||
#define ASOF_INDEX_BUFFER_TYPES(T) \
|
||||
Buffer<std::unique_ptr<AsofIndex<T, TExprOpcode::LT>>>, Buffer<std::unique_ptr<AsofIndex<T, TExprOpcode::LE>>>, \
|
||||
Buffer<std::unique_ptr<AsofIndex<T, TExprOpcode::GT>>>, \
|
||||
Buffer<std::unique_ptr<AsofIndex<T, TExprOpcode::GE>>>
|
||||
|
||||
using AsofIndexBufferVariant =
|
||||
std::variant<ASOF_INDEX_BUFFER_TYPES(int64_t), // 0-3: Buffer<AsofIndex<int64_t, OP>*>
|
||||
ASOF_INDEX_BUFFER_TYPES(DateValue), // 4-7: Buffer<AsofIndex<DateValue, OP>*>
|
||||
ASOF_INDEX_BUFFER_TYPES(TimestampValue) // 8-11: Buffer<AsofIndex<TimestampValue, OP>*>
|
||||
>;
|
||||
|
||||
#undef ASOF_INDEX_BUFFER_TYPES
|
||||
|
||||
struct JoinHashTableItems {
|
||||
//TODO: memory continues problem?
|
||||
ChunkPtr build_chunk = nullptr;
|
||||
|
|
@ -63,6 +120,7 @@ struct JoinHashTableItems {
|
|||
// about the bucket-chained hash table of this kind.
|
||||
Buffer<uint32_t> first;
|
||||
Buffer<uint32_t> next;
|
||||
Buffer<uint8_t> fps;
|
||||
|
||||
Buffer<uint8_t> key_bitset;
|
||||
struct DenseGroup {
|
||||
|
|
@ -98,9 +156,27 @@ struct JoinHashTableItems {
|
|||
bool enable_late_materialization = false;
|
||||
bool is_collision_free_and_unique = false;
|
||||
|
||||
AsofJoinConditionDesc asof_join_condition_desc;
|
||||
|
||||
AsofIndexBufferVariant asof_index_vector;
|
||||
|
||||
float get_keys_per_bucket() const { return keys_per_bucket; }
|
||||
bool ht_cache_miss_serious() const { return cache_miss_serious; }
|
||||
|
||||
void resize_asof_index_vector(size_t size) {
|
||||
std::visit([size](auto& buffer) { buffer.resize(size); }, asof_index_vector);
|
||||
}
|
||||
|
||||
void finalize_asof_index_vector() {
|
||||
std::visit(
|
||||
[](auto& buffer) {
|
||||
for (auto& ptr : buffer) {
|
||||
if (ptr) ptr->sort();
|
||||
}
|
||||
},
|
||||
asof_index_vector);
|
||||
}
|
||||
|
||||
void calculate_ht_info(size_t key_bytes) {
|
||||
if (used_buckets != 0) {
|
||||
// to avoid redo
|
||||
|
|
@ -175,6 +251,7 @@ struct HashTableProbeState {
|
|||
RuntimeProfile::Counter* output_probe_column_timer = nullptr;
|
||||
RuntimeProfile::Counter* output_build_column_timer = nullptr;
|
||||
RuntimeProfile::Counter* probe_counter = nullptr;
|
||||
ColumnPtr asof_temporal_condition_column = nullptr;
|
||||
|
||||
HashTableProbeState()
|
||||
: build_index_column(UInt32Column::create()),
|
||||
|
|
@ -270,9 +347,215 @@ struct HashTableParam {
|
|||
std::set<SlotId> predicate_slots;
|
||||
std::vector<JoinKeyDesc> join_keys;
|
||||
|
||||
AsofJoinConditionDesc asof_join_condition_desc;
|
||||
|
||||
RuntimeProfile::Counter* search_ht_timer = nullptr;
|
||||
RuntimeProfile::Counter* output_build_column_timer = nullptr;
|
||||
RuntimeProfile::Counter* output_probe_column_timer = nullptr;
|
||||
RuntimeProfile::Counter* probe_counter = nullptr;
|
||||
};
|
||||
|
||||
inline bool is_asof_join(TJoinOp::type join_type) {
|
||||
return join_type == TJoinOp::ASOF_INNER_JOIN || join_type == TJoinOp::ASOF_LEFT_OUTER_JOIN;
|
||||
}
|
||||
|
||||
constexpr size_t get_asof_variant_index(LogicalType logical_type, TExprOpcode::type opcode) {
|
||||
size_t base = (logical_type == TYPE_BIGINT) ? 0 : (logical_type == TYPE_DATE) ? 4 : 8;
|
||||
size_t offset =
|
||||
(opcode == TExprOpcode::LT) ? 0 : (opcode == TExprOpcode::LE) ? 1 : (opcode == TExprOpcode::GT) ? 2 : 3;
|
||||
return base + offset;
|
||||
}
|
||||
|
||||
#define CREATE_ASOF_VECTOR_CASE(TYPE, BASE_INDEX) \
|
||||
case BASE_INDEX + 0: \
|
||||
return Buffer<std::unique_ptr<AsofIndex<TYPE, TExprOpcode::LT>>>{}; \
|
||||
case BASE_INDEX + 1: \
|
||||
return Buffer<std::unique_ptr<AsofIndex<TYPE, TExprOpcode::LE>>>{}; \
|
||||
case BASE_INDEX + 2: \
|
||||
return Buffer<std::unique_ptr<AsofIndex<TYPE, TExprOpcode::GT>>>{}; \
|
||||
case BASE_INDEX + 3: \
|
||||
return Buffer<std::unique_ptr<AsofIndex<TYPE, TExprOpcode::GE>>>{};
|
||||
|
||||
inline AsofIndexBufferVariant create_asof_index_vector(size_t variant_index) {
|
||||
switch (variant_index) {
|
||||
CREATE_ASOF_VECTOR_CASE(int64_t, 0)
|
||||
CREATE_ASOF_VECTOR_CASE(DateValue, 4)
|
||||
CREATE_ASOF_VECTOR_CASE(TimestampValue, 8)
|
||||
default:
|
||||
__builtin_unreachable();
|
||||
}
|
||||
}
|
||||
#undef CREATE_ASOF_BUFFER_CASE
|
||||
|
||||
template <size_t VariantIndex>
|
||||
constexpr auto& get_asof_index_vector_static(JoinHashTableItems* table_items) {
|
||||
static_assert(VariantIndex < 12, "Invalid variant index");
|
||||
return std::get<VariantIndex>(table_items->asof_index_vector);
|
||||
}
|
||||
|
||||
template <size_t VariantIndex>
|
||||
#define CREATE_ASOF_INDEX_CASE(TYPE, BASE_INDEX) \
|
||||
if constexpr (VariantIndex == BASE_INDEX + 0) { \
|
||||
vector[asof_lookup_index] = std::make_unique<AsofIndex<TYPE, TExprOpcode::LT>>(); \
|
||||
} else if constexpr (VariantIndex == BASE_INDEX + 1) { \
|
||||
vector[asof_lookup_index] = std::make_unique<AsofIndex<TYPE, TExprOpcode::LE>>(); \
|
||||
} else if constexpr (VariantIndex == BASE_INDEX + 2) { \
|
||||
vector[asof_lookup_index] = std::make_unique<AsofIndex<TYPE, TExprOpcode::GT>>(); \
|
||||
} else if constexpr (VariantIndex == BASE_INDEX + 3) { \
|
||||
vector[asof_lookup_index] = std::make_unique<AsofIndex<TYPE, TExprOpcode::GE>>(); \
|
||||
} else
|
||||
|
||||
void create_asof_index(JoinHashTableItems* table_items, uint32_t asof_lookup_index) {
|
||||
auto& vector = get_asof_index_vector_static<VariantIndex>(table_items);
|
||||
|
||||
CREATE_ASOF_INDEX_CASE(int64_t, 0)
|
||||
CREATE_ASOF_INDEX_CASE(DateValue, 4)
|
||||
CREATE_ASOF_INDEX_CASE(TimestampValue, 8) { static_assert(VariantIndex < 12, "Invalid variant index"); }
|
||||
}
|
||||
|
||||
#undef CREATE_ASOF_INDEX_CASE
|
||||
|
||||
template <LogicalType LT, TExprOpcode::type OP>
|
||||
class AsofJoinTemporalRowProcessor {
|
||||
public:
|
||||
template <typename EquiJoinIndexLocator>
|
||||
static void process_rows(JoinHashTableItems* table_items, const auto& keys,
|
||||
const ImmBuffer<uint8_t>* equi_join_key_nulls,
|
||||
EquiJoinIndexLocator&& equi_join_index_locator) {
|
||||
using AsofCppType = RunTimeCppType<LT>;
|
||||
static constexpr size_t variant_index = get_asof_variant_index(LT, OP);
|
||||
|
||||
const ColumnPtr& asof_temporal_col =
|
||||
table_items->build_chunk->get_column_by_slot_id(table_items->asof_join_condition_desc.build_slot_id);
|
||||
const auto* data_col = ColumnHelper::get_data_column_by_type<LT>(asof_temporal_col.get());
|
||||
const NullColumn* asof_temporal_col_nulls_column = ColumnHelper::get_null_column(asof_temporal_col);
|
||||
const Buffer<uint8_t>* asof_temporal_col_nulls =
|
||||
asof_temporal_col_nulls_column ? &const_cast<NullColumn*>(asof_temporal_col_nulls_column)->get_data()
|
||||
: nullptr;
|
||||
const AsofCppType* __restrict asof_temporal_data = data_col->immutable_data().data();
|
||||
|
||||
const bool has_equi_join_key_nulls = (equi_join_key_nulls != nullptr);
|
||||
const bool has_asof_temporal_nulls = (asof_temporal_col_nulls != nullptr);
|
||||
|
||||
auto process_rows_impl = [&]<bool HasEquiJoinKeyNulls, bool HasAsofTemporalNulls>() {
|
||||
auto& asof_index_vector = get_asof_index_vector_static<variant_index>(table_items);
|
||||
const uint32_t num_rows = table_items->row_count + 1;
|
||||
|
||||
const uint8_t* __restrict asof_temporal_null_data =
|
||||
HasAsofTemporalNulls ? asof_temporal_col_nulls->data() : nullptr;
|
||||
const uint8_t* __restrict equi_key_null_data = HasEquiJoinKeyNulls ? equi_join_key_nulls->data() : nullptr;
|
||||
|
||||
auto is_null_row = [&](uint32_t i) {
|
||||
if constexpr (HasEquiJoinKeyNulls) {
|
||||
if (equi_key_null_data[i] != 0) return true;
|
||||
}
|
||||
if constexpr (HasAsofTemporalNulls) {
|
||||
if (asof_temporal_null_data[i] != 0) return true;
|
||||
}
|
||||
return false;
|
||||
};
|
||||
|
||||
for (uint32_t i = 1; i < num_rows; ++i) {
|
||||
if (is_null_row(i)) continue;
|
||||
|
||||
uint32_t equi_join_bucket_index = equi_join_index_locator(table_items, keys, i);
|
||||
|
||||
if (!asof_index_vector[equi_join_bucket_index]) {
|
||||
create_asof_index<variant_index>(table_items, equi_join_bucket_index);
|
||||
}
|
||||
asof_index_vector[equi_join_bucket_index]->add_row(asof_temporal_data[i], i);
|
||||
}
|
||||
};
|
||||
|
||||
if (!has_equi_join_key_nulls && !has_asof_temporal_nulls) {
|
||||
process_rows_impl.template operator()<false, false>();
|
||||
} else if (has_equi_join_key_nulls && !has_asof_temporal_nulls) {
|
||||
process_rows_impl.template operator()<true, false>();
|
||||
} else if (!has_equi_join_key_nulls) {
|
||||
process_rows_impl.template operator()<false, true>();
|
||||
} else {
|
||||
process_rows_impl.template operator()<true, true>();
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
struct AsofJoinTemporalTypeOpcodeDispatcher {
|
||||
template <typename Func>
|
||||
static void dispatch(LogicalType asof_type, TExprOpcode::type opcode, Func&& func) {
|
||||
switch (asof_type) {
|
||||
case TYPE_BIGINT:
|
||||
dispatch_impl<TYPE_BIGINT>(opcode, std::forward<Func>(func));
|
||||
break;
|
||||
case TYPE_DATE:
|
||||
dispatch_impl<TYPE_DATE>(opcode, std::forward<Func>(func));
|
||||
break;
|
||||
case TYPE_DATETIME:
|
||||
dispatch_impl<TYPE_DATETIME>(opcode, std::forward<Func>(func));
|
||||
break;
|
||||
default:
|
||||
LOG(ERROR) << "ASOF JOIN: Unsupported type: " << asof_type;
|
||||
CHECK(false) << "ASOF JOIN: Unsupported type";
|
||||
__builtin_unreachable();
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
template <LogicalType ASOF_LT, typename Func>
|
||||
static void dispatch_impl(TExprOpcode::type opcode, Func&& func) {
|
||||
switch (opcode) {
|
||||
case TExprOpcode::LT:
|
||||
func(std::integral_constant<LogicalType, ASOF_LT>{},
|
||||
std::integral_constant<TExprOpcode::type, TExprOpcode::LT>{});
|
||||
break;
|
||||
case TExprOpcode::LE:
|
||||
func(std::integral_constant<LogicalType, ASOF_LT>{},
|
||||
std::integral_constant<TExprOpcode::type, TExprOpcode::LE>{});
|
||||
break;
|
||||
case TExprOpcode::GT:
|
||||
func(std::integral_constant<LogicalType, ASOF_LT>{},
|
||||
std::integral_constant<TExprOpcode::type, TExprOpcode::GT>{});
|
||||
break;
|
||||
case TExprOpcode::GE:
|
||||
func(std::integral_constant<LogicalType, ASOF_LT>{},
|
||||
std::integral_constant<TExprOpcode::type, TExprOpcode::GE>{});
|
||||
break;
|
||||
default:
|
||||
__builtin_unreachable();
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
class AsofJoinDispatcher {
|
||||
public:
|
||||
template <typename EquiJoinIndexLocator>
|
||||
static void dispatch_and_process(JoinHashTableItems* table_items, const auto& keys,
|
||||
const ImmBuffer<uint8_t>* equi_join_key_nulls,
|
||||
EquiJoinIndexLocator&& equi_join_index_locator) {
|
||||
LogicalType asof_type = table_items->asof_join_condition_desc.build_logical_type;
|
||||
TExprOpcode::type opcode = table_items->asof_join_condition_desc.condition_op;
|
||||
|
||||
auto body = [&](auto tag_lt, auto tag_op) {
|
||||
static constexpr LogicalType Lt = decltype(tag_lt)::value;
|
||||
static constexpr TExprOpcode::type Op = decltype(tag_op)::value;
|
||||
AsofJoinTemporalRowProcessor<Lt, Op>::process_rows(
|
||||
table_items, keys, equi_join_key_nulls,
|
||||
std::forward<EquiJoinIndexLocator>(equi_join_index_locator));
|
||||
};
|
||||
|
||||
AsofJoinTemporalTypeOpcodeDispatcher::dispatch(asof_type, opcode, body);
|
||||
}
|
||||
};
|
||||
|
||||
class AsofJoinProbeDispatcher {
|
||||
public:
|
||||
template <typename Func>
|
||||
static void dispatch(LogicalType asof_type, TExprOpcode::type opcode, Func&& body) {
|
||||
AsofJoinTemporalTypeOpcodeDispatcher::dispatch(asof_type, opcode, [&](auto tag_lt, auto tag_op) {
|
||||
static constexpr LogicalType Lt = decltype(tag_lt)::value;
|
||||
static constexpr TExprOpcode::type Op = decltype(tag_op)::value;
|
||||
body.template operator()<Lt, Op>();
|
||||
});
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace starrocks
|
||||
|
|
|
|||
|
|
@ -45,7 +45,8 @@ namespace starrocks {
|
|||
M(RANGE_DIRECT_MAPPING_SET) \
|
||||
M(DENSE_RANGE_DIRECT_MAPPING) \
|
||||
M(LINEAR_CHAINED) \
|
||||
M(LINEAR_CHAINED_SET)
|
||||
M(LINEAR_CHAINED_SET) \
|
||||
M(LINEAR_CHAINED_ASOF)
|
||||
|
||||
#define APPLY_JOIN_KEY_CONSTRUCTOR_UNARY_TYPE(M) \
|
||||
M(ONE_KEY_BOOLEAN) \
|
||||
|
|
@ -117,7 +118,20 @@ namespace starrocks {
|
|||
M(LINEAR_CHAINED_SET_DECIMAL32) \
|
||||
M(LINEAR_CHAINED_SET_DECIMAL64) \
|
||||
M(LINEAR_CHAINED_SET_DECIMAL128) \
|
||||
M(LINEAR_CHAINED_SET_VARCHAR)
|
||||
M(LINEAR_CHAINED_SET_VARCHAR) \
|
||||
\
|
||||
M(LINEAR_CHAINED_ASOF_INT) \
|
||||
M(LINEAR_CHAINED_ASOF_BIGINT) \
|
||||
M(LINEAR_CHAINED_ASOF_LARGEINT) \
|
||||
M(LINEAR_CHAINED_ASOF_FLOAT) \
|
||||
M(LINEAR_CHAINED_ASOF_DOUBLE) \
|
||||
M(LINEAR_CHAINED_ASOF_DATE) \
|
||||
M(LINEAR_CHAINED_ASOF_DATETIME) \
|
||||
M(LINEAR_CHAINED_ASOF_DECIMALV2) \
|
||||
M(LINEAR_CHAINED_ASOF_DECIMAL32) \
|
||||
M(LINEAR_CHAINED_ASOF_DECIMAL64) \
|
||||
M(LINEAR_CHAINED_ASOF_DECIMAL128) \
|
||||
M(LINEAR_CHAINED_ASOF_VARCHAR)
|
||||
|
||||
enum class JoinKeyConstructorType {
|
||||
#define NAME_TO_ENUM(NAME) NAME,
|
||||
|
|
@ -294,6 +308,27 @@ REGISTER_JOIN_MAP_METHOD_TYPE(LINEAR_CHAINED_SET, TYPE_DECIMAL64, LinearChainedJ
|
|||
REGISTER_JOIN_MAP_METHOD_TYPE(LINEAR_CHAINED_SET, TYPE_DECIMAL128, LinearChainedJoinHashSet,
|
||||
LINEAR_CHAINED_SET_DECIMAL128);
|
||||
REGISTER_JOIN_MAP_METHOD_TYPE(LINEAR_CHAINED_SET, TYPE_VARCHAR, LinearChainedJoinHashSet, LINEAR_CHAINED_SET_VARCHAR);
|
||||
REGISTER_JOIN_MAP_METHOD_TYPE(LINEAR_CHAINED_ASOF, TYPE_INT, LinearChainedAsofJoinHashMap, LINEAR_CHAINED_ASOF_INT);
|
||||
REGISTER_JOIN_MAP_METHOD_TYPE(LINEAR_CHAINED_ASOF, TYPE_BIGINT, LinearChainedAsofJoinHashMap,
|
||||
LINEAR_CHAINED_ASOF_BIGINT);
|
||||
REGISTER_JOIN_MAP_METHOD_TYPE(LINEAR_CHAINED_ASOF, TYPE_LARGEINT, LinearChainedAsofJoinHashMap,
|
||||
LINEAR_CHAINED_ASOF_LARGEINT);
|
||||
REGISTER_JOIN_MAP_METHOD_TYPE(LINEAR_CHAINED_ASOF, TYPE_FLOAT, LinearChainedAsofJoinHashMap, LINEAR_CHAINED_ASOF_FLOAT);
|
||||
REGISTER_JOIN_MAP_METHOD_TYPE(LINEAR_CHAINED_ASOF, TYPE_DOUBLE, LinearChainedAsofJoinHashMap,
|
||||
LINEAR_CHAINED_ASOF_DOUBLE);
|
||||
REGISTER_JOIN_MAP_METHOD_TYPE(LINEAR_CHAINED_ASOF, TYPE_DATE, LinearChainedAsofJoinHashMap, LINEAR_CHAINED_ASOF_DATE);
|
||||
REGISTER_JOIN_MAP_METHOD_TYPE(LINEAR_CHAINED_ASOF, TYPE_DATETIME, LinearChainedAsofJoinHashMap,
|
||||
LINEAR_CHAINED_ASOF_DATETIME);
|
||||
REGISTER_JOIN_MAP_METHOD_TYPE(LINEAR_CHAINED_ASOF, TYPE_DECIMALV2, LinearChainedAsofJoinHashMap,
|
||||
LINEAR_CHAINED_ASOF_DECIMALV2);
|
||||
REGISTER_JOIN_MAP_METHOD_TYPE(LINEAR_CHAINED_ASOF, TYPE_DECIMAL32, LinearChainedAsofJoinHashMap,
|
||||
LINEAR_CHAINED_ASOF_DECIMAL32);
|
||||
REGISTER_JOIN_MAP_METHOD_TYPE(LINEAR_CHAINED_ASOF, TYPE_DECIMAL64, LinearChainedAsofJoinHashMap,
|
||||
LINEAR_CHAINED_ASOF_DECIMAL64);
|
||||
REGISTER_JOIN_MAP_METHOD_TYPE(LINEAR_CHAINED_ASOF, TYPE_DECIMAL128, LinearChainedAsofJoinHashMap,
|
||||
LINEAR_CHAINED_ASOF_DECIMAL128);
|
||||
REGISTER_JOIN_MAP_METHOD_TYPE(LINEAR_CHAINED_ASOF, TYPE_VARCHAR, LinearChainedAsofJoinHashMap,
|
||||
LINEAR_CHAINED_ASOF_VARCHAR);
|
||||
|
||||
#undef REGISTER_JOIN_MAP_TYPE
|
||||
|
||||
|
|
|
|||
|
|
@ -27,6 +27,14 @@ Status HashJoinerFactory::prepare(RuntimeState* state) {
|
|||
RETURN_IF_ERROR(Expr::open(_param._common_expr_ctxs, state));
|
||||
RETURN_IF_ERROR(Expr::open(_param._other_join_conjunct_ctxs, state));
|
||||
RETURN_IF_ERROR(Expr::open(_param._conjunct_ctxs, state));
|
||||
if (_param._asof_join_condition_build_expr_ctx != nullptr) {
|
||||
RETURN_IF_ERROR(_param._asof_join_condition_build_expr_ctx->prepare(state));
|
||||
RETURN_IF_ERROR(_param._asof_join_condition_build_expr_ctx->open(state));
|
||||
}
|
||||
if (_param._asof_join_condition_probe_expr_ctx != nullptr) {
|
||||
RETURN_IF_ERROR(_param._asof_join_condition_probe_expr_ctx->prepare(state));
|
||||
RETURN_IF_ERROR(_param._asof_join_condition_probe_expr_ctx->open(state));
|
||||
}
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
|
|
@ -36,6 +44,12 @@ void HashJoinerFactory::close(RuntimeState* state) {
|
|||
Expr::close(_param._other_join_conjunct_ctxs, state);
|
||||
Expr::close(_param._probe_expr_ctxs, state);
|
||||
Expr::close(_param._build_expr_ctxs, state);
|
||||
if (_param._asof_join_condition_build_expr_ctx != nullptr) {
|
||||
_param._asof_join_condition_build_expr_ctx->close(state);
|
||||
}
|
||||
if (_param._asof_join_condition_probe_expr_ctx != nullptr) {
|
||||
_param._asof_join_condition_probe_expr_ctx->close(state);
|
||||
}
|
||||
}
|
||||
|
||||
HashJoinerPtr HashJoinerFactory::create_builder(int32_t builder_dop, int32_t builder_driver_seq) {
|
||||
|
|
|
|||
|
|
@ -49,6 +49,62 @@ INSERT INTO user_status VALUES
|
|||
(102, '2024-01-02 12:00:00', 'PREMIUM', 950);
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT o.order_id, o.user_id, o.order_time, us.status_time, us.status, us.credit_score
|
||||
FROM orders o
|
||||
ASOF INNER JOIN user_status us ON o.user_id = us.user_id AND o.order_time >= us.status_time
|
||||
ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 101 2024-01-01 10:00:00 2024-01-01 08:00:00 NORMAL 750
|
||||
2 101 2024-01-01 15:30:00 2024-01-01 14:00:00 VIP 850
|
||||
3 102 2024-01-01 11:00:00 2024-01-01 09:00:00 NORMAL 700
|
||||
4 102 2024-01-01 16:00:00 2024-01-01 13:00:00 VIP 800
|
||||
5 101 2024-01-02 09:00:00 2024-01-02 08:00:00 PREMIUM 900
|
||||
6 102 2024-01-02 14:00:00 2024-01-02 12:00:00 PREMIUM 950
|
||||
-- !result
|
||||
SELECT o.order_id, o.user_id, o.order_time, us.status_time, us.status, us.credit_score
|
||||
FROM orders o
|
||||
ASOF LEFT JOIN user_status us ON o.user_id = us.user_id AND o.order_time >= us.status_time
|
||||
ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 101 2024-01-01 10:00:00 2024-01-01 08:00:00 NORMAL 750
|
||||
2 101 2024-01-01 15:30:00 2024-01-01 14:00:00 VIP 850
|
||||
3 102 2024-01-01 11:00:00 2024-01-01 09:00:00 NORMAL 700
|
||||
4 102 2024-01-01 16:00:00 2024-01-01 13:00:00 VIP 800
|
||||
5 101 2024-01-02 09:00:00 2024-01-02 08:00:00 PREMIUM 900
|
||||
6 102 2024-01-02 14:00:00 2024-01-02 12:00:00 PREMIUM 950
|
||||
-- !result
|
||||
SELECT o.order_id, o.user_id, o.order_time, us.status_time, us.status, us.credit_score
|
||||
FROM orders o
|
||||
ASOF INNER JOIN user_status us ON o.user_id = us.user_id AND o.order_time > us.status_time
|
||||
ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 101 2024-01-01 10:00:00 2024-01-01 08:00:00 NORMAL 750
|
||||
2 101 2024-01-01 15:30:00 2024-01-01 14:00:00 VIP 850
|
||||
3 102 2024-01-01 11:00:00 2024-01-01 09:00:00 NORMAL 700
|
||||
4 102 2024-01-01 16:00:00 2024-01-01 13:00:00 VIP 800
|
||||
5 101 2024-01-02 09:00:00 2024-01-02 08:00:00 PREMIUM 900
|
||||
6 102 2024-01-02 14:00:00 2024-01-02 12:00:00 PREMIUM 950
|
||||
-- !result
|
||||
SELECT o.order_id, o.user_id, o.order_time, us.status_time, us.status, us.credit_score
|
||||
FROM orders o
|
||||
ASOF INNER JOIN user_status us ON o.user_id = us.user_id AND o.order_time >= us.status_time
|
||||
WHERE o.amount > 150
|
||||
ORDER BY o.order_id;
|
||||
-- result:
|
||||
2 101 2024-01-01 15:30:00 2024-01-01 14:00:00 VIP 850
|
||||
4 102 2024-01-01 16:00:00 2024-01-01 13:00:00 VIP 800
|
||||
5 101 2024-01-02 09:00:00 2024-01-02 08:00:00 PREMIUM 900
|
||||
6 102 2024-01-02 14:00:00 2024-01-02 12:00:00 PREMIUM 950
|
||||
-- !result
|
||||
SELECT o.user_id, COUNT(*) as order_count, MAX(us.credit_score) as max_credit_score
|
||||
FROM orders o
|
||||
ASOF LEFT JOIN user_status us ON o.user_id = us.user_id AND o.order_time >= us.status_time
|
||||
GROUP BY o.user_id
|
||||
ORDER BY o.user_id;
|
||||
-- result:
|
||||
101 3 900
|
||||
102 3 950
|
||||
-- !result
|
||||
function: assert_explain_contains('SELECT * FROM orders o ASOF LEFT JOIN user_status us ON o.user_id = us.user_id AND o.order_time >= us.status_time', 'ASOF LEFT OUTER JOIN')
|
||||
-- result:
|
||||
None
|
||||
|
|
|
|||
|
|
@ -0,0 +1,169 @@
|
|||
-- name: test_asof_join_basic
|
||||
DROP DATABASE IF EXISTS test_asof_join_basic;
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE DATABASE test_asof_join_basic;
|
||||
-- result:
|
||||
-- !result
|
||||
use test_asof_join_basic;
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE orders_datetime (
|
||||
`order_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`order_time` datetime NOT NULL,
|
||||
`amount` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`order_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE prices_datetime (
|
||||
`product_id` int(11) NOT NULL,
|
||||
`price_time` datetime NOT NULL,
|
||||
`price` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`product_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE orders_date (
|
||||
`order_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`order_date` date NOT NULL,
|
||||
`amount` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`order_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE prices_date (
|
||||
`product_id` int(11) NOT NULL,
|
||||
`price_date` date NOT NULL,
|
||||
`price` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`product_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE orders_bigint (
|
||||
`order_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`order_timestamp` bigint NOT NULL,
|
||||
`amount` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`order_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE prices_bigint (
|
||||
`product_id` int(11) NOT NULL,
|
||||
`price_timestamp` bigint NOT NULL,
|
||||
`price` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`product_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO orders_datetime VALUES
|
||||
(1, 101, '2024-01-01 10:00:00', 100.00),
|
||||
(2, 101, '2024-01-01 15:30:00', 200.00),
|
||||
(3, 102, '2024-01-01 11:00:00', 150.00),
|
||||
(4, 102, '2024-01-01 16:00:00', 300.00),
|
||||
(5, 101, '2024-01-02 09:00:00', 250.00),
|
||||
(6, 102, '2024-01-02 14:00:00', 180.00);
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO prices_datetime VALUES
|
||||
(101, '2024-01-01 08:00:00', 95.00),
|
||||
(101, '2024-01-01 14:00:00', 105.00),
|
||||
(101, '2024-01-02 08:00:00', 110.00),
|
||||
(102, '2024-01-01 09:00:00', 90.00),
|
||||
(102, '2024-01-01 13:00:00', 100.00),
|
||||
(102, '2024-01-02 12:00:00', 115.00);
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO orders_date VALUES
|
||||
(1, 201, '2024-01-01', 100.00),
|
||||
(2, 201, '2024-01-02', 200.00),
|
||||
(3, 202, '2024-01-01', 150.00),
|
||||
(4, 202, '2024-01-03', 300.00);
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO prices_date VALUES
|
||||
(201, '2023-12-31', 95.00),
|
||||
(201, '2024-01-01', 105.00),
|
||||
(202, '2023-12-31', 90.00),
|
||||
(202, '2024-01-02', 100.00);
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO orders_bigint VALUES
|
||||
(1, 301, 1704067200, 100.00), -- 2024-01-01 10:00:00
|
||||
(2, 301, 1704084600, 200.00), -- 2024-01-01 15:30:00
|
||||
(3, 302, 1704070800, 150.00), -- 2024-01-01 11:00:00
|
||||
(4, 302, 1704086400, 300.00); -- 2024-01-01 16:00:00
|
||||
|
||||
INSERT INTO prices_bigint VALUES
|
||||
(301, 1704060000, 95.00), -- 2024-01-01 08:00:00
|
||||
(301, 1704081600, 105.00), -- 2024-01-01 14:00:00
|
||||
(302, 1704063600, 90.00), -- 2024-01-01 09:00:00
|
||||
(302, 1704078000, 100.00); -- 2024-01-01 13:00:00
|
||||
|
||||
SELECT o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_datetime o
|
||||
ASOF INNER JOIN prices_datetime p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
ORDER BY o.order_id;
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_datetime o
|
||||
ASOF LEFT JOIN prices_datetime p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 101 2024-01-01 10:00:00 2024-01-01 08:00:00 95.00
|
||||
2 101 2024-01-01 15:30:00 2024-01-01 14:00:00 105.00
|
||||
3 102 2024-01-01 11:00:00 2024-01-01 09:00:00 90.00
|
||||
4 102 2024-01-01 16:00:00 2024-01-01 13:00:00 100.00
|
||||
5 101 2024-01-02 09:00:00 2024-01-02 08:00:00 110.00
|
||||
6 102 2024-01-02 14:00:00 2024-01-02 12:00:00 115.00
|
||||
-- !result
|
||||
SELECT o.order_id, o.user_id, o.order_date, p.price_date, p.price
|
||||
FROM orders_date o
|
||||
ASOF INNER JOIN prices_date p ON o.user_id = p.product_id AND o.order_date >= p.price_date
|
||||
ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 201 2024-01-01 2024-01-01 105.00
|
||||
2 201 2024-01-02 2024-01-01 105.00
|
||||
3 202 2024-01-01 2023-12-31 90.00
|
||||
4 202 2024-01-03 2024-01-02 100.00
|
||||
-- !result
|
||||
SELECT o.order_id, o.user_id, o.order_date, p.price_date, p.price
|
||||
FROM orders_date o
|
||||
ASOF LEFT JOIN prices_date p ON o.user_id = p.product_id AND o.order_date >= p.price_date
|
||||
ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 201 2024-01-01 2024-01-01 105.00
|
||||
2 201 2024-01-02 2024-01-01 105.00
|
||||
3 202 2024-01-01 2023-12-31 90.00
|
||||
4 202 2024-01-03 2024-01-02 100.00
|
||||
-- !result
|
||||
SELECT o.order_id, o.user_id, o.order_timestamp, p.price_timestamp, p.price
|
||||
FROM orders_bigint o
|
||||
ASOF INNER JOIN prices_bigint p ON o.user_id = p.product_id AND o.order_timestamp >= p.price_timestamp
|
||||
ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 301 1704067200 1704060000 95.00
|
||||
2 301 1704084600 1704081600 105.00
|
||||
3 302 1704070800 1704063600 90.00
|
||||
4 302 1704086400 1704078000 100.00
|
||||
-- !result
|
||||
SELECT o.order_id, o.user_id, o.order_timestamp, p.price_timestamp, p.price
|
||||
FROM orders_bigint o
|
||||
ASOF LEFT JOIN prices_bigint p ON o.user_id = p.product_id AND o.order_timestamp >= p.price_timestamp
|
||||
ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 301 1704067200 1704060000 95.00
|
||||
2 301 1704084600 1704081600 105.00
|
||||
3 302 1704070800 1704063600 90.00
|
||||
4 302 1704086400 1704078000 100.00
|
||||
-- !result
|
||||
|
|
@ -0,0 +1,415 @@
|
|||
-- name: test_asof_join_complex
|
||||
DROP DATABASE IF EXISTS test_asof_join_complex;
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE DATABASE test_asof_join_complex;
|
||||
-- result:
|
||||
-- !result
|
||||
use test_asof_join_complex;
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE transactions (
|
||||
`txn_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`account_id` varchar(20) NOT NULL,
|
||||
`txn_time` datetime NOT NULL,
|
||||
`amount` decimal(15,2) NOT NULL,
|
||||
`txn_type` varchar(20) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`txn_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE exchange_rates (
|
||||
`currency_pair` varchar(10) NOT NULL,
|
||||
`rate_time` datetime NOT NULL,
|
||||
`exchange_rate` decimal(10,6) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`currency_pair`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE user_profiles (
|
||||
`user_id` int(11) NOT NULL,
|
||||
`profile_time` datetime NOT NULL,
|
||||
`risk_level` varchar(20) NOT NULL,
|
||||
`credit_score` int(11) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`user_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE market_events (
|
||||
`event_id` int(11) NOT NULL,
|
||||
`event_time` datetime NOT NULL,
|
||||
`currency` varchar(20) NOT NULL,
|
||||
`event_type` varchar(30) NOT NULL,
|
||||
`market_impact` decimal(5,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`event_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO transactions VALUES
|
||||
(1, 101, 'USD', '2024-01-01 10:00:00', 1000.00, 'DEPOSIT'),
|
||||
(2, 101, 'USD', '2024-01-01 15:30:00', 500.00, 'WITHDRAWAL'),
|
||||
(3, 102, 'EUR', '2024-01-01 11:00:00', 2000.00, 'DEPOSIT'),
|
||||
(4, 102, 'EUR', '2024-01-01 16:00:00', 800.00, 'WITHDRAWAL'),
|
||||
(5, 101, 'USD', '2024-01-02 09:00:00', 1500.00, 'DEPOSIT'),
|
||||
(6, 102, 'EUR', '2024-01-02 14:00:00', 1200.00, 'WITHDRAWAL');
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO exchange_rates VALUES
|
||||
('USD', '2024-01-01 08:00:00', 0.900000),
|
||||
('USD', '2024-01-01 14:00:00', 0.950000),
|
||||
('USD', '2024-01-02 08:00:00', 0.960000),
|
||||
('EUR', '2024-01-01 08:00:00', 1.100000),
|
||||
('EUR', '2024-01-01 13:00:00', 1.120000),
|
||||
('EUR', '2024-01-02 08:00:00', 1.130000);
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO user_profiles VALUES
|
||||
(101, '2024-01-01 09:00:00', 'LOW', 760),
|
||||
(101, '2024-01-01 11:30:00', 'MEDIUM', 805),
|
||||
(101, '2024-01-02 08:00:00', 'HIGH', 860),
|
||||
(102, '2024-01-01 10:00:00', 'LOW', 710),
|
||||
(102, '2024-01-01 12:30:00', 'MEDIUM', 770),
|
||||
(102, '2024-01-02 12:00:00', 'HIGH', 810);
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO market_events VALUES
|
||||
(1, '2024-01-01 08:00:00', 'USD', 'MARKET_OPEN', 0.50),
|
||||
(2, '2024-01-01 12:00:00', 'USD', 'FED_ANNOUNCEMENT', 1.20),
|
||||
(3, '2024-01-01 16:00:00', 'USD', 'MARKET_CLOSE', -0.30),
|
||||
(4, '2024-01-02 08:30:00', 'EUR', 'MARKET_OPEN', 0.25);
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT *
|
||||
FROM transactions t
|
||||
ASOF LEFT JOIN exchange_rates er ON t.account_id = er.currency_pair AND t.txn_time >= er.rate_time
|
||||
ASOF LEFT JOIN user_profiles up ON t.user_id = up.user_id AND t.txn_time >= up.profile_time
|
||||
ASOF LEFT JOIN market_events me ON t.account_id = me.currency AND t.txn_time >= me.event_time
|
||||
WHERE t.amount > 1000
|
||||
ORDER BY t.txn_id;
|
||||
-- result:
|
||||
3 102 EUR 2024-01-01 11:00:00 2000.00 DEPOSIT EUR 2024-01-01 08:00:00 1.100000 102 2024-01-01 10:00:00 LOW 710 None None None None None
|
||||
5 101 USD 2024-01-02 09:00:00 1500.00 DEPOSIT USD 2024-01-02 08:00:00 0.960000 101 2024-01-02 08:00:00 HIGH 860 3 2024-01-01 16:00:00 USD MARKET_CLOSE -0.30
|
||||
6 102 EUR 2024-01-02 14:00:00 1200.00 WITHDRAWAL EUR 2024-01-02 08:00:00 1.130000 102 2024-01-02 12:00:00 HIGH 810 4 2024-01-02 08:30:00 EUR MARKET_OPEN 0.25
|
||||
-- !result
|
||||
SELECT
|
||||
t.txn_id,
|
||||
t.user_id,
|
||||
DATE(t.txn_time) as txn_date,
|
||||
HOUR(t.txn_time) as txn_hour,
|
||||
er.exchange_rate,
|
||||
up.risk_level
|
||||
FROM transactions t
|
||||
ASOF INNER JOIN exchange_rates er
|
||||
ON t.account_id = er.currency_pair
|
||||
AND date_trunc('hour', t.txn_time) >= date_add(er.rate_time, INTERVAL 1 HOUR)
|
||||
ASOF INNER JOIN user_profiles up
|
||||
ON t.user_id = up.user_id
|
||||
AND date_trunc('minute', t.txn_time) >= date_add(up.profile_time, INTERVAL 30 MINUTE)
|
||||
WHERE DATE(t.txn_time) = '2024-01-01'
|
||||
ORDER BY t.txn_id;
|
||||
-- result:
|
||||
1 101 2024-01-01 10 0.900000 LOW
|
||||
2 101 2024-01-01 15 0.950000 MEDIUM
|
||||
3 102 2024-01-01 11 1.100000 LOW
|
||||
4 102 2024-01-01 16 1.120000 MEDIUM
|
||||
-- !result
|
||||
SELECT
|
||||
t.txn_id,
|
||||
t.user_id,
|
||||
t.txn_time,
|
||||
t.amount,
|
||||
er.exchange_rate,
|
||||
up.risk_level,
|
||||
up.credit_score
|
||||
FROM transactions t
|
||||
ASOF INNER JOIN exchange_rates er ON t.account_id = er.currency_pair AND t.txn_time >= er.rate_time
|
||||
ASOF INNER JOIN user_profiles up ON t.user_id = up.user_id AND t.txn_time >= up.profile_time
|
||||
WHERE t.amount > 500
|
||||
AND up.credit_score > 750
|
||||
AND er.exchange_rate > 0.85
|
||||
ORDER BY t.txn_id;
|
||||
-- result:
|
||||
1 101 2024-01-01 10:00:00 1000.00 0.900000 LOW 760
|
||||
4 102 2024-01-01 16:00:00 800.00 1.120000 MEDIUM 770
|
||||
5 101 2024-01-02 09:00:00 1500.00 0.960000 HIGH 860
|
||||
6 102 2024-01-02 14:00:00 1200.00 1.130000 HIGH 810
|
||||
-- !result
|
||||
SELECT
|
||||
t.user_id,
|
||||
COUNT(*) as txn_count,
|
||||
SUM(t.amount) as total_amount,
|
||||
AVG(er.exchange_rate) as avg_rate,
|
||||
MAX(up.credit_score) as max_credit_score
|
||||
FROM transactions t
|
||||
ASOF INNER JOIN exchange_rates er ON t.account_id = er.currency_pair AND t.txn_time >= er.rate_time
|
||||
ASOF INNER JOIN user_profiles up ON t.user_id = up.user_id AND t.txn_time >= up.profile_time
|
||||
GROUP BY t.user_id
|
||||
ORDER BY t.user_id;
|
||||
-- result:
|
||||
101 3 3000.00 0.936666666667 860
|
||||
102 3 4000.00 1.116666666667 810
|
||||
-- !result
|
||||
SELECT
|
||||
t.txn_id,
|
||||
t.user_id,
|
||||
t.txn_time,
|
||||
t.amount,
|
||||
er.exchange_rate,
|
||||
up.risk_level
|
||||
FROM transactions t
|
||||
ASOF INNER JOIN exchange_rates er ON t.account_id = er.currency_pair AND t.txn_time >= er.rate_time
|
||||
ASOF INNER JOIN user_profiles up ON t.user_id = up.user_id AND t.txn_time >= up.profile_time
|
||||
WHERE t.user_id IN (
|
||||
SELECT DISTINCT user_id
|
||||
FROM transactions
|
||||
WHERE amount > 1000
|
||||
)
|
||||
ORDER BY t.txn_id;
|
||||
-- result:
|
||||
1 101 2024-01-01 10:00:00 1000.00 0.900000 LOW
|
||||
2 101 2024-01-01 15:30:00 500.00 0.950000 MEDIUM
|
||||
3 102 2024-01-01 11:00:00 2000.00 1.100000 LOW
|
||||
4 102 2024-01-01 16:00:00 800.00 1.120000 MEDIUM
|
||||
5 101 2024-01-02 09:00:00 1500.00 0.960000 HIGH
|
||||
6 102 2024-01-02 14:00:00 1200.00 1.130000 HIGH
|
||||
-- !result
|
||||
SELECT
|
||||
t.txn_id,
|
||||
t.user_id,
|
||||
t.txn_time,
|
||||
t.amount,
|
||||
er.exchange_rate,
|
||||
up.risk_level
|
||||
FROM transactions t
|
||||
ASOF INNER JOIN exchange_rates er ON t.account_id = er.currency_pair AND t.txn_time > er.rate_time
|
||||
ASOF INNER JOIN user_profiles up ON t.user_id = up.user_id AND t.txn_time <= up.profile_time
|
||||
WHERE t.txn_type = 'DEPOSIT'
|
||||
ORDER BY t.txn_id;
|
||||
-- result:
|
||||
1 101 2024-01-01 10:00:00 1000.00 0.900000 MEDIUM
|
||||
3 102 2024-01-01 11:00:00 2000.00 1.100000 MEDIUM
|
||||
-- !result
|
||||
SELECT
|
||||
t.txn_id,
|
||||
t.user_id,
|
||||
t.txn_time,
|
||||
er.exchange_rate,
|
||||
me.event_type
|
||||
FROM transactions t
|
||||
ASOF INNER JOIN exchange_rates er
|
||||
ON t.account_id = er.currency_pair
|
||||
AND date_trunc('day', t.txn_time) >= date_add(er.rate_time, INTERVAL 0 DAY)
|
||||
ASOF INNER JOIN market_events me
|
||||
ON t.account_id = me.currency
|
||||
AND date_add(t.txn_time, INTERVAL -30 MINUTE) >= date_trunc('minute', me.event_time)
|
||||
WHERE t.txn_type = 'WITHDRAWAL'
|
||||
ORDER BY t.txn_id;
|
||||
-- result:
|
||||
6 102 2024-01-02 14:00:00 1.120000 MARKET_OPEN
|
||||
-- !result
|
||||
CREATE TABLE events_dt6 (
|
||||
`event_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`event_time` datetime NOT NULL
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(`event_id`) PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE snapshots_dt6 (
|
||||
`user_id` int(11) NOT NULL,
|
||||
`snapshot_time` datetime NOT NULL,
|
||||
`snapshot_data` varchar(50) NOT NULL
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(`user_id`) PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO snapshots_dt6 VALUES
|
||||
(701,'2024-01-01 10:00:00.100000','S1'),
|
||||
(701,'2024-01-01 10:00:00.300000','S2'),
|
||||
(701,'2024-01-01 10:00:00.900000','S3');
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO snapshots_dt6 VALUES
|
||||
(702,'2024-01-01 10:00:00.123456','T1'),
|
||||
(702,'2024-01-01 10:00:00.223456','T2'),
|
||||
(702,'2024-01-01 10:00:00.323456','T3');
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO events_dt6 VALUES
|
||||
(1,701,'2024-01-01 10:00:00.350000'), -- between S2 and S3 (>= should pick S2)
|
||||
(2,701,'2024-01-01 10:00:00.100001'), -- just after S1
|
||||
(3,702,'2024-01-01 10:00:00.323455'), -- just before T3 (>= picks T2)
|
||||
(4,702,'2024-01-01 10:00:00.500000'), -- after T3 (>= picks T3)
|
||||
(5,702,'2024-01-01 10:00:00.123456'), -- equal to T1 boundary
|
||||
(6,702,'2024-01-01 09:59:59.999999'); -- before first snapshot (no match for >=)
|
||||
|
||||
SELECT /* DT6-INNER-GE */ e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events_dt6 e
|
||||
ASOF INNER JOIN snapshots_dt6 s ON e.user_id = s.user_id AND e.event_time >= s.snapshot_time
|
||||
ORDER BY e.event_id;
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT /* DT6-LEFT-GE */ e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events_dt6 e
|
||||
ASOF LEFT JOIN snapshots_dt6 s ON e.user_id = s.user_id AND e.event_time >= s.snapshot_time
|
||||
ORDER BY e.event_id;
|
||||
-- result:
|
||||
1 701 2024-01-01 10:00:00.350000 2024-01-01 10:00:00.300000 S2
|
||||
2 701 2024-01-01 10:00:00.100001 2024-01-01 10:00:00.100000 S1
|
||||
3 702 2024-01-01 10:00:00.323455 2024-01-01 10:00:00.223456 T2
|
||||
4 702 2024-01-01 10:00:00.500000 2024-01-01 10:00:00.323456 T3
|
||||
5 702 2024-01-01 10:00:00.123456 2024-01-01 10:00:00.123456 T1
|
||||
6 702 2024-01-01 09:59:59.999999 None None
|
||||
-- !result
|
||||
SELECT /* DT6-INNER-LT */ e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events_dt6 e
|
||||
ASOF INNER JOIN snapshots_dt6 s ON e.user_id = s.user_id AND e.event_time < s.snapshot_time
|
||||
WHERE e.user_id IN (701,702)
|
||||
ORDER BY e.event_id;
|
||||
-- result:
|
||||
1 701 2024-01-01 10:00:00.350000 2024-01-01 10:00:00.900000 S3
|
||||
2 701 2024-01-01 10:00:00.100001 2024-01-01 10:00:00.300000 S2
|
||||
3 702 2024-01-01 10:00:00.323455 2024-01-01 10:00:00.323456 T3
|
||||
5 702 2024-01-01 10:00:00.123456 2024-01-01 10:00:00.223456 T2
|
||||
6 702 2024-01-01 09:59:59.999999 2024-01-01 10:00:00.123456 T1
|
||||
-- !result
|
||||
SELECT /* DT6-INNER-GE */ e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events_dt6 e
|
||||
ASOF INNER JOIN snapshots_dt6 s ON e.user_id = s.user_id AND cast(e.event_time as date) >= cast(s.snapshot_time as date)
|
||||
ORDER BY e.event_id;
|
||||
-- result:
|
||||
1 701 2024-01-01 10:00:00.350000 2024-01-01 10:00:00.100000 S1
|
||||
2 701 2024-01-01 10:00:00.100001 2024-01-01 10:00:00.100000 S1
|
||||
3 702 2024-01-01 10:00:00.323455 2024-01-01 10:00:00.123456 T1
|
||||
4 702 2024-01-01 10:00:00.500000 2024-01-01 10:00:00.123456 T1
|
||||
5 702 2024-01-01 10:00:00.123456 2024-01-01 10:00:00.123456 T1
|
||||
6 702 2024-01-01 09:59:59.999999 2024-01-01 10:00:00.123456 T1
|
||||
-- !result
|
||||
CREATE TABLE orders (
|
||||
`id` int(11) NULL COMMENT "订单ID",
|
||||
`product` varchar(50) NULL COMMENT "产品名称",
|
||||
`order_time` datetime NULL COMMENT "订单时间",
|
||||
`quantity` int(11) NULL COMMENT "订单数量",
|
||||
`max_price` decimal(10, 2) NULL COMMENT "最大价格"
|
||||
) ENGINE=OLAP
|
||||
DUPLICATE KEY(`id`)
|
||||
DISTRIBUTED BY HASH(`id`) BUCKETS 1
|
||||
PROPERTIES (
|
||||
"replication_num" = "1"
|
||||
);
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE prices (
|
||||
`id` int(11) NULL COMMENT "价格记录ID",
|
||||
`product` varchar(50) NULL COMMENT "产品名称",
|
||||
`price_time` datetime NULL COMMENT "价格时间",
|
||||
`price` decimal(10, 2) NULL COMMENT "价格",
|
||||
`volume` int(11) NULL COMMENT "成交量"
|
||||
) ENGINE=OLAP
|
||||
DUPLICATE KEY(`id`)
|
||||
DISTRIBUTED BY HASH(`id`) BUCKETS 1
|
||||
PROPERTIES (
|
||||
"replication_num" = "1"
|
||||
);
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO orders VALUES
|
||||
(1, 'A', '2024-01-01 10:00:00', 100, 100.00),
|
||||
(2, 'A', '2024-01-01 11:30:00', 150, 60.00),
|
||||
(3, 'A', '2024-01-01 14:00:00', 200, 50.00),
|
||||
(4, 'A', '2024-01-01 16:00:00', 50, 120.00),
|
||||
(5, 'B', '2024-01-01 09:30:00', 400, 25.00),
|
||||
(6, 'B', '2024-01-01 12:00:00', 100, 30.00),
|
||||
(7, 'B', '2024-01-01 15:30:00', 80, 40.00),
|
||||
(8, 'C', '2024-01-01 13:00:00', 500, 20.00),
|
||||
(9, 'C', '2024-01-01 17:00:00', 160, 50.00);
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO prices VALUES
|
||||
(1, 'A', '2024-01-01 08:00:00', 80.00, 1000),
|
||||
(16, 'A', '2024-01-01 09:00:00', 60.00, 1000),
|
||||
(2, 'A', '2024-01-01 09:30:00', 85.00, 1000),
|
||||
(3, 'A', '2024-01-01 10:30:00', 53.33, 1500),
|
||||
(4, 'A', '2024-01-01 12:00:00', 40.00, 1200),
|
||||
(5, 'A', '2024-01-01 15:00:00', 120.00, 800),
|
||||
(6, 'A', '2024-01-01 18:00:00', 90.00, 900),
|
||||
(7, 'B', '2024-01-01 08:30:00', 20.00, 2000),
|
||||
(8, 'B', '2024-01-01 10:00:00', 30.00, 1800),
|
||||
(9, 'B', '2024-01-01 11:00:00', 80.00, 1500),
|
||||
(17, 'B', '2024-01-01 13:00:00', 50.00, 2000),
|
||||
(10, 'B', '2024-01-01 14:00:00', 100.00, 1400),
|
||||
(11, 'B', '2024-01-01 16:30:00', 35.00, 1400),
|
||||
(12, 'C', '2024-01-01 09:00:00', 16.00, 3000),
|
||||
(13, 'C', '2024-01-01 12:30:00', 18.00, 2800),
|
||||
(18, 'C', '2024-01-01 15:00:00', 30.00, 3000),
|
||||
(14, 'C', '2024-01-01 16:00:00', 50.00, 1000),
|
||||
(15, 'C', '2024-01-01 19:00:00', 45.00, 900);
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT
|
||||
o.id AS order_id,
|
||||
o.product,
|
||||
o.order_time,
|
||||
o.quantity,
|
||||
o.max_price,
|
||||
p.id AS price_id,
|
||||
p.product AS price_product,
|
||||
p.price_time,
|
||||
p.price,
|
||||
p.volume,
|
||||
o.quantity * p.price AS total_amount,
|
||||
TIMESTAMPDIFF(MINUTE, p.price_time, o.order_time) AS time_diff_min,
|
||||
CASE
|
||||
WHEN p.id IS NULL THEN 'NO_MATCH'
|
||||
ELSE 'MATCHED'
|
||||
END AS match_status
|
||||
FROM orders o
|
||||
ASOF LEFT JOIN prices p
|
||||
ON o.product = p.product
|
||||
AND o.order_time >= p.price_time
|
||||
AND CAST(o.quantity * p.price AS BIGINT) = CAST(8000 AS BIGINT)
|
||||
ORDER BY o.product, o.order_time;
|
||||
-- result:
|
||||
1 A 2024-01-01 10:00:00 100 100.00 None None None None None None None NO_MATCH
|
||||
2 A 2024-01-01 11:30:00 150 60.00 None None None None None None None NO_MATCH
|
||||
3 A 2024-01-01 14:00:00 200 50.00 4 A 2024-01-01 12:00:00 40.00 1200 8000.00 120 MATCHED
|
||||
4 A 2024-01-01 16:00:00 50 120.00 None None None None None None None NO_MATCH
|
||||
5 B 2024-01-01 09:30:00 400 25.00 7 B 2024-01-01 08:30:00 20.00 2000 8000.00 60 MATCHED
|
||||
6 B 2024-01-01 12:00:00 100 30.00 9 B 2024-01-01 11:00:00 80.00 1500 8000.00 60 MATCHED
|
||||
7 B 2024-01-01 15:30:00 80 40.00 10 B 2024-01-01 14:00:00 100.00 1400 8000.00 90 MATCHED
|
||||
8 C 2024-01-01 13:00:00 500 20.00 None None None None None None None NO_MATCH
|
||||
9 C 2024-01-01 17:00:00 160 50.00 14 C 2024-01-01 16:00:00 50.00 1000 8000.00 60 MATCHED
|
||||
-- !result
|
||||
SELECT
|
||||
o.id AS order_id,
|
||||
o.product,
|
||||
o.order_time,
|
||||
o.quantity,
|
||||
o.max_price,
|
||||
p.id AS price_id,
|
||||
p.product AS price_product,
|
||||
p.price_time,
|
||||
p.price,
|
||||
p.volume,
|
||||
o.quantity * p.price AS total_amount,
|
||||
TIMESTAMPDIFF(MINUTE, p.price_time, o.order_time) AS time_diff_min,
|
||||
CASE
|
||||
WHEN p.id IS NULL THEN 'NO_MATCH'
|
||||
ELSE 'MATCHED'
|
||||
END AS match_status
|
||||
FROM orders o
|
||||
ASOF JOIN prices p
|
||||
ON o.product = p.product
|
||||
AND o.order_time >= p.price_time
|
||||
AND CAST(o.quantity * p.price AS BIGINT) = CAST(8000 AS BIGINT)
|
||||
ORDER BY o.product, o.order_time;
|
||||
-- result:
|
||||
3 A 2024-01-01 14:00:00 200 50.00 4 A 2024-01-01 12:00:00 40.00 1200 8000.00 120 MATCHED
|
||||
5 B 2024-01-01 09:30:00 400 25.00 7 B 2024-01-01 08:30:00 20.00 2000 8000.00 60 MATCHED
|
||||
6 B 2024-01-01 12:00:00 100 30.00 9 B 2024-01-01 11:00:00 80.00 1500 8000.00 60 MATCHED
|
||||
7 B 2024-01-01 15:30:00 80 40.00 10 B 2024-01-01 14:00:00 100.00 1400 8000.00 90 MATCHED
|
||||
9 C 2024-01-01 17:00:00 160 50.00 14 C 2024-01-01 16:00:00 50.00 1000 8000.00 60 MATCHED
|
||||
-- !result
|
||||
|
|
@ -0,0 +1,390 @@
|
|||
-- name: test_asof_join_edge_cases
|
||||
DROP DATABASE IF EXISTS test_asof_join_edge_cases;
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE DATABASE test_asof_join_edge_cases;
|
||||
-- result:
|
||||
-- !result
|
||||
use test_asof_join_edge_cases;
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE events (
|
||||
`event_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`event_time` datetime NOT NULL,
|
||||
`event_data` varchar(50) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`event_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE snapshots (
|
||||
`user_id` int(11) NOT NULL,
|
||||
`snapshot_time` datetime NOT NULL,
|
||||
`snapshot_data` varchar(50) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`user_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE events_with_nulls (
|
||||
`event_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`event_time` datetime NULL,
|
||||
`event_data` varchar(50) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`event_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE snapshots_with_nulls (
|
||||
`user_id` int(11) NOT NULL,
|
||||
`snapshot_time` datetime NULL,
|
||||
`snapshot_data` varchar(50) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`user_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO events VALUES
|
||||
(1, 101, '2024-01-01 10:00:00', 'EVENT_A'),
|
||||
(2, 101, '2024-01-01 15:30:00', 'EVENT_B'),
|
||||
(3, 102, '2024-01-01 11:00:00', 'EVENT_C'),
|
||||
(4, 102, '2024-01-01 16:00:00', 'EVENT_D'),
|
||||
(5, 103, '2024-01-01 12:00:00', 'EVENT_E');
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO snapshots VALUES
|
||||
(101, '2024-01-01 08:00:00', 'SNAPSHOT_1'),
|
||||
(101, '2024-01-01 14:00:00', 'SNAPSHOT_2'),
|
||||
(102, '2024-01-01 09:00:00', 'SNAPSHOT_3'),
|
||||
(102, '2024-01-01 13:00:00', 'SNAPSHOT_4'),
|
||||
(103, '2024-01-01 11:30:00', 'SNAPSHOT_5');
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO snapshots VALUES (101, '2024-01-01 10:00:00', 'SNAPSHOT_1_10AM');
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO events_with_nulls VALUES
|
||||
(1, 201, '2024-01-01 10:00:00', 'EVENT_A'),
|
||||
(2, 201, NULL, 'EVENT_B'),
|
||||
(3, 202, '2024-01-01 11:00:00', 'EVENT_C'),
|
||||
(4, 202, '2024-01-01 16:00:00', 'EVENT_D'),
|
||||
(5, 203, NULL, 'EVENT_E');
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO snapshots_with_nulls VALUES
|
||||
(201, '2024-01-01 08:00:00', 'SNAPSHOT_1'),
|
||||
(201, NULL, 'SNAPSHOT_2'),
|
||||
(202, '2024-01-01 09:00:00', 'SNAPSHOT_3'),
|
||||
(202, '2024-01-01 13:00:00', 'SNAPSHOT_4'),
|
||||
(203, '2024-01-01 11:30:00', 'SNAPSHOT_5');
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events e
|
||||
ASOF INNER JOIN snapshots s ON e.user_id = s.user_id AND e.event_time < s.snapshot_time
|
||||
WHERE e.user_id = 101
|
||||
ORDER BY e.event_id;
|
||||
-- result:
|
||||
1 101 2024-01-01 10:00:00 2024-01-01 14:00:00 SNAPSHOT_2
|
||||
-- !result
|
||||
SELECT e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events e
|
||||
ASOF LEFT JOIN snapshots s ON e.user_id = s.user_id AND e.event_time < s.snapshot_time
|
||||
WHERE e.user_id = 101
|
||||
ORDER BY e.event_id;
|
||||
-- result:
|
||||
1 101 2024-01-01 10:00:00 2024-01-01 14:00:00 SNAPSHOT_2
|
||||
2 101 2024-01-01 15:30:00 None None
|
||||
-- !result
|
||||
SELECT e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events e
|
||||
ASOF INNER JOIN snapshots s ON e.user_id = s.user_id AND e.event_time >= s.snapshot_time
|
||||
ORDER BY e.event_id;
|
||||
-- result:
|
||||
1 101 2024-01-01 10:00:00 2024-01-01 10:00:00 SNAPSHOT_1_10AM
|
||||
2 101 2024-01-01 15:30:00 2024-01-01 14:00:00 SNAPSHOT_2
|
||||
3 102 2024-01-01 11:00:00 2024-01-01 09:00:00 SNAPSHOT_3
|
||||
4 102 2024-01-01 16:00:00 2024-01-01 13:00:00 SNAPSHOT_4
|
||||
5 103 2024-01-01 12:00:00 2024-01-01 11:30:00 SNAPSHOT_5
|
||||
-- !result
|
||||
SELECT e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events e
|
||||
ASOF INNER JOIN snapshots s ON e.user_id = s.user_id + 1000 AND e.event_time >= s.snapshot_time
|
||||
ORDER BY e.event_id;
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE empty_snapshots (
|
||||
`user_id` int(11) NOT NULL,
|
||||
`snapshot_time` datetime NOT NULL,
|
||||
`snapshot_data` varchar(50) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`user_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events e
|
||||
ASOF INNER JOIN empty_snapshots s ON e.user_id = s.user_id AND e.event_time >= s.snapshot_time
|
||||
ORDER BY e.event_id;
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events e
|
||||
ASOF LEFT JOIN empty_snapshots s ON e.user_id = s.user_id AND e.event_time >= s.snapshot_time
|
||||
ORDER BY e.event_id;
|
||||
-- result:
|
||||
1 101 2024-01-01 10:00:00 None None
|
||||
2 101 2024-01-01 15:30:00 None None
|
||||
3 102 2024-01-01 11:00:00 None None
|
||||
4 102 2024-01-01 16:00:00 None None
|
||||
5 103 2024-01-01 12:00:00 None None
|
||||
-- !result
|
||||
SELECT e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events_with_nulls e
|
||||
ASOF INNER JOIN snapshots_with_nulls s ON e.user_id = s.user_id AND e.event_time >= s.snapshot_time
|
||||
WHERE e.event_time IS NULL
|
||||
ORDER BY e.event_id;
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events_with_nulls e
|
||||
ASOF LEFT JOIN snapshots_with_nulls s ON e.user_id = s.user_id AND e.event_time >= s.snapshot_time
|
||||
WHERE e.event_time IS NULL
|
||||
ORDER BY e.event_id;
|
||||
-- result:
|
||||
2 201 None None None
|
||||
5 203 None None None
|
||||
-- !result
|
||||
SELECT e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events e
|
||||
ASOF INNER JOIN snapshots_with_nulls s ON e.user_id = s.user_id AND e.event_time >= s.snapshot_time
|
||||
ORDER BY e.event_id;
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events_with_nulls e
|
||||
ASOF LEFT JOIN snapshots_with_nulls s ON e.user_id = s.user_id AND e.event_time >= s.snapshot_time
|
||||
ORDER BY e.event_id;
|
||||
-- result:
|
||||
1 201 2024-01-01 10:00:00 2024-01-01 08:00:00 SNAPSHOT_1
|
||||
2 201 None None None
|
||||
3 202 2024-01-01 11:00:00 2024-01-01 09:00:00 SNAPSHOT_3
|
||||
4 202 2024-01-01 16:00:00 2024-01-01 13:00:00 SNAPSHOT_4
|
||||
5 203 None None None
|
||||
-- !result
|
||||
CREATE TABLE events_large_gap (
|
||||
`event_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`event_time` datetime NOT NULL,
|
||||
`event_data` varchar(50) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`event_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE snapshots_large_gap (
|
||||
`user_id` int(11) NOT NULL,
|
||||
`snapshot_time` datetime NOT NULL,
|
||||
`snapshot_data` varchar(50) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`user_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO events_large_gap VALUES
|
||||
(1, 301, '2024-01-01 10:00:00', 'EVENT_A'),
|
||||
(2, 301, '2024-01-01 15:30:00', 'EVENT_B'),
|
||||
(3, 302, '2024-01-01 11:00:00', 'EVENT_C');
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO snapshots_large_gap VALUES
|
||||
(301, '2023-12-01 08:00:00', 'SNAPSHOT_1'),
|
||||
(301, '2023-12-15 14:00:00', 'SNAPSHOT_2'),
|
||||
(302, '2023-11-01 09:00:00', 'SNAPSHOT_3');
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events_large_gap e
|
||||
ASOF INNER JOIN snapshots_large_gap s ON e.user_id = s.user_id AND e.event_time >= s.snapshot_time
|
||||
ORDER BY e.event_id;
|
||||
-- result:
|
||||
1 301 2024-01-01 10:00:00 2023-12-15 14:00:00 SNAPSHOT_2
|
||||
2 301 2024-01-01 15:30:00 2023-12-15 14:00:00 SNAPSHOT_2
|
||||
3 302 2024-01-01 11:00:00 2023-11-01 09:00:00 SNAPSHOT_3
|
||||
-- !result
|
||||
CREATE TABLE events_dups (
|
||||
`event_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`event_time` datetime NOT NULL
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(`event_id`) PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE snapshots_dups (
|
||||
`user_id` int(11) NOT NULL,
|
||||
`snapshot_time` datetime NOT NULL,
|
||||
`snapshot_data` varchar(50) NOT NULL
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(`user_id`) PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO events_dups VALUES (1,401,'2024-01-01 10:00:00');
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO snapshots_dups VALUES
|
||||
(401,'2024-01-01 10:00:00','SNAP_D1'),
|
||||
(401,'2024-01-01 10:00:00','SNAP_D2');
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events_dups e
|
||||
ASOF INNER JOIN snapshots_dups s ON e.user_id=s.user_id AND e.event_time>=s.snapshot_time
|
||||
ORDER BY e.event_id;
|
||||
-- result:
|
||||
1 401 2024-01-01 10:00:00 2024-01-01 10:00:00 SNAP_D1
|
||||
-- !result
|
||||
CREATE TABLE events_ops (
|
||||
`event_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`event_time` datetime NOT NULL
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(`event_id`) PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE snapshots_ops (
|
||||
`user_id` int(11) NOT NULL,
|
||||
`snapshot_time` datetime NOT NULL
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(`user_id`) PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO events_ops VALUES (1,501,'2024-01-01 12:00:00');
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO snapshots_ops VALUES (501,'2024-01-01 12:00:00');
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT /* OP-< */ e.event_id, e.event_time, s.snapshot_time
|
||||
FROM events_ops e ASOF INNER JOIN snapshots_ops s
|
||||
ON e.user_id=s.user_id AND e.event_time < s.snapshot_time;
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT /* OP-<= */ e.event_id, e.event_time, s.snapshot_time
|
||||
FROM events_ops e ASOF INNER JOIN snapshots_ops s
|
||||
ON e.user_id=s.user_id AND e.event_time <= s.snapshot_time;
|
||||
-- result:
|
||||
1 2024-01-01 12:00:00 2024-01-01 12:00:00
|
||||
-- !result
|
||||
SELECT /* OP-> */ e.event_id, e.event_time, s.snapshot_time
|
||||
FROM events_ops e ASOF INNER JOIN snapshots_ops s
|
||||
ON e.user_id=s.user_id AND e.event_time > s.snapshot_time;
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT /* OP->= */ e.event_id, e.event_time, s.snapshot_time
|
||||
FROM events_ops e ASOF INNER JOIN snapshots_ops s
|
||||
ON e.user_id=s.user_id AND e.event_time >= s.snapshot_time;
|
||||
-- result:
|
||||
1 2024-01-01 12:00:00 2024-01-01 12:00:00
|
||||
-- !result
|
||||
CREATE TABLE events_ooo (
|
||||
`event_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`event_time` datetime NOT NULL
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(`event_id`) PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE snapshots_ooo (
|
||||
`user_id` int(11) NOT NULL,
|
||||
`snapshot_time` datetime NOT NULL,
|
||||
`snapshot_data` varchar(50) NOT NULL
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(`user_id`) PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO events_ooo VALUES (1,551,'2024-01-01 10:30:00');
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO snapshots_ooo VALUES (551,'2024-01-01 10:00:00','S10'),(551,'2024-01-01 09:00:00','S09');
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT e.event_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events_ooo e ASOF INNER JOIN snapshots_ooo s
|
||||
ON e.user_id=s.user_id AND e.event_time>=s.snapshot_time
|
||||
ORDER BY e.event_id;
|
||||
-- result:
|
||||
1 2024-01-01 10:30:00 2024-01-01 10:00:00 S10
|
||||
-- !result
|
||||
CREATE TABLE events_bounds (
|
||||
`event_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`event_time` datetime NOT NULL
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(`event_id`) PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE snapshots_bounds (
|
||||
`user_id` int(11) NOT NULL,
|
||||
`snapshot_time` datetime NOT NULL,
|
||||
`snapshot_data` varchar(50) NOT NULL
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(`user_id`) PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO snapshots_bounds VALUES (601,'2024-01-01 09:00:00','SB09'),(601,'2024-01-01 18:00:00','SB18');
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO events_bounds VALUES (1,601,'2024-01-01 07:00:00'),(2,601,'2024-01-01 20:00:00');
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT /* B-INNER-BEFORE */ e.event_id, e.event_time, s.snapshot_time
|
||||
FROM events_bounds e ASOF INNER JOIN snapshots_bounds s
|
||||
ON e.user_id=s.user_id AND e.event_time>=s.snapshot_time
|
||||
WHERE e.event_id=1;
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT /* B-LEFT-BEFORE */ e.event_id, e.event_time, s.snapshot_time
|
||||
FROM events_bounds e ASOF LEFT JOIN snapshots_bounds s
|
||||
ON e.user_id=s.user_id AND e.event_time>=s.snapshot_time
|
||||
WHERE e.event_id=1;
|
||||
-- result:
|
||||
1 2024-01-01 07:00:00 None
|
||||
-- !result
|
||||
SELECT /* B-INNER-AFTER */ e.event_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events_bounds e ASOF INNER JOIN snapshots_bounds s
|
||||
ON e.user_id=s.user_id AND e.event_time>=s.snapshot_time
|
||||
WHERE e.event_id=2;
|
||||
-- result:
|
||||
2 2024-01-01 20:00:00 2024-01-01 18:00:00 SB18
|
||||
-- !result
|
||||
SELECT /* B-LEFT-AFTER */ e.event_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events_bounds e ASOF LEFT JOIN snapshots_bounds s
|
||||
ON e.user_id=s.user_id AND e.event_time>=s.snapshot_time
|
||||
WHERE e.event_id=2;
|
||||
-- result:
|
||||
2 2024-01-01 20:00:00 2024-01-01 18:00:00 SB18
|
||||
-- !result
|
||||
CREATE TABLE events_equi_null (
|
||||
`event_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NULL,
|
||||
`event_time` datetime NOT NULL
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(`event_id`) PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE snapshots_equi_null (
|
||||
`user_id` int(11) NULL,
|
||||
`snapshot_time` datetime NOT NULL
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(`user_id`) PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO events_equi_null VALUES (1,NULL,'2024-01-01 12:00:00');
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO snapshots_equi_null VALUES (NULL,'2024-01-01 11:00:00');
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT /* EN-INNER */ e.event_id, e.user_id, e.event_time, s.snapshot_time
|
||||
FROM events_equi_null e ASOF INNER JOIN snapshots_equi_null s
|
||||
ON e.user_id=s.user_id AND e.event_time>=s.snapshot_time;
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT /* EN-LEFT */ e.event_id, e.user_id, e.event_time, s.snapshot_time
|
||||
FROM events_equi_null e ASOF LEFT JOIN snapshots_equi_null s
|
||||
ON e.user_id=s.user_id AND e.event_time>=s.snapshot_time;
|
||||
-- result:
|
||||
1 None 2024-01-01 12:00:00 None
|
||||
-- !result
|
||||
|
|
@ -0,0 +1,354 @@
|
|||
-- name: test_asof_join_hash_tables
|
||||
DROP DATABASE IF EXISTS test_asof_join_hash_tables;
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE DATABASE test_asof_join_hash_tables;
|
||||
-- result:
|
||||
-- !result
|
||||
use test_asof_join_hash_tables;
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE orders_tinyint (
|
||||
`order_id` int(11) NOT NULL,
|
||||
`user_id` tinyint NOT NULL, -- TINYINT triggers DirectMapping
|
||||
`order_time` datetime NOT NULL,
|
||||
`amount` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`order_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE prices_tinyint (
|
||||
`product_id` tinyint NOT NULL,
|
||||
`price_time` datetime NOT NULL,
|
||||
`price` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`product_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE orders_range (
|
||||
`order_id` int(11) NOT NULL,
|
||||
`user_id` int NOT NULL, -- Small range to trigger RangeDirectMapping
|
||||
`order_time` datetime NOT NULL,
|
||||
`amount` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`order_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE prices_range (
|
||||
`product_id` int NOT NULL,
|
||||
`price_time` datetime NOT NULL,
|
||||
`price` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`product_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE orders_dense (
|
||||
`order_id` int(11) NOT NULL,
|
||||
`user_id` int NOT NULL,
|
||||
`order_time` datetime NOT NULL,
|
||||
`amount` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`order_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE prices_dense (
|
||||
`product_id` int NOT NULL,
|
||||
`price_time` datetime NOT NULL,
|
||||
`price` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`product_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE orders_linear (
|
||||
`order_id` int(11) NOT NULL,
|
||||
`user_id` int NOT NULL, -- Large range to trigger LinearChained
|
||||
`order_time` datetime NOT NULL,
|
||||
`amount` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`order_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE prices_linear (
|
||||
`product_id` int NOT NULL,
|
||||
`price_time` datetime NOT NULL,
|
||||
`price` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`product_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE orders_asof_fallback (
|
||||
`order_id` int(11) NOT NULL,
|
||||
`user_id` int NOT NULL, -- Large range that doesn't fit other optimizations
|
||||
`order_time` datetime NOT NULL,
|
||||
`amount` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`order_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE prices_asof_fallback (
|
||||
`product_id` int NOT NULL,
|
||||
`price_time` datetime NOT NULL,
|
||||
`price` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`product_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE orders_string (
|
||||
`order_id` int(11) NOT NULL,
|
||||
`user_code` varchar(20) NOT NULL,
|
||||
`order_time` datetime NOT NULL,
|
||||
`amount` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`order_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE prices_string (
|
||||
`product_code` varchar(20) NOT NULL,
|
||||
`price_time` datetime NOT NULL,
|
||||
`price` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`product_code`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE orders_multi_key (
|
||||
`order_id` int(11) NOT NULL,
|
||||
`user_id` int NOT NULL,
|
||||
`region_id` int NOT NULL, -- Multiple keys trigger SERIALIZED_FIXED_SIZE
|
||||
`order_time` datetime NOT NULL,
|
||||
`amount` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`order_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE prices_multi_key (
|
||||
`product_id` int NOT NULL,
|
||||
`region_id` int NOT NULL,
|
||||
`price_time` datetime NOT NULL,
|
||||
`price` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`product_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO orders_tinyint VALUES
|
||||
(1, 1, '2024-01-01 10:00:00', 100.00),
|
||||
(2, 2, '2024-01-01 15:30:00', 200.00),
|
||||
(3, 3, '2024-01-01 11:00:00', 150.00),
|
||||
(4, 4, '2024-01-01 16:00:00', 300.00),
|
||||
(5, 5, '2024-01-02 09:00:00', 250.00),
|
||||
(6, 6, '2024-01-02 14:00:00', 180.00);
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO prices_tinyint VALUES
|
||||
(1, '2024-01-01 08:00:00', 95.00),
|
||||
(1, '2024-01-01 14:00:00', 105.00),
|
||||
(2, '2024-01-01 09:00:00', 90.00),
|
||||
(2, '2024-01-01 13:00:00', 100.00),
|
||||
(3, '2024-01-01 10:00:00', 85.00),
|
||||
(4, '2024-01-01 11:00:00', 110.00),
|
||||
(5, '2024-01-01 12:00:00', 120.00),
|
||||
(6, '2024-01-01 13:00:00', 130.00);
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO orders_range VALUES
|
||||
(1, 100, '2024-01-01 10:00:00', 100.00),
|
||||
(2, 101, '2024-01-01 15:30:00', 200.00),
|
||||
(3, 102, '2024-01-01 11:00:00', 150.00),
|
||||
(4, 103, '2024-01-01 16:00:00', 300.00),
|
||||
(5, 104, '2024-01-02 09:00:00', 250.00),
|
||||
(6, 105, '2024-01-02 14:00:00', 180.00);
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO prices_range VALUES
|
||||
(100, '2024-01-01 08:00:00', 95.00),
|
||||
(100, '2024-01-01 14:00:00', 105.00),
|
||||
(101, '2024-01-01 09:00:00', 90.00),
|
||||
(101, '2024-01-01 13:00:00', 100.00),
|
||||
(102, '2024-01-01 10:00:00', 85.00),
|
||||
(103, '2024-01-01 11:00:00', 110.00),
|
||||
(104, '2024-01-01 12:00:00', 120.00),
|
||||
(105, '2024-01-01 13:00:00', 130.00);
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO prices_dense
|
||||
SELECT
|
||||
1500000000 + (generate_series - 1) * 10 + 10 as product_id, -- range: 1500000010 to 1505000000
|
||||
'2024-01-01 08:00:00' + INTERVAL (generate_series - 1) SECOND as price_time,
|
||||
95.00 + (generate_series - 1) % 100 as price
|
||||
FROM TABLE(generate_series(1, 5000000));
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO orders_dense
|
||||
SELECT
|
||||
generate_series as order_id,
|
||||
1500000000 + generate_series * 10 as user_id, -- range: 1500000010 to 1500000000+500000*10 = 1505000000
|
||||
'2024-01-01 10:00:00' + INTERVAL (generate_series - 1) SECOND as order_time,
|
||||
100.00 + (generate_series - 1) % 1000 as amount
|
||||
FROM TABLE(generate_series(1, 500000));
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO orders_linear VALUES
|
||||
(1, 10000, '2024-01-01 10:00:00', 100.00),
|
||||
(2, 10001, '2024-01-01 15:30:00', 200.00),
|
||||
(3, 10002, '2024-01-01 11:00:00', 150.00),
|
||||
(4, 10003, '2024-01-01 16:00:00', 300.00),
|
||||
(5, 10004, '2024-01-02 09:00:00', 250.00),
|
||||
(6, 10005, '2024-01-02 14:00:00', 180.00);
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO prices_linear VALUES
|
||||
(10000, '2024-01-01 08:00:00', 95.00),
|
||||
(10000, '2024-01-01 14:00:00', 105.00),
|
||||
(10001, '2024-01-01 09:00:00', 90.00),
|
||||
(10001, '2024-01-01 13:00:00', 100.00),
|
||||
(10002, '2024-01-01 10:00:00', 85.00),
|
||||
(10003, '2024-01-01 11:00:00', 110.00),
|
||||
(10004, '2024-01-01 12:00:00', 120.00),
|
||||
(10005, '2024-01-01 13:00:00', 130.00);
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO orders_asof_fallback VALUES
|
||||
(1, 100000, '2024-01-01 10:00:00', 100.00),
|
||||
(2, 100001, '2024-01-01 15:30:00', 200.00),
|
||||
(3, 100002, '2024-01-01 11:00:00', 150.00),
|
||||
(4, 100003, '2024-01-01 16:00:00', 300.00);
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO prices_asof_fallback VALUES
|
||||
(100000, '2024-01-01 08:00:00', 95.00),
|
||||
(100000, '2024-01-01 14:00:00', 105.00),
|
||||
(100001, '2024-01-01 09:00:00', 90.00),
|
||||
(100001, '2024-01-01 13:00:00', 100.00),
|
||||
(100002, '2024-01-01 10:00:00', 85.00),
|
||||
(100003, '2024-01-01 11:00:00', 110.00);
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO orders_string VALUES
|
||||
(1, 'USER001', '2024-01-01 10:00:00', 100.00),
|
||||
(2, 'USER002', '2024-01-01 15:30:00', 200.00),
|
||||
(3, 'USER003', '2024-01-01 11:00:00', 150.00),
|
||||
(4, 'USER004', '2024-01-01 16:00:00', 300.00);
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO prices_string VALUES
|
||||
('USER001', '2024-01-01 08:00:00', 95.00),
|
||||
('USER001', '2024-01-01 14:00:00', 105.00),
|
||||
('USER002', '2024-01-01 09:00:00', 90.00),
|
||||
('USER002', '2024-01-01 13:00:00', 100.00),
|
||||
('USER003', '2024-01-01 10:00:00', 85.00),
|
||||
('USER004', '2024-01-01 11:00:00', 110.00);
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO orders_multi_key VALUES
|
||||
(1, 100, 1, '2024-01-01 10:00:00', 100.00),
|
||||
(2, 101, 1, '2024-01-01 15:30:00', 200.00),
|
||||
(3, 102, 2, '2024-01-01 11:00:00', 150.00),
|
||||
(4, 103, 2, '2024-01-01 16:00:00', 300.00);
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO prices_multi_key VALUES
|
||||
(100, 1, '2024-01-01 08:00:00', 95.00),
|
||||
(100, 1, '2024-01-01 14:00:00', 105.00),
|
||||
(101, 1, '2024-01-01 09:00:00', 90.00),
|
||||
(101, 1, '2024-01-01 13:00:00', 100.00),
|
||||
(102, 2, '2024-01-01 10:00:00', 85.00),
|
||||
(103, 2, '2024-01-01 11:00:00', 110.00);
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_tinyint o
|
||||
ASOF INNER JOIN prices_tinyint p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 1 2024-01-01 10:00:00 2024-01-01 08:00:00 95.00
|
||||
2 2 2024-01-01 15:30:00 2024-01-01 13:00:00 100.00
|
||||
3 3 2024-01-01 11:00:00 2024-01-01 10:00:00 85.00
|
||||
4 4 2024-01-01 16:00:00 2024-01-01 11:00:00 110.00
|
||||
5 5 2024-01-02 09:00:00 2024-01-01 12:00:00 120.00
|
||||
6 6 2024-01-02 14:00:00 2024-01-01 13:00:00 130.00
|
||||
-- !result
|
||||
SELECT o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_range o
|
||||
ASOF INNER JOIN prices_range p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 100 2024-01-01 10:00:00 2024-01-01 08:00:00 95.00
|
||||
2 101 2024-01-01 15:30:00 2024-01-01 13:00:00 100.00
|
||||
3 102 2024-01-01 11:00:00 2024-01-01 10:00:00 85.00
|
||||
4 103 2024-01-01 16:00:00 2024-01-01 11:00:00 110.00
|
||||
5 104 2024-01-02 09:00:00 2024-01-01 12:00:00 120.00
|
||||
6 105 2024-01-02 14:00:00 2024-01-01 13:00:00 130.00
|
||||
-- !result
|
||||
SELECT count(*)
|
||||
FROM orders_dense o
|
||||
ASOF INNER JOIN [broadcast] prices_dense p ON o.user_id = p.product_id AND o.order_time >= p.price_time;
|
||||
-- result:
|
||||
500000
|
||||
-- !result
|
||||
set enable_hash_join_range_direct_mapping_opt=false;
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_linear o
|
||||
ASOF INNER JOIN prices_linear p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 10000 2024-01-01 10:00:00 2024-01-01 08:00:00 95.00
|
||||
2 10001 2024-01-01 15:30:00 2024-01-01 13:00:00 100.00
|
||||
3 10002 2024-01-01 11:00:00 2024-01-01 10:00:00 85.00
|
||||
4 10003 2024-01-01 16:00:00 2024-01-01 11:00:00 110.00
|
||||
5 10004 2024-01-02 09:00:00 2024-01-01 12:00:00 120.00
|
||||
6 10005 2024-01-02 14:00:00 2024-01-01 13:00:00 130.00
|
||||
-- !result
|
||||
set enable_hash_join_linear_chained_opt=false;
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_asof_fallback o
|
||||
ASOF INNER JOIN prices_asof_fallback p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 100000 2024-01-01 10:00:00 2024-01-01 08:00:00 95.00
|
||||
2 100001 2024-01-01 15:30:00 2024-01-01 13:00:00 100.00
|
||||
3 100002 2024-01-01 11:00:00 2024-01-01 10:00:00 85.00
|
||||
4 100003 2024-01-01 16:00:00 2024-01-01 11:00:00 110.00
|
||||
-- !result
|
||||
set enable_hash_join_range_direct_mapping_opt=true;
|
||||
-- result:
|
||||
-- !result
|
||||
set enable_hash_join_linear_chained_opt=true;
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT o.order_id, o.user_code, o.order_time, p.price_time, p.price
|
||||
FROM orders_string o
|
||||
ASOF INNER JOIN prices_string p ON o.user_code = p.product_code AND o.order_time >= p.price_time
|
||||
ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 USER001 2024-01-01 10:00:00 2024-01-01 08:00:00 95.00
|
||||
2 USER002 2024-01-01 15:30:00 2024-01-01 13:00:00 100.00
|
||||
3 USER003 2024-01-01 11:00:00 2024-01-01 10:00:00 85.00
|
||||
4 USER004 2024-01-01 16:00:00 2024-01-01 11:00:00 110.00
|
||||
-- !result
|
||||
SELECT o.order_id, o.user_id, o.region_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_multi_key o
|
||||
ASOF INNER JOIN prices_multi_key p ON o.user_id = p.product_id AND o.region_id = p.region_id AND o.order_time >= p.price_time
|
||||
ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 100 1 2024-01-01 10:00:00 2024-01-01 08:00:00 95.00
|
||||
2 101 1 2024-01-01 15:30:00 2024-01-01 13:00:00 100.00
|
||||
3 102 2 2024-01-01 11:00:00 2024-01-01 10:00:00 85.00
|
||||
4 103 2 2024-01-01 16:00:00 2024-01-01 11:00:00 110.00
|
||||
-- !result
|
||||
|
|
@ -0,0 +1,329 @@
|
|||
-- name: test_asof_join_nulls
|
||||
DROP DATABASE IF EXISTS test_asof_join_nulls;
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE DATABASE test_asof_join_nulls;
|
||||
-- result:
|
||||
-- !result
|
||||
USE test_asof_join_nulls;
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE orders_nulls (
|
||||
`order_id` int,
|
||||
`user_id` int, -- nullable equi key
|
||||
`order_time` datetime, -- nullable temporal
|
||||
`amount` decimal(10,2)
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`order_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE prices_nulls (
|
||||
`product_id` int, -- nullable equi key
|
||||
`price_time` datetime, -- nullable temporal
|
||||
`price` decimal(10,2)
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`product_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO orders_nulls VALUES
|
||||
(1, 101, '2024-01-01 10:00:00', 100.00),
|
||||
(2, 101, '2024-01-01 12:00:00', 120.00),
|
||||
(3, 102, '2024-01-02 10:00:00', 130.00),
|
||||
(4, NULL, '2024-01-01 11:00:00', 90.00),
|
||||
(5, 101, NULL, 110.00),
|
||||
(6, 102, NULL, 140.00);
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO prices_nulls VALUES
|
||||
(101, '2024-01-01 09:00:00', 9.50),
|
||||
(101, '2024-01-01 11:30:00', 10.50),
|
||||
(101, '2024-01-01 12:30:00', 10.80),
|
||||
(102, '2024-01-02 08:00:00', 12.00),
|
||||
(102, '2024-01-02 12:00:00', 12.50),
|
||||
(NULL, '2024-01-01 08:00:00', 8.88),
|
||||
(101, NULL, 10.00),
|
||||
(102, NULL, 12.34);
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT /* A1 */ o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_nulls o
|
||||
ASOF INNER JOIN prices_nulls p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
WHERE o.order_id IN (1,2,3)
|
||||
ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 101 2024-01-01 10:00:00 2024-01-01 09:00:00 9.50
|
||||
2 101 2024-01-01 12:00:00 2024-01-01 11:30:00 10.50
|
||||
3 102 2024-01-02 10:00:00 2024-01-02 08:00:00 12.00
|
||||
-- !result
|
||||
SELECT /* A2 */ o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_nulls o
|
||||
ASOF LEFT JOIN prices_nulls p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
WHERE o.order_id IN (1,2,3)
|
||||
ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 101 2024-01-01 10:00:00 2024-01-01 09:00:00 9.50
|
||||
2 101 2024-01-01 12:00:00 2024-01-01 11:30:00 10.50
|
||||
3 102 2024-01-02 10:00:00 2024-01-02 08:00:00 12.00
|
||||
-- !result
|
||||
SELECT /* B1 */ o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_nulls o
|
||||
ASOF INNER JOIN prices_nulls p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
WHERE o.order_id = 4
|
||||
ORDER BY o.order_id;
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT /* B2 */ o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_nulls o
|
||||
ASOF LEFT JOIN prices_nulls p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
WHERE o.order_id = 4
|
||||
ORDER BY o.order_id;
|
||||
-- result:
|
||||
4 None 2024-01-01 11:00:00 None None
|
||||
-- !result
|
||||
SELECT /* C1 */ o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_nulls o
|
||||
ASOF INNER JOIN prices_nulls p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
WHERE o.order_id IN (5,6)
|
||||
ORDER BY o.order_id;
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT /* C2 */ o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_nulls o
|
||||
ASOF LEFT JOIN prices_nulls p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
WHERE o.order_id IN (5,6)
|
||||
ORDER BY o.order_id;
|
||||
-- result:
|
||||
5 101 None None None
|
||||
6 102 None None None
|
||||
-- !result
|
||||
SELECT /* D1 */ o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_nulls o
|
||||
ASOF INNER JOIN prices_nulls p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
WHERE o.order_id IN (1,2,3)
|
||||
ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 101 2024-01-01 10:00:00 2024-01-01 09:00:00 9.50
|
||||
2 101 2024-01-01 12:00:00 2024-01-01 11:30:00 10.50
|
||||
3 102 2024-01-02 10:00:00 2024-01-02 08:00:00 12.00
|
||||
-- !result
|
||||
SELECT /* D2 */ o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_nulls o
|
||||
ASOF LEFT JOIN prices_nulls p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
WHERE o.order_id IN (1,2,3)
|
||||
ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 101 2024-01-01 10:00:00 2024-01-01 09:00:00 9.50
|
||||
2 101 2024-01-01 12:00:00 2024-01-01 11:30:00 10.50
|
||||
3 102 2024-01-02 10:00:00 2024-01-02 08:00:00 12.00
|
||||
-- !result
|
||||
SELECT /* E1 */ o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_nulls o
|
||||
ASOF LEFT JOIN prices_nulls p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
WHERE o.order_id IN (1,4,5)
|
||||
ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 101 2024-01-01 10:00:00 2024-01-01 09:00:00 9.50
|
||||
4 None 2024-01-01 11:00:00 None None
|
||||
5 101 None None None
|
||||
-- !result
|
||||
CREATE TABLE dm_orders_nulls (
|
||||
order_id int,
|
||||
user_id tinyint,
|
||||
order_time datetime,
|
||||
amount decimal(10,2)
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(order_id) PROPERTIES ("replication_num"="1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE dm_prices_nulls (
|
||||
product_id tinyint,
|
||||
price_time datetime,
|
||||
price decimal(10,2)
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(product_id) PROPERTIES ("replication_num"="1");
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO dm_orders_nulls VALUES
|
||||
(1, 1, '2024-01-01 10:00:00', 10.0),
|
||||
(2, 1, NULL, 20.0),
|
||||
(3, NULL,'2024-01-01 11:00:00', 30.0);
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO dm_prices_nulls VALUES
|
||||
(1, '2024-01-01 09:00:00', 9.0),
|
||||
(1, NULL, 9.5),
|
||||
(NULL,'2024-01-01 08:00:00',8.8);
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT /* DM-INNER */ o.order_id, o.user_id, o.order_time, p.price_time
|
||||
FROM dm_orders_nulls o ASOF INNER JOIN dm_prices_nulls p
|
||||
ON o.user_id=p.product_id AND o.order_time>=p.price_time ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 1 2024-01-01 10:00:00 2024-01-01 09:00:00
|
||||
-- !result
|
||||
SELECT /* DM-LEFT */ o.order_id, o.user_id, o.order_time, p.price_time
|
||||
FROM dm_orders_nulls o ASOF LEFT JOIN dm_prices_nulls p
|
||||
ON o.user_id=p.product_id AND o.order_time>=p.price_time ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 1 2024-01-01 10:00:00 2024-01-01 09:00:00
|
||||
2 1 None None
|
||||
3 None 2024-01-01 11:00:00 None
|
||||
-- !result
|
||||
CREATE TABLE rdm_orders_nulls (
|
||||
order_id int,
|
||||
user_id int,
|
||||
order_time datetime,
|
||||
amount decimal(10,2)
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(order_id) PROPERTIES ("replication_num"="1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE rdm_prices_nulls (
|
||||
product_id int,
|
||||
price_time datetime,
|
||||
price decimal(10,2)
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(product_id) PROPERTIES ("replication_num"="1");
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO rdm_orders_nulls VALUES
|
||||
(1, 100, '2024-01-01 10:00:00', 10.0),
|
||||
(2, 100, NULL, 20.0),
|
||||
(3, NULL,'2024-01-01 12:00:00', 30.0);
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO rdm_prices_nulls VALUES
|
||||
(100,'2024-01-01 09:00:00', 9.0),
|
||||
(100,'2024-01-01 09:30:00', 9.5),
|
||||
(NULL,'2024-01-01 08:00:00',8.8),
|
||||
(100, NULL, 9.9);
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT /* RDM-INNER */ o.order_id, o.user_id, o.order_time, p.price_time
|
||||
FROM rdm_orders_nulls o ASOF INNER JOIN rdm_prices_nulls p
|
||||
ON o.user_id=p.product_id AND o.order_time>=p.price_time ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 100 2024-01-01 10:00:00 2024-01-01 09:30:00
|
||||
-- !result
|
||||
SELECT /* RDM-LEFT */ o.order_id, o.user_id, o.order_time, p.price_time
|
||||
FROM rdm_orders_nulls o ASOF LEFT JOIN rdm_prices_nulls p
|
||||
ON o.user_id=p.product_id AND o.order_time>=p.price_time ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 100 2024-01-01 10:00:00 2024-01-01 09:30:00
|
||||
2 100 None None
|
||||
3 None 2024-01-01 12:00:00 None
|
||||
-- !result
|
||||
CREATE TABLE dense_orders_nulls (
|
||||
order_id int,
|
||||
user_id int,
|
||||
order_time datetime,
|
||||
amount decimal(10,2)
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(order_id) PROPERTIES ("replication_num"="1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE dense_prices_nulls (
|
||||
product_id int,
|
||||
price_time datetime,
|
||||
price decimal(10,2)
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(product_id) PROPERTIES ("replication_num"="1");
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO dense_prices_nulls
|
||||
SELECT
|
||||
1600000000 + (generate_series - 1) * 10 + 10,
|
||||
date_add('2024-01-01 08:00:00', INTERVAL (generate_series - 1) SECOND),
|
||||
9.0 + (generate_series % 100)
|
||||
FROM TABLE(generate_series(1, 500000));
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO dense_orders_nulls VALUES
|
||||
(1, 1600000010, '2024-01-01 10:00:00', 10.0),
|
||||
(2, 1600001010, NULL, 20.0),
|
||||
(3, NULL, '2024-01-01 10:30:00', 30.0);
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT count(*)
|
||||
FROM dense_orders_nulls o ASOF INNER JOIN [broadcast] dense_prices_nulls p
|
||||
ON o.user_id=p.product_id AND o.order_time>=p.price_time;
|
||||
-- result:
|
||||
1
|
||||
-- !result
|
||||
SELECT count(*)
|
||||
FROM dense_orders_nulls o ASOF LEFT JOIN [broadcast] dense_prices_nulls p
|
||||
ON o.user_id=p.product_id AND o.order_time>=p.price_time;
|
||||
-- result:
|
||||
3
|
||||
-- !result
|
||||
set enable_hash_join_range_direct_mapping_opt=false;
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE lc_orders_nulls (
|
||||
order_id int,
|
||||
user_id int,
|
||||
order_time datetime,
|
||||
amount decimal(10,2)
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(order_id) PROPERTIES ("replication_num"="1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE lc_prices_nulls (
|
||||
product_id int,
|
||||
price_time datetime,
|
||||
price decimal(10,2)
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(product_id) PROPERTIES ("replication_num"="1");
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO lc_prices_nulls
|
||||
SELECT
|
||||
1700000000 + (generate_series - 1) * 1000,
|
||||
date_add('2024-01-01 08:00:00', INTERVAL (generate_series - 1) SECOND),
|
||||
20.0 + (generate_series % 10)
|
||||
FROM TABLE(generate_series(1, 5));
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO lc_orders_nulls VALUES
|
||||
(1, 1700000000, '2024-01-01 10:00:00', 10.0),
|
||||
(2, 1700050000, NULL, 20.0),
|
||||
(3, NULL, '2024-01-01 10:30:00', 30.0);
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT /* LC-INNER */ o.order_id, o.user_id, o.order_time, p.price_time
|
||||
FROM lc_orders_nulls o ASOF INNER JOIN lc_prices_nulls p
|
||||
ON o.user_id=p.product_id AND o.order_time>=p.price_time ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 1700000000 2024-01-01 10:00:00 2024-01-01 08:00:00
|
||||
-- !result
|
||||
SELECT /* LC-LEFT */ o.order_id, o.user_id, o.order_time, p.price_time
|
||||
FROM lc_orders_nulls o ASOF LEFT JOIN lc_prices_nulls p
|
||||
ON o.user_id=p.product_id AND o.order_time>=p.price_time ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 1700000000 2024-01-01 10:00:00 2024-01-01 08:00:00
|
||||
2 1700050000 None None
|
||||
3 None 2024-01-01 10:30:00 None
|
||||
-- !result
|
||||
CREATE TABLE lca_orders_nulls LIKE lc_orders_nulls;
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE lca_prices_nulls LIKE lc_prices_nulls;
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO lca_prices_nulls VALUES (1800000000,'2024-01-01 08:00:00', 30.0);
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO lca_orders_nulls VALUES (1,1800000000,'2024-01-01 10:00:00',10.0),(2,NULL,'2024-01-01 11:00:00',20.0),(3,1800000000,NULL,30.0);
|
||||
-- result:
|
||||
-- !result
|
||||
set enable_hash_join_linear_chained_opt=false;
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT /* LCA-LEFT */ o.order_id, o.user_id, o.order_time, p.price_time
|
||||
FROM lca_orders_nulls o ASOF LEFT JOIN lca_prices_nulls p
|
||||
ON o.user_id=p.product_id AND o.order_time>=p.price_time ORDER BY o.order_id;
|
||||
-- result:
|
||||
1 1800000000 2024-01-01 10:00:00 2024-01-01 08:00:00
|
||||
2 None 2024-01-01 11:00:00 None
|
||||
3 1800000000 None None
|
||||
-- !result
|
||||
set enable_hash_join_linear_chained_opt=true;
|
||||
-- result:
|
||||
-- !result
|
||||
set enable_hash_join_range_direct_mapping_opt=true;
|
||||
-- result:
|
||||
-- !result
|
||||
|
|
@ -0,0 +1,138 @@
|
|||
-- name: test_asof_join_operators
|
||||
DROP DATABASE IF EXISTS test_asof_join_operators;
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE DATABASE test_asof_join_operators;
|
||||
-- result:
|
||||
-- !result
|
||||
use test_asof_join_operators;
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE events (
|
||||
`event_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`event_time` datetime NOT NULL,
|
||||
`event_type` varchar(20) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`event_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE user_changes (
|
||||
`user_id` int(11) NOT NULL,
|
||||
`change_time` datetime NOT NULL,
|
||||
`change_type` varchar(20) NOT NULL,
|
||||
`new_value` varchar(50) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`user_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO events VALUES
|
||||
(1, 101, '2024-01-01 10:00:00', 'LOGIN'),
|
||||
(2, 101, '2024-01-01 15:30:00', 'PURCHASE'),
|
||||
(3, 102, '2024-01-01 11:00:00', 'LOGIN'),
|
||||
(4, 102, '2024-01-01 16:00:00', 'PURCHASE'),
|
||||
(5, 101, '2024-01-02 09:00:00', 'LOGOUT'),
|
||||
(6, 102, '2024-01-02 14:00:00', 'LOGOUT'),
|
||||
(7, 101, '2024-01-01 12:00:00', 'EXACT_MATCH_1'),
|
||||
(8, 102, '2024-01-01 12:00:00', 'EXACT_MATCH_2'),
|
||||
(9, 103, '2024-01-01 12:00:00', 'EXACT_MATCH_3');
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO user_changes VALUES
|
||||
(101, '2024-01-01 08:00:00', 'STATUS', 'ACTIVE'),
|
||||
(101, '2024-01-01 14:00:00', 'STATUS', 'VIP'),
|
||||
(101, '2024-01-02 08:00:00', 'STATUS', 'PREMIUM'),
|
||||
(102, '2024-01-01 09:00:00', 'STATUS', 'ACTIVE'),
|
||||
(102, '2024-01-01 13:00:00', 'STATUS', 'VIP'),
|
||||
(102, '2024-01-02 12:00:00', 'STATUS', 'PREMIUM'),
|
||||
(101, '2024-01-01 12:00:00', 'STATUS', 'EXACT_STATUS_1'),
|
||||
(102, '2024-01-01 12:00:00', 'STATUS', 'EXACT_STATUS_2'),
|
||||
(103, '2024-01-01 12:00:00', 'STATUS', 'EXACT_STATUS_3');
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT e.event_id, e.user_id, e.event_time, uc.change_time, uc.change_type, uc.new_value
|
||||
FROM events e
|
||||
ASOF INNER JOIN user_changes uc ON e.user_id = uc.user_id AND e.event_time >= uc.change_time
|
||||
ORDER BY e.event_id;
|
||||
-- result:
|
||||
1 101 2024-01-01 10:00:00 2024-01-01 08:00:00 STATUS ACTIVE
|
||||
2 101 2024-01-01 15:30:00 2024-01-01 14:00:00 STATUS VIP
|
||||
3 102 2024-01-01 11:00:00 2024-01-01 09:00:00 STATUS ACTIVE
|
||||
4 102 2024-01-01 16:00:00 2024-01-01 13:00:00 STATUS VIP
|
||||
5 101 2024-01-02 09:00:00 2024-01-02 08:00:00 STATUS PREMIUM
|
||||
6 102 2024-01-02 14:00:00 2024-01-02 12:00:00 STATUS PREMIUM
|
||||
7 101 2024-01-01 12:00:00 2024-01-01 12:00:00 STATUS EXACT_STATUS_1
|
||||
8 102 2024-01-01 12:00:00 2024-01-01 12:00:00 STATUS EXACT_STATUS_2
|
||||
9 103 2024-01-01 12:00:00 2024-01-01 12:00:00 STATUS EXACT_STATUS_3
|
||||
-- !result
|
||||
SELECT e.event_id, e.user_id, e.event_time, uc.change_time, uc.change_type, uc.new_value
|
||||
FROM events e
|
||||
ASOF INNER JOIN user_changes uc ON e.user_id = uc.user_id AND e.event_time > uc.change_time
|
||||
ORDER BY e.event_id;
|
||||
-- result:
|
||||
1 101 2024-01-01 10:00:00 2024-01-01 08:00:00 STATUS ACTIVE
|
||||
2 101 2024-01-01 15:30:00 2024-01-01 14:00:00 STATUS VIP
|
||||
3 102 2024-01-01 11:00:00 2024-01-01 09:00:00 STATUS ACTIVE
|
||||
4 102 2024-01-01 16:00:00 2024-01-01 13:00:00 STATUS VIP
|
||||
5 101 2024-01-02 09:00:00 2024-01-02 08:00:00 STATUS PREMIUM
|
||||
6 102 2024-01-02 14:00:00 2024-01-02 12:00:00 STATUS PREMIUM
|
||||
7 101 2024-01-01 12:00:00 2024-01-01 08:00:00 STATUS ACTIVE
|
||||
8 102 2024-01-01 12:00:00 2024-01-01 09:00:00 STATUS ACTIVE
|
||||
-- !result
|
||||
SELECT e.event_id, e.user_id, e.event_time, uc.change_time, uc.change_type, uc.new_value
|
||||
FROM events e
|
||||
ASOF INNER JOIN user_changes uc ON e.user_id = uc.user_id AND e.event_time <= uc.change_time
|
||||
ORDER BY e.event_id;
|
||||
-- result:
|
||||
1 101 2024-01-01 10:00:00 2024-01-01 12:00:00 STATUS EXACT_STATUS_1
|
||||
2 101 2024-01-01 15:30:00 2024-01-02 08:00:00 STATUS PREMIUM
|
||||
3 102 2024-01-01 11:00:00 2024-01-01 12:00:00 STATUS EXACT_STATUS_2
|
||||
4 102 2024-01-01 16:00:00 2024-01-02 12:00:00 STATUS PREMIUM
|
||||
7 101 2024-01-01 12:00:00 2024-01-01 12:00:00 STATUS EXACT_STATUS_1
|
||||
8 102 2024-01-01 12:00:00 2024-01-01 12:00:00 STATUS EXACT_STATUS_2
|
||||
9 103 2024-01-01 12:00:00 2024-01-01 12:00:00 STATUS EXACT_STATUS_3
|
||||
-- !result
|
||||
SELECT e.event_id, e.user_id, e.event_time, uc.change_time, uc.change_type, uc.new_value
|
||||
FROM events e
|
||||
ASOF INNER JOIN user_changes uc ON e.user_id = uc.user_id AND e.event_time < uc.change_time
|
||||
ORDER BY e.event_id;
|
||||
-- result:
|
||||
1 101 2024-01-01 10:00:00 2024-01-01 12:00:00 STATUS EXACT_STATUS_1
|
||||
2 101 2024-01-01 15:30:00 2024-01-02 08:00:00 STATUS PREMIUM
|
||||
3 102 2024-01-01 11:00:00 2024-01-01 12:00:00 STATUS EXACT_STATUS_2
|
||||
4 102 2024-01-01 16:00:00 2024-01-02 12:00:00 STATUS PREMIUM
|
||||
7 101 2024-01-01 12:00:00 2024-01-01 14:00:00 STATUS VIP
|
||||
8 102 2024-01-01 12:00:00 2024-01-01 13:00:00 STATUS VIP
|
||||
-- !result
|
||||
SELECT e.event_id, e.user_id, e.event_time, uc.change_time, uc.change_type, uc.new_value
|
||||
FROM events e
|
||||
ASOF LEFT JOIN user_changes uc ON e.user_id = uc.user_id AND e.event_time >= uc.change_time
|
||||
ORDER BY e.event_id;
|
||||
-- result:
|
||||
1 101 2024-01-01 10:00:00 2024-01-01 08:00:00 STATUS ACTIVE
|
||||
2 101 2024-01-01 15:30:00 2024-01-01 14:00:00 STATUS VIP
|
||||
3 102 2024-01-01 11:00:00 2024-01-01 09:00:00 STATUS ACTIVE
|
||||
4 102 2024-01-01 16:00:00 2024-01-01 13:00:00 STATUS VIP
|
||||
5 101 2024-01-02 09:00:00 2024-01-02 08:00:00 STATUS PREMIUM
|
||||
6 102 2024-01-02 14:00:00 2024-01-02 12:00:00 STATUS PREMIUM
|
||||
7 101 2024-01-01 12:00:00 2024-01-01 12:00:00 STATUS EXACT_STATUS_1
|
||||
8 102 2024-01-01 12:00:00 2024-01-01 12:00:00 STATUS EXACT_STATUS_2
|
||||
9 103 2024-01-01 12:00:00 2024-01-01 12:00:00 STATUS EXACT_STATUS_3
|
||||
-- !result
|
||||
SELECT e.event_id, e.user_id, e.event_time, uc.change_time, uc.change_type, uc.new_value
|
||||
FROM events e
|
||||
ASOF LEFT JOIN user_changes uc ON e.user_id = uc.user_id AND e.event_time > uc.change_time
|
||||
ORDER BY e.event_id;
|
||||
-- result:
|
||||
1 101 2024-01-01 10:00:00 2024-01-01 08:00:00 STATUS ACTIVE
|
||||
2 101 2024-01-01 15:30:00 2024-01-01 14:00:00 STATUS VIP
|
||||
3 102 2024-01-01 11:00:00 2024-01-01 09:00:00 STATUS ACTIVE
|
||||
4 102 2024-01-01 16:00:00 2024-01-01 13:00:00 STATUS VIP
|
||||
5 101 2024-01-02 09:00:00 2024-01-02 08:00:00 STATUS PREMIUM
|
||||
6 102 2024-01-02 14:00:00 2024-01-02 12:00:00 STATUS PREMIUM
|
||||
7 101 2024-01-01 12:00:00 2024-01-01 08:00:00 STATUS ACTIVE
|
||||
8 102 2024-01-01 12:00:00 2024-01-01 09:00:00 STATUS ACTIVE
|
||||
9 103 2024-01-01 12:00:00 None None None
|
||||
-- !result
|
||||
|
|
@ -0,0 +1,70 @@
|
|||
-- name: test_asof_join_performance
|
||||
DROP DATABASE IF EXISTS test_asof_join_performance;
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE DATABASE test_asof_join_performance;
|
||||
-- result:
|
||||
-- !result
|
||||
use test_asof_join_performance;
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE sessions (
|
||||
visitor_id BIGINT NOT NULL COMMENT '访客ID',
|
||||
session_start DATETIME NOT NULL COMMENT '会话开始时间',
|
||||
session_id BIGINT NOT NULL COMMENT '会话ID',
|
||||
session_duration INT COMMENT '会话时长秒'
|
||||
) ENGINE=OLAP
|
||||
DUPLICATE KEY(visitor_id, session_start)
|
||||
DISTRIBUTED BY HASH(visitor_id) BUCKETS 4
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
CREATE TABLE visitors (
|
||||
visitor_id BIGINT NOT NULL COMMENT '访客ID',
|
||||
first_visit DATETIME NOT NULL COMMENT '首次访问时间',
|
||||
starting_session_id BIGINT NOT NULL COMMENT '起始会话ID',
|
||||
visitor_type VARCHAR(20) COMMENT '访客类型'
|
||||
) ENGINE=OLAP
|
||||
DUPLICATE KEY(visitor_id, first_visit)
|
||||
DISTRIBUTED BY HASH(visitor_id) BUCKETS 4
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO sessions (visitor_id, session_start, session_id, session_duration)
|
||||
SELECT
|
||||
(number % 10000) as visitor_id,
|
||||
'2024-01-01 08:00:00' + INTERVAL number SECOND as session_start,
|
||||
number as session_id,
|
||||
30 + (number % 200000) as session_duration
|
||||
FROM TABLE(generate_series(1, 1000000)) as t(number)
|
||||
ORDER BY visitor_id, session_start;
|
||||
-- result:
|
||||
-- !result
|
||||
INSERT INTO visitors (visitor_id, first_visit, starting_session_id, visitor_type)
|
||||
SELECT
|
||||
(number % 10000) as visitor_id,
|
||||
'2024-01-01 07:30:00' + INTERVAL number SECOND as first_visit,
|
||||
number as starting_session_id,
|
||||
'aaaa'
|
||||
FROM TABLE(generate_series(1, 50000000)) as t(number)
|
||||
ORDER BY visitor_id, first_visit;
|
||||
-- result:
|
||||
-- !result
|
||||
SELECT
|
||||
COUNT(*) as total_sessions,
|
||||
COUNT(v.first_visit) as valid_matches
|
||||
FROM sessions s
|
||||
ASOF JOIN visitors v
|
||||
ON s.visitor_id = v.visitor_id AND v.first_visit <= s.session_start;
|
||||
-- result:
|
||||
1000000 1000000
|
||||
-- !result
|
||||
SELECT
|
||||
COUNT(*) as total_sessions,
|
||||
COUNT(v.first_visit) as valid_matches
|
||||
FROM sessions s
|
||||
ASOF LEFT JOIN visitors v
|
||||
ON s.visitor_id = v.visitor_id AND v.first_visit <= s.session_start;
|
||||
-- result:
|
||||
1000000 1000000
|
||||
-- !result
|
||||
|
|
@ -1,4 +1,5 @@
|
|||
-- name: test_asof_join
|
||||
drop database if exists test_asof_join;
|
||||
CREATE DATABASE test_asof_join;
|
||||
use test_asof_join;
|
||||
|
||||
|
|
@ -42,5 +43,38 @@ INSERT INTO user_status VALUES
|
|||
(102, '2024-01-01 13:00:00', 'VIP', 800),
|
||||
(102, '2024-01-02 12:00:00', 'PREMIUM', 950);
|
||||
|
||||
-- Test 1: ASOF INNER JOIN
|
||||
SELECT o.order_id, o.user_id, o.order_time, us.status_time, us.status, us.credit_score
|
||||
FROM orders o
|
||||
ASOF INNER JOIN user_status us ON o.user_id = us.user_id AND o.order_time >= us.status_time
|
||||
ORDER BY o.order_id;
|
||||
|
||||
-- Test 2: ASOF LEFT OUTER JOIN
|
||||
SELECT o.order_id, o.user_id, o.order_time, us.status_time, us.status, us.credit_score
|
||||
FROM orders o
|
||||
ASOF LEFT JOIN user_status us ON o.user_id = us.user_id AND o.order_time >= us.status_time
|
||||
ORDER BY o.order_id;
|
||||
|
||||
-- Test 3: ASOF JOIN with different temporal operators
|
||||
SELECT o.order_id, o.user_id, o.order_time, us.status_time, us.status, us.credit_score
|
||||
FROM orders o
|
||||
ASOF INNER JOIN user_status us ON o.user_id = us.user_id AND o.order_time > us.status_time
|
||||
ORDER BY o.order_id;
|
||||
|
||||
-- Test 4: ASOF JOIN with WHERE filter
|
||||
SELECT o.order_id, o.user_id, o.order_time, us.status_time, us.status, us.credit_score
|
||||
FROM orders o
|
||||
ASOF INNER JOIN user_status us ON o.user_id = us.user_id AND o.order_time >= us.status_time
|
||||
WHERE o.amount > 150
|
||||
ORDER BY o.order_id;
|
||||
|
||||
-- Test 5: ASOF JOIN with aggregation
|
||||
SELECT o.user_id, COUNT(*) as order_count, MAX(us.credit_score) as max_credit_score
|
||||
FROM orders o
|
||||
ASOF LEFT JOIN user_status us ON o.user_id = us.user_id AND o.order_time >= us.status_time
|
||||
GROUP BY o.user_id
|
||||
ORDER BY o.user_id;
|
||||
|
||||
-- Test 6: Verify ASOF JOIN in execution plan
|
||||
function: assert_explain_contains('SELECT * FROM orders o ASOF LEFT JOIN user_status us ON o.user_id = us.user_id AND o.order_time >= us.status_time', 'ASOF LEFT OUTER JOIN')
|
||||
|
||||
|
|
|
|||
|
|
@ -0,0 +1,136 @@
|
|||
-- name: test_asof_join_basic
|
||||
|
||||
DROP DATABASE IF EXISTS test_asof_join_basic;
|
||||
CREATE DATABASE test_asof_join_basic;
|
||||
use test_asof_join_basic;
|
||||
|
||||
-- Test tables for different temporal types
|
||||
CREATE TABLE orders_datetime (
|
||||
`order_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`order_time` datetime NOT NULL,
|
||||
`amount` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`order_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
CREATE TABLE prices_datetime (
|
||||
`product_id` int(11) NOT NULL,
|
||||
`price_time` datetime NOT NULL,
|
||||
`price` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`product_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
CREATE TABLE orders_date (
|
||||
`order_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`order_date` date NOT NULL,
|
||||
`amount` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`order_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
CREATE TABLE prices_date (
|
||||
`product_id` int(11) NOT NULL,
|
||||
`price_date` date NOT NULL,
|
||||
`price` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`product_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
CREATE TABLE orders_bigint (
|
||||
`order_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`order_timestamp` bigint NOT NULL,
|
||||
`amount` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`order_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
CREATE TABLE prices_bigint (
|
||||
`product_id` int(11) NOT NULL,
|
||||
`price_timestamp` bigint NOT NULL,
|
||||
`price` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`product_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
-- Insert test data for datetime type
|
||||
INSERT INTO orders_datetime VALUES
|
||||
(1, 101, '2024-01-01 10:00:00', 100.00),
|
||||
(2, 101, '2024-01-01 15:30:00', 200.00),
|
||||
(3, 102, '2024-01-01 11:00:00', 150.00),
|
||||
(4, 102, '2024-01-01 16:00:00', 300.00),
|
||||
(5, 101, '2024-01-02 09:00:00', 250.00),
|
||||
(6, 102, '2024-01-02 14:00:00', 180.00);
|
||||
|
||||
INSERT INTO prices_datetime VALUES
|
||||
(101, '2024-01-01 08:00:00', 95.00),
|
||||
(101, '2024-01-01 14:00:00', 105.00),
|
||||
(101, '2024-01-02 08:00:00', 110.00),
|
||||
(102, '2024-01-01 09:00:00', 90.00),
|
||||
(102, '2024-01-01 13:00:00', 100.00),
|
||||
(102, '2024-01-02 12:00:00', 115.00);
|
||||
|
||||
-- Insert test data for date type
|
||||
INSERT INTO orders_date VALUES
|
||||
(1, 201, '2024-01-01', 100.00),
|
||||
(2, 201, '2024-01-02', 200.00),
|
||||
(3, 202, '2024-01-01', 150.00),
|
||||
(4, 202, '2024-01-03', 300.00);
|
||||
|
||||
INSERT INTO prices_date VALUES
|
||||
(201, '2023-12-31', 95.00),
|
||||
(201, '2024-01-01', 105.00),
|
||||
(202, '2023-12-31', 90.00),
|
||||
(202, '2024-01-02', 100.00);
|
||||
|
||||
-- Insert test data for bigint timestamp type
|
||||
INSERT INTO orders_bigint VALUES
|
||||
(1, 301, 1704067200, 100.00), -- 2024-01-01 10:00:00
|
||||
(2, 301, 1704084600, 200.00), -- 2024-01-01 15:30:00
|
||||
(3, 302, 1704070800, 150.00), -- 2024-01-01 11:00:00
|
||||
(4, 302, 1704086400, 300.00); -- 2024-01-01 16:00:00
|
||||
|
||||
INSERT INTO prices_bigint VALUES
|
||||
(301, 1704060000, 95.00), -- 2024-01-01 08:00:00
|
||||
(301, 1704081600, 105.00), -- 2024-01-01 14:00:00
|
||||
(302, 1704063600, 90.00), -- 2024-01-01 09:00:00
|
||||
(302, 1704078000, 100.00); -- 2024-01-01 13:00:00
|
||||
|
||||
-- Test 1: ASOF INNER JOIN with datetime
|
||||
SELECT o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_datetime o
|
||||
ASOF INNER JOIN prices_datetime p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
ORDER BY o.order_id;
|
||||
|
||||
-- Test 2: ASOF LEFT OUTER JOIN with datetime
|
||||
SELECT o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_datetime o
|
||||
ASOF LEFT JOIN prices_datetime p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
ORDER BY o.order_id;
|
||||
|
||||
-- Test 3: ASOF INNER JOIN with date
|
||||
SELECT o.order_id, o.user_id, o.order_date, p.price_date, p.price
|
||||
FROM orders_date o
|
||||
ASOF INNER JOIN prices_date p ON o.user_id = p.product_id AND o.order_date >= p.price_date
|
||||
ORDER BY o.order_id;
|
||||
|
||||
-- Test 4: ASOF LEFT OUTER JOIN with date
|
||||
SELECT o.order_id, o.user_id, o.order_date, p.price_date, p.price
|
||||
FROM orders_date o
|
||||
ASOF LEFT JOIN prices_date p ON o.user_id = p.product_id AND o.order_date >= p.price_date
|
||||
ORDER BY o.order_id;
|
||||
|
||||
-- Test 5: ASOF INNER JOIN with bigint timestamp
|
||||
SELECT o.order_id, o.user_id, o.order_timestamp, p.price_timestamp, p.price
|
||||
FROM orders_bigint o
|
||||
ASOF INNER JOIN prices_bigint p ON o.user_id = p.product_id AND o.order_timestamp >= p.price_timestamp
|
||||
ORDER BY o.order_id;
|
||||
|
||||
-- Test 6: ASOF LEFT OUTER JOIN with bigint timestamp
|
||||
SELECT o.order_id, o.user_id, o.order_timestamp, p.price_timestamp, p.price
|
||||
FROM orders_bigint o
|
||||
ASOF LEFT JOIN prices_bigint p ON o.user_id = p.product_id AND o.order_timestamp >= p.price_timestamp
|
||||
ORDER BY o.order_id;
|
||||
|
|
@ -0,0 +1,347 @@
|
|||
-- name: test_asof_join_complex
|
||||
|
||||
DROP DATABASE IF EXISTS test_asof_join_complex;
|
||||
CREATE DATABASE test_asof_join_complex;
|
||||
use test_asof_join_complex;
|
||||
|
||||
CREATE TABLE transactions (
|
||||
`txn_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`account_id` varchar(20) NOT NULL,
|
||||
`txn_time` datetime NOT NULL,
|
||||
`amount` decimal(15,2) NOT NULL,
|
||||
`txn_type` varchar(20) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`txn_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
CREATE TABLE exchange_rates (
|
||||
`currency_pair` varchar(10) NOT NULL,
|
||||
`rate_time` datetime NOT NULL,
|
||||
`exchange_rate` decimal(10,6) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`currency_pair`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
CREATE TABLE user_profiles (
|
||||
`user_id` int(11) NOT NULL,
|
||||
`profile_time` datetime NOT NULL,
|
||||
`risk_level` varchar(20) NOT NULL,
|
||||
`credit_score` int(11) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`user_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
CREATE TABLE market_events (
|
||||
`event_id` int(11) NOT NULL,
|
||||
`event_time` datetime NOT NULL,
|
||||
`currency` varchar(20) NOT NULL,
|
||||
`event_type` varchar(30) NOT NULL,
|
||||
`market_impact` decimal(5,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`event_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
-- Insert test data
|
||||
INSERT INTO transactions VALUES
|
||||
(1, 101, 'USD', '2024-01-01 10:00:00', 1000.00, 'DEPOSIT'),
|
||||
(2, 101, 'USD', '2024-01-01 15:30:00', 500.00, 'WITHDRAWAL'),
|
||||
(3, 102, 'EUR', '2024-01-01 11:00:00', 2000.00, 'DEPOSIT'),
|
||||
(4, 102, 'EUR', '2024-01-01 16:00:00', 800.00, 'WITHDRAWAL'),
|
||||
(5, 101, 'USD', '2024-01-02 09:00:00', 1500.00, 'DEPOSIT'),
|
||||
(6, 102, 'EUR', '2024-01-02 14:00:00', 1200.00, 'WITHDRAWAL');
|
||||
|
||||
INSERT INTO exchange_rates VALUES
|
||||
('USD', '2024-01-01 08:00:00', 0.900000),
|
||||
('USD', '2024-01-01 14:00:00', 0.950000),
|
||||
('USD', '2024-01-02 08:00:00', 0.960000),
|
||||
('EUR', '2024-01-01 08:00:00', 1.100000),
|
||||
('EUR', '2024-01-01 13:00:00', 1.120000),
|
||||
('EUR', '2024-01-02 08:00:00', 1.130000);
|
||||
|
||||
INSERT INTO user_profiles VALUES
|
||||
(101, '2024-01-01 09:00:00', 'LOW', 760),
|
||||
(101, '2024-01-01 11:30:00', 'MEDIUM', 805),
|
||||
(101, '2024-01-02 08:00:00', 'HIGH', 860),
|
||||
(102, '2024-01-01 10:00:00', 'LOW', 710),
|
||||
(102, '2024-01-01 12:30:00', 'MEDIUM', 770),
|
||||
(102, '2024-01-02 12:00:00', 'HIGH', 810);
|
||||
|
||||
INSERT INTO market_events VALUES
|
||||
(1, '2024-01-01 08:00:00', 'USD', 'MARKET_OPEN', 0.50),
|
||||
(2, '2024-01-01 12:00:00', 'USD', 'FED_ANNOUNCEMENT', 1.20),
|
||||
(3, '2024-01-01 16:00:00', 'USD', 'MARKET_CLOSE', -0.30),
|
||||
(4, '2024-01-02 08:30:00', 'EUR', 'MARKET_OPEN', 0.25);
|
||||
|
||||
-- Test 1: Multiple ASOF JOINs with different temporal conditions
|
||||
SELECT *
|
||||
FROM transactions t
|
||||
ASOF LEFT JOIN exchange_rates er ON t.account_id = er.currency_pair AND t.txn_time >= er.rate_time
|
||||
ASOF LEFT JOIN user_profiles up ON t.user_id = up.user_id AND t.txn_time >= up.profile_time
|
||||
ASOF LEFT JOIN market_events me ON t.account_id = me.currency AND t.txn_time >= me.event_time
|
||||
WHERE t.amount > 1000
|
||||
ORDER BY t.txn_id;
|
||||
|
||||
-- Test 2: ASOF JOIN with time functions wrapping both sides of temporal predicates
|
||||
SELECT
|
||||
t.txn_id,
|
||||
t.user_id,
|
||||
DATE(t.txn_time) as txn_date,
|
||||
HOUR(t.txn_time) as txn_hour,
|
||||
er.exchange_rate,
|
||||
up.risk_level
|
||||
FROM transactions t
|
||||
ASOF INNER JOIN exchange_rates er
|
||||
ON t.account_id = er.currency_pair
|
||||
AND date_trunc('hour', t.txn_time) >= date_add(er.rate_time, INTERVAL 1 HOUR)
|
||||
ASOF INNER JOIN user_profiles up
|
||||
ON t.user_id = up.user_id
|
||||
AND date_trunc('minute', t.txn_time) >= date_add(up.profile_time, INTERVAL 30 MINUTE)
|
||||
WHERE DATE(t.txn_time) = '2024-01-01'
|
||||
ORDER BY t.txn_id;
|
||||
|
||||
-- Test 3: ASOF JOIN with complex WHERE conditions
|
||||
SELECT
|
||||
t.txn_id,
|
||||
t.user_id,
|
||||
t.txn_time,
|
||||
t.amount,
|
||||
er.exchange_rate,
|
||||
up.risk_level,
|
||||
up.credit_score
|
||||
FROM transactions t
|
||||
ASOF INNER JOIN exchange_rates er ON t.account_id = er.currency_pair AND t.txn_time >= er.rate_time
|
||||
ASOF INNER JOIN user_profiles up ON t.user_id = up.user_id AND t.txn_time >= up.profile_time
|
||||
WHERE t.amount > 500
|
||||
AND up.credit_score > 750
|
||||
AND er.exchange_rate > 0.85
|
||||
ORDER BY t.txn_id;
|
||||
|
||||
-- Test 4: ASOF JOIN with aggregation
|
||||
SELECT
|
||||
t.user_id,
|
||||
COUNT(*) as txn_count,
|
||||
SUM(t.amount) as total_amount,
|
||||
AVG(er.exchange_rate) as avg_rate,
|
||||
MAX(up.credit_score) as max_credit_score
|
||||
FROM transactions t
|
||||
ASOF INNER JOIN exchange_rates er ON t.account_id = er.currency_pair AND t.txn_time >= er.rate_time
|
||||
ASOF INNER JOIN user_profiles up ON t.user_id = up.user_id AND t.txn_time >= up.profile_time
|
||||
GROUP BY t.user_id
|
||||
ORDER BY t.user_id;
|
||||
|
||||
-- Test 5: ASOF JOIN with subquery
|
||||
SELECT
|
||||
t.txn_id,
|
||||
t.user_id,
|
||||
t.txn_time,
|
||||
t.amount,
|
||||
er.exchange_rate,
|
||||
up.risk_level
|
||||
FROM transactions t
|
||||
ASOF INNER JOIN exchange_rates er ON t.account_id = er.currency_pair AND t.txn_time >= er.rate_time
|
||||
ASOF INNER JOIN user_profiles up ON t.user_id = up.user_id AND t.txn_time >= up.profile_time
|
||||
WHERE t.user_id IN (
|
||||
SELECT DISTINCT user_id
|
||||
FROM transactions
|
||||
WHERE amount > 1000
|
||||
)
|
||||
ORDER BY t.txn_id;
|
||||
|
||||
-- Test 6: ASOF JOIN with different temporal operators
|
||||
SELECT
|
||||
t.txn_id,
|
||||
t.user_id,
|
||||
t.txn_time,
|
||||
t.amount,
|
||||
er.exchange_rate,
|
||||
up.risk_level
|
||||
FROM transactions t
|
||||
ASOF INNER JOIN exchange_rates er ON t.account_id = er.currency_pair AND t.txn_time > er.rate_time
|
||||
ASOF INNER JOIN user_profiles up ON t.user_id = up.user_id AND t.txn_time <= up.profile_time
|
||||
WHERE t.txn_type = 'DEPOSIT'
|
||||
ORDER BY t.txn_id;
|
||||
|
||||
-- Test 7: ASOF JOIN with more function-wrapped temporal predicates (both sides)
|
||||
SELECT
|
||||
t.txn_id,
|
||||
t.user_id,
|
||||
t.txn_time,
|
||||
er.exchange_rate,
|
||||
me.event_type
|
||||
FROM transactions t
|
||||
ASOF INNER JOIN exchange_rates er
|
||||
ON t.account_id = er.currency_pair
|
||||
AND date_trunc('day', t.txn_time) >= date_add(er.rate_time, INTERVAL 0 DAY)
|
||||
ASOF INNER JOIN market_events me
|
||||
ON t.account_id = me.currency
|
||||
AND date_add(t.txn_time, INTERVAL -30 MINUTE) >= date_trunc('minute', me.event_time)
|
||||
WHERE t.txn_type = 'WITHDRAWAL'
|
||||
ORDER BY t.txn_id;
|
||||
|
||||
-- Test 8: DATETIME(6) microsecond precision with non-equality temporal predicates
|
||||
CREATE TABLE events_dt6 (
|
||||
`event_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`event_time` datetime NOT NULL
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(`event_id`) PROPERTIES ("replication_num" = "1");
|
||||
|
||||
CREATE TABLE snapshots_dt6 (
|
||||
`user_id` int(11) NOT NULL,
|
||||
`snapshot_time` datetime NOT NULL,
|
||||
`snapshot_data` varchar(50) NOT NULL
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(`user_id`) PROPERTIES ("replication_num" = "1");
|
||||
|
||||
-- Multiple snapshots per equi key (user 701) at ms precision
|
||||
INSERT INTO snapshots_dt6 VALUES
|
||||
(701,'2024-01-01 10:00:00.100000','S1'),
|
||||
(701,'2024-01-01 10:00:00.300000','S2'),
|
||||
(701,'2024-01-01 10:00:00.900000','S3');
|
||||
|
||||
-- Multiple snapshots per equi key (user 702) at us precision
|
||||
INSERT INTO snapshots_dt6 VALUES
|
||||
(702,'2024-01-01 10:00:00.123456','T1'),
|
||||
(702,'2024-01-01 10:00:00.223456','T2'),
|
||||
(702,'2024-01-01 10:00:00.323456','T3');
|
||||
|
||||
-- Events around boundaries, ensuring multiple candidates exist per user
|
||||
INSERT INTO events_dt6 VALUES
|
||||
(1,701,'2024-01-01 10:00:00.350000'), -- between S2 and S3 (>= should pick S2)
|
||||
(2,701,'2024-01-01 10:00:00.100001'), -- just after S1
|
||||
(3,702,'2024-01-01 10:00:00.323455'), -- just before T3 (>= picks T2)
|
||||
(4,702,'2024-01-01 10:00:00.500000'), -- after T3 (>= picks T3)
|
||||
(5,702,'2024-01-01 10:00:00.123456'), -- equal to T1 boundary
|
||||
(6,702,'2024-01-01 09:59:59.999999'); -- before first snapshot (no match for >=)
|
||||
|
||||
-- 8a: INNER with >= on DATETIME(6)
|
||||
SELECT /* DT6-INNER-GE */ e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events_dt6 e
|
||||
ASOF INNER JOIN snapshots_dt6 s ON e.user_id = s.user_id AND e.event_time >= s.snapshot_time
|
||||
ORDER BY e.event_id;
|
||||
|
||||
-- 8b: LEFT with >= on DATETIME(6) (retains before-first row id=6)
|
||||
SELECT /* DT6-LEFT-GE */ e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events_dt6 e
|
||||
ASOF LEFT JOIN snapshots_dt6 s ON e.user_id = s.user_id AND e.event_time >= s.snapshot_time
|
||||
ORDER BY e.event_id;
|
||||
|
||||
-- 8c: INNER with < on DATETIME(6) (choose nearest future snapshot if implementation supports)
|
||||
SELECT /* DT6-INNER-LT */ e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events_dt6 e
|
||||
ASOF INNER JOIN snapshots_dt6 s ON e.user_id = s.user_id AND e.event_time < s.snapshot_time
|
||||
WHERE e.user_id IN (701,702)
|
||||
ORDER BY e.event_id;
|
||||
|
||||
-- 8d: compare with date
|
||||
SELECT /* DT6-INNER-GE */ e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events_dt6 e
|
||||
ASOF INNER JOIN snapshots_dt6 s ON e.user_id = s.user_id AND cast(e.event_time as date) >= cast(s.snapshot_time as date)
|
||||
ORDER BY e.event_id;
|
||||
|
||||
|
||||
-- Test 9: test asof join with other join conjunct
|
||||
CREATE TABLE orders (
|
||||
`id` int(11) NULL COMMENT "订单ID",
|
||||
`product` varchar(50) NULL COMMENT "产品名称",
|
||||
`order_time` datetime NULL COMMENT "订单时间",
|
||||
`quantity` int(11) NULL COMMENT "订单数量",
|
||||
`max_price` decimal(10, 2) NULL COMMENT "最大价格"
|
||||
) ENGINE=OLAP
|
||||
DUPLICATE KEY(`id`)
|
||||
DISTRIBUTED BY HASH(`id`) BUCKETS 1
|
||||
PROPERTIES (
|
||||
"replication_num" = "1"
|
||||
);
|
||||
|
||||
CREATE TABLE prices (
|
||||
`id` int(11) NULL COMMENT "价格记录ID",
|
||||
`product` varchar(50) NULL COMMENT "产品名称",
|
||||
`price_time` datetime NULL COMMENT "价格时间",
|
||||
`price` decimal(10, 2) NULL COMMENT "价格",
|
||||
`volume` int(11) NULL COMMENT "成交量"
|
||||
) ENGINE=OLAP
|
||||
DUPLICATE KEY(`id`)
|
||||
DISTRIBUTED BY HASH(`id`) BUCKETS 1
|
||||
PROPERTIES (
|
||||
"replication_num" = "1"
|
||||
);
|
||||
|
||||
INSERT INTO orders VALUES
|
||||
(1, 'A', '2024-01-01 10:00:00', 100, 100.00),
|
||||
(2, 'A', '2024-01-01 11:30:00', 150, 60.00),
|
||||
(3, 'A', '2024-01-01 14:00:00', 200, 50.00),
|
||||
(4, 'A', '2024-01-01 16:00:00', 50, 120.00),
|
||||
(5, 'B', '2024-01-01 09:30:00', 400, 25.00),
|
||||
(6, 'B', '2024-01-01 12:00:00', 100, 30.00),
|
||||
(7, 'B', '2024-01-01 15:30:00', 80, 40.00),
|
||||
(8, 'C', '2024-01-01 13:00:00', 500, 20.00),
|
||||
(9, 'C', '2024-01-01 17:00:00', 160, 50.00);
|
||||
|
||||
INSERT INTO prices VALUES
|
||||
(1, 'A', '2024-01-01 08:00:00', 80.00, 1000),
|
||||
(16, 'A', '2024-01-01 09:00:00', 60.00, 1000),
|
||||
(2, 'A', '2024-01-01 09:30:00', 85.00, 1000),
|
||||
(3, 'A', '2024-01-01 10:30:00', 53.33, 1500),
|
||||
(4, 'A', '2024-01-01 12:00:00', 40.00, 1200),
|
||||
(5, 'A', '2024-01-01 15:00:00', 120.00, 800),
|
||||
(6, 'A', '2024-01-01 18:00:00', 90.00, 900),
|
||||
(7, 'B', '2024-01-01 08:30:00', 20.00, 2000),
|
||||
(8, 'B', '2024-01-01 10:00:00', 30.00, 1800),
|
||||
(9, 'B', '2024-01-01 11:00:00', 80.00, 1500),
|
||||
(17, 'B', '2024-01-01 13:00:00', 50.00, 2000),
|
||||
(10, 'B', '2024-01-01 14:00:00', 100.00, 1400),
|
||||
(11, 'B', '2024-01-01 16:30:00', 35.00, 1400),
|
||||
(12, 'C', '2024-01-01 09:00:00', 16.00, 3000),
|
||||
(13, 'C', '2024-01-01 12:30:00', 18.00, 2800),
|
||||
(18, 'C', '2024-01-01 15:00:00', 30.00, 3000),
|
||||
(14, 'C', '2024-01-01 16:00:00', 50.00, 1000),
|
||||
(15, 'C', '2024-01-01 19:00:00', 45.00, 900);
|
||||
|
||||
SELECT
|
||||
o.id AS order_id,
|
||||
o.product,
|
||||
o.order_time,
|
||||
o.quantity,
|
||||
o.max_price,
|
||||
p.id AS price_id,
|
||||
p.product AS price_product,
|
||||
p.price_time,
|
||||
p.price,
|
||||
p.volume,
|
||||
o.quantity * p.price AS total_amount,
|
||||
TIMESTAMPDIFF(MINUTE, p.price_time, o.order_time) AS time_diff_min,
|
||||
CASE
|
||||
WHEN p.id IS NULL THEN 'NO_MATCH'
|
||||
ELSE 'MATCHED'
|
||||
END AS match_status
|
||||
FROM orders o
|
||||
ASOF LEFT JOIN prices p
|
||||
ON o.product = p.product
|
||||
AND o.order_time >= p.price_time
|
||||
AND CAST(o.quantity * p.price AS BIGINT) = CAST(8000 AS BIGINT)
|
||||
ORDER BY o.product, o.order_time;
|
||||
|
||||
SELECT
|
||||
o.id AS order_id,
|
||||
o.product,
|
||||
o.order_time,
|
||||
o.quantity,
|
||||
o.max_price,
|
||||
p.id AS price_id,
|
||||
p.product AS price_product,
|
||||
p.price_time,
|
||||
p.price,
|
||||
p.volume,
|
||||
o.quantity * p.price AS total_amount,
|
||||
TIMESTAMPDIFF(MINUTE, p.price_time, o.order_time) AS time_diff_min,
|
||||
CASE
|
||||
WHEN p.id IS NULL THEN 'NO_MATCH'
|
||||
ELSE 'MATCHED'
|
||||
END AS match_status
|
||||
FROM orders o
|
||||
ASOF JOIN prices p
|
||||
ON o.product = p.product
|
||||
AND o.order_time >= p.price_time
|
||||
AND CAST(o.quantity * p.price AS BIGINT) = CAST(8000 AS BIGINT)
|
||||
ORDER BY o.product, o.order_time;
|
||||
|
||||
|
|
@ -0,0 +1,300 @@
|
|||
-- name: test_asof_join_edge_cases
|
||||
|
||||
DROP DATABASE IF EXISTS test_asof_join_edge_cases;
|
||||
CREATE DATABASE test_asof_join_edge_cases;
|
||||
use test_asof_join_edge_cases;
|
||||
|
||||
CREATE TABLE events (
|
||||
`event_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`event_time` datetime NOT NULL,
|
||||
`event_data` varchar(50) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`event_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
CREATE TABLE snapshots (
|
||||
`user_id` int(11) NOT NULL,
|
||||
`snapshot_time` datetime NOT NULL,
|
||||
`snapshot_data` varchar(50) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`user_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
CREATE TABLE events_with_nulls (
|
||||
`event_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`event_time` datetime NULL,
|
||||
`event_data` varchar(50) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`event_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
CREATE TABLE snapshots_with_nulls (
|
||||
`user_id` int(11) NOT NULL,
|
||||
`snapshot_time` datetime NULL,
|
||||
`snapshot_data` varchar(50) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`user_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
-- Insert test data for normal cases
|
||||
INSERT INTO events VALUES
|
||||
(1, 101, '2024-01-01 10:00:00', 'EVENT_A'),
|
||||
(2, 101, '2024-01-01 15:30:00', 'EVENT_B'),
|
||||
(3, 102, '2024-01-01 11:00:00', 'EVENT_C'),
|
||||
(4, 102, '2024-01-01 16:00:00', 'EVENT_D'),
|
||||
(5, 103, '2024-01-01 12:00:00', 'EVENT_E');
|
||||
|
||||
INSERT INTO snapshots VALUES
|
||||
(101, '2024-01-01 08:00:00', 'SNAPSHOT_1'),
|
||||
(101, '2024-01-01 14:00:00', 'SNAPSHOT_2'),
|
||||
(102, '2024-01-01 09:00:00', 'SNAPSHOT_3'),
|
||||
(102, '2024-01-01 13:00:00', 'SNAPSHOT_4'),
|
||||
(103, '2024-01-01 11:30:00', 'SNAPSHOT_5');
|
||||
|
||||
-- Ensure exact temporal match exists for user 101 at 10:00:00 (for Test 3)
|
||||
INSERT INTO snapshots VALUES (101, '2024-01-01 10:00:00', 'SNAPSHOT_1_10AM');
|
||||
|
||||
-- Insert test data with NULL temporal values
|
||||
INSERT INTO events_with_nulls VALUES
|
||||
(1, 201, '2024-01-01 10:00:00', 'EVENT_A'),
|
||||
(2, 201, NULL, 'EVENT_B'),
|
||||
(3, 202, '2024-01-01 11:00:00', 'EVENT_C'),
|
||||
(4, 202, '2024-01-01 16:00:00', 'EVENT_D'),
|
||||
(5, 203, NULL, 'EVENT_E');
|
||||
|
||||
INSERT INTO snapshots_with_nulls VALUES
|
||||
(201, '2024-01-01 08:00:00', 'SNAPSHOT_1'),
|
||||
(201, NULL, 'SNAPSHOT_2'),
|
||||
(202, '2024-01-01 09:00:00', 'SNAPSHOT_3'),
|
||||
(202, '2024-01-01 13:00:00', 'SNAPSHOT_4'),
|
||||
(203, '2024-01-01 11:30:00', 'SNAPSHOT_5');
|
||||
|
||||
-- Test 1: INNER with e.event_time < s.snapshot_time (returns only events earlier than first future snapshot)
|
||||
SELECT e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events e
|
||||
ASOF INNER JOIN snapshots s ON e.user_id = s.user_id AND e.event_time < s.snapshot_time
|
||||
WHERE e.user_id = 101
|
||||
ORDER BY e.event_id;
|
||||
|
||||
-- Test 2: LEFT with e.event_time < s.snapshot_time (keeps probe rows; unmatched get NULL build)
|
||||
SELECT e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events e
|
||||
ASOF LEFT JOIN snapshots s ON e.user_id = s.user_id AND e.event_time < s.snapshot_time
|
||||
WHERE e.user_id = 101
|
||||
ORDER BY e.event_id;
|
||||
|
||||
|
||||
-- Test 4: Multiple temporal matches (should return closest match)
|
||||
SELECT e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events e
|
||||
ASOF INNER JOIN snapshots s ON e.user_id = s.user_id AND e.event_time >= s.snapshot_time
|
||||
ORDER BY e.event_id;
|
||||
|
||||
-- Test 5: No equi-join matches (should return no rows)
|
||||
SELECT e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events e
|
||||
ASOF INNER JOIN snapshots s ON e.user_id = s.user_id + 1000 AND e.event_time >= s.snapshot_time
|
||||
ORDER BY e.event_id;
|
||||
|
||||
-- Test 6: Empty build table (should return no rows for INNER JOIN)
|
||||
CREATE TABLE empty_snapshots (
|
||||
`user_id` int(11) NOT NULL,
|
||||
`snapshot_time` datetime NOT NULL,
|
||||
`snapshot_data` varchar(50) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`user_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
SELECT e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events e
|
||||
ASOF INNER JOIN empty_snapshots s ON e.user_id = s.user_id AND e.event_time >= s.snapshot_time
|
||||
ORDER BY e.event_id;
|
||||
|
||||
-- Test 7: Empty build table (should return probe rows with NULL for LEFT JOIN)
|
||||
SELECT e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events e
|
||||
ASOF LEFT JOIN empty_snapshots s ON e.user_id = s.user_id AND e.event_time >= s.snapshot_time
|
||||
ORDER BY e.event_id;
|
||||
|
||||
-- Test 8a: Probe temporal NULL (INNER) -> rows with e.event_time IS NULL are dropped
|
||||
SELECT e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events_with_nulls e
|
||||
ASOF INNER JOIN snapshots_with_nulls s ON e.user_id = s.user_id AND e.event_time >= s.snapshot_time
|
||||
WHERE e.event_time IS NULL
|
||||
ORDER BY e.event_id;
|
||||
|
||||
-- Test 8b: Probe temporal NULL (LEFT) -> retain probe rows, build side NULL
|
||||
SELECT e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events_with_nulls e
|
||||
ASOF LEFT JOIN snapshots_with_nulls s ON e.user_id = s.user_id AND e.event_time >= s.snapshot_time
|
||||
WHERE e.event_time IS NULL
|
||||
ORDER BY e.event_id;
|
||||
|
||||
-- Test 9: NULL temporal values in build table (build NULL snapshot_time should be ignored)
|
||||
SELECT e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events e
|
||||
ASOF INNER JOIN snapshots_with_nulls s ON e.user_id = s.user_id AND e.event_time >= s.snapshot_time
|
||||
ORDER BY e.event_id;
|
||||
|
||||
-- Test 10: Both tables have NULL temporal values (LEFT retains probe side)
|
||||
SELECT e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events_with_nulls e
|
||||
ASOF LEFT JOIN snapshots_with_nulls s ON e.user_id = s.user_id AND e.event_time >= s.snapshot_time
|
||||
ORDER BY e.event_id;
|
||||
|
||||
-- Test 11: Large temporal gaps
|
||||
CREATE TABLE events_large_gap (
|
||||
`event_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`event_time` datetime NOT NULL,
|
||||
`event_data` varchar(50) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`event_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
CREATE TABLE snapshots_large_gap (
|
||||
`user_id` int(11) NOT NULL,
|
||||
`snapshot_time` datetime NOT NULL,
|
||||
`snapshot_data` varchar(50) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`user_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
INSERT INTO events_large_gap VALUES
|
||||
(1, 301, '2024-01-01 10:00:00', 'EVENT_A'),
|
||||
(2, 301, '2024-01-01 15:30:00', 'EVENT_B'),
|
||||
(3, 302, '2024-01-01 11:00:00', 'EVENT_C');
|
||||
|
||||
INSERT INTO snapshots_large_gap VALUES
|
||||
(301, '2023-12-01 08:00:00', 'SNAPSHOT_1'),
|
||||
(301, '2023-12-15 14:00:00', 'SNAPSHOT_2'),
|
||||
(302, '2023-11-01 09:00:00', 'SNAPSHOT_3');
|
||||
|
||||
SELECT e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events_large_gap e
|
||||
ASOF INNER JOIN snapshots_large_gap s ON e.user_id = s.user_id AND e.event_time >= s.snapshot_time
|
||||
ORDER BY e.event_id;
|
||||
|
||||
-- Test 12: Duplicate snapshot_time for the same user (tie-breaking on equal times)
|
||||
CREATE TABLE events_dups (
|
||||
`event_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`event_time` datetime NOT NULL
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(`event_id`) PROPERTIES ("replication_num" = "1");
|
||||
CREATE TABLE snapshots_dups (
|
||||
`user_id` int(11) NOT NULL,
|
||||
`snapshot_time` datetime NOT NULL,
|
||||
`snapshot_data` varchar(50) NOT NULL
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(`user_id`) PROPERTIES ("replication_num" = "1");
|
||||
INSERT INTO events_dups VALUES (1,401,'2024-01-01 10:00:00');
|
||||
INSERT INTO snapshots_dups VALUES
|
||||
(401,'2024-01-01 10:00:00','SNAP_D1'),
|
||||
(401,'2024-01-01 10:00:00','SNAP_D2');
|
||||
SELECT e.event_id, e.user_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events_dups e
|
||||
ASOF INNER JOIN snapshots_dups s ON e.user_id=s.user_id AND e.event_time>=s.snapshot_time
|
||||
ORDER BY e.event_id;
|
||||
|
||||
-- Test 13: Operator edge behavior (<, <=, >, >=) at equality boundary
|
||||
CREATE TABLE events_ops (
|
||||
`event_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`event_time` datetime NOT NULL
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(`event_id`) PROPERTIES ("replication_num" = "1");
|
||||
CREATE TABLE snapshots_ops (
|
||||
`user_id` int(11) NOT NULL,
|
||||
`snapshot_time` datetime NOT NULL
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(`user_id`) PROPERTIES ("replication_num" = "1");
|
||||
INSERT INTO events_ops VALUES (1,501,'2024-01-01 12:00:00');
|
||||
INSERT INTO snapshots_ops VALUES (501,'2024-01-01 12:00:00');
|
||||
-- < should NOT match on equality
|
||||
SELECT /* OP-< */ e.event_id, e.event_time, s.snapshot_time
|
||||
FROM events_ops e ASOF INNER JOIN snapshots_ops s
|
||||
ON e.user_id=s.user_id AND e.event_time < s.snapshot_time;
|
||||
-- <= should match on equality
|
||||
SELECT /* OP-<= */ e.event_id, e.event_time, s.snapshot_time
|
||||
FROM events_ops e ASOF INNER JOIN snapshots_ops s
|
||||
ON e.user_id=s.user_id AND e.event_time <= s.snapshot_time;
|
||||
-- > should NOT match on equality
|
||||
SELECT /* OP-> */ e.event_id, e.event_time, s.snapshot_time
|
||||
FROM events_ops e ASOF INNER JOIN snapshots_ops s
|
||||
ON e.user_id=s.user_id AND e.event_time > s.snapshot_time;
|
||||
-- >= should match on equality
|
||||
SELECT /* OP->= */ e.event_id, e.event_time, s.snapshot_time
|
||||
FROM events_ops e ASOF INNER JOIN snapshots_ops s
|
||||
ON e.user_id=s.user_id AND e.event_time >= s.snapshot_time;
|
||||
|
||||
-- Test 14: Out-of-order build insertion should still select by time correctly
|
||||
CREATE TABLE events_ooo (
|
||||
`event_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`event_time` datetime NOT NULL
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(`event_id`) PROPERTIES ("replication_num" = "1");
|
||||
CREATE TABLE snapshots_ooo (
|
||||
`user_id` int(11) NOT NULL,
|
||||
`snapshot_time` datetime NOT NULL,
|
||||
`snapshot_data` varchar(50) NOT NULL
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(`user_id`) PROPERTIES ("replication_num" = "1");
|
||||
INSERT INTO events_ooo VALUES (1,551,'2024-01-01 10:30:00');
|
||||
-- Insert later time first, then earlier time
|
||||
INSERT INTO snapshots_ooo VALUES (551,'2024-01-01 10:00:00','S10'),(551,'2024-01-01 09:00:00','S09');
|
||||
SELECT e.event_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events_ooo e ASOF INNER JOIN snapshots_ooo s
|
||||
ON e.user_id=s.user_id AND e.event_time>=s.snapshot_time
|
||||
ORDER BY e.event_id;
|
||||
|
||||
-- Test 15: Before-first and after-last boundaries
|
||||
CREATE TABLE events_bounds (
|
||||
`event_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`event_time` datetime NOT NULL
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(`event_id`) PROPERTIES ("replication_num" = "1");
|
||||
CREATE TABLE snapshots_bounds (
|
||||
`user_id` int(11) NOT NULL,
|
||||
`snapshot_time` datetime NOT NULL,
|
||||
`snapshot_data` varchar(50) NOT NULL
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(`user_id`) PROPERTIES ("replication_num" = "1");
|
||||
INSERT INTO snapshots_bounds VALUES (601,'2024-01-01 09:00:00','SB09'),(601,'2024-01-01 18:00:00','SB18');
|
||||
INSERT INTO events_bounds VALUES (1,601,'2024-01-01 07:00:00'),(2,601,'2024-01-01 20:00:00');
|
||||
-- INNER before-first -> no rows
|
||||
SELECT /* B-INNER-BEFORE */ e.event_id, e.event_time, s.snapshot_time
|
||||
FROM events_bounds e ASOF INNER JOIN snapshots_bounds s
|
||||
ON e.user_id=s.user_id AND e.event_time>=s.snapshot_time
|
||||
WHERE e.event_id=1;
|
||||
-- LEFT before-first -> keep probe, NULL build
|
||||
SELECT /* B-LEFT-BEFORE */ e.event_id, e.event_time, s.snapshot_time
|
||||
FROM events_bounds e ASOF LEFT JOIN snapshots_bounds s
|
||||
ON e.user_id=s.user_id AND e.event_time>=s.snapshot_time
|
||||
WHERE e.event_id=1;
|
||||
-- INNER after-last -> matches last snapshot
|
||||
SELECT /* B-INNER-AFTER */ e.event_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events_bounds e ASOF INNER JOIN snapshots_bounds s
|
||||
ON e.user_id=s.user_id AND e.event_time>=s.snapshot_time
|
||||
WHERE e.event_id=2;
|
||||
-- LEFT after-last -> also matches last snapshot
|
||||
SELECT /* B-LEFT-AFTER */ e.event_id, e.event_time, s.snapshot_time, s.snapshot_data
|
||||
FROM events_bounds e ASOF LEFT JOIN snapshots_bounds s
|
||||
ON e.user_id=s.user_id AND e.event_time>=s.snapshot_time
|
||||
WHERE e.event_id=2;
|
||||
|
||||
-- Test 16: Equi NULL with non-null temporal
|
||||
CREATE TABLE events_equi_null (
|
||||
`event_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NULL,
|
||||
`event_time` datetime NOT NULL
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(`event_id`) PROPERTIES ("replication_num" = "1");
|
||||
CREATE TABLE snapshots_equi_null (
|
||||
`user_id` int(11) NULL,
|
||||
`snapshot_time` datetime NOT NULL
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(`user_id`) PROPERTIES ("replication_num" = "1");
|
||||
INSERT INTO events_equi_null VALUES (1,NULL,'2024-01-01 12:00:00');
|
||||
INSERT INTO snapshots_equi_null VALUES (NULL,'2024-01-01 11:00:00');
|
||||
-- INNER should drop, LEFT should retain probe with NULL build
|
||||
SELECT /* EN-INNER */ e.event_id, e.user_id, e.event_time, s.snapshot_time
|
||||
FROM events_equi_null e ASOF INNER JOIN snapshots_equi_null s
|
||||
ON e.user_id=s.user_id AND e.event_time>=s.snapshot_time;
|
||||
SELECT /* EN-LEFT */ e.event_id, e.user_id, e.event_time, s.snapshot_time
|
||||
FROM events_equi_null e ASOF LEFT JOIN snapshots_equi_null s
|
||||
ON e.user_id=s.user_id AND e.event_time>=s.snapshot_time;
|
||||
|
|
@ -0,0 +1,302 @@
|
|||
-- name: test_asof_join_hash_tables
|
||||
|
||||
DROP DATABASE IF EXISTS test_asof_join_hash_tables;
|
||||
CREATE DATABASE test_asof_join_hash_tables;
|
||||
use test_asof_join_hash_tables;
|
||||
|
||||
-- Test 1: DirectMappingJoinHashMap (TINYINT, SMALLINT, BOOLEAN)
|
||||
CREATE TABLE orders_tinyint (
|
||||
`order_id` int(11) NOT NULL,
|
||||
`user_id` tinyint NOT NULL, -- TINYINT triggers DirectMapping
|
||||
`order_time` datetime NOT NULL,
|
||||
`amount` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`order_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
CREATE TABLE prices_tinyint (
|
||||
`product_id` tinyint NOT NULL,
|
||||
`price_time` datetime NOT NULL,
|
||||
`price` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`product_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
-- Test 2: RangeDirectMappingJoinHashMap (INT with small range)
|
||||
CREATE TABLE orders_range (
|
||||
`order_id` int(11) NOT NULL,
|
||||
`user_id` int NOT NULL, -- Small range to trigger RangeDirectMapping
|
||||
`order_time` datetime NOT NULL,
|
||||
`amount` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`order_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
CREATE TABLE prices_range (
|
||||
`product_id` int NOT NULL,
|
||||
`price_time` datetime NOT NULL,
|
||||
`price` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`product_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
-- Test 3: DenseRangeDirectMappingJoinHashMap (INT with range that bypasses RangeDirectMapping)
|
||||
-- Note: DENSE_RANGE_DIRECT_MAPPING is very difficult to trigger in practice
|
||||
-- It requires value_interval > L2_CACHE_SIZE AND value_interval/4 + row_count*4 <= bucket_size*1.1*4
|
||||
-- This is rarely used in real scenarios, so we'll use a simpler approach
|
||||
CREATE TABLE orders_dense (
|
||||
`order_id` int(11) NOT NULL,
|
||||
`user_id` int NOT NULL,
|
||||
`order_time` datetime NOT NULL,
|
||||
`amount` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`order_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
CREATE TABLE prices_dense (
|
||||
`product_id` int NOT NULL,
|
||||
`price_time` datetime NOT NULL,
|
||||
`price` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`product_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
-- Test 4: LinearChainedJoinHashMap (INT with large range, small data)
|
||||
CREATE TABLE orders_linear (
|
||||
`order_id` int(11) NOT NULL,
|
||||
`user_id` int NOT NULL, -- Large range to trigger LinearChained
|
||||
`order_time` datetime NOT NULL,
|
||||
`amount` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`order_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
CREATE TABLE prices_linear (
|
||||
`product_id` int NOT NULL,
|
||||
`price_time` datetime NOT NULL,
|
||||
`price` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`product_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
-- Test 5: LinearChainedAsofJoinHashMap (fallback for ASOF JOIN)
|
||||
CREATE TABLE orders_asof_fallback (
|
||||
`order_id` int(11) NOT NULL,
|
||||
`user_id` int NOT NULL, -- Large range that doesn't fit other optimizations
|
||||
`order_time` datetime NOT NULL,
|
||||
`amount` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`order_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
CREATE TABLE prices_asof_fallback (
|
||||
`product_id` int NOT NULL,
|
||||
`price_time` datetime NOT NULL,
|
||||
`price` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`product_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
-- Test 6: SERIALIZED_VARCHAR (string keys)
|
||||
CREATE TABLE orders_string (
|
||||
`order_id` int(11) NOT NULL,
|
||||
`user_code` varchar(20) NOT NULL,
|
||||
`order_time` datetime NOT NULL,
|
||||
`amount` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`order_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
CREATE TABLE prices_string (
|
||||
`product_code` varchar(20) NOT NULL,
|
||||
`price_time` datetime NOT NULL,
|
||||
`price` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`product_code`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
-- Test 7: Multiple keys (SERIALIZED_FIXED_SIZE_INT)
|
||||
CREATE TABLE orders_multi_key (
|
||||
`order_id` int(11) NOT NULL,
|
||||
`user_id` int NOT NULL,
|
||||
`region_id` int NOT NULL, -- Multiple keys trigger SERIALIZED_FIXED_SIZE
|
||||
`order_time` datetime NOT NULL,
|
||||
`amount` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`order_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
CREATE TABLE prices_multi_key (
|
||||
`product_id` int NOT NULL,
|
||||
`region_id` int NOT NULL,
|
||||
`price_time` datetime NOT NULL,
|
||||
`price` decimal(10,2) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`product_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
-- Insert data for DirectMappingJoinHashMap (TINYINT range 1-10)
|
||||
INSERT INTO orders_tinyint VALUES
|
||||
(1, 1, '2024-01-01 10:00:00', 100.00),
|
||||
(2, 2, '2024-01-01 15:30:00', 200.00),
|
||||
(3, 3, '2024-01-01 11:00:00', 150.00),
|
||||
(4, 4, '2024-01-01 16:00:00', 300.00),
|
||||
(5, 5, '2024-01-02 09:00:00', 250.00),
|
||||
(6, 6, '2024-01-02 14:00:00', 180.00);
|
||||
|
||||
INSERT INTO prices_tinyint VALUES
|
||||
(1, '2024-01-01 08:00:00', 95.00),
|
||||
(1, '2024-01-01 14:00:00', 105.00),
|
||||
(2, '2024-01-01 09:00:00', 90.00),
|
||||
(2, '2024-01-01 13:00:00', 100.00),
|
||||
(3, '2024-01-01 10:00:00', 85.00),
|
||||
(4, '2024-01-01 11:00:00', 110.00),
|
||||
(5, '2024-01-01 12:00:00', 120.00),
|
||||
(6, '2024-01-01 13:00:00', 130.00);
|
||||
|
||||
-- Insert data for RangeDirectMappingJoinHashMap (INT range 100-110, small range)
|
||||
-- This should trigger RANGE_DIRECT_MAPPING because value_interval = 11 <= bucket_size
|
||||
INSERT INTO orders_range VALUES
|
||||
(1, 100, '2024-01-01 10:00:00', 100.00),
|
||||
(2, 101, '2024-01-01 15:30:00', 200.00),
|
||||
(3, 102, '2024-01-01 11:00:00', 150.00),
|
||||
(4, 103, '2024-01-01 16:00:00', 300.00),
|
||||
(5, 104, '2024-01-02 09:00:00', 250.00),
|
||||
(6, 105, '2024-01-02 14:00:00', 180.00);
|
||||
|
||||
INSERT INTO prices_range VALUES
|
||||
(100, '2024-01-01 08:00:00', 95.00),
|
||||
(100, '2024-01-01 14:00:00', 105.00),
|
||||
(101, '2024-01-01 09:00:00', 90.00),
|
||||
(101, '2024-01-01 13:00:00', 100.00),
|
||||
(102, '2024-01-01 10:00:00', 85.00),
|
||||
(103, '2024-01-01 11:00:00', 110.00),
|
||||
(104, '2024-01-01 12:00:00', 120.00),
|
||||
(105, '2024-01-01 13:00:00', 130.00);
|
||||
|
||||
INSERT INTO prices_dense
|
||||
SELECT
|
||||
1500000000 + (generate_series - 1) * 10 + 10 as product_id, -- range: 1500000010 to 1505000000
|
||||
'2024-01-01 08:00:00' + INTERVAL (generate_series - 1) SECOND as price_time,
|
||||
95.00 + (generate_series - 1) % 100 as price
|
||||
FROM TABLE(generate_series(1, 5000000));
|
||||
|
||||
-- Probe-side can be smaller. Insert 500K rows into orders_dense covering same key lattice.
|
||||
INSERT INTO orders_dense
|
||||
SELECT
|
||||
generate_series as order_id,
|
||||
1500000000 + generate_series * 10 as user_id, -- range: 1500000010 to 1500000000+500000*10 = 1505000000
|
||||
'2024-01-01 10:00:00' + INTERVAL (generate_series - 1) SECOND as order_time,
|
||||
100.00 + (generate_series - 1) % 1000 as amount
|
||||
FROM TABLE(generate_series(1, 500000));
|
||||
|
||||
-- Insert data for LinearChainedJoinHashMap (INT range 10000-20000, large range)
|
||||
INSERT INTO orders_linear VALUES
|
||||
(1, 10000, '2024-01-01 10:00:00', 100.00),
|
||||
(2, 10001, '2024-01-01 15:30:00', 200.00),
|
||||
(3, 10002, '2024-01-01 11:00:00', 150.00),
|
||||
(4, 10003, '2024-01-01 16:00:00', 300.00),
|
||||
(5, 10004, '2024-01-02 09:00:00', 250.00),
|
||||
(6, 10005, '2024-01-02 14:00:00', 180.00);
|
||||
|
||||
INSERT INTO prices_linear VALUES
|
||||
(10000, '2024-01-01 08:00:00', 95.00),
|
||||
(10000, '2024-01-01 14:00:00', 105.00),
|
||||
(10001, '2024-01-01 09:00:00', 90.00),
|
||||
(10001, '2024-01-01 13:00:00', 100.00),
|
||||
(10002, '2024-01-01 10:00:00', 85.00),
|
||||
(10003, '2024-01-01 11:00:00', 110.00),
|
||||
(10004, '2024-01-01 12:00:00', 120.00),
|
||||
(10005, '2024-01-01 13:00:00', 130.00);
|
||||
|
||||
-- Insert data for LinearChainedAsofJoinHashMap (fallback case)
|
||||
INSERT INTO orders_asof_fallback VALUES
|
||||
(1, 100000, '2024-01-01 10:00:00', 100.00),
|
||||
(2, 100001, '2024-01-01 15:30:00', 200.00),
|
||||
(3, 100002, '2024-01-01 11:00:00', 150.00),
|
||||
(4, 100003, '2024-01-01 16:00:00', 300.00);
|
||||
|
||||
INSERT INTO prices_asof_fallback VALUES
|
||||
(100000, '2024-01-01 08:00:00', 95.00),
|
||||
(100000, '2024-01-01 14:00:00', 105.00),
|
||||
(100001, '2024-01-01 09:00:00', 90.00),
|
||||
(100001, '2024-01-01 13:00:00', 100.00),
|
||||
(100002, '2024-01-01 10:00:00', 85.00),
|
||||
(100003, '2024-01-01 11:00:00', 110.00);
|
||||
|
||||
-- Insert data for SERIALIZED_VARCHAR (string keys)
|
||||
INSERT INTO orders_string VALUES
|
||||
(1, 'USER001', '2024-01-01 10:00:00', 100.00),
|
||||
(2, 'USER002', '2024-01-01 15:30:00', 200.00),
|
||||
(3, 'USER003', '2024-01-01 11:00:00', 150.00),
|
||||
(4, 'USER004', '2024-01-01 16:00:00', 300.00);
|
||||
|
||||
INSERT INTO prices_string VALUES
|
||||
('USER001', '2024-01-01 08:00:00', 95.00),
|
||||
('USER001', '2024-01-01 14:00:00', 105.00),
|
||||
('USER002', '2024-01-01 09:00:00', 90.00),
|
||||
('USER002', '2024-01-01 13:00:00', 100.00),
|
||||
('USER003', '2024-01-01 10:00:00', 85.00),
|
||||
('USER004', '2024-01-01 11:00:00', 110.00);
|
||||
|
||||
-- Insert data for SERIALIZED_FIXED_SIZE_INT (multiple keys)
|
||||
INSERT INTO orders_multi_key VALUES
|
||||
(1, 100, 1, '2024-01-01 10:00:00', 100.00),
|
||||
(2, 101, 1, '2024-01-01 15:30:00', 200.00),
|
||||
(3, 102, 2, '2024-01-01 11:00:00', 150.00),
|
||||
(4, 103, 2, '2024-01-01 16:00:00', 300.00);
|
||||
|
||||
INSERT INTO prices_multi_key VALUES
|
||||
(100, 1, '2024-01-01 08:00:00', 95.00),
|
||||
(100, 1, '2024-01-01 14:00:00', 105.00),
|
||||
(101, 1, '2024-01-01 09:00:00', 90.00),
|
||||
(101, 1, '2024-01-01 13:00:00', 100.00),
|
||||
(102, 2, '2024-01-01 10:00:00', 85.00),
|
||||
(103, 2, '2024-01-01 11:00:00', 110.00);
|
||||
|
||||
|
||||
-- Test DirectMappingJoinHashMap
|
||||
SELECT o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_tinyint o
|
||||
ASOF INNER JOIN prices_tinyint p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
ORDER BY o.order_id;
|
||||
|
||||
-- Test RangeDirectMappingJoinHashMap
|
||||
SELECT o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_range o
|
||||
ASOF INNER JOIN prices_range p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
ORDER BY o.order_id;
|
||||
|
||||
-- Test DenseRangeDirectMappingJoinHashMap
|
||||
-- This should trigger DENSE_RANGE_DIRECT_MAPPING due to large value_interval and sufficient row_count
|
||||
SELECT count(*)
|
||||
FROM orders_dense o
|
||||
ASOF INNER JOIN [broadcast] prices_dense p ON o.user_id = p.product_id AND o.order_time >= p.price_time;
|
||||
|
||||
set enable_hash_join_range_direct_mapping_opt=false;
|
||||
-- Test LinearChainedJoinHashMap
|
||||
SELECT o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_linear o
|
||||
ASOF INNER JOIN prices_linear p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
ORDER BY o.order_id;
|
||||
|
||||
-- Test LinearChainedAsofJoinHashMap (fallback)
|
||||
set enable_hash_join_linear_chained_opt=false;
|
||||
SELECT o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_asof_fallback o
|
||||
ASOF INNER JOIN prices_asof_fallback p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
ORDER BY o.order_id;
|
||||
set enable_hash_join_range_direct_mapping_opt=true;
|
||||
set enable_hash_join_linear_chained_opt=true;
|
||||
|
||||
-- Test SERIALIZED_VARCHAR
|
||||
SELECT o.order_id, o.user_code, o.order_time, p.price_time, p.price
|
||||
FROM orders_string o
|
||||
ASOF INNER JOIN prices_string p ON o.user_code = p.product_code AND o.order_time >= p.price_time
|
||||
ORDER BY o.order_id;
|
||||
|
||||
-- Test SERIALIZED_FIXED_SIZE_INT (multiple keys)
|
||||
SELECT o.order_id, o.user_id, o.region_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_multi_key o
|
||||
ASOF INNER JOIN prices_multi_key p ON o.user_id = p.product_id AND o.region_id = p.region_id AND o.order_time >= p.price_time
|
||||
ORDER BY o.order_id;
|
||||
|
|
@ -0,0 +1,249 @@
|
|||
-- name: test_asof_join_nulls
|
||||
|
||||
DROP DATABASE IF EXISTS test_asof_join_nulls;
|
||||
CREATE DATABASE test_asof_join_nulls;
|
||||
USE test_asof_join_nulls;
|
||||
|
||||
-- Probe table: allow NULLs on equi and temporal columns
|
||||
CREATE TABLE orders_nulls (
|
||||
`order_id` int,
|
||||
`user_id` int, -- nullable equi key
|
||||
`order_time` datetime, -- nullable temporal
|
||||
`amount` decimal(10,2)
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`order_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
-- Build table: allow NULLs on equi and temporal columns
|
||||
CREATE TABLE prices_nulls (
|
||||
`product_id` int, -- nullable equi key
|
||||
`price_time` datetime, -- nullable temporal
|
||||
`price` decimal(10,2)
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`product_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
-- Data layout notes:
|
||||
-- user_id/product_id: 101, 102, NULL
|
||||
-- *_time: some NULLs to test temporal-null behavior
|
||||
|
||||
INSERT INTO orders_nulls VALUES
|
||||
-- Baseline non-null temporal & equi
|
||||
(1, 101, '2024-01-01 10:00:00', 100.00),
|
||||
(2, 101, '2024-01-01 12:00:00', 120.00),
|
||||
(3, 102, '2024-01-02 10:00:00', 130.00),
|
||||
-- Probe equi NULL
|
||||
(4, NULL, '2024-01-01 11:00:00', 90.00),
|
||||
-- Probe temporal NULL
|
||||
(5, 101, NULL, 110.00),
|
||||
(6, 102, NULL, 140.00);
|
||||
|
||||
INSERT INTO prices_nulls VALUES
|
||||
-- Build rows for 101 with increasing times
|
||||
(101, '2024-01-01 09:00:00', 9.50),
|
||||
(101, '2024-01-01 11:30:00', 10.50),
|
||||
(101, '2024-01-01 12:30:00', 10.80),
|
||||
-- Build rows for 102 with increasing times
|
||||
(102, '2024-01-02 08:00:00', 12.00),
|
||||
(102, '2024-01-02 12:00:00', 12.50),
|
||||
-- Build equi NULL
|
||||
(NULL, '2024-01-01 08:00:00', 8.88),
|
||||
-- Build temporal NULL
|
||||
(101, NULL, 10.00),
|
||||
(102, NULL, 12.34);
|
||||
|
||||
-- Case A: equi non-null, temporal non-null (baseline) - INNER
|
||||
SELECT /* A1 */ o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_nulls o
|
||||
ASOF INNER JOIN prices_nulls p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
WHERE o.order_id IN (1,2,3)
|
||||
ORDER BY o.order_id;
|
||||
|
||||
-- Case A: baseline - LEFT
|
||||
SELECT /* A2 */ o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_nulls o
|
||||
ASOF LEFT JOIN prices_nulls p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
WHERE o.order_id IN (1,2,3)
|
||||
ORDER BY o.order_id;
|
||||
|
||||
-- Case B: probe equi NULL - INNER (should drop rows with NULL equi key)
|
||||
SELECT /* B1 */ o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_nulls o
|
||||
ASOF INNER JOIN prices_nulls p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
WHERE o.order_id = 4
|
||||
ORDER BY o.order_id;
|
||||
|
||||
-- Case B: probe equi NULL - LEFT (should retain probe row, NULL build side)
|
||||
SELECT /* B2 */ o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_nulls o
|
||||
ASOF LEFT JOIN prices_nulls p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
WHERE o.order_id = 4
|
||||
ORDER BY o.order_id;
|
||||
|
||||
-- Case C: probe temporal NULL - INNER (ASOF predicate can't evaluate -> drop)
|
||||
SELECT /* C1 */ o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_nulls o
|
||||
ASOF INNER JOIN prices_nulls p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
WHERE o.order_id IN (5,6)
|
||||
ORDER BY o.order_id;
|
||||
|
||||
-- Case C: probe temporal NULL - LEFT (should retain probe row, NULL build side)
|
||||
SELECT /* C2 */ o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_nulls o
|
||||
ASOF LEFT JOIN prices_nulls p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
WHERE o.order_id IN (5,6)
|
||||
ORDER BY o.order_id;
|
||||
|
||||
-- Case D: build temporal NULL - INNER (build rows with NULL price_time should be ignored)
|
||||
SELECT /* D1 */ o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_nulls o
|
||||
ASOF INNER JOIN prices_nulls p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
WHERE o.order_id IN (1,2,3)
|
||||
ORDER BY o.order_id;
|
||||
|
||||
-- Case D: build temporal NULL - LEFT (same expectation, presence of NULL price_time rows doesn't affect)
|
||||
SELECT /* D2 */ o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_nulls o
|
||||
ASOF LEFT JOIN prices_nulls p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
WHERE o.order_id IN (1,2,3)
|
||||
ORDER BY o.order_id;
|
||||
|
||||
-- Case E: build equi NULL rows should not participate - LEFT shows no matches from those build rows
|
||||
SELECT /* E1 */ o.order_id, o.user_id, o.order_time, p.price_time, p.price
|
||||
FROM orders_nulls o
|
||||
ASOF LEFT JOIN prices_nulls p ON o.user_id = p.product_id AND o.order_time >= p.price_time
|
||||
WHERE o.order_id IN (1,4,5)
|
||||
ORDER BY o.order_id;
|
||||
|
||||
-- -----------------------------------------------------------------------------------
|
||||
-- Additional null-handling tests across hash map types
|
||||
-- -----------------------------------------------------------------------------------
|
||||
|
||||
-- 1) DirectMappingJoinHashMap (TINYINT)
|
||||
CREATE TABLE dm_orders_nulls (
|
||||
order_id int,
|
||||
user_id tinyint,
|
||||
order_time datetime,
|
||||
amount decimal(10,2)
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(order_id) PROPERTIES ("replication_num"="1");
|
||||
CREATE TABLE dm_prices_nulls (
|
||||
product_id tinyint,
|
||||
price_time datetime,
|
||||
price decimal(10,2)
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(product_id) PROPERTIES ("replication_num"="1");
|
||||
INSERT INTO dm_orders_nulls VALUES
|
||||
(1, 1, '2024-01-01 10:00:00', 10.0),
|
||||
(2, 1, NULL, 20.0),
|
||||
(3, NULL,'2024-01-01 11:00:00', 30.0);
|
||||
INSERT INTO dm_prices_nulls VALUES
|
||||
(1, '2024-01-01 09:00:00', 9.0),
|
||||
(1, NULL, 9.5),
|
||||
(NULL,'2024-01-01 08:00:00',8.8);
|
||||
SELECT /* DM-INNER */ o.order_id, o.user_id, o.order_time, p.price_time
|
||||
FROM dm_orders_nulls o ASOF INNER JOIN dm_prices_nulls p
|
||||
ON o.user_id=p.product_id AND o.order_time>=p.price_time ORDER BY o.order_id;
|
||||
SELECT /* DM-LEFT */ o.order_id, o.user_id, o.order_time, p.price_time
|
||||
FROM dm_orders_nulls o ASOF LEFT JOIN dm_prices_nulls p
|
||||
ON o.user_id=p.product_id AND o.order_time>=p.price_time ORDER BY o.order_id;
|
||||
|
||||
-- 2) RangeDirectMappingJoinHashMap (INT small range)
|
||||
CREATE TABLE rdm_orders_nulls (
|
||||
order_id int,
|
||||
user_id int,
|
||||
order_time datetime,
|
||||
amount decimal(10,2)
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(order_id) PROPERTIES ("replication_num"="1");
|
||||
CREATE TABLE rdm_prices_nulls (
|
||||
product_id int,
|
||||
price_time datetime,
|
||||
price decimal(10,2)
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(product_id) PROPERTIES ("replication_num"="1");
|
||||
INSERT INTO rdm_orders_nulls VALUES
|
||||
(1, 100, '2024-01-01 10:00:00', 10.0),
|
||||
(2, 100, NULL, 20.0),
|
||||
(3, NULL,'2024-01-01 12:00:00', 30.0);
|
||||
INSERT INTO rdm_prices_nulls VALUES
|
||||
(100,'2024-01-01 09:00:00', 9.0),
|
||||
(100,'2024-01-01 09:30:00', 9.5),
|
||||
(NULL,'2024-01-01 08:00:00',8.8),
|
||||
(100, NULL, 9.9);
|
||||
SELECT /* RDM-INNER */ o.order_id, o.user_id, o.order_time, p.price_time
|
||||
FROM rdm_orders_nulls o ASOF INNER JOIN rdm_prices_nulls p
|
||||
ON o.user_id=p.product_id AND o.order_time>=p.price_time ORDER BY o.order_id;
|
||||
SELECT /* RDM-LEFT */ o.order_id, o.user_id, o.order_time, p.price_time
|
||||
FROM rdm_orders_nulls o ASOF LEFT JOIN rdm_prices_nulls p
|
||||
ON o.user_id=p.product_id AND o.order_time>=p.price_time ORDER BY o.order_id;
|
||||
|
||||
-- 3) DenseRangeDirectMappingJoinHashMap (INT large interval, moderate rows)
|
||||
CREATE TABLE dense_orders_nulls (
|
||||
order_id int,
|
||||
user_id int,
|
||||
order_time datetime,
|
||||
amount decimal(10,2)
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(order_id) PROPERTIES ("replication_num"="1");
|
||||
CREATE TABLE dense_prices_nulls (
|
||||
product_id int,
|
||||
price_time datetime,
|
||||
price decimal(10,2)
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(product_id) PROPERTIES ("replication_num"="1");
|
||||
-- build side ~500k rows to satisfy Dense
|
||||
INSERT INTO dense_prices_nulls
|
||||
SELECT
|
||||
1600000000 + (generate_series - 1) * 10 + 10,
|
||||
date_add('2024-01-01 08:00:00', INTERVAL (generate_series - 1) SECOND),
|
||||
9.0 + (generate_series % 100)
|
||||
FROM TABLE(generate_series(1, 500000));
|
||||
-- probe side small, with a few NULLs
|
||||
INSERT INTO dense_orders_nulls VALUES
|
||||
(1, 1600000010, '2024-01-01 10:00:00', 10.0),
|
||||
(2, 1600001010, NULL, 20.0),
|
||||
(3, NULL, '2024-01-01 10:30:00', 30.0);
|
||||
SELECT count(*)
|
||||
FROM dense_orders_nulls o ASOF INNER JOIN [broadcast] dense_prices_nulls p
|
||||
ON o.user_id=p.product_id AND o.order_time>=p.price_time;
|
||||
SELECT count(*)
|
||||
FROM dense_orders_nulls o ASOF LEFT JOIN [broadcast] dense_prices_nulls p
|
||||
ON o.user_id=p.product_id AND o.order_time>=p.price_time;
|
||||
|
||||
-- 4) LinearChainedJoinHashMap
|
||||
set enable_hash_join_range_direct_mapping_opt=false;
|
||||
CREATE TABLE lc_orders_nulls (
|
||||
order_id int,
|
||||
user_id int,
|
||||
order_time datetime,
|
||||
amount decimal(10,2)
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(order_id) PROPERTIES ("replication_num"="1");
|
||||
CREATE TABLE lc_prices_nulls (
|
||||
product_id int,
|
||||
price_time datetime,
|
||||
price decimal(10,2)
|
||||
) ENGINE=OLAP DISTRIBUTED BY HASH(product_id) PROPERTIES ("replication_num"="1");
|
||||
-- build side small so Dense memory test fails
|
||||
INSERT INTO lc_prices_nulls
|
||||
SELECT
|
||||
1700000000 + (generate_series - 1) * 1000,
|
||||
date_add('2024-01-01 08:00:00', INTERVAL (generate_series - 1) SECOND),
|
||||
20.0 + (generate_series % 10)
|
||||
FROM TABLE(generate_series(1, 5));
|
||||
INSERT INTO lc_orders_nulls VALUES
|
||||
(1, 1700000000, '2024-01-01 10:00:00', 10.0),
|
||||
(2, 1700050000, NULL, 20.0),
|
||||
(3, NULL, '2024-01-01 10:30:00', 30.0);
|
||||
SELECT /* LC-INNER */ o.order_id, o.user_id, o.order_time, p.price_time
|
||||
FROM lc_orders_nulls o ASOF INNER JOIN lc_prices_nulls p
|
||||
ON o.user_id=p.product_id AND o.order_time>=p.price_time ORDER BY o.order_id;
|
||||
SELECT /* LC-LEFT */ o.order_id, o.user_id, o.order_time, p.price_time
|
||||
FROM lc_orders_nulls o ASOF LEFT JOIN lc_prices_nulls p
|
||||
ON o.user_id=p.product_id AND o.order_time>=p.price_time ORDER BY o.order_id;
|
||||
|
||||
-- 5) LinearChainedAsofJoinHashMap (fallback): disable linear chained opt then ASOF fallback kicks in
|
||||
CREATE TABLE lca_orders_nulls LIKE lc_orders_nulls;
|
||||
CREATE TABLE lca_prices_nulls LIKE lc_prices_nulls;
|
||||
INSERT INTO lca_prices_nulls VALUES (1800000000,'2024-01-01 08:00:00', 30.0);
|
||||
INSERT INTO lca_orders_nulls VALUES (1,1800000000,'2024-01-01 10:00:00',10.0),(2,NULL,'2024-01-01 11:00:00',20.0),(3,1800000000,NULL,30.0);
|
||||
set enable_hash_join_linear_chained_opt=false;
|
||||
SELECT /* LCA-LEFT */ o.order_id, o.user_id, o.order_time, p.price_time
|
||||
FROM lca_orders_nulls o ASOF LEFT JOIN lca_prices_nulls p
|
||||
ON o.user_id=p.product_id AND o.order_time>=p.price_time ORDER BY o.order_id;
|
||||
set enable_hash_join_linear_chained_opt=true;
|
||||
set enable_hash_join_range_direct_mapping_opt=true;
|
||||
|
|
@ -0,0 +1,83 @@
|
|||
-- name: test_asof_join_operators
|
||||
|
||||
DROP DATABASE IF EXISTS test_asof_join_operators;
|
||||
CREATE DATABASE test_asof_join_operators;
|
||||
use test_asof_join_operators;
|
||||
|
||||
CREATE TABLE events (
|
||||
`event_id` int(11) NOT NULL,
|
||||
`user_id` int(11) NOT NULL,
|
||||
`event_time` datetime NOT NULL,
|
||||
`event_type` varchar(20) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`event_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
CREATE TABLE user_changes (
|
||||
`user_id` int(11) NOT NULL,
|
||||
`change_time` datetime NOT NULL,
|
||||
`change_type` varchar(20) NOT NULL,
|
||||
`new_value` varchar(50) NOT NULL
|
||||
) ENGINE=OLAP
|
||||
DISTRIBUTED BY HASH(`user_id`)
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
-- Insert test data
|
||||
INSERT INTO events VALUES
|
||||
(1, 101, '2024-01-01 10:00:00', 'LOGIN'),
|
||||
(2, 101, '2024-01-01 15:30:00', 'PURCHASE'),
|
||||
(3, 102, '2024-01-01 11:00:00', 'LOGIN'),
|
||||
(4, 102, '2024-01-01 16:00:00', 'PURCHASE'),
|
||||
(5, 101, '2024-01-02 09:00:00', 'LOGOUT'),
|
||||
(6, 102, '2024-01-02 14:00:00', 'LOGOUT'),
|
||||
(7, 101, '2024-01-01 12:00:00', 'EXACT_MATCH_1'),
|
||||
(8, 102, '2024-01-01 12:00:00', 'EXACT_MATCH_2'),
|
||||
(9, 103, '2024-01-01 12:00:00', 'EXACT_MATCH_3');
|
||||
|
||||
INSERT INTO user_changes VALUES
|
||||
(101, '2024-01-01 08:00:00', 'STATUS', 'ACTIVE'),
|
||||
(101, '2024-01-01 14:00:00', 'STATUS', 'VIP'),
|
||||
(101, '2024-01-02 08:00:00', 'STATUS', 'PREMIUM'),
|
||||
(102, '2024-01-01 09:00:00', 'STATUS', 'ACTIVE'),
|
||||
(102, '2024-01-01 13:00:00', 'STATUS', 'VIP'),
|
||||
(102, '2024-01-02 12:00:00', 'STATUS', 'PREMIUM'),
|
||||
(101, '2024-01-01 12:00:00', 'STATUS', 'EXACT_STATUS_1'),
|
||||
(102, '2024-01-01 12:00:00', 'STATUS', 'EXACT_STATUS_2'),
|
||||
(103, '2024-01-01 12:00:00', 'STATUS', 'EXACT_STATUS_3');
|
||||
|
||||
-- Test 1: ASOF JOIN with >= operator (default behavior)
|
||||
SELECT e.event_id, e.user_id, e.event_time, uc.change_time, uc.change_type, uc.new_value
|
||||
FROM events e
|
||||
ASOF INNER JOIN user_changes uc ON e.user_id = uc.user_id AND e.event_time >= uc.change_time
|
||||
ORDER BY e.event_id;
|
||||
|
||||
-- Test 2: ASOF JOIN with > operator
|
||||
SELECT e.event_id, e.user_id, e.event_time, uc.change_time, uc.change_type, uc.new_value
|
||||
FROM events e
|
||||
ASOF INNER JOIN user_changes uc ON e.user_id = uc.user_id AND e.event_time > uc.change_time
|
||||
ORDER BY e.event_id;
|
||||
|
||||
-- Test 3: ASOF JOIN with <= operator
|
||||
SELECT e.event_id, e.user_id, e.event_time, uc.change_time, uc.change_type, uc.new_value
|
||||
FROM events e
|
||||
ASOF INNER JOIN user_changes uc ON e.user_id = uc.user_id AND e.event_time <= uc.change_time
|
||||
ORDER BY e.event_id;
|
||||
|
||||
-- Test 4: ASOF JOIN with < operator
|
||||
SELECT e.event_id, e.user_id, e.event_time, uc.change_time, uc.change_type, uc.new_value
|
||||
FROM events e
|
||||
ASOF INNER JOIN user_changes uc ON e.user_id = uc.user_id AND e.event_time < uc.change_time
|
||||
ORDER BY e.event_id;
|
||||
|
||||
-- Test 5: ASOF LEFT JOIN with >= operator
|
||||
SELECT e.event_id, e.user_id, e.event_time, uc.change_time, uc.change_type, uc.new_value
|
||||
FROM events e
|
||||
ASOF LEFT JOIN user_changes uc ON e.user_id = uc.user_id AND e.event_time >= uc.change_time
|
||||
ORDER BY e.event_id;
|
||||
|
||||
-- Test 6: ASOF LEFT JOIN with > operator
|
||||
SELECT e.event_id, e.user_id, e.event_time, uc.change_time, uc.change_type, uc.new_value
|
||||
FROM events e
|
||||
ASOF LEFT JOIN user_changes uc ON e.user_id = uc.user_id AND e.event_time > uc.change_time
|
||||
ORDER BY e.event_id;
|
||||
|
||||
|
|
@ -0,0 +1,59 @@
|
|||
-- name: test_asof_join_performance
|
||||
|
||||
DROP DATABASE IF EXISTS test_asof_join_performance;
|
||||
CREATE DATABASE test_asof_join_performance;
|
||||
use test_asof_join_performance;
|
||||
|
||||
CREATE TABLE sessions (
|
||||
visitor_id BIGINT NOT NULL COMMENT '访客ID',
|
||||
session_start DATETIME NOT NULL COMMENT '会话开始时间',
|
||||
session_id BIGINT NOT NULL COMMENT '会话ID',
|
||||
session_duration INT COMMENT '会话时长秒'
|
||||
) ENGINE=OLAP
|
||||
DUPLICATE KEY(visitor_id, session_start)
|
||||
DISTRIBUTED BY HASH(visitor_id) BUCKETS 4
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
CREATE TABLE visitors (
|
||||
visitor_id BIGINT NOT NULL COMMENT '访客ID',
|
||||
first_visit DATETIME NOT NULL COMMENT '首次访问时间',
|
||||
starting_session_id BIGINT NOT NULL COMMENT '起始会话ID',
|
||||
visitor_type VARCHAR(20) COMMENT '访客类型'
|
||||
) ENGINE=OLAP
|
||||
DUPLICATE KEY(visitor_id, first_visit)
|
||||
DISTRIBUTED BY HASH(visitor_id) BUCKETS 4
|
||||
PROPERTIES ("replication_num" = "1");
|
||||
|
||||
INSERT INTO sessions (visitor_id, session_start, session_id, session_duration)
|
||||
SELECT
|
||||
(number % 10000) as visitor_id,
|
||||
'2024-01-01 08:00:00' + INTERVAL number SECOND as session_start,
|
||||
number as session_id,
|
||||
30 + (number % 200000) as session_duration
|
||||
FROM TABLE(generate_series(1, 1000000)) as t(number)
|
||||
ORDER BY visitor_id, session_start;
|
||||
|
||||
INSERT INTO visitors (visitor_id, first_visit, starting_session_id, visitor_type)
|
||||
SELECT
|
||||
(number % 10000) as visitor_id,
|
||||
'2024-01-01 07:30:00' + INTERVAL number SECOND as first_visit,
|
||||
number as starting_session_id,
|
||||
'aaaa'
|
||||
FROM TABLE(generate_series(1, 50000000)) as t(number)
|
||||
ORDER BY visitor_id, first_visit;
|
||||
|
||||
SELECT
|
||||
COUNT(*) as total_sessions,
|
||||
COUNT(v.first_visit) as valid_matches
|
||||
FROM sessions s
|
||||
ASOF JOIN visitors v
|
||||
ON s.visitor_id = v.visitor_id AND v.first_visit <= s.session_start;
|
||||
|
||||
SELECT
|
||||
COUNT(*) as total_sessions,
|
||||
COUNT(v.first_visit) as valid_matches
|
||||
FROM sessions s
|
||||
ASOF LEFT JOIN visitors v
|
||||
ON s.visitor_id = v.visitor_id AND v.first_visit <= s.session_start;
|
||||
|
||||
|
||||
Loading…
Reference in New Issue