risingwave_expr_macro/
context.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
// 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 itertools::Itertools;
use proc_macro2::TokenStream;
use quote::{quote, quote_spanned, ToTokens};
use syn::parse::{Parse, ParseStream};
use syn::{Error, FnArg, Ident, ItemFn, Pat, PatType, Result, ReturnType, Token, Type, Visibility};

use crate::utils::extend_vis_with_super;

/// See [`super::define_context!`].
#[derive(Debug, Clone)]
pub(super) struct DefineContextField {
    vis: Visibility,
    name: Ident,
    ty: Type,
}

/// See [`super::define_context!`].
#[derive(Debug, Clone)]
pub(super) struct DefineContextAttr {
    fields: Vec<DefineContextField>,
}

impl Parse for DefineContextField {
    fn parse(input: ParseStream<'_>) -> Result<Self> {
        let vis: Visibility = input.parse()?;
        let name: Ident = input.parse()?;
        input.parse::<Token![:]>()?;
        let ty: Type = input.parse()?;

        Ok(Self { vis, name, ty })
    }
}

impl Parse for DefineContextAttr {
    fn parse(input: ParseStream<'_>) -> Result<Self> {
        let fields = input.parse_terminated(DefineContextField::parse, Token![,])?;
        Ok(Self {
            fields: fields.into_iter().collect(),
        })
    }
}

impl DefineContextField {
    pub(super) fn gen(self) -> Result<TokenStream> {
        let Self { vis, name, ty } = self;

        // We create a sub mod, so we need to extend the vis of getter.
        let vis: Visibility = extend_vis_with_super(vis);

        {
            let name_s = name.to_string();
            if name_s.to_uppercase() != name_s {
                return Err(Error::new_spanned(
                    name,
                    "the name of context variable should be uppercase",
                ));
            }
        }

        Ok(quote! {
            #[allow(non_snake_case)]
            pub mod #name {
                use super::*;
                pub type Type = #ty;

                tokio::task_local! {
                    static LOCAL_KEY: #ty;
                }

                #vis fn try_with<F, R>(f: F) -> Result<R, risingwave_expr::ExprError>
                where
                    F: FnOnce(&#ty) -> R
                {
                    LOCAL_KEY.try_with(f).map_err(|_| risingwave_expr::ContextUnavailable::new(stringify!(#name))).map_err(Into::into)
                }

                pub fn scope<F>(value: #ty, f: F) -> tokio::task::futures::TaskLocalFuture<#ty, F>
                where
                    F: std::future::Future
                {
                    LOCAL_KEY.scope(value, f)
                }

                pub fn sync_scope<F, R>(value: #ty, f: F) -> R
                where
                    F: FnOnce() -> R
                {
                    LOCAL_KEY.sync_scope(value, f)
                }
            }
        })
    }
}

impl DefineContextAttr {
    pub(super) fn gen(self) -> Result<TokenStream> {
        let generated_fields: Vec<TokenStream> = self
            .fields
            .into_iter()
            .map(DefineContextField::gen)
            .try_collect()?;
        Ok(quote! {
            #(#generated_fields)*
        })
    }
}

pub struct CaptureContextAttr {
    /// The context variables which are captured.
    captures: Vec<Ident>,
}

impl Parse for CaptureContextAttr {
    fn parse(input: ParseStream<'_>) -> Result<Self> {
        let captures = input.parse_terminated(Ident::parse, Token![,])?;
        Ok(Self {
            captures: captures.into_iter().collect(),
        })
    }
}

pub(super) fn generate_captured_function(
    attr: CaptureContextAttr,
    mut user_fn: ItemFn,
) -> Result<TokenStream> {
    let CaptureContextAttr { captures } = attr;
    let is_async = user_fn.sig.asyncness.is_some();
    let mut orig_user_fn = user_fn.clone();
    if is_async {
        // Modify the return type to impl Future<Output = output> + Send + 'static for the original function.
        let output_type = match &orig_user_fn.sig.output {
            ReturnType::Type(_, ty) => ty.clone(),
            ReturnType::Default => Box::new(syn::parse_quote!(())),
        };
        orig_user_fn.sig.output = ReturnType::Type(
            syn::token::RArrow::default(),
            Box::new(
                syn::parse_quote!(impl std::future::Future<Output = #output_type> + Send + 'static),
            ),
        );
        orig_user_fn.sig.asyncness = None;

        // Generate clone statements for each input
        let input_def: Vec<TokenStream> = orig_user_fn
            .sig
            .inputs
            .iter()
            .map(|arg| {
                if let FnArg::Typed(PatType { pat, .. }) = arg {
                    if let Pat::Ident(ident) = pat.as_ref() {
                        let ident_name = &ident.ident;
                        return quote! {
                            let #ident_name = #ident_name.clone();
                        };
                    }
                }
                quote! {}
            })
            .collect();

        // Wrap the original function body in async move { ... }.
        let orig_body = &orig_user_fn.block;
        orig_user_fn.block = Box::new(syn::parse_quote!({
            #(#input_def)*
            async move { #orig_body }
        }));
    }

    let sig = &mut user_fn.sig;

    let name = sig.ident.clone();

    // Modify the name.
    {
        let new_name = format!("{}_captured", name);
        let new_name = Ident::new(&new_name, sig.ident.span());
        sig.ident = new_name;
    }

    if is_async {
        // Ensure the function is async
        sig.asyncness = Some(syn::token::Async::default());
    }

    // Modify the inputs of sig.
    let inputs = &mut sig.inputs;
    if inputs.len() < captures.len() {
        return Err(syn::Error::new_spanned(
            inputs,
            format!("expected at least {} inputs", captures.len()),
        ));
    }

    let arg_names: Vec<_> = inputs
        .iter()
        .map(|arg| {
            let FnArg::Typed(arg) = arg else {
                return Err(syn::Error::new_spanned(
                    arg,
                    "receiver is not allowed in captured function",
                ));
            };
            Ok(arg.pat.to_token_stream())
        })
        .try_collect()?;

    let (captured_inputs, remained_inputs) = {
        let mut inputs = inputs.iter().cloned();
        let inputs = inputs.by_ref();
        let captured_inputs = inputs.take(captures.len()).collect_vec();
        let remained_inputs = inputs.collect_vec();
        (captured_inputs, remained_inputs)
    };
    *inputs = remained_inputs.into_iter().collect();

    let call_old_fn = quote! {
        #name(#(#arg_names),*)
    };

    let new_body = {
        let mut scoped = quote! {
            #call_old_fn
        };

        #[allow(clippy::disallowed_methods)]
        for (context, arg) in captures.into_iter().zip(captured_inputs.into_iter()) {
            let FnArg::Typed(arg) = arg else {
                return Err(syn::Error::new_spanned(
                    arg,
                    "receiver is not allowed in captured function",
                ));
            };
            let name = arg.pat.into_token_stream();
            // TODO: Can we add an assertion here that `&<<#context::Type> as Deref>::Target` is same as `#arg.ty`?
            scoped = if is_async {
                quote_spanned! { context.span()=>
                    #context::try_with(|#name| { #scoped })
                }
            } else {
                quote_spanned! { context.span()=>
                    #context::try_with(|#name| { #scoped }).flatten()
                }
            };
        }
        scoped
    };
    let new_user_fn = {
        let vis = user_fn.vis;
        let sig = user_fn.sig;
        if is_async {
            quote! {
                #vis #sig {
                    {#new_body}?.await
                }
            }
        } else {
            quote! {
                #vis #sig {
                    {#new_body}.map_err(Into::into)
                }
            }
        }
    };

    Ok(quote! {
        #orig_user_fn
        #new_user_fn
    })
}