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
use crate::api;
use crate::api::checkpoint::CheckpointFunction;
use crate::api::element::{Element, Record};
use crate::api::function::{Context, InputFormat, InputSplit, InputSplitSource, NamedFunction};
use crate::api::properties::{ChannelBaseOn, SystemProperties};
use crate::api::runtime::TaskId;
use crate::channel::select::ChannelSelect;
use crate::channel::{ElementReceiver, TryRecvError};
use crate::dag::execution_graph::ExecutionEdge;
use crate::pub_sub::{memory, network, DEFAULT_CHANNEL_SIZE};

pub(crate) struct SystemInputFormat {
    memory_receiver: Option<ElementReceiver>,
    network_receiver: Option<ElementReceiver>,

    task_id: TaskId,
}

impl SystemInputFormat {
    pub fn new() -> Self {
        SystemInputFormat {
            memory_receiver: None,
            network_receiver: None,
            task_id: TaskId::default(),
        }
    }

    fn subscribe_log(&self, context: &Context) {
        let mut parents: Vec<String> = context
            .parents
            .iter()
            .map(|(node, edge)| {
                format!(
                    "Node: {:?} --{:?}--> {:?}",
                    node.task_id, edge, &context.task_id
                )
            })
            .collect();
        parents.sort();
        info!("subscribe\n  {}", parents.join("\n  "));
    }
}

impl InputFormat for SystemInputFormat {
    fn open(&mut self, _input_split: InputSplit, context: &Context) -> api::Result<()> {
        self.subscribe_log(context);

        self.task_id = context.task_id.clone();

        let channel_size = context
            .application_properties
            .get_pub_sub_channel_size()
            .unwrap_or(DEFAULT_CHANNEL_SIZE);
        let channel_base_on = context
            .application_properties
            .get_pub_sub_channel_base()
            .unwrap_or(ChannelBaseOn::UnBounded);

        let mut memory_jobs = Vec::new();
        let mut network_jobs = Vec::new();
        context
            .parents
            .iter()
            .for_each(|(execution_node, execution_edge)| match execution_edge {
                ExecutionEdge::Memory => memory_jobs.push(execution_node.task_id.clone()),
                ExecutionEdge::Network => network_jobs.push(execution_node.task_id.clone()),
            });

        if memory_jobs.len() > 0 {
            let rx = memory::subscribe(
                &memory_jobs,
                &context.task_id,
                channel_size,
                channel_base_on,
            );
            self.memory_receiver = Some(rx);
        }
        if network_jobs.len() > 0 {
            let rx = network::subscribe(
                &network_jobs,
                &context.task_id,
                channel_size,
                channel_base_on,
            );
            self.network_receiver = Some(rx);
        }

        Ok(())
    }

    fn record_iter(&mut self) -> Box<dyn Iterator<Item = Record> + Send> {
        unimplemented!()
    }

    fn element_iter(&mut self) -> Box<dyn Iterator<Item = Element> + Send> {
        let mut receivers = Vec::new();
        if let Some(n) = &self.memory_receiver {
            receivers.push(n.clone());
        }
        if let Some(n) = &self.network_receiver {
            receivers.push(n.clone());
        }

        match receivers.len() {
            0 => panic!("unsupported"),
            1 => Box::new(ChannelIterator::new(receivers.remove(0))),
            _ => Box::new(MultiChannelIterator::new(receivers)),
        }
    }

    fn close(&mut self) -> crate::api::Result<()> {
        Ok(())
    }
}

impl InputSplitSource for SystemInputFormat {}

impl NamedFunction for SystemInputFormat {
    fn name(&self) -> &str {
        "SystemInputFormat"
    }
}

impl CheckpointFunction for SystemInputFormat {}

struct ChannelIterator {
    receiver: ElementReceiver,
}

impl ChannelIterator {
    pub fn new(receiver: ElementReceiver) -> Self {
        ChannelIterator { receiver }
    }
}

impl Iterator for ChannelIterator {
    type Item = Element;

    fn next(&mut self) -> Option<Self::Item> {
        match self.receiver.recv() {
            Ok(element) => {
                // if element.is_stream_status() && element.as_stream_status().end {
                //     return None;
                // }
                return Some(element);
            }
            Err(_e) => {
                panic!("network_receiver Disconnected");
            }
        }
    }
}

pub struct MultiChannelIterator {
    receivers: Vec<ElementReceiver>,
}

impl MultiChannelIterator {
    pub fn new(receivers: Vec<ElementReceiver>) -> Self {
        MultiChannelIterator { receivers }
    }
}

impl Iterator for MultiChannelIterator {
    type Item = Element;

    fn next(&mut self) -> Option<Self::Item> {
        // Build a list of operations.
        let mut sel = ChannelSelect::new();
        for r in &self.receivers {
            sel.recv(r);
        }

        loop {
            // Wait until a receive operation becomes ready and try executing it.
            let index = sel.ready();
            let res = self.receivers[index].try_recv();

            match res {
                Ok(element) => {
                    // if element.is_stream_status() && element.as_stream_status().end {
                    //     return None;
                    // }
                    return Some(element);
                }
                Err(TryRecvError::Empty) => continue,
                Err(TryRecvError::Disconnected) => panic!("the channel is Disconnected"),
            }
        }
    }
}