risingwave_stream/executor/join/
hash_join.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use std::alloc::Global;
use std::cmp::Ordering;
use std::ops::{Bound, Deref, DerefMut, RangeBounds};
use std::sync::Arc;

use anyhow::{anyhow, Context};
use futures::future::{join, try_join};
use futures::{pin_mut, stream, StreamExt};
use futures_async_stream::for_await;
use join_row_set::JoinRowSet;
use local_stats_alloc::{SharedStatsAlloc, StatsAlloc};
use risingwave_common::bitmap::Bitmap;
use risingwave_common::hash::{HashKey, PrecomputedBuildHasher};
use risingwave_common::metrics::LabelGuardedIntCounter;
use risingwave_common::row::{OwnedRow, Row, RowExt};
use risingwave_common::types::{DataType, ScalarImpl};
use risingwave_common::util::epoch::EpochPair;
use risingwave_common::util::iter_util::ZipEqFast;
use risingwave_common::util::row_serde::OrderedRowSerde;
use risingwave_common::util::sort_util::OrderType;
use risingwave_common_estimate_size::EstimateSize;
use risingwave_storage::store::PrefetchOptions;
use risingwave_storage::StateStore;
use thiserror_ext::AsReport;

use super::row::{DegreeType, EncodedJoinRow};
use crate::cache::ManagedLruCache;
use crate::common::metrics::MetricsInfo;
use crate::common::table::state_table::StateTable;
use crate::consistency::{consistency_error, consistency_panic, enable_strict_consistency};
use crate::executor::error::StreamExecutorResult;
use crate::executor::join::row::JoinRow;
use crate::executor::monitor::StreamingMetrics;
use crate::task::{ActorId, AtomicU64Ref, FragmentId};

/// Memcomparable encoding.
type PkType = Vec<u8>;
type InequalKeyType = Vec<u8>;

pub type StateValueType = EncodedJoinRow;
pub type HashValueType = Box<JoinEntryState>;

impl EstimateSize for HashValueType {
    fn estimated_heap_size(&self) -> usize {
        self.as_ref().estimated_heap_size()
    }
}

/// The wrapper for [`JoinEntryState`] which should be `Some` most of the time in the hash table.
///
/// When the executor is operating on the specific entry of the map, it can hold the ownership of
/// the entry by taking the value out of the `Option`, instead of holding a mutable reference to the
/// map, which can make the compiler happy.
struct HashValueWrapper(Option<HashValueType>);

impl EstimateSize for HashValueWrapper {
    fn estimated_heap_size(&self) -> usize {
        self.0.estimated_heap_size()
    }
}

impl HashValueWrapper {
    const MESSAGE: &'static str = "the state should always be `Some`";

    /// Take the value out of the wrapper. Panic if the value is `None`.
    pub fn take(&mut self) -> HashValueType {
        self.0.take().expect(Self::MESSAGE)
    }
}

impl Deref for HashValueWrapper {
    type Target = HashValueType;

    fn deref(&self) -> &Self::Target {
        self.0.as_ref().expect(Self::MESSAGE)
    }
}

impl DerefMut for HashValueWrapper {
    fn deref_mut(&mut self) -> &mut Self::Target {
        self.0.as_mut().expect(Self::MESSAGE)
    }
}

type JoinHashMapInner<K> =
    ManagedLruCache<K, HashValueWrapper, PrecomputedBuildHasher, SharedStatsAlloc<Global>>;

pub struct JoinHashMapMetrics {
    /// Basic information
    /// How many times have we hit the cache of join executor
    lookup_miss_count: usize,
    total_lookup_count: usize,
    /// How many times have we miss the cache when insert row
    insert_cache_miss_count: usize,

    // Metrics
    join_lookup_total_count_metric: LabelGuardedIntCounter<4>,
    join_lookup_miss_count_metric: LabelGuardedIntCounter<4>,
    join_insert_cache_miss_count_metrics: LabelGuardedIntCounter<4>,
}

impl JoinHashMapMetrics {
    pub fn new(
        metrics: &StreamingMetrics,
        actor_id: ActorId,
        fragment_id: FragmentId,
        side: &'static str,
        join_table_id: u32,
    ) -> Self {
        let actor_id = actor_id.to_string();
        let fragment_id = fragment_id.to_string();
        let join_table_id = join_table_id.to_string();
        let join_lookup_total_count_metric = metrics
            .join_lookup_total_count
            .with_guarded_label_values(&[(side), &join_table_id, &actor_id, &fragment_id]);
        let join_lookup_miss_count_metric = metrics
            .join_lookup_miss_count
            .with_guarded_label_values(&[(side), &join_table_id, &actor_id, &fragment_id]);
        let join_insert_cache_miss_count_metrics = metrics
            .join_insert_cache_miss_count
            .with_guarded_label_values(&[(side), &join_table_id, &actor_id, &fragment_id]);

        Self {
            lookup_miss_count: 0,
            total_lookup_count: 0,
            insert_cache_miss_count: 0,
            join_lookup_total_count_metric,
            join_lookup_miss_count_metric,
            join_insert_cache_miss_count_metrics,
        }
    }

    pub fn flush(&mut self) {
        self.join_lookup_total_count_metric
            .inc_by(self.total_lookup_count as u64);
        self.join_lookup_miss_count_metric
            .inc_by(self.lookup_miss_count as u64);
        self.join_insert_cache_miss_count_metrics
            .inc_by(self.insert_cache_miss_count as u64);
        self.total_lookup_count = 0;
        self.lookup_miss_count = 0;
        self.insert_cache_miss_count = 0;
    }
}

/// Inequality key description for `AsOf` join.
struct InequalityKeyDesc {
    idx: usize,
    serializer: OrderedRowSerde,
}

impl InequalityKeyDesc {
    /// Serialize the inequality key from a row.
    pub fn serialize_inequal_key_from_row(&self, row: impl Row) -> InequalKeyType {
        let indices = vec![self.idx];
        let inequality_key = row.project(&indices);
        inequality_key.memcmp_serialize(&self.serializer)
    }
}

pub struct JoinHashMap<K: HashKey, S: StateStore> {
    /// Store the join states.
    inner: JoinHashMapInner<K>,
    /// Data types of the join key columns
    join_key_data_types: Vec<DataType>,
    /// Null safe bitmap for each join pair
    null_matched: K::Bitmap,
    /// The memcomparable serializer of primary key.
    pk_serializer: OrderedRowSerde,
    /// State table. Contains the data from upstream.
    state: TableInner<S>,
    /// Degree table.
    ///
    /// The degree is generated from the hash join executor.
    /// Each row in `state` has a corresponding degree in `degree state`.
    /// A degree value `d` in for a row means the row has `d` matched row in the other join side.
    ///
    /// It will only be used when needed in a side.
    ///
    /// - Full Outer: both side
    /// - Left Outer/Semi/Anti: left side
    /// - Right Outer/Semi/Anti: right side
    /// - Inner: neither side.
    ///
    /// Should be set to `None` if `need_degree_table` was set to `false`.
    degree_state: Option<TableInner<S>>,
    /// If degree table is need
    need_degree_table: bool,
    /// Pk is part of the join key.
    pk_contained_in_jk: bool,
    /// Inequality key description for `AsOf` join.
    inequality_key_desc: Option<InequalityKeyDesc>,
    /// Metrics of the hash map
    metrics: JoinHashMapMetrics,
}

pub struct TableInner<S: StateStore> {
    /// Indices of the (cache) pk in a state row
    pk_indices: Vec<usize>,
    /// Indices of the join key in a state row
    join_key_indices: Vec<usize>,
    // This should be identical to the pk in state table.
    order_key_indices: Vec<usize>,
    pub(crate) table: StateTable<S>,
}

impl<S: StateStore> TableInner<S> {
    pub fn new(pk_indices: Vec<usize>, join_key_indices: Vec<usize>, table: StateTable<S>) -> Self {
        let order_key_indices = table.pk_indices().to_vec();
        Self {
            pk_indices,
            join_key_indices,
            order_key_indices,
            table,
        }
    }

    fn error_context(&self, row: &impl Row) -> String {
        let pk = row.project(&self.pk_indices);
        let jk = row.project(&self.join_key_indices);
        format!(
            "join key: {}, pk: {}, row: {}, state_table_id: {}",
            jk.display(),
            pk.display(),
            row.display(),
            self.table.table_id()
        )
    }
}

impl<K: HashKey, S: StateStore> JoinHashMap<K, S> {
    /// Create a [`JoinHashMap`] with the given LRU capacity.
    #[allow(clippy::too_many_arguments)]
    pub fn new(
        watermark_sequence: AtomicU64Ref,
        join_key_data_types: Vec<DataType>,
        state_join_key_indices: Vec<usize>,
        state_all_data_types: Vec<DataType>,
        state_table: StateTable<S>,
        state_pk_indices: Vec<usize>,
        degree_state: Option<TableInner<S>>,
        null_matched: K::Bitmap,
        pk_contained_in_jk: bool,
        inequality_key_idx: Option<usize>,
        metrics: Arc<StreamingMetrics>,
        actor_id: ActorId,
        fragment_id: FragmentId,
        side: &'static str,
    ) -> Self {
        let alloc = StatsAlloc::new(Global).shared();
        // TODO: unify pk encoding with state table.
        let pk_data_types = state_pk_indices
            .iter()
            .map(|i| state_all_data_types[*i].clone())
            .collect();
        let pk_serializer = OrderedRowSerde::new(
            pk_data_types,
            vec![OrderType::ascending(); state_pk_indices.len()],
        );

        let inequality_key_desc = inequality_key_idx.map(|idx| {
            let serializer = OrderedRowSerde::new(
                vec![state_all_data_types[idx].clone()],
                vec![OrderType::ascending()],
            );
            InequalityKeyDesc { idx, serializer }
        });

        let join_table_id = state_table.table_id();
        let state = TableInner {
            pk_indices: state_pk_indices,
            join_key_indices: state_join_key_indices,
            order_key_indices: state_table.pk_indices().to_vec(),
            table: state_table,
        };

        let need_degree_table = degree_state.is_some();

        let metrics_info = MetricsInfo::new(
            metrics.clone(),
            join_table_id,
            actor_id,
            format!("hash join {}", side),
        );

        let cache = ManagedLruCache::unbounded_with_hasher_in(
            watermark_sequence,
            metrics_info,
            PrecomputedBuildHasher,
            alloc,
        );

        Self {
            inner: cache,
            join_key_data_types,
            null_matched,
            pk_serializer,
            state,
            degree_state,
            need_degree_table,
            pk_contained_in_jk,
            inequality_key_desc,
            metrics: JoinHashMapMetrics::new(&metrics, actor_id, fragment_id, side, join_table_id),
        }
    }

    pub async fn init(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> {
        self.state.table.init_epoch(epoch).await?;
        if let Some(degree_state) = &mut self.degree_state {
            degree_state.table.init_epoch(epoch).await?;
        }
        Ok(())
    }

    /// Update the vnode bitmap and manipulate the cache if necessary.
    pub fn update_vnode_bitmap(&mut self, vnode_bitmap: Arc<Bitmap>) -> bool {
        let (_previous_vnode_bitmap, cache_may_stale) =
            self.state.table.update_vnode_bitmap(vnode_bitmap.clone());
        let _ = self
            .degree_state
            .as_mut()
            .map(|degree_state| degree_state.table.update_vnode_bitmap(vnode_bitmap.clone()));

        if cache_may_stale {
            self.inner.clear();
        }

        cache_may_stale
    }

    pub fn update_watermark(&mut self, watermark: ScalarImpl) {
        // TODO: remove data in cache.
        self.state.table.update_watermark(watermark.clone());
        if let Some(degree_state) = &mut self.degree_state {
            degree_state.table.update_watermark(watermark);
        }
    }

    /// Take the state for the given `key` out of the hash table and return it. One **MUST** call
    /// `update_state` after some operations to put the state back.
    ///
    /// If the state does not exist in the cache, fetch the remote storage and return. If it still
    /// does not exist in the remote storage, a [`JoinEntryState`] with empty cache will be
    /// returned.
    ///
    /// Note: This will NOT remove anything from remote storage.
    pub async fn take_state<'a>(&mut self, key: &K) -> StreamExecutorResult<HashValueType> {
        self.metrics.total_lookup_count += 1;
        let state = if self.inner.contains(key) {
            // Do not update the LRU statistics here with `peek_mut` since we will put the state
            // back.
            let mut state = self.inner.peek_mut(key).unwrap();
            state.take()
        } else {
            self.metrics.lookup_miss_count += 1;
            self.fetch_cached_state(key).await?.into()
        };
        Ok(state)
    }

    /// Fetch cache from the state store. Should only be called if the key does not exist in memory.
    /// Will return a empty `JoinEntryState` even when state does not exist in remote.
    async fn fetch_cached_state(&self, key: &K) -> StreamExecutorResult<JoinEntryState> {
        let key = key.deserialize(&self.join_key_data_types)?;

        let mut entry_state = JoinEntryState::default();

        if self.need_degree_table {
            let sub_range: &(Bound<OwnedRow>, Bound<OwnedRow>) =
                &(Bound::Unbounded, Bound::Unbounded);
            let table_iter_fut =
                self.state
                    .table
                    .iter_with_prefix(&key, sub_range, PrefetchOptions::default());
            let degree_state = self.degree_state.as_ref().unwrap();
            let degree_table_iter_fut =
                degree_state
                    .table
                    .iter_with_prefix(&key, sub_range, PrefetchOptions::default());

            let (table_iter, degree_table_iter) =
                try_join(table_iter_fut, degree_table_iter_fut).await?;

            let mut pinned_table_iter = std::pin::pin!(table_iter);
            let mut pinned_degree_table_iter = std::pin::pin!(degree_table_iter);

            // For better tolerating inconsistent stream, we have to first buffer all rows and
            // degree rows, and check the number of them, then iterate on them.
            let mut rows = vec![];
            let mut degree_rows = vec![];
            let mut inconsistency_happened = false;
            loop {
                let (row, degree_row) =
                    join(pinned_table_iter.next(), pinned_degree_table_iter.next()).await;
                let (row, degree_row) = match (row, degree_row) {
                    (None, None) => break,
                    (None, Some(_)) => {
                        inconsistency_happened = true;
                        consistency_panic!(
                            "mismatched row and degree table of join key: {:?}, degree table has more rows",
                            &key
                        );
                        break;
                    }
                    (Some(_), None) => {
                        inconsistency_happened = true;
                        consistency_panic!(
                            "mismatched row and degree table of join key: {:?}, input table has more rows",
                            &key
                        );
                        break;
                    }
                    (Some(r), Some(d)) => (r, d),
                };

                let row = row?;
                let degree_row = degree_row?;
                rows.push(row);
                degree_rows.push(degree_row);
            }

            if inconsistency_happened {
                // Pk-based row-degree pairing.
                assert_ne!(rows.len(), degree_rows.len());

                let row_iter = stream::iter(rows.into_iter()).peekable();
                let degree_row_iter = stream::iter(degree_rows.into_iter()).peekable();
                pin_mut!(row_iter);
                pin_mut!(degree_row_iter);

                loop {
                    match join(row_iter.as_mut().peek(), degree_row_iter.as_mut().peek()).await {
                        (None, _) | (_, None) => break,
                        (Some(row), Some(degree_row)) => match row.key().cmp(degree_row.key()) {
                            Ordering::Greater => {
                                degree_row_iter.next().await;
                            }
                            Ordering::Less => {
                                row_iter.next().await;
                            }
                            Ordering::Equal => {
                                let row = row_iter.next().await.unwrap();
                                let degree_row = degree_row_iter.next().await.unwrap();

                                let pk = row
                                    .as_ref()
                                    .project(&self.state.pk_indices)
                                    .memcmp_serialize(&self.pk_serializer);
                                let degree_i64 = degree_row
                                    .datum_at(degree_row.len() - 1)
                                    .expect("degree should not be NULL");
                                let inequality_key = self
                                    .inequality_key_desc
                                    .as_ref()
                                    .map(|desc| desc.serialize_inequal_key_from_row(row.row()));
                                entry_state
                                    .insert(
                                        pk,
                                        JoinRow::new(row.row(), degree_i64.into_int64() as u64)
                                            .encode(),
                                        inequality_key,
                                    )
                                    .with_context(|| self.state.error_context(row.row()))?;
                            }
                        },
                    }
                }
            } else {
                // 1 to 1 row-degree pairing.
                // Actually it's possible that both the input data table and the degree table missed
                // some equal number of rows, but let's ignore this case because it should be rare.

                assert_eq!(rows.len(), degree_rows.len());

                #[for_await]
                for (row, degree_row) in
                    stream::iter(rows.into_iter().zip_eq_fast(degree_rows.into_iter()))
                {
                    let pk1 = row.key();
                    let pk2 = degree_row.key();
                    debug_assert_eq!(
                        pk1, pk2,
                        "mismatched pk in degree table: pk1: {pk1:?}, pk2: {pk2:?}",
                    );
                    let pk = row
                        .as_ref()
                        .project(&self.state.pk_indices)
                        .memcmp_serialize(&self.pk_serializer);
                    let inequality_key = self
                        .inequality_key_desc
                        .as_ref()
                        .map(|desc| desc.serialize_inequal_key_from_row(row.row()));
                    let degree_i64 = degree_row
                        .datum_at(degree_row.len() - 1)
                        .expect("degree should not be NULL");
                    entry_state
                        .insert(
                            pk,
                            JoinRow::new(row.row(), degree_i64.into_int64() as u64).encode(),
                            inequality_key,
                        )
                        .with_context(|| self.state.error_context(row.row()))?;
                }
            }
        } else {
            let sub_range: &(Bound<OwnedRow>, Bound<OwnedRow>) =
                &(Bound::Unbounded, Bound::Unbounded);
            let table_iter = self
                .state
                .table
                .iter_with_prefix(&key, sub_range, PrefetchOptions::default())
                .await?;

            #[for_await]
            for entry in table_iter {
                let row = entry?;
                let pk = row
                    .as_ref()
                    .project(&self.state.pk_indices)
                    .memcmp_serialize(&self.pk_serializer);
                let inequality_key = self
                    .inequality_key_desc
                    .as_ref()
                    .map(|desc| desc.serialize_inequal_key_from_row(row.row()));
                entry_state
                    .insert(pk, JoinRow::new(row.row(), 0).encode(), inequality_key)
                    .with_context(|| self.state.error_context(row.row()))?;
            }
        };

        Ok(entry_state)
    }

    pub async fn flush(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> {
        self.metrics.flush();
        self.state.table.commit(epoch).await?;
        if let Some(degree_state) = &mut self.degree_state {
            degree_state.table.commit(epoch).await?;
        }
        Ok(())
    }

    pub async fn try_flush(&mut self) -> StreamExecutorResult<()> {
        self.state.table.try_flush().await?;
        if let Some(degree_state) = &mut self.degree_state {
            degree_state.table.try_flush().await?;
        }
        Ok(())
    }

    /// Insert a join row
    pub fn insert(&mut self, key: &K, value: JoinRow<impl Row>) -> StreamExecutorResult<()> {
        let pk = self.serialize_pk_from_row(&value.row);

        let inequality_key = self
            .inequality_key_desc
            .as_ref()
            .map(|desc| desc.serialize_inequal_key_from_row(&value.row));

        // TODO(yuhao): avoid this `contains`.
        // https://github.com/risingwavelabs/risingwave/issues/9233
        if self.inner.contains(key) {
            // Update cache
            let mut entry = self.inner.get_mut(key).unwrap();
            entry
                .insert(pk, value.encode(), inequality_key)
                .with_context(|| self.state.error_context(&value.row))?;
        } else if self.pk_contained_in_jk {
            // Refill cache when the join key exist in neither cache or storage.
            self.metrics.insert_cache_miss_count += 1;
            let mut state = JoinEntryState::default();
            state
                .insert(pk, value.encode(), inequality_key)
                .with_context(|| self.state.error_context(&value.row))?;
            self.update_state(key, state.into());
        }

        // Update the flush buffer.
        if let Some(degree_state) = self.degree_state.as_mut() {
            let (row, degree) = value.to_table_rows(&self.state.order_key_indices);
            self.state.table.insert(row);
            degree_state.table.insert(degree);
        } else {
            self.state.table.insert(value.row);
        }
        Ok(())
    }

    /// Insert a row.
    /// Used when the side does not need to update degree.
    pub fn insert_row(&mut self, key: &K, value: impl Row) -> StreamExecutorResult<()> {
        let join_row = JoinRow::new(&value, 0);
        self.insert(key, join_row)?;
        Ok(())
    }

    /// Delete a join row
    pub fn delete(&mut self, key: &K, value: JoinRow<impl Row>) -> StreamExecutorResult<()> {
        if let Some(mut entry) = self.inner.get_mut(key) {
            let pk = (&value.row)
                .project(&self.state.pk_indices)
                .memcmp_serialize(&self.pk_serializer);
            let inequality_key = self
                .inequality_key_desc
                .as_ref()
                .map(|desc| desc.serialize_inequal_key_from_row(&value.row));
            entry
                .remove(pk, inequality_key.as_ref())
                .with_context(|| self.state.error_context(&value.row))?;
        }

        // If no cache maintained, only update the state table.
        let (row, degree) = value.to_table_rows(&self.state.order_key_indices);
        self.state.table.delete(row);
        let degree_state = self.degree_state.as_mut().unwrap();
        degree_state.table.delete(degree);
        Ok(())
    }

    /// Delete a row
    /// Used when the side does not need to update degree.
    pub fn delete_row(&mut self, key: &K, value: impl Row) -> StreamExecutorResult<()> {
        if let Some(mut entry) = self.inner.get_mut(key) {
            let pk = (&value)
                .project(&self.state.pk_indices)
                .memcmp_serialize(&self.pk_serializer);

            let inequality_key = self
                .inequality_key_desc
                .as_ref()
                .map(|desc| desc.serialize_inequal_key_from_row(&value));
            entry
                .remove(pk, inequality_key.as_ref())
                .with_context(|| self.state.error_context(&value))?;
        }

        // If no cache maintained, only update the state table.
        self.state.table.delete(value);
        Ok(())
    }

    /// Update a [`JoinEntryState`] into the hash table.
    pub fn update_state(&mut self, key: &K, state: HashValueType) {
        self.inner.put(key.clone(), HashValueWrapper(Some(state)));
    }

    /// Manipulate the degree of the given [`JoinRow`] and [`EncodedJoinRow`] with `action`, both in
    /// memory and in the degree table.
    fn manipulate_degree(
        &mut self,
        join_row_ref: &mut StateValueType,
        join_row: &mut JoinRow<OwnedRow>,
        action: impl Fn(&mut DegreeType),
    ) {
        // TODO: no need to `into_owned_row` here due to partial borrow.
        let old_degree = join_row
            .to_table_rows(&self.state.order_key_indices)
            .1
            .into_owned_row();

        action(&mut join_row_ref.degree);
        action(&mut join_row.degree);

        let new_degree = join_row.to_table_rows(&self.state.order_key_indices).1;
        let degree_state = self.degree_state.as_mut().unwrap();
        degree_state.table.update(old_degree, new_degree);
    }

    /// Increment the degree of the given [`JoinRow`] and [`EncodedJoinRow`] with `action`, both in
    /// memory and in the degree table.
    pub fn inc_degree(
        &mut self,
        join_row_ref: &mut StateValueType,
        join_row: &mut JoinRow<OwnedRow>,
    ) {
        self.manipulate_degree(join_row_ref, join_row, |d| *d += 1)
    }

    /// Decrement the degree of the given [`JoinRow`] and [`EncodedJoinRow`] with `action`, both in
    /// memory and in the degree table.
    pub fn dec_degree(
        &mut self,
        join_row_ref: &mut StateValueType,
        join_row: &mut JoinRow<OwnedRow>,
    ) {
        self.manipulate_degree(join_row_ref, join_row, |d| {
            *d = d.checked_sub(1).unwrap_or_else(|| {
                consistency_panic!("Tried to decrement zero join row degree");
                0
            });
        })
    }

    /// Evict the cache.
    pub fn evict(&mut self) {
        self.inner.evict();
    }

    /// Cached entry count for this hash table.
    pub fn entry_count(&self) -> usize {
        self.inner.len()
    }

    pub fn null_matched(&self) -> &K::Bitmap {
        &self.null_matched
    }

    pub fn table_id(&self) -> u32 {
        self.state.table.table_id()
    }

    pub fn join_key_data_types(&self) -> &[DataType] {
        &self.join_key_data_types
    }

    /// Return true if the inequality key is null.
    /// # Panics
    /// Panics if the inequality key is not set.
    pub fn check_inequal_key_null(&self, row: &impl Row) -> bool {
        let desc = self.inequality_key_desc.as_ref().unwrap();
        row.datum_at(desc.idx).is_none()
    }

    /// Serialize the inequality key from a row.
    /// # Panics
    /// Panics if the inequality key is not set.
    pub fn serialize_inequal_key_from_row(&self, row: impl Row) -> InequalKeyType {
        self.inequality_key_desc
            .as_ref()
            .unwrap()
            .serialize_inequal_key_from_row(&row)
    }

    pub fn serialize_pk_from_row(&self, row: impl Row) -> PkType {
        row.project(&self.state.pk_indices)
            .memcmp_serialize(&self.pk_serializer)
    }
}

use risingwave_common_estimate_size::KvSize;
use thiserror::Error;

use super::*;

/// We manages a `HashMap` in memory for all entries belonging to a join key.
/// When evicted, `cached` does not hold any entries.
///
/// If a `JoinEntryState` exists for a join key, the all records under this
/// join key will be presented in the cache.
#[derive(Default)]
pub struct JoinEntryState {
    /// The full copy of the state.
    cached: JoinRowSet<PkType, StateValueType>,
    /// Index used for AS OF join. The key is inequal column value. The value is the primary key in `cached`.
    inequality_index: JoinRowSet<InequalKeyType, JoinRowSet<PkType, ()>>,
    kv_heap_size: KvSize,
}

impl EstimateSize for JoinEntryState {
    fn estimated_heap_size(&self) -> usize {
        // TODO: Add btreemap internal size.
        // https://github.com/risingwavelabs/risingwave/issues/9713
        self.kv_heap_size.size()
    }
}

#[derive(Error, Debug)]
pub enum JoinEntryError {
    #[error("double inserting a join state entry")]
    Occupied,
    #[error("removing a join state entry but it is not in the cache")]
    Remove,
    #[error("retrieving a pk from the inequality index but it is not in the cache")]
    InequalIndex,
}

impl JoinEntryState {
    /// Insert into the cache.
    pub fn insert(
        &mut self,
        key: PkType,
        value: StateValueType,
        inequality_key: Option<InequalKeyType>,
    ) -> Result<&mut StateValueType, JoinEntryError> {
        let mut removed = false;
        if !enable_strict_consistency() {
            // strict consistency is off, let's remove existing (if any) first
            if let Some(old_value) = self.cached.remove(&key) {
                if let Some(inequality_key) = inequality_key.as_ref() {
                    self.remove_pk_from_inequality_index(&key, inequality_key);
                }
                self.kv_heap_size.sub(&key, &old_value);
                removed = true;
            }
        }

        self.kv_heap_size.add(&key, &value);

        if let Some(inequality_key) = inequality_key {
            self.insert_pk_to_inequality_index(key.clone(), inequality_key);
        }
        let ret = self.cached.try_insert(key.clone(), value);

        if !enable_strict_consistency() {
            assert!(ret.is_ok(), "we have removed existing entry, if any");
            if removed {
                // if not silent, we should log the error
                consistency_error!(?key, "double inserting a join state entry");
            }
        }

        ret.map_err(|_| JoinEntryError::Occupied)
    }

    /// Delete from the cache.
    pub fn remove(
        &mut self,
        pk: PkType,
        inequality_key: Option<&InequalKeyType>,
    ) -> Result<(), JoinEntryError> {
        if let Some(value) = self.cached.remove(&pk) {
            self.kv_heap_size.sub(&pk, &value);
            if let Some(inequality_key) = inequality_key {
                self.remove_pk_from_inequality_index(&pk, inequality_key);
            }
            Ok(())
        } else if enable_strict_consistency() {
            Err(JoinEntryError::Remove)
        } else {
            consistency_error!(?pk, "removing a join state entry but it's not in the cache");
            Ok(())
        }
    }

    fn remove_pk_from_inequality_index(&mut self, pk: &PkType, inequality_key: &InequalKeyType) {
        if let Some(pk_set) = self.inequality_index.get_mut(inequality_key) {
            if pk_set.remove(pk).is_none() {
                if enable_strict_consistency() {
                    panic!("removing a pk that it not in the inequality index");
                } else {
                    consistency_error!(?pk, "removing a pk that it not in the inequality index");
                };
            } else {
                self.kv_heap_size.sub(pk, &());
            }
            if pk_set.is_empty() {
                self.inequality_index.remove(inequality_key);
            }
        }
    }

    fn insert_pk_to_inequality_index(&mut self, pk: PkType, inequality_key: InequalKeyType) {
        if let Some(pk_set) = self.inequality_index.get_mut(&inequality_key) {
            let pk_size = pk.estimated_size();
            if pk_set.try_insert(pk, ()).is_err() {
                if enable_strict_consistency() {
                    panic!("inserting a pk that it already in the inequality index");
                } else {
                    consistency_error!("inserting a pk that it already in the inequality index");
                };
            } else {
                self.kv_heap_size.add_size(pk_size);
            }
        } else {
            let mut pk_set = JoinRowSet::default();
            pk_set.try_insert(pk, ()).unwrap();
            self.inequality_index
                .try_insert(inequality_key, pk_set)
                .unwrap();
        }
    }

    pub fn get(
        &self,
        pk: &PkType,
        data_types: &[DataType],
    ) -> Option<StreamExecutorResult<JoinRow<OwnedRow>>> {
        self.cached
            .get(pk)
            .map(|encoded| encoded.decode(data_types))
    }

    /// Note: the first item in the tuple is the mutable reference to the value in this entry, while
    /// the second item is the decoded value. To mutate the degree, one **must not** forget to apply
    /// the changes to the first item.
    ///
    /// WARNING: Should not change the heap size of `StateValueType` with the mutable reference.
    pub fn values_mut<'a>(
        &'a mut self,
        data_types: &'a [DataType],
    ) -> impl Iterator<
        Item = (
            &'a mut StateValueType,
            StreamExecutorResult<JoinRow<OwnedRow>>,
        ),
    > + 'a {
        self.cached.values_mut().map(|encoded| {
            let decoded = encoded.decode(data_types);
            (encoded, decoded)
        })
    }

    pub fn len(&self) -> usize {
        self.cached.len()
    }

    /// Range scan the cache using the inequality index.
    pub fn range_by_inequality<'a, R>(
        &'a self,
        range: R,
        data_types: &'a [DataType],
    ) -> impl Iterator<Item = StreamExecutorResult<JoinRow<OwnedRow>>> + 'a
    where
        R: RangeBounds<InequalKeyType> + 'a,
    {
        self.inequality_index.range(range).flat_map(|(_, pk_set)| {
            pk_set
                .keys()
                .flat_map(|pk| self.get_by_indexed_pk(pk, data_types))
        })
    }

    /// Get the records whose inequality key upper bound satisfy the given bound.
    pub fn upper_bound_by_inequality<'a>(
        &'a self,
        bound: Bound<&InequalKeyType>,
        data_types: &'a [DataType],
    ) -> Option<StreamExecutorResult<JoinRow<OwnedRow>>> {
        if let Some((_, pk_set)) = self.inequality_index.upper_bound(bound) {
            if let Some(pk) = pk_set.first_key_sorted() {
                self.get_by_indexed_pk(pk, data_types)
            } else {
                panic!("pk set for a index record must has at least one element");
            }
        } else {
            None
        }
    }

    pub fn get_by_indexed_pk(
        &self,
        pk: &PkType,
        data_types: &[DataType],
    ) -> Option<StreamExecutorResult<JoinRow<OwnedRow>>>
where {
        if let Some(value) = self.cached.get(pk) {
            Some(value.decode(data_types))
        } else if enable_strict_consistency() {
            Some(Err(anyhow!(JoinEntryError::InequalIndex).into()))
        } else {
            consistency_error!(?pk, "{}", JoinEntryError::InequalIndex.as_report());
            None
        }
    }

    /// Get the records whose inequality key lower bound satisfy the given bound.
    pub fn lower_bound_by_inequality<'a>(
        &'a self,
        bound: Bound<&InequalKeyType>,
        data_types: &'a [DataType],
    ) -> Option<StreamExecutorResult<JoinRow<OwnedRow>>> {
        if let Some((_, pk_set)) = self.inequality_index.lower_bound(bound) {
            if let Some(pk) = pk_set.first_key_sorted() {
                self.get_by_indexed_pk(pk, data_types)
            } else {
                panic!("pk set for a index record must has at least one element");
            }
        } else {
            None
        }
    }

    pub fn get_first_by_inequality<'a>(
        &'a self,
        inequality_key: &InequalKeyType,
        data_types: &'a [DataType],
    ) -> Option<StreamExecutorResult<JoinRow<OwnedRow>>> {
        if let Some(pk_set) = self.inequality_index.get(inequality_key) {
            if let Some(pk) = pk_set.first_key_sorted() {
                self.get_by_indexed_pk(pk, data_types)
            } else {
                panic!("pk set for a index record must has at least one element");
            }
        } else {
            None
        }
    }

    pub fn inequality_index(&self) -> &JoinRowSet<InequalKeyType, JoinRowSet<PkType, ()>> {
        &self.inequality_index
    }
}

#[cfg(test)]
mod tests {
    use itertools::Itertools;
    use risingwave_common::array::*;
    use risingwave_common::util::iter_util::ZipEqDebug;

    use super::*;

    fn insert_chunk(
        managed_state: &mut JoinEntryState,
        pk_indices: &[usize],
        col_types: &[DataType],
        inequality_key_idx: Option<usize>,
        data_chunk: &DataChunk,
    ) {
        let pk_col_type = pk_indices
            .iter()
            .map(|idx| col_types[*idx].clone())
            .collect_vec();
        let pk_serializer =
            OrderedRowSerde::new(pk_col_type, vec![OrderType::ascending(); pk_indices.len()]);
        let inequality_key_type = inequality_key_idx.map(|idx| col_types[idx].clone());
        let inequality_key_serializer = inequality_key_type
            .map(|data_type| OrderedRowSerde::new(vec![data_type], vec![OrderType::ascending()]));
        for row_ref in data_chunk.rows() {
            let row: OwnedRow = row_ref.into_owned_row();
            let value_indices = (0..row.len() - 1).collect_vec();
            let pk = pk_indices.iter().map(|idx| row[*idx].clone()).collect_vec();
            // Pk is only a `i64` here, so encoding method does not matter.
            let pk = OwnedRow::new(pk)
                .project(&value_indices)
                .memcmp_serialize(&pk_serializer);
            let inequality_key = inequality_key_idx.map(|idx| {
                (&row)
                    .project(&[idx])
                    .memcmp_serialize(inequality_key_serializer.as_ref().unwrap())
            });
            let join_row = JoinRow { row, degree: 0 };
            managed_state
                .insert(pk, join_row.encode(), inequality_key)
                .unwrap();
        }
    }

    fn check(
        managed_state: &mut JoinEntryState,
        col_types: &[DataType],
        col1: &[i64],
        col2: &[i64],
    ) {
        for ((_, matched_row), (d1, d2)) in managed_state
            .values_mut(col_types)
            .zip_eq_debug(col1.iter().zip_eq_debug(col2.iter()))
        {
            let matched_row = matched_row.unwrap();
            assert_eq!(matched_row.row[0], Some(ScalarImpl::Int64(*d1)));
            assert_eq!(matched_row.row[1], Some(ScalarImpl::Int64(*d2)));
            assert_eq!(matched_row.degree, 0);
        }
    }

    #[tokio::test]
    async fn test_managed_join_state() {
        let mut managed_state = JoinEntryState::default();
        let col_types = vec![DataType::Int64, DataType::Int64];
        let pk_indices = [0];

        let col1 = [3, 2, 1];
        let col2 = [4, 5, 6];
        let data_chunk1 = DataChunk::from_pretty(
            "I I
             3 4
             2 5
             1 6",
        );

        // `Vec` in state
        insert_chunk(
            &mut managed_state,
            &pk_indices,
            &col_types,
            None,
            &data_chunk1,
        );
        check(&mut managed_state, &col_types, &col1, &col2);

        // `BtreeMap` in state
        let col1 = [1, 2, 3, 4, 5];
        let col2 = [6, 5, 4, 9, 8];
        let data_chunk2 = DataChunk::from_pretty(
            "I I
             5 8
             4 9",
        );
        insert_chunk(
            &mut managed_state,
            &pk_indices,
            &col_types,
            None,
            &data_chunk2,
        );
        check(&mut managed_state, &col_types, &col1, &col2);
    }

    #[tokio::test]
    async fn test_managed_join_state_w_inequality_index() {
        let mut managed_state = JoinEntryState::default();
        let col_types = vec![DataType::Int64, DataType::Int64];
        let pk_indices = [0];
        let inequality_key_idx = Some(1);
        let inequality_key_serializer =
            OrderedRowSerde::new(vec![DataType::Int64], vec![OrderType::ascending()]);

        let col1 = [3, 2, 1];
        let col2 = [4, 5, 5];
        let data_chunk1 = DataChunk::from_pretty(
            "I I
             3 4
             2 5
             1 5",
        );

        // `Vec` in state
        insert_chunk(
            &mut managed_state,
            &pk_indices,
            &col_types,
            inequality_key_idx,
            &data_chunk1,
        );
        check(&mut managed_state, &col_types, &col1, &col2);
        let bound = OwnedRow::new(vec![Some(ScalarImpl::Int64(5))])
            .memcmp_serialize(&inequality_key_serializer);
        let row = managed_state
            .upper_bound_by_inequality(Bound::Included(&bound), &col_types)
            .unwrap()
            .unwrap();
        assert_eq!(row.row[0], Some(ScalarImpl::Int64(1)));
        let row = managed_state
            .upper_bound_by_inequality(Bound::Excluded(&bound), &col_types)
            .unwrap()
            .unwrap();
        assert_eq!(row.row[0], Some(ScalarImpl::Int64(3)));

        // `BtreeMap` in state
        let col1 = [1, 2, 3, 4, 5];
        let col2 = [5, 5, 4, 4, 8];
        let data_chunk2 = DataChunk::from_pretty(
            "I I
             5 8
             4 4",
        );
        insert_chunk(
            &mut managed_state,
            &pk_indices,
            &col_types,
            inequality_key_idx,
            &data_chunk2,
        );
        check(&mut managed_state, &col_types, &col1, &col2);

        let bound = OwnedRow::new(vec![Some(ScalarImpl::Int64(8))])
            .memcmp_serialize(&inequality_key_serializer);
        let row = managed_state.lower_bound_by_inequality(Bound::Excluded(&bound), &col_types);
        assert!(row.is_none());
    }
}