risingwave_connector/source/kafka/
client_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
// 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::collections::BTreeMap;
use std::sync::Arc;
use std::thread;

use anyhow::anyhow;
use aws_config::Region;
use aws_sdk_s3::config::SharedCredentialsProvider;
use rdkafka::client::{BrokerAddr, OAuthToken};
use rdkafka::consumer::ConsumerContext;
use rdkafka::message::DeliveryResult;
use rdkafka::producer::ProducerContext;
use rdkafka::{ClientContext, Statistics};

use super::private_link::{BrokerAddrRewriter, PrivateLinkContextRole};
use super::stats::RdKafkaStats;
use crate::connector_common::AwsAuthProps;
use crate::error::ConnectorResult;

struct IamAuthEnv {
    credentials_provider: SharedCredentialsProvider,
    region: Region,
    // XXX(runji): madsim does not support `Handle` for now
    #[cfg(not(madsim))]
    rt: tokio::runtime::Handle,
}

pub struct KafkaContextCommon {
    // For VPC PrivateLink support
    addr_rewriter: BrokerAddrRewriter,

    // identifier is required when reporting metrics as a label, usually it is compose by connector
    // format (source or sink) and corresponding id (source_id or sink_id)
    // identifier and metrics should be set at the same time
    identifier: Option<String>,
    metrics: Option<Arc<RdKafkaStats>>,

    /// Credential and region for AWS MSK
    auth: Option<IamAuthEnv>,
}

impl KafkaContextCommon {
    pub async fn new(
        broker_rewrite_map: Option<BTreeMap<String, String>>,
        identifier: Option<String>,
        metrics: Option<Arc<RdKafkaStats>>,
        auth: AwsAuthProps,
        is_aws_msk_iam: bool,
    ) -> ConnectorResult<Self> {
        let addr_rewriter =
            BrokerAddrRewriter::new(PrivateLinkContextRole::Consumer, broker_rewrite_map)?;
        let auth = if is_aws_msk_iam {
            let config = auth.build_config().await?;
            let credentials_provider = config
                .credentials_provider()
                .ok_or_else(|| anyhow!("missing aws credentials_provider"))?;
            let region = config
                .region()
                .ok_or_else(|| anyhow!("missing aws region"))?
                .clone();
            Some(IamAuthEnv {
                credentials_provider,
                region,
                #[cfg(not(madsim))]
                rt: tokio::runtime::Handle::current(),
            })
        } else {
            None
        };
        Ok(Self {
            addr_rewriter,
            identifier,
            metrics,
            auth,
        })
    }
}

impl KafkaContextCommon {
    fn stats(&self, statistics: Statistics) {
        if let Some(metrics) = &self.metrics
            && let Some(id) = &self.identifier
        {
            metrics.report(id.as_str(), &statistics);
        }
    }

    fn rewrite_broker_addr(&self, addr: BrokerAddr) -> BrokerAddr {
        self.addr_rewriter.rewrite_broker_addr(addr)
    }

    // XXX(runji): oauth is ignored in simulation
    #[cfg_or_panic::cfg_or_panic(not(madsim))]
    fn generate_oauth_token(
        &self,
        _oauthbearer_config: Option<&str>,
    ) -> Result<OAuthToken, Box<dyn std::error::Error>> {
        use aws_msk_iam_sasl_signer::generate_auth_token_from_credentials_provider;
        use tokio::time::{timeout, Duration};

        if let Some(IamAuthEnv {
            credentials_provider,
            region,
            rt,
        }) = &self.auth
        {
            let region = region.clone();
            let credentials_provider = credentials_provider.clone();
            let rt = rt.clone();
            let (token, expiration_time_ms) = {
                let handle = thread::spawn(move || {
                    rt.block_on(async {
                        timeout(
                            Duration::from_secs(10),
                            generate_auth_token_from_credentials_provider(
                                region,
                                credentials_provider,
                            ),
                        )
                        .await
                    })
                });
                handle.join().unwrap()??
            };
            Ok(OAuthToken {
                token,
                principal_name: "".to_string(),
                lifetime_ms: expiration_time_ms,
            })
        } else {
            Err("must provide AWS IAM credential".into())
        }
    }

    fn enable_refresh_oauth_token(&self) -> bool {
        self.auth.is_some()
    }
}

pub type BoxConsumerContext = Box<dyn ConsumerContext>;

/// Kafka consumer context used for private link, IAM auth, and metrics
pub struct RwConsumerContext {
    common: KafkaContextCommon,
}

impl RwConsumerContext {
    pub fn new(common: KafkaContextCommon) -> Self {
        Self { common }
    }
}

impl ClientContext for RwConsumerContext {
    /// this func serves as a callback when `poll` is completed.
    fn stats(&self, statistics: Statistics) {
        self.common.stats(statistics);
    }

    fn rewrite_broker_addr(&self, addr: BrokerAddr) -> BrokerAddr {
        self.common.rewrite_broker_addr(addr)
    }

    fn generate_oauth_token(
        &self,
        oauthbearer_config: Option<&str>,
    ) -> Result<OAuthToken, Box<dyn std::error::Error>> {
        self.common.generate_oauth_token(oauthbearer_config)
    }

    fn enable_refresh_oauth_token(&self) -> bool {
        self.common.enable_refresh_oauth_token()
    }
}

// required by the trait bound of BaseConsumer
impl ConsumerContext for RwConsumerContext {}

/// Kafka producer context used for private link, IAM auth, and metrics
pub struct RwProducerContext {
    common: KafkaContextCommon,
}

impl RwProducerContext {
    pub fn new(common: KafkaContextCommon) -> Self {
        Self { common }
    }
}

impl ClientContext for RwProducerContext {
    fn stats(&self, statistics: Statistics) {
        self.common.stats(statistics);
    }

    fn rewrite_broker_addr(&self, addr: BrokerAddr) -> BrokerAddr {
        self.common.rewrite_broker_addr(addr)
    }

    fn generate_oauth_token(
        &self,
        oauthbearer_config: Option<&str>,
    ) -> Result<OAuthToken, Box<dyn std::error::Error>> {
        self.common.generate_oauth_token(oauthbearer_config)
    }

    fn enable_refresh_oauth_token(&self) -> bool {
        self.common.enable_refresh_oauth_token()
    }
}

impl ProducerContext for RwProducerContext {
    type DeliveryOpaque = ();

    fn delivery(&self, _: &DeliveryResult<'_>, _: Self::DeliveryOpaque) {}
}