From 07911cfa3d02ed90cb1f2e3a8926fbfd456d6b91 Mon Sep 17 00:00:00 2001 From: lmatz Date: Mon, 26 Aug 2024 12:51:25 +0800 Subject: [PATCH] chore: add license to file sink (#18227) --- e2e_test/sink/license.slt | 42 +++++++++++++++++++ .../src/sink/file_sink/opendal_sink.rs | 3 ++ src/license/src/feature.rs | 1 + 3 files changed, 46 insertions(+) diff --git a/e2e_test/sink/license.slt b/e2e_test/sink/license.slt index 852d7c0fe7bfc..e38470d1c70d7 100644 --- a/e2e_test/sink/license.slt +++ b/e2e_test/sink/license.slt @@ -7,6 +7,32 @@ ALTER SYSTEM SET license_key TO ''; statement ok CREATE TABLE t (k INT); +statement error +CREATE SINK file_sink +FROM + t +WITH +( + connector = 's3', + s3.region_name = 'us-east-1', + s3.bucket_name = 'test', + s3.path = '', + s3.file_type = 'parquet', + type = 'append-only', + force_append_only='true' +) FORMAT PLAIN ENCODE PARQUET(force_append_only='true'); +---- +db error: ERROR: Failed to run the query + +Caused by these errors (recent errors listed first): + 1: gRPC request to meta service failed: Internal error + 2: failed to validate sink + 3: Internal error + 4: feature FileSink is only available for tier Paid and above, while the current tier is Free + +Hint: You may want to set a license key with `ALTER SYSTEM SET license_key = '...';` command. + + statement error CREATE SINK dynamodb_sink FROM @@ -109,6 +135,22 @@ ALTER SYSTEM SET license_key TO DEFAULT; statement ok flush; +statement ok +CREATE SINK file_sink +FROM + t +WITH +( + connector = 's3', + s3.region_name = 'us-east-1', + s3.bucket_name = 'test', + s3.path = '', + s3.file_type = 'parquet', + type = 'append-only', + force_append_only='true' +) FORMAT PLAIN ENCODE PARQUET(force_append_only='true'); + + statement error CREATE SINK dynamodb_sink FROM diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs index f157d0c861d17..4b201bba131e3 100644 --- a/src/connector/src/sink/file_sink/opendal_sink.rs +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -95,6 +95,9 @@ impl Sink for FileSink { const SINK_NAME: &'static str = S::SINK_NAME; async fn validate(&self) -> Result<()> { + risingwave_common::license::Feature::FileSink + .check_available() + .map_err(|e| anyhow::anyhow!(e))?; if !self.is_append_only { return Err(SinkError::Config(anyhow!( "File sink only supports append-only mode at present. \ diff --git a/src/license/src/feature.rs b/src/license/src/feature.rs index 6047e0414b24e..b7082c01dd7b4 100644 --- a/src/license/src/feature.rs +++ b/src/license/src/feature.rs @@ -57,6 +57,7 @@ macro_rules! for_all_features { { SqlServerCdcSource, Paid, "CDC source connector for Sql Server." }, { CdcAutoSchemaChange, Paid, "Auto replicate upstream DDL to CDC Table." }, { IcebergSinkWithGlue, Paid, "Delivering data to Iceberg with Glue catalog." }, + { FileSink, Paid, "Delivering data to object storage."}, } }; }