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
//!
//! # Partition & Partitions Metadata
//!
//! Partition metadata information on cached in the local Controller.
//!

use std::sync::Arc;

use tracing::debug;
use async_trait::async_trait;

use fluvio_types::SpuId;

use crate::store::*;
use crate::core::*;
use super::*;

pub type SharedPartitionStore<C> = Arc<PartitionLocalStore<C>>;

pub type PartitionMetadata<C> = MetadataStoreObject<PartitionSpec, C>;
pub type PartitionLocalStore<C> = LocalStore<PartitionSpec, C>;
pub type DefaultPartitionMd = PartitionMetadata<String>;
pub type DefaultPartitionStore = PartitionLocalStore<u32>;

pub trait PartitionMd<C: MetadataItem> {
    fn with_replicas(key: ReplicaKey, replicas: Vec<SpuId>) -> Self;

    fn quick<S: Into<String>>(partition: ((S, i32), Vec<i32>)) -> Self;
}

impl<C: MetadataItem> PartitionMd<C> for PartitionMetadata<C> {
    /// create new partition with replica map.
    /// first element of replicas is leader
    fn with_replicas(key: ReplicaKey, replicas: Vec<SpuId>) -> Self {
        let spec: PartitionSpec = replicas.into();
        Self::new(key, spec, PartitionStatus::default())
    }

    fn quick<S: Into<String>>(partition: ((S, i32), Vec<i32>)) -> Self {
        let (replica_key, replicas) = partition;
        Self::with_replicas(replica_key.into(), replicas)
    }
}

#[async_trait]
pub trait PartitionLocalStorePolicy<C>
where
    C: MetadataItem,
{
    async fn names(&self) -> Vec<ReplicaKey>;

    async fn topic_partitions(&self, topic: &str) -> Vec<PartitionMetadata<C>>;

    /// find all partitions that has spu in the replicas
    async fn partition_spec_for_spu(&self, target_spu: i32) -> Vec<(ReplicaKey, PartitionSpec)>;

    async fn count_topic_partitions(&self, topic: &str) -> i32;

    // return partitions that belong to this topic
    async fn topic_partitions_list(&self, topic: &str) -> Vec<ReplicaKey>;

    async fn table_fmt(&self) -> String;

    /// replica msg for target spu
    async fn replica_for_spu(&self, target_spu: SpuId) -> Vec<Replica>;

    async fn leaders(&self) -> Vec<ReplicaLeader>;

    fn bulk_load<S: Into<String>>(partitions: Vec<((S, i32), Vec<i32>)>) -> Self;
}

#[async_trait]
impl<C> PartitionLocalStorePolicy<C> for PartitionLocalStore<C>
where
    C: MetadataItem + Send + Sync,
{
    async fn names(&self) -> Vec<ReplicaKey> {
        self.read().await.keys().cloned().collect()
    }

    async fn topic_partitions(&self, topic: &str) -> Vec<PartitionMetadata<C>> {
        let mut res: Vec<PartitionMetadata<C>> = Vec::default();
        for (name, partition) in self.read().await.iter() {
            if name.topic == topic {
                res.push(partition.inner().clone());
            }
        }
        res
    }

    /// find all partitions that has spu in the replicas
    async fn partition_spec_for_spu(&self, target_spu: i32) -> Vec<(ReplicaKey, PartitionSpec)> {
        let mut res = vec![];
        for (name, partition) in self.read().await.iter() {
            if partition.spec.replicas.contains(&target_spu) {
                res.push((name.clone(), partition.spec.clone()));
            }
        }
        res
    }

    async fn count_topic_partitions(&self, topic: &str) -> i32 {
        let mut count: i32 = 0;
        for (name, _) in self.read().await.iter() {
            if name.topic == topic {
                count += 1;
            }
        }
        count
    }

    // return partitions that belong to this topic
    async fn topic_partitions_list(&self, topic: &str) -> Vec<ReplicaKey> {
        self.read()
            .await
            .keys()
            .filter_map(|name| {
                if name.topic == topic {
                    Some(name.clone())
                } else {
                    None
                }
            })
            .collect()
    }

    async fn table_fmt(&self) -> String {
        let mut table = String::new();

        let partition_hdr = format!(
            "{n:<18}   {l:<6}  {r}\n",
            n = "PARTITION",
            l = "LEADER",
            r = "LIVE-REPLICAS",
        );
        table.push_str(&partition_hdr);

        for (name, partition) in self.read().await.iter() {
            let mut leader = String::from("-");
            let mut _lrs = String::from("[]");

            if partition.spec.leader >= 0 {
                leader = format!("{}", partition.spec.leader);
                //   lrs = partition.live_replicas_str();
            }
            let row = format!("{n:<18} {l:<6} \n", n = name.to_string(), l = leader,);
            table.push_str(&row);
        }

        table
    }

    /// replica msg for target spu
    async fn replica_for_spu(&self, target_spu: SpuId) -> Vec<Replica> {
        let msgs: Vec<Replica> = self
            .partition_spec_for_spu(target_spu)
            .await
            .into_iter()
            .map(|(replica_key, partition_spec)| {
                Replica::new(replica_key, partition_spec.leader, partition_spec.replicas)
            })
            .collect();
        debug!(
            "{} computing replica msg for spu y: {}, msg: {}",
            self,
            target_spu,
            msgs.len()
        );
        msgs
    }

    async fn leaders(&self) -> Vec<ReplicaLeader> {
        self.read()
            .await
            .iter()
            .map(|(key, value)| ReplicaLeader {
                id: key.clone(),
                leader: value.spec.leader,
            })
            .collect()
    }

    fn bulk_load<S: Into<String>>(partitions: Vec<((S, i32), Vec<i32>)>) -> Self {
        let elements = partitions
            .into_iter()
            .map(|(replica_key, replicas)| PartitionMetadata::quick((replica_key, replicas)))
            .collect();
        Self::bulk_new(elements)
    }
}

#[cfg(test)]
pub mod test {

    use fluvio_future::test_async;
    use super::*;

    #[test_async]
    async fn test_partitions_to_replica_msgs() -> Result<(), ()> {
        let partitions = DefaultPartitionStore::bulk_load(vec![(("topic1", 0), vec![10, 11, 12])]);
        let replica_msg = partitions.replica_for_spu(10).await;
        assert_eq!(replica_msg.len(), 1);
        Ok(())
    }
}