-
Notifications
You must be signed in to change notification settings - Fork 585
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
feat(sink): implement snowflake sink #15429
Changes from 5 commits
82c03f9
03c5aa0
4f7cac9
e9da466
805b6a2
f0657e2
b6bdd34
827a1d9
e19c3ea
53ef2c5
25d3aef
fb0cade
cd4168c
7a9fdf9
db090a9
95310bf
e46b51c
cd6f587
4caa11f
00d548d
805c44f
5b26ccd
8bc0bf4
ce20818
b7fa1bc
5c7ac05
287fa2c
4c10695
ccfd638
7ddbdf8
3fc56b9
9ceab04
af53821
d877b14
dbc468a
426cb61
4b257d5
6373fdc
583964a
9e52dc2
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,155 @@ | ||
// 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::HashMap; | ||
use std::sync::Arc; | ||
|
||
use anyhow::anyhow; | ||
use async_trait::async_trait; | ||
use risingwave_common::array::StreamChunk; | ||
use risingwave_common::buffer::Bitmap; | ||
use risingwave_common::catalog::Schema; | ||
use serde::Deserialize; | ||
use serde_derive::Serialize; | ||
use serde_json::Value; | ||
use serde_with::serde_as; | ||
use with_options::WithOptions; | ||
|
||
use super::encoder::JsonEncoder; | ||
use super::writer::LogSinkerOf; | ||
use super::{SinkError, SinkParam}; | ||
use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkWriter, SinkWriterParam}; | ||
|
||
pub const SNOWFLAKE_SINK: &str = "snowflake"; | ||
|
||
// TODO: add comments | ||
#[derive(Deserialize, Debug, Clone, WithOptions)] | ||
pub struct SnowflakeCommon { | ||
#[serde(rename = "snowflake.database")] | ||
pub database: String, | ||
|
||
#[serde(rename = "snowflake.database.schema")] | ||
pub schema: String, | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can't we query the schema from snowflow? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I haven't found a solution to query schema from snowflow, but the There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. You can access the table schema via https://docs.snowflake.com/en/sql-reference/info-schema/tables selecting from There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do we have the ability to query the schema from external REST API request? 🤔 There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. You may have to use a snowflake rust driver, like https://docs.rs/snowflake-api/latest/snowflake_api/, which is based on SQL REST API https://docs.rs/snowflake-api/latest/snowflake_api/, and issue a query to
It's just an idea. I never tried it. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Not sure whether this (query the schema from external REST API request) can be done by the same credentials provided below. Asking users to provide a schema also LGTM. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Provide an entire schema seems impossible, the |
||
|
||
#[serde(rename = "snowflake.database.schema.pipe")] | ||
pub pipe: String, | ||
|
||
#[serde(rename = "snowflake.account")] | ||
pub account: String, | ||
|
||
#[serde(rename = "snowflake.private.key")] | ||
pub private_key: String, | ||
|
||
#[serde(rename = "snowflake.private.key.passphrase")] | ||
pub private_key_passphrase: Option<String>, | ||
|
||
#[serde(rename = "snowflake.role")] | ||
pub role: String, | ||
|
||
#[serde(rename = "snowflake.jwt_token")] | ||
pub jwt_token: String, | ||
|
||
#[serde(rename = "snowflake.s3")] | ||
pub s3: String, | ||
} | ||
|
||
#[serde_as] | ||
#[derive(Clone, Debug, Deserialize, WithOptions)] | ||
pub struct SnowflakeConfig { | ||
#[serde(flatten)] | ||
pub common: SnowflakeCommon, | ||
} | ||
|
||
impl SnowflakeConfig { | ||
pub fn from_hashmap(properties: HashMap<String, String>) -> Result<Self> { | ||
let config = | ||
serde_json::from_value::<SnowflakeConfig>(serde_json::to_value(properties).unwrap()) | ||
.map_err(|e| SinkError::Config(anyhow!(e)))?; | ||
Ok(config) | ||
} | ||
} | ||
|
||
#[derive(Debug)] | ||
pub struct SnowflakeSink { | ||
pub config: SnowflakeConfig, | ||
schema: Schema, | ||
pk_indices: Vec<usize>, | ||
is_append_only: bool, | ||
} | ||
|
||
impl Sink for SnowflakeSink { | ||
type Coordinator = DummySinkCommitCoordinator; | ||
type LogSinker = LogSinkerOf<SnowflakeSinkWriter>; | ||
|
||
const SINK_NAME: &'static str = SNOWFLAKE_SINK; | ||
|
||
async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result<Self::LogSinker> { | ||
todo!() | ||
} | ||
|
||
async fn validate(&self) -> Result<()> { | ||
todo!() | ||
} | ||
} | ||
|
||
pub struct SnowflakeSinkWriter { | ||
pub config: SnowflakeSink, | ||
schema: Schema, | ||
pk_indices: Vec<usize>, | ||
is_append_only: bool, | ||
client: Option<SnowflakeClient>, | ||
row_encoder: JsonEncoder, | ||
} | ||
|
||
impl TryFrom<SinkParam> for SnowflakeSink { | ||
type Error = SinkError; | ||
|
||
fn try_from(param: SinkParam) -> std::result::Result<Self, Self::Error> { | ||
let schema = param.schema(); | ||
let config = SnowflakeConfig::from_hashmap(param.properties)?; | ||
Ok(SnowflakeSink { | ||
config, | ||
schema, | ||
pk_indices: param.downstream_pk, | ||
is_append_only: param.sink_type.is_append_only(), | ||
}) | ||
} | ||
} | ||
|
||
#[async_trait] | ||
impl SinkWriter for SnowflakeSinkWriter { | ||
async fn begin_epoch(&mut self, epoch: u64) -> Result<()> { | ||
todo!() | ||
} | ||
|
||
async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { | ||
todo!() | ||
} | ||
|
||
async fn barrier(&mut self, is_checkpoint: bool) -> Result<Self::CommitMetadata> { | ||
todo!() | ||
} | ||
|
||
async fn abort(&mut self) -> Result<()> { | ||
Ok(()) | ||
} | ||
|
||
async fn update_vnode_bitmap(&mut self, _vnode_bitmap: Arc<Bitmap>) -> Result<()> { | ||
Ok(()) | ||
} | ||
} | ||
|
||
pub struct SnowflakeClient {} | ||
|
||
impl SnowflakeClient {} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,51 @@ | ||
use http::request::Builder; | ||
xzhseh marked this conversation as resolved.
Show resolved
Hide resolved
|
||
use hyper::body::{Body, Sender}; | ||
use hyper::client::HttpConnector; | ||
use hyper::{body, Client, Request, StatusCode}; | ||
use hyper_tls::HttpsConnector; | ||
|
||
use std::collections::HashMap; | ||
|
||
use super::{Result, SinkError}; | ||
|
||
const SNOWFLAKE_HOST_ADDR: &str = "snowflakecomputing.com"; | ||
const SNOWFLAKE_REQUEST_ID: &str = "RW_SNOWFLAKE_SINK"; | ||
|
||
#[derive(Debug)] | ||
pub struct SnowflakeInserterBuilder { | ||
url: String, | ||
header: HashMap<String, String>, | ||
} | ||
|
||
impl SnowflakeInserterBuilder { | ||
pub fn new(account: String, db: String, schema: String, pipe: String, header: HashMap<String, String>) -> Self { | ||
// TODO: ensure if we need user to *explicitly* provide the request id | ||
let url = format!("https://{}.{}/v1/data/pipes/{}.{}.{}/insertFiles?request_id={}", | ||
account, | ||
SNOWFLAKE_HOST_ADDR, | ||
db, | ||
schema. | ||
pipe, | ||
SNOWFLAKE_REQUEST_ID); | ||
|
||
Self { | ||
url, | ||
header, | ||
} | ||
} | ||
|
||
fn build_request_and_client() -> (Builder, Client<HttpsConnector<HttpConnector>>) { | ||
|
||
} | ||
|
||
pub async fn build(&self) -> Result<SnowflakeInserter> { | ||
|
||
} | ||
} | ||
|
||
#[derive(Debug)] | ||
pub struct SnowflakeInserter { | ||
sender: Option<Sender>, | ||
join_handle: Option<JoinHandle<Result<Vec<u8>>, | ||
buffer: BytesMut, | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
May have a separate folder
snowflake
to hold the files related to snowflake.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
currently we only have
snowflake.rs
for the core sinking logic, plussnowflake_connector.rs
for the helper clients (i.e., rest api client, s3 client) implementations - let's keep it simple at present, and move things around when it gets bigger in the future.