diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml new file mode 100644 index 0000000000000..ab8ba3d9d7eb9 --- /dev/null +++ b/.pre-commit-config.yaml @@ -0,0 +1,26 @@ +# Usage: install pre-commit, and then run `pre-commit install` to install git hooks +# See https://pre-commit.com for more information +# See https://pre-commit.com/hooks.html for more hooks +repos: +- repo: https://github.com/pre-commit/pre-commit-hooks + rev: v2.3.0 + hooks: + - id: end-of-file-fixer + - id: trailing-whitespace +- repo: local + hooks: + - id: rustfmt + name: rustfmt + entry: rustfmt --edition 2021 + language: system + types: [rust] + - id: typos + name: typos + entry: typos -w + language: system + - id: cargo sort + name: cargo sort + entry: cargo sort -g -w + language: system + files: 'Cargo.toml' + pass_filenames: false diff --git a/.typos.toml b/.typos.toml index 6b42f7b01bf25..567904f5c319b 100644 --- a/.typos.toml +++ b/.typos.toml @@ -6,6 +6,7 @@ inout = "inout" # This is a SQL keyword! numer = "numer" # numerator nd = "nd" # N-dimentional / 2nd steam = "stream" # You played with Steam games too much. +ser = "ser" # Serialization # Some weird short variable names ot = "ot" bui = "bui" # BackwardUserIterator diff --git a/ci/Dockerfile b/ci/Dockerfile index 616af35fd118e..a1b6857a45c20 100644 --- a/ci/Dockerfile +++ b/ci/Dockerfile @@ -12,7 +12,7 @@ ENV LANG en_US.utf8 RUN sed -i 's|http://archive.ubuntu.com/ubuntu|http://us-east-2.ec2.archive.ubuntu.com/ubuntu/|g' /etc/apt/sources.list RUN apt-get update -yy && \ DEBIAN_FRONTEND=noninteractive apt-get -y install sudo make build-essential cmake protobuf-compiler curl parallel python3 python3-pip python3-venv software-properties-common \ - openssl libssl-dev libsasl2-dev libcurl4-openssl-dev pkg-config bash openjdk-11-jdk wget unzip git tmux lld postgresql-client kcat netcat-openbsd mysql-client \ + openssl libssl-dev libsasl2-dev libcurl4-openssl-dev pkg-config bash openjdk-17-jdk wget unzip git tmux lld postgresql-client kcat netcat-openbsd mysql-client \ maven zstd libzstd-dev locales \ python3.12 python3.12-dev \ && rm -rf /var/lib/{apt,dpkg,cache,log}/ diff --git a/ci/docker-compose.yml b/ci/docker-compose.yml index 60d2d8946717c..15274be94be9b 100644 --- a/ci/docker-compose.yml +++ b/ci/docker-compose.yml @@ -61,7 +61,8 @@ services: - "29092:29092" - "9092:9092" - "9644:9644" - - "8081:8081" + # Don't use Redpanda's schema registry, use the separated service instead + # - "8081:8081" environment: {} container_name: message_queue healthcheck: @@ -89,6 +90,7 @@ services: - mysql - db - message_queue + - schemaregistry - elasticsearch - clickhouse-server - redis-server diff --git a/ci/scripts/e2e-kafka-sink-test.sh b/ci/scripts/e2e-kafka-sink-test.sh index 206ce4ba1d75d..7cab1ae1f76f7 100755 --- a/ci/scripts/e2e-kafka-sink-test.sh +++ b/ci/scripts/e2e-kafka-sink-test.sh @@ -154,16 +154,16 @@ cp src/connector/src/test_data/proto_recursive/recursive.pb ./proto-recursive rpk topic create test-rw-sink-append-only-protobuf rpk topic create test-rw-sink-append-only-protobuf-csr-a rpk topic create test-rw-sink-append-only-protobuf-csr-hi -python3 e2e_test/sink/kafka/register_schema.py 'http://message_queue:8081' 'test-rw-sink-append-only-protobuf-csr-a-value' src/connector/src/test_data/test-index-array.proto -python3 e2e_test/sink/kafka/register_schema.py 'http://message_queue:8081' 'test-rw-sink-append-only-protobuf-csr-hi-value' src/connector/src/test_data/test-index-array.proto +python3 e2e_test/sink/kafka/register_schema.py 'http://schemaregistry:8082' 'test-rw-sink-append-only-protobuf-csr-a-value' src/connector/src/test_data/test-index-array.proto +python3 e2e_test/sink/kafka/register_schema.py 'http://schemaregistry:8082' 'test-rw-sink-append-only-protobuf-csr-hi-value' src/connector/src/test_data/test-index-array.proto sqllogictest -p 4566 -d dev 'e2e_test/sink/kafka/protobuf.slt' rpk topic delete test-rw-sink-append-only-protobuf rpk topic delete test-rw-sink-append-only-protobuf-csr-a rpk topic delete test-rw-sink-append-only-protobuf-csr-hi echo "testing avro" -python3 e2e_test/sink/kafka/register_schema.py 'http://message_queue:8081' 'test-rw-sink-upsert-avro-value' src/connector/src/test_data/all-types.avsc -python3 e2e_test/sink/kafka/register_schema.py 'http://message_queue:8081' 'test-rw-sink-upsert-avro-key' src/connector/src/test_data/all-types.avsc 'string_field,int32_field' +python3 e2e_test/sink/kafka/register_schema.py 'http://schemaregistry:8082' 'test-rw-sink-upsert-avro-value' src/connector/src/test_data/all-types.avsc +python3 e2e_test/sink/kafka/register_schema.py 'http://schemaregistry:8082' 'test-rw-sink-upsert-avro-key' src/connector/src/test_data/all-types.avsc 'string_field,int32_field' rpk topic create test-rw-sink-upsert-avro sqllogictest -p 4566 -d dev 'e2e_test/sink/kafka/avro.slt' rpk topic delete test-rw-sink-upsert-avro diff --git a/ci/scripts/e2e-source-test.sh b/ci/scripts/e2e-source-test.sh index 5127731256c6b..35b7965f12bb3 100755 --- a/ci/scripts/e2e-source-test.sh +++ b/ci/scripts/e2e-source-test.sh @@ -137,11 +137,11 @@ export RISINGWAVE_CI=true RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ risedev ci-start ci-1cn-1fe python3 -m pip install --break-system-packages requests protobuf confluent-kafka -python3 e2e_test/schema_registry/pb.py "message_queue:29092" "http://message_queue:8081" "sr_pb_test" 20 user +python3 e2e_test/schema_registry/pb.py "message_queue:29092" "http://schemaregistry:8082" "sr_pb_test" 20 user echo "make sure google/protobuf/source_context.proto is NOT in schema registry" -curl --silent 'http://message_queue:8081/subjects'; echo -# curl --silent --head -X GET 'http://message_queue:8081/subjects/google%2Fprotobuf%2Fsource_context.proto/versions' | grep 404 -curl --silent 'http://message_queue:8081/subjects' | grep -v 'google/protobuf/source_context.proto' +curl --silent 'http://schemaregistry:8082/subjects'; echo +# curl --silent --head -X GET 'http://schemaregistry:8082/subjects/google%2Fprotobuf%2Fsource_context.proto/versions' | grep 404 +curl --silent 'http://schemaregistry:8082/subjects' | grep -v 'google/protobuf/source_context.proto' risedev slt './e2e_test/schema_registry/pb.slt' risedev slt './e2e_test/schema_registry/alter_sr.slt' diff --git a/ci/scripts/release.sh b/ci/scripts/release.sh index ee6479362f2ed..94fd38e2c9c75 100755 --- a/ci/scripts/release.sh +++ b/ci/scripts/release.sh @@ -23,7 +23,7 @@ echo "--- Install dependencies" dnf install -y perl-core wget python3 python3-devel cyrus-sasl-devel rsync openssl-devel echo "--- Install java and maven" -dnf install -y java-11-openjdk java-11-openjdk-devel +dnf install -y java-17-openjdk java-17-openjdk-devel pip3 install toml-cli wget https://rw-ci-deps-dist.s3.amazonaws.com/apache-maven-3.9.3-bin.tar.gz && tar -zxvf apache-maven-3.9.3-bin.tar.gz export PATH="${REPO_ROOT}/apache-maven-3.9.3/bin:$PATH" diff --git a/ci/scripts/run-backfill-tests.sh b/ci/scripts/run-backfill-tests.sh index b641065853337..ac552cfcdcdd0 100755 --- a/ci/scripts/run-backfill-tests.sh +++ b/ci/scripts/run-backfill-tests.sh @@ -23,7 +23,7 @@ TEST_DIR=$PWD/e2e_test BACKGROUND_DDL_DIR=$TEST_DIR/background_ddl COMMON_DIR=$BACKGROUND_DDL_DIR/common -CLUSTER_PROFILE='ci-1cn-1fe-kafka-with-recovery' +CLUSTER_PROFILE='ci-1cn-1fe-user-kafka-with-recovery' echo "--- Configuring cluster profiles" if [[ -n "${BUILDKITE:-}" ]]; then echo "Running in buildkite" @@ -187,14 +187,14 @@ test_sink_backfill_recovery() { # Restart restart_cluster - sleep 3 + sleep 5 # Sink back into rw run_sql "CREATE TABLE table_kafka (v1 int primary key) WITH ( connector = 'kafka', topic = 's_kafka', - properties.bootstrap.server = 'localhost:29092', + properties.bootstrap.server = 'message_queue:29092', ) FORMAT DEBEZIUM ENCODE JSON;" sleep 10 diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index e097d2d587994..14f3a23161c80 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -704,7 +704,7 @@ steps: - "build" plugins: - docker-compose#v5.1.0: - run: rw-build-env + run: source-test-env config: ci/docker-compose.yml mount-buildkite-agent: true - ./ci/plugins/upload-failure-logs diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index 9ced38891a75b..3a4e87307231d 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -700,7 +700,7 @@ steps: - "build" plugins: - docker-compose#v5.1.0: - run: rw-build-env + run: source-test-env config: ci/docker-compose.yml mount-buildkite-agent: true - ./ci/plugins/upload-failure-logs diff --git a/docker/Dockerfile b/docker/Dockerfile index 167815a988131..b4d2cf73ee85f 100644 --- a/docker/Dockerfile +++ b/docker/Dockerfile @@ -3,7 +3,7 @@ FROM ubuntu:24.04 AS base ENV LANG en_US.utf8 RUN apt-get update \ - && apt-get -y install ca-certificates build-essential libsasl2-dev openjdk-11-jdk software-properties-common python3.12 python3.12-dev openssl pkg-config + && apt-get -y install ca-certificates build-essential libsasl2-dev openjdk-17-jdk software-properties-common python3.12 python3.12-dev openssl pkg-config FROM base AS rust-base diff --git a/docker/Dockerfile.hdfs b/docker/Dockerfile.hdfs index 2e49564ccb570..b6eba07c421c0 100644 --- a/docker/Dockerfile.hdfs +++ b/docker/Dockerfile.hdfs @@ -3,7 +3,7 @@ FROM ubuntu:24.04 AS base ENV LANG en_US.utf8 RUN apt-get update \ - && apt-get -y install ca-certificates build-essential libsasl2-dev openjdk-11-jdk software-properties-common python3.12 python3.12-dev openssl pkg-config + && apt-get -y install ca-certificates build-essential libsasl2-dev openjdk-17-jdk software-properties-common python3.12 python3.12-dev openssl pkg-config FROM base AS dashboard-builder @@ -113,7 +113,7 @@ RUN cd /risingwave/java && mvn -B package -Dmaven.test.skip=true -Dno-build-rust tar -zxvf /risingwave/java/connector-node/assembly/target/risingwave-connector-1.0.0.tar.gz -C /risingwave/bin/connector-node FROM ubuntu:24.04 as image-base -RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get -y install ca-certificates openjdk-11-jdk wget libsasl2-dev && rm -rf /var/lib/{apt,dpkg,cache,log}/ +RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get -y install ca-certificates openjdk-17-jdk wget libsasl2-dev && rm -rf /var/lib/{apt,dpkg,cache,log}/ FROM image-base as risingwave LABEL org.opencontainers.image.source https://github.com/risingwavelabs/risingwave diff --git a/e2e_test/backfill/sink/create_sink.slt b/e2e_test/backfill/sink/create_sink.slt index bc9fba04da5c8..017eb8e693de2 100644 --- a/e2e_test/backfill/sink/create_sink.slt +++ b/e2e_test/backfill/sink/create_sink.slt @@ -20,7 +20,7 @@ from t x join t y on x.v1 = y.v1 with ( connector='kafka', - properties.bootstrap.server='localhost:29092', + properties.bootstrap.server='message_queue:29092', topic='s_kafka', primary_key='v1', allow.auto.create.topics=true, diff --git a/e2e_test/schema_registry/alter_sr.slt b/e2e_test/schema_registry/alter_sr.slt index 8daf41d87b633..d703c0401a35e 100644 --- a/e2e_test/schema_registry/alter_sr.slt +++ b/e2e_test/schema_registry/alter_sr.slt @@ -9,7 +9,7 @@ CREATE SOURCE src_user WITH ( scan.startup.mode = 'earliest' ) FORMAT PLAIN ENCODE PROTOBUF( - schema.registry = 'http://message_queue:8081', + schema.registry = 'http://schemaregistry:8082', message = 'test.User' ); @@ -24,7 +24,7 @@ CREATE TABLE t_user WITH ( scan.startup.mode = 'earliest' ) FORMAT PLAIN ENCODE PROTOBUF( - schema.registry = 'http://message_queue:8081', + schema.registry = 'http://schemaregistry:8082', message = 'test.User' ); @@ -36,7 +36,7 @@ SELECT age FROM t_user; # Push more events with extended fields system ok -python3 e2e_test/schema_registry/pb.py "message_queue:29092" "http://message_queue:8081" "sr_pb_test" 5 user_with_more_fields +python3 e2e_test/schema_registry/pb.py "message_queue:29092" "http://schemaregistry:8082" "sr_pb_test" 5 user_with_more_fields sleep 5s @@ -58,7 +58,7 @@ SELECT COUNT(*), MAX(age), MIN(age), SUM(age) FROM mv_user_more; # Push more events with extended fields system ok -python3 e2e_test/schema_registry/pb.py "message_queue:29092" "http://message_queue:8081" "sr_pb_test" 5 user_with_more_fields +python3 e2e_test/schema_registry/pb.py "message_queue:29092" "http://schemaregistry:8082" "sr_pb_test" 5 user_with_more_fields sleep 5s diff --git a/e2e_test/schema_registry/pb.slt b/e2e_test/schema_registry/pb.slt index d9c0edca1b21c..7b60b4fa8d7a4 100644 --- a/e2e_test/schema_registry/pb.slt +++ b/e2e_test/schema_registry/pb.slt @@ -9,7 +9,7 @@ create table sr_pb_test with ( properties.bootstrap.server = 'message_queue:29092', scan.startup.mode = 'earliest') FORMAT plain ENCODE protobuf( - schema.registry = 'http://message_queue:8081', + schema.registry = 'http://schemaregistry:8082', message = 'test.User' ); @@ -21,7 +21,7 @@ create table sr_pb_test_bk with ( properties.bootstrap.server = 'message_queue:29092', scan.startup.mode = 'earliest') FORMAT plain ENCODE protobuf( - schema.registry = 'http://message_queue:8081,http://message_queue:8081', + schema.registry = 'http://schemaregistry:8082,http://schemaregistry:8082', message = 'test.User' ); diff --git a/e2e_test/sink/kafka/avro.slt b/e2e_test/sink/kafka/avro.slt index d9fa53bc589ac..1cf27b811d9be 100644 --- a/e2e_test/sink/kafka/avro.slt +++ b/e2e_test/sink/kafka/avro.slt @@ -6,7 +6,7 @@ with ( topic = 'test-rw-sink-upsert-avro', properties.bootstrap.server = 'message_queue:29092') format upsert encode avro ( - schema.registry = 'http://message_queue:8081'); + schema.registry = 'http://schemaregistry:8082'); statement ok create table into_kafka ( @@ -40,7 +40,7 @@ create sink sink0 from into_kafka with ( properties.bootstrap.server = 'message_queue:29092', primary_key = 'int32_field,string_field') format upsert encode avro ( - schema.registry = 'http://message_queue:8081'); + schema.registry = 'http://schemaregistry:8082'); sleep 2s @@ -72,7 +72,7 @@ create sink sink_err from into_kafka with ( properties.bootstrap.server = 'message_queue:29092', primary_key = 'int32_field,string_field') format upsert encode avro ( - schema.registry = 'http://message_queue:8081'); + schema.registry = 'http://schemaregistry:8082'); statement error field not in avro create sink sink_err as select 1 as extra_column, * from into_kafka with ( @@ -81,7 +81,7 @@ create sink sink_err as select 1 as extra_column, * from into_kafka with ( properties.bootstrap.server = 'message_queue:29092', primary_key = 'int32_field,string_field') format upsert encode avro ( - schema.registry = 'http://message_queue:8081'); + schema.registry = 'http://schemaregistry:8082'); statement error unrecognized create sink sink_err from into_kafka with ( @@ -90,7 +90,7 @@ create sink sink_err from into_kafka with ( properties.bootstrap.server = 'message_queue:29092', primary_key = 'int32_field,string_field') format upsert encode avro ( - schema.registry = 'http://message_queue:8081', + schema.registry = 'http://schemaregistry:8082', schema.registry.name.strategy = 'typo'); statement error empty field key.message @@ -100,7 +100,7 @@ create sink sink_err from into_kafka with ( properties.bootstrap.server = 'message_queue:29092', primary_key = 'int32_field,string_field') format upsert encode avro ( - schema.registry = 'http://message_queue:8081', + schema.registry = 'http://schemaregistry:8082', schema.registry.name.strategy = 'record_name_strategy'); statement ok diff --git a/e2e_test/sink/kafka/protobuf.slt b/e2e_test/sink/kafka/protobuf.slt index 0c74cc8a0b369..c3f6f0d3ad8e2 100644 --- a/e2e_test/sink/kafka/protobuf.slt +++ b/e2e_test/sink/kafka/protobuf.slt @@ -13,7 +13,7 @@ create table from_kafka_csr_trivial with ( topic = 'test-rw-sink-append-only-protobuf-csr-a', properties.bootstrap.server = 'message_queue:29092') format plain encode protobuf ( - schema.registry = 'http://message_queue:8081', + schema.registry = 'http://schemaregistry:8082', message = 'test.package.MessageA'); statement ok @@ -22,7 +22,7 @@ create table from_kafka_csr_nested with ( topic = 'test-rw-sink-append-only-protobuf-csr-hi', properties.bootstrap.server = 'message_queue:29092') format plain encode protobuf ( - schema.registry = 'http://message_queue:8081', + schema.registry = 'http://schemaregistry:8082', message = 'test.package.MessageH.MessageI'); statement ok @@ -68,7 +68,7 @@ create sink sink_csr_trivial as select string_field as field_a from into_kafka w properties.bootstrap.server = 'message_queue:29092') format plain encode protobuf ( force_append_only = true, - schema.registry = 'http://message_queue:8081', + schema.registry = 'http://schemaregistry:8082', message = 'test.package.MessageA'); statement ok @@ -78,7 +78,7 @@ create sink sink_csr_nested as select sint32_field as field_i from into_kafka wi properties.bootstrap.server = 'message_queue:29092') format plain encode protobuf ( force_append_only = true, - schema.registry = 'http://message_queue:8081', + schema.registry = 'http://schemaregistry:8082', message = 'test.package.MessageH.MessageI'); sleep 2s diff --git a/e2e_test/source/basic/nosim_kafka.slt b/e2e_test/source/basic/nosim_kafka.slt index 12626b6926fdf..f143471e0f269 100644 --- a/e2e_test/source/basic/nosim_kafka.slt +++ b/e2e_test/source/basic/nosim_kafka.slt @@ -1,3 +1,6 @@ +control substitution on + +# FIXME: does this really work?? # Start with nosim to avoid running in deterministic test @@ -7,18 +10,18 @@ CREATE TABLE upsert_avro_json_default_key ( primary key (rw_key) ) INCLUDE KEY AS rw_key WITH ( connector = 'kafka', - properties.bootstrap.server = 'message_queue:29092', + properties.bootstrap.server = '${RISEDEV_KAFKA_BOOTSTRAP_SERVERS}', topic = 'upsert_avro_json') -FORMAT UPSERT ENCODE AVRO (schema.registry = 'http://message_queue:8081'); +FORMAT UPSERT ENCODE AVRO (schema.registry = '${RISEDEV_SCHEMA_REGISTRY_URL}'); statement ok CREATE TABLE upsert_student_avro_json ( primary key (rw_key) ) INCLUDE KEY AS rw_key WITH ( connector = 'kafka', - properties.bootstrap.server = 'message_queue:29092', + properties.bootstrap.server = '${RISEDEV_KAFKA_BOOTSTRAP_SERVERS}', topic = 'upsert_student_avro_json') -FORMAT UPSERT ENCODE AVRO (schema.registry = 'http://message_queue:8081'); +FORMAT UPSERT ENCODE AVRO (schema.registry = '${RISEDEV_SCHEMA_REGISTRY_URL}'); # TODO: Uncomment this when we add test data kafka key with format `"ID":id` @@ -28,35 +31,35 @@ FORMAT UPSERT ENCODE AVRO (schema.registry = 'http://message_queue:8081'); # ) # WITH ( # connector = 'kafka', -# properties.bootstrap.server = 'message_queue:29092', +# properties.bootstrap.server = '${RISEDEV_KAFKA_BOOTSTRAP_SERVERS}', # topic = 'upsert_avro_json') -# FORMAT UPSERT ENCODE AVRO (schema.registry = 'http://message_queue:8081'); +# FORMAT UPSERT ENCODE AVRO (schema.registry = '${RISEDEV_SCHEMA_REGISTRY_URL}'); statement ok CREATE TABLE debezium_non_compact (PRIMARY KEY(order_id)) with ( connector = 'kafka', kafka.topic = 'debezium_non_compact_avro_json', - kafka.brokers = 'message_queue:29092', + kafka.brokers = '${RISEDEV_KAFKA_BOOTSTRAP_SERVERS}', kafka.scan.startup.mode = 'earliest' -) FORMAT DEBEZIUM ENCODE AVRO (schema.registry = 'http://message_queue:8081'); +) FORMAT DEBEZIUM ENCODE AVRO (schema.registry = '${RISEDEV_SCHEMA_REGISTRY_URL}'); statement ok CREATE TABLE debezium_compact (PRIMARY KEY(order_id)) with ( connector = 'kafka', kafka.topic = 'debezium_compact_avro_json', - kafka.brokers = 'message_queue:29092', + kafka.brokers = '${RISEDEV_KAFKA_BOOTSTRAP_SERVERS}', kafka.scan.startup.mode = 'earliest' -) FORMAT DEBEZIUM ENCODE AVRO (schema.registry = 'http://message_queue:8081'); +) FORMAT DEBEZIUM ENCODE AVRO (schema.registry = '${RISEDEV_SCHEMA_REGISTRY_URL}'); statement ok CREATE TABLE kafka_json_schema_plain with ( connector = 'kafka', kafka.topic = 'kafka_json_schema', - kafka.brokers = 'message_queue:29092', + kafka.brokers = '${RISEDEV_KAFKA_BOOTSTRAP_SERVERS}', kafka.scan.startup.mode = 'earliest' -) FORMAT PLAIN ENCODE JSON (schema.registry = 'http://schemaregistry:8082'); +) FORMAT PLAIN ENCODE JSON (schema.registry = '${RISEDEV_SCHEMA_REGISTRY_URL}'); statement ok CREATE TABLE kafka_json_schema_upsert (PRIMARY KEY(rw_key)) @@ -64,9 +67,9 @@ INCLUDE KEY AS rw_key with ( connector = 'kafka', kafka.topic = 'kafka_upsert_json_schema', - kafka.brokers = 'message_queue:29092', + kafka.brokers = '${RISEDEV_KAFKA_BOOTSTRAP_SERVERS}', kafka.scan.startup.mode = 'earliest' -) FORMAT UPSERT ENCODE JSON (schema.registry = 'http://schemaregistry:8082'); +) FORMAT UPSERT ENCODE JSON (schema.registry = '${RISEDEV_SCHEMA_REGISTRY_URL}'); statement ok flush; diff --git a/e2e_test/source/basic/schema_registry.slt b/e2e_test/source/basic/schema_registry.slt index 76f867b2b1d0e..4673e441e80c6 100644 --- a/e2e_test/source/basic/schema_registry.slt +++ b/e2e_test/source/basic/schema_registry.slt @@ -5,7 +5,7 @@ create source s1 () with ( topic = 'upsert_avro_json-record', properties.bootstrap.server = 'message_queue:29092' ) format plain encode avro ( - schema.registry = 'http://message_queue:8081', + schema.registry = 'http://schemaregistry:8082', schema.registry.name.strategy = 'no sense', message = 'CPLM.OBJ_ATTRIBUTE_VALUE', ); @@ -17,7 +17,7 @@ create source s1 () with ( topic = 'upsert_avro_json-record', properties.bootstrap.server = 'message_queue:29092' ) format plain encode avro ( - schema.registry = 'http://message_queue:8081', + schema.registry = 'http://schemaregistry:8082', schema.registry.name.strategy = 'record_name_strategy', message = 'CPLM.OBJ_ATTRIBUTE_VALUE', key.message = 'string' @@ -29,7 +29,7 @@ create source s1 () with ( topic = 'upsert_avro_json-record', properties.bootstrap.server = 'message_queue:29092' ) format plain encode avro ( - schema.registry = 'http://message_queue:8081', + schema.registry = 'http://schemaregistry:8082', schema.registry.name.strategy = 'record_name_strategy', message = 'CPLM.OBJ_ATTRIBUTE_VALUE', ); @@ -41,7 +41,7 @@ create table t1 () with ( topic = 'upsert_avro_json-topic-record', properties.bootstrap.server = 'message_queue:29092' ) format upsert encode avro ( - schema.registry = 'http://message_queue:8081', + schema.registry = 'http://schemaregistry:8082', schema.registry.name.strategy = 'topic_record_name_strategy', message = 'CPLM.OBJ_ATTRIBUTE_VALUE' ); @@ -54,7 +54,7 @@ with ( topic = 'upsert_avro_json-topic-record', properties.bootstrap.server = 'message_queue:29092' ) format upsert encode avro ( - schema.registry = 'http://message_queue:8081', + schema.registry = 'http://schemaregistry:8082', schema.registry.name.strategy = 'topic_record_name_strategy', message = 'CPLM.OBJ_ATTRIBUTE_VALUE', key.message = 'string' diff --git a/e2e_test/source/cdc/cdc.share_stream.slt b/e2e_test/source/cdc/cdc.share_stream.slt index e07a0c1d773ef..d30d9c53dc6fe 100644 --- a/e2e_test/source/cdc/cdc.share_stream.slt +++ b/e2e_test/source/cdc/cdc.share_stream.slt @@ -27,10 +27,9 @@ statement error Should not create MATERIALIZED VIEW or SELECT directly on shared create materialized view mv as select * from mysql_mytest; statement error The upstream table name must contain database name prefix* -create table products_test ( id INT, +create table products_test ( id INT PRIMARY KEY, name STRING, - description STRING, - PRIMARY KEY (id) + description STRING ) from mysql_mytest table 'products'; statement ok @@ -233,12 +232,11 @@ CREATE TABLE IF NOT EXISTS postgres_all_types( statement error The upstream table name must contain schema name prefix* CREATE TABLE person_new ( - id int, + id int PRIMARY KEY, name varchar, email_address varchar, credit_card varchar, - city varchar, - PRIMARY KEY (id) + city varchar ) FROM pg_source TABLE 'person'; statement ok diff --git a/e2e_test/subscription/main.py b/e2e_test/subscription/main.py index c7fcc56a35ac5..3ffaefd02cee6 100644 --- a/e2e_test/subscription/main.py +++ b/e2e_test/subscription/main.py @@ -33,8 +33,7 @@ def execute_insert(sql,conn): conn.commit() cur.close() -def check_rows_data(expect_vec,rows,status): - row = rows[0] +def check_rows_data(expect_vec,row,status): value_len = len(row) for index, value in enumerate(row): if index == value_len - 1: @@ -56,7 +55,7 @@ def test_cursor_snapshot(): execute_insert("declare cur subscription cursor for sub",conn) row = execute_query("fetch next from cur",conn) - check_rows_data([1,2],row,1) + check_rows_data([1,2],row[0],1) row = execute_query("fetch next from cur",conn) assert row == [] execute_insert("close cur",conn) @@ -75,7 +74,7 @@ def test_cursor_snapshot_log_store(): execute_insert("declare cur subscription cursor for sub",conn) row = execute_query("fetch next from cur",conn) - check_rows_data([1,2],row,1) + check_rows_data([1,2],row[0],1) row = execute_query("fetch next from cur",conn) assert row == [] execute_insert("insert into t1 values(4,4)",conn) @@ -83,9 +82,9 @@ def test_cursor_snapshot_log_store(): execute_insert("insert into t1 values(5,5)",conn) execute_insert("flush",conn) row = execute_query("fetch next from cur",conn) - check_rows_data([4,4],row,1) + check_rows_data([4,4],row[0],1) row = execute_query("fetch next from cur",conn) - check_rows_data([5,5],row,1) + check_rows_data([5,5],row[0],1) row = execute_query("fetch next from cur",conn) assert row == [] execute_insert("close cur",conn) @@ -109,11 +108,11 @@ def test_cursor_since_begin(): execute_insert("insert into t1 values(6,6)",conn) execute_insert("flush",conn) row = execute_query("fetch next from cur",conn) - check_rows_data([4,4],row,1) + check_rows_data([4,4],row[0],1) row = execute_query("fetch next from cur",conn) - check_rows_data([5,5],row,1) + check_rows_data([5,5],row[0],1) row = execute_query("fetch next from cur",conn) - check_rows_data([6,6],row,1) + check_rows_data([6,6],row[0],1) row = execute_query("fetch next from cur",conn) assert row == [] execute_insert("close cur",conn) @@ -138,7 +137,7 @@ def test_cursor_since_now(): execute_insert("insert into t1 values(6,6)",conn) execute_insert("flush",conn) row = execute_query("fetch next from cur",conn) - check_rows_data([6,6],row,1) + check_rows_data([6,6],row[0],1) row = execute_query("fetch next from cur",conn) assert row == [] execute_insert("close cur",conn) @@ -164,27 +163,27 @@ def test_cursor_since_rw_timestamp(): row = execute_query("fetch next from cur",conn) valuelen = len(row[0]) rw_timestamp_1 = row[0][valuelen - 1] - check_rows_data([4,4],row,1) + check_rows_data([4,4],row[0],1) row = execute_query("fetch next from cur",conn) valuelen = len(row[0]) rw_timestamp_2 = row[0][valuelen - 1] - 1 - check_rows_data([5,5],row,1) + check_rows_data([5,5],row[0],1) row = execute_query("fetch next from cur",conn) valuelen = len(row[0]) rw_timestamp_3 = row[0][valuelen - 1] + 1 - check_rows_data([6,6],row,1) + check_rows_data([6,6],row[0],1) row = execute_query("fetch next from cur",conn) assert row == [] execute_insert("close cur",conn) execute_insert(f"declare cur subscription cursor for sub since {rw_timestamp_1}",conn) row = execute_query("fetch next from cur",conn) - check_rows_data([4,4],row,1) + check_rows_data([4,4],row[0],1) execute_insert("close cur",conn) execute_insert(f"declare cur subscription cursor for sub since {rw_timestamp_2}",conn) row = execute_query("fetch next from cur",conn) - check_rows_data([5,5],row,1) + check_rows_data([5,5],row[0],1) execute_insert("close cur",conn) execute_insert(f"declare cur subscription cursor for sub since {rw_timestamp_3}",conn) @@ -206,7 +205,7 @@ def test_cursor_op(): execute_insert("declare cur subscription cursor for sub",conn) row = execute_query("fetch next from cur",conn) - check_rows_data([1,2],row,1) + check_rows_data([1,2],row[0],1) row = execute_query("fetch next from cur",conn) assert row == [] @@ -215,24 +214,96 @@ def test_cursor_op(): execute_insert("update t1 set v2 = 10 where v1 = 4",conn) execute_insert("flush",conn) row = execute_query("fetch next from cur",conn) - check_rows_data([4,4],row,1) + check_rows_data([4,4],row[0],1) row = execute_query("fetch next from cur",conn) - check_rows_data([4,4],row,4) + check_rows_data([4,4],row[0],4) row = execute_query("fetch next from cur",conn) - check_rows_data([4,10],row,3) + check_rows_data([4,10],row[0],3) row = execute_query("fetch next from cur",conn) assert row == [] execute_insert("delete from t1 where v1 = 4",conn) execute_insert("flush",conn) row = execute_query("fetch next from cur",conn) - check_rows_data([4,10],row,2) + check_rows_data([4,10],row[0],2) row = execute_query("fetch next from cur",conn) assert row == [] execute_insert("close cur",conn) drop_table_subscription() +def test_cursor_with_table_alter(): + print(f"test_cursor_with_table_alter") + create_table_subscription() + conn = psycopg2.connect( + host="localhost", + port="4566", + user="root", + database="dev" + ) + + execute_insert("declare cur subscription cursor for sub",conn) + execute_insert("alter table t1 add v3 int",conn) + execute_insert("insert into t1 values(4,4,4)",conn) + execute_insert("flush",conn) + row = execute_query("fetch next from cur",conn) + check_rows_data([1,2],row[0],1) + row = execute_query("fetch next from cur",conn) + check_rows_data([4,4,4],row[0],1) + execute_insert("insert into t1 values(5,5,5)",conn) + execute_insert("flush",conn) + row = execute_query("fetch next from cur",conn) + check_rows_data([5,5,5],row[0],1) + execute_insert("alter table t1 drop column v2",conn) + execute_insert("insert into t1 values(6,6)",conn) + execute_insert("flush",conn) + row = execute_query("fetch next from cur",conn) + check_rows_data([6,6],row[0],1) + drop_table_subscription() + +def test_cursor_fetch_n(): + print(f"test_cursor_with_table_alter") + create_table_subscription() + conn = psycopg2.connect( + host="localhost", + port="4566", + user="root", + database="dev" + ) + + execute_insert("declare cur subscription cursor for sub",conn) + execute_insert("insert into t1 values(4,4)",conn) + execute_insert("flush",conn) + execute_insert("insert into t1 values(5,5)",conn) + execute_insert("flush",conn) + execute_insert("insert into t1 values(6,6)",conn) + execute_insert("flush",conn) + execute_insert("insert into t1 values(7,7)",conn) + execute_insert("flush",conn) + execute_insert("insert into t1 values(8,8)",conn) + execute_insert("flush",conn) + execute_insert("insert into t1 values(9,9)",conn) + execute_insert("flush",conn) + execute_insert("insert into t1 values(10,10)",conn) + execute_insert("flush",conn) + execute_insert("update t1 set v2 = 100 where v1 = 10",conn) + execute_insert("flush",conn) + row = execute_query("fetch 6 from cur",conn) + assert len(row) == 6 + check_rows_data([1,2],row[0],1) + check_rows_data([4,4],row[1],1) + check_rows_data([5,5],row[2],1) + check_rows_data([6,6],row[3],1) + check_rows_data([7,7],row[4],1) + check_rows_data([8,8],row[5],1) + row = execute_query("fetch 6 from cur",conn) + assert len(row) == 4 + check_rows_data([9,9],row[0],1) + check_rows_data([10,10],row[1],1) + check_rows_data([10,10],row[2],4) + check_rows_data([10,100],row[3],3) + drop_table_subscription() + if __name__ == "__main__": test_cursor_snapshot() test_cursor_op() @@ -240,3 +311,5 @@ def test_cursor_op(): test_cursor_since_rw_timestamp() test_cursor_since_now() test_cursor_since_begin() + test_cursor_with_table_alter() + test_cursor_fetch_n() diff --git a/proto/hummock.proto b/proto/hummock.proto index 8d68ec168ef21..7caf27e155deb 100644 --- a/proto/hummock.proto +++ b/proto/hummock.proto @@ -382,6 +382,8 @@ message CompactTask { map table_watermarks = 24; // The table schemas that are at least as new as the one used to create `input_ssts`. map table_schemas = 25; + // Max sub compaction task numbers + uint32 max_sub_compaction = 26; } message LevelHandler { diff --git a/risedev.yml b/risedev.yml index 65f84882c682c..f730a69d5f444 100644 --- a/risedev.yml +++ b/risedev.yml @@ -861,6 +861,10 @@ profile: user-managed: true address: message_queue port: 29092 + - use: schema-registry + user-managed: true + address: schemaregistry + port: 8082 ci-inline-source-test: config-path: src/config/ci-recovery.toml @@ -950,7 +954,7 @@ profile: - use: frontend - use: compactor - ci-1cn-1fe-kafka-with-recovery: + ci-1cn-1fe-user-kafka-with-recovery: config-path: src/config/ci-recovery.toml steps: - use: minio @@ -962,7 +966,9 @@ profile: - use: frontend - use: compactor - use: kafka - persist-data: true + user-managed: true + address: message_queue + port: 29092 ci-meta-backup-test-etcd: config-path: src/config/ci-meta-backup-test.toml @@ -1431,9 +1437,8 @@ template: # Listen port of KRaft controller controller-port: 29093 - - # Listen address - listen-address: ${address} + # Listen port for other services in docker (schema-registry) + docker-port: 29094 # The docker image. Can be overridden to use a different version. image: "confluentinc/cp-kafka:7.6.1" @@ -1446,6 +1451,23 @@ template: user-managed: false + schema-registry: + # Id to be picked-up by services + id: schema-registry-${port} + + # Advertise address + address: "127.0.0.1" + + # Listen port of Schema Registry + port: 8081 + + # The docker image. Can be overridden to use a different version. + image: "confluentinc/cp-schema-registry:7.6.1" + + user-managed: false + + provide-kafka: "kafka*" + # Google pubsub emulator service pubsub: id: pubsub-${port} diff --git a/scripts/source/prepare_ci_kafka.sh b/scripts/source/prepare_ci_kafka.sh index e50229a73759f..9f3e2f473ca9b 100755 --- a/scripts/source/prepare_ci_kafka.sh +++ b/scripts/source/prepare_ci_kafka.sh @@ -56,7 +56,7 @@ for filename in $kafka_data_files; do if [[ "$topic" = *bin ]]; then kcat -P -b message_queue:29092 -t "$topic" "$filename" elif [[ "$topic" = *avro_json ]]; then - python3 source/schema_registry_producer.py "message_queue:29092" "http://message_queue:8081" "$filename" "topic" "avro" + python3 source/schema_registry_producer.py "message_queue:29092" "http://schemaregistry:8082" "$filename" "topic" "avro" elif [[ "$topic" = *json_schema ]]; then python3 source/schema_registry_producer.py "message_queue:29092" "http://schemaregistry:8082" "$filename" "topic" "json" else @@ -72,9 +72,9 @@ for i in {0..100}; do echo "key$i:{\"a\": $i}" | kcat -P -b message_queue:29092 # write schema with name strategy ## topic: upsert_avro_json-record, key subject: string, value subject: CPLM.OBJ_ATTRIBUTE_VALUE -(python3 source/schema_registry_producer.py "message_queue:29092" "http://message_queue:8081" source/test_data/upsert_avro_json.1 "record" "avro") & +(python3 source/schema_registry_producer.py "message_queue:29092" "http://schemaregistry:8082" source/test_data/upsert_avro_json.1 "record" "avro") & ## topic: upsert_avro_json-topic-record, ## key subject: upsert_avro_json-topic-record-string ## value subject: upsert_avro_json-topic-record-CPLM.OBJ_ATTRIBUTE_VALUE -(python3 source/schema_registry_producer.py "message_queue:29092" "http://message_queue:8081" source/test_data/upsert_avro_json.1 "topic-record" "avro") & +(python3 source/schema_registry_producer.py "message_queue:29092" "http://schemaregistry:8082" source/test_data/upsert_avro_json.1 "topic-record" "avro") & wait diff --git a/scripts/source/schema_registry_producer.py b/scripts/source/schema_registry_producer.py index 79a3d4db1b40f..a88861b65bd26 100644 --- a/scripts/source/schema_registry_producer.py +++ b/scripts/source/schema_registry_producer.py @@ -39,8 +39,11 @@ def load_avro_json(encoded, schema): if __name__ == '__main__': - if len(sys.argv) < 5: - print("datagen.py ") + if len(sys.argv) <= 5: + print( + "usage: schema_registry_producer.py " + ) + exit(1) broker_list = sys.argv[1] schema_registry_url = sys.argv[2] file = sys.argv[3] diff --git a/src/batch/src/executor/hash_agg.rs b/src/batch/src/executor/hash_agg.rs index cb4adcecdc8c7..00f7366655e0d 100644 --- a/src/batch/src/executor/hash_agg.rs +++ b/src/batch/src/executor/hash_agg.rs @@ -188,6 +188,8 @@ pub struct HashAggExecutor { chunk_size: usize, mem_context: MemoryContext, enable_spill: bool, + /// The upper bound of memory usage for this executor. + memory_upper_bound: Option, shutdown_rx: ShutdownToken, _phantom: PhantomData, } @@ -205,7 +207,7 @@ impl HashAggExecutor { enable_spill: bool, shutdown_rx: ShutdownToken, ) -> Self { - Self::new_with_init_agg_state( + Self::new_inner( aggs, group_key_columns, group_key_types, @@ -216,12 +218,13 @@ impl HashAggExecutor { chunk_size, mem_context, enable_spill, + None, shutdown_rx, ) } #[allow(clippy::too_many_arguments)] - fn new_with_init_agg_state( + fn new_inner( aggs: Arc>, group_key_columns: Vec, group_key_types: Vec, @@ -232,6 +235,7 @@ impl HashAggExecutor { chunk_size: usize, mem_context: MemoryContext, enable_spill: bool, + memory_upper_bound: Option, shutdown_rx: ShutdownToken, ) -> Self { HashAggExecutor { @@ -245,6 +249,7 @@ impl HashAggExecutor { chunk_size, mem_context, enable_spill, + memory_upper_bound, shutdown_rx, _phantom: PhantomData, } @@ -461,6 +466,22 @@ impl AggSpillManager { Ok(Self::read_stream(r)) } + async fn estimate_partition_size(&self, partition: usize) -> Result { + let agg_state_partition_file_name = format!("agg-state-p{}", partition); + let agg_state_size = self + .op + .stat(&agg_state_partition_file_name) + .await? + .content_length(); + let input_partition_file_name = format!("input-chunks-p{}", partition); + let input_size = self + .op + .stat(&input_partition_file_name) + .await? + .content_length(); + Ok(agg_state_size + input_size) + } + async fn clear_partition(&mut self, partition: usize) -> Result<()> { let agg_state_partition_file_name = format!("agg-state-p{}", partition); self.op.delete(&agg_state_partition_file_name).await?; @@ -470,11 +491,18 @@ impl AggSpillManager { } } +const SPILL_AT_LEAST_MEMORY: u64 = 1024 * 1024; + impl HashAggExecutor { #[try_stream(boxed, ok = DataChunk, error = BatchError)] async fn do_execute(self: Box) { let child_schema = self.child.schema().clone(); let mut need_to_spill = false; + // If the memory upper bound is less than 1MB, we don't need to check memory usage. + let check_memory = match self.memory_upper_bound { + Some(upper_bound) => upper_bound > SPILL_AT_LEAST_MEMORY, + None => true, + }; // hash map for each agg groups let mut groups = AggHashMap::::with_hasher_in( @@ -508,7 +536,7 @@ impl HashAggExecutor { groups.try_insert(key, agg_states).unwrap(); } - if !self.mem_context.add(memory_usage_diff) { + if !self.mem_context.add(memory_usage_diff) && check_memory { warn!("not enough memory to load one partition agg state after spill which is not a normal case, so keep going"); } } @@ -553,7 +581,7 @@ impl HashAggExecutor { } } // update memory usage - if !self.mem_context.add(memory_usage_diff) { + if !self.mem_context.add(memory_usage_diff) && check_memory { if self.enable_spill { need_to_spill = true; break; @@ -624,26 +652,28 @@ impl HashAggExecutor { // Process each partition one by one. for i in 0..agg_spill_manager.partition_num { + let partition_size = agg_spill_manager.estimate_partition_size(i).await?; + let agg_state_stream = agg_spill_manager.read_agg_state_partition(i).await?; let input_stream = agg_spill_manager.read_input_partition(i).await?; - let sub_hash_agg_executor: HashAggExecutor = - HashAggExecutor::new_with_init_agg_state( - self.aggs.clone(), - self.group_key_columns.clone(), - self.group_key_types.clone(), + let sub_hash_agg_executor: HashAggExecutor = HashAggExecutor::new_inner( + self.aggs.clone(), + self.group_key_columns.clone(), + self.group_key_types.clone(), + self.schema.clone(), + Box::new(WrapStreamExecutor::new(child_schema.clone(), input_stream)), + Some(Box::new(WrapStreamExecutor::new( self.schema.clone(), - Box::new(WrapStreamExecutor::new(child_schema.clone(), input_stream)), - Some(Box::new(WrapStreamExecutor::new( - self.schema.clone(), - agg_state_stream, - ))), - format!("{}-sub{}", self.identity.clone(), i), - self.chunk_size, - self.mem_context.clone(), - self.enable_spill, - self.shutdown_rx.clone(), - ); + agg_state_stream, + ))), + format!("{}-sub{}", self.identity.clone(), i), + self.chunk_size, + self.mem_context.clone(), + self.enable_spill, + Some(partition_size), + self.shutdown_rx.clone(), + ); debug!( "create sub_hash_agg {} for hash_agg {} to spill", diff --git a/src/common/src/config.rs b/src/common/src/config.rs index c8da0f6dce5e9..26e8bcaf1f56b 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -721,10 +721,6 @@ pub struct StorageConfig { #[serde(default = "default::storage::min_sst_size_for_streaming_upload")] pub min_sst_size_for_streaming_upload: u64, - /// Max sub compaction task numbers - #[serde(default = "default::storage::max_sub_compaction")] - pub max_sub_compaction: u32, - #[serde(default = "default::storage::max_concurrent_compaction_task_number")] pub max_concurrent_compaction_task_number: u64, @@ -1461,10 +1457,6 @@ pub mod default { 32 * 1024 * 1024 } - pub fn max_sub_compaction() -> u32 { - 4 - } - pub fn max_concurrent_compaction_task_number() -> u64 { 16 } diff --git a/src/config/docs.md b/src/config/docs.md index 018c9dd41087c..0a024ba992db0 100644 --- a/src/config/docs.md +++ b/src/config/docs.md @@ -121,7 +121,6 @@ This page is automatically generated by `./risedev generate-example-config` | max_prefetch_block_number | max prefetch block number | 16 | | max_preload_io_retry_times | | 3 | | max_preload_wait_time_mill | | 0 | -| max_sub_compaction | Max sub compaction task numbers | 4 | | max_version_pinning_duration_sec | | 10800 | | mem_table_spill_threshold | The spill threshold for mem table. | 4194304 | | meta_cache_capacity_mb | DEPRECATED: This config will be deprecated in the future version, use `storage.cache.meta_cache_capacity_mb` instead. | | diff --git a/src/config/example.toml b/src/config/example.toml index 00b1ef759e5f9..93546c7bdd238 100644 --- a/src/config/example.toml +++ b/src/config/example.toml @@ -136,7 +136,6 @@ compactor_memory_available_proportion = 0.8 sstable_id_remote_fetch_number = 10 min_sstable_size_mb = 32 min_sst_size_for_streaming_upload = 33554432 -max_sub_compaction = 4 max_concurrent_compaction_task_number = 16 max_preload_wait_time_mill = 0 max_version_pinning_duration_sec = 10800 diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 64e3d159daa71..e73fb35e63267 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -179,7 +179,7 @@ prost-build = "0.12" protobuf-src = "1" [[bench]] -name = "parser" +name = "debezium_json_parser" harness = false [[bench]] @@ -187,7 +187,11 @@ name = "nexmark_integration" harness = false [[bench]] -name = "json_parser" +name = "json_parser_case_insensitive" +harness = false + +[[bench]] +name = "json_vs_plain_parser" harness = false [lints] diff --git a/src/connector/benches/debezium_json_parser.rs b/src/connector/benches/debezium_json_parser.rs new file mode 100644 index 0000000000000..e448fa17ad1db --- /dev/null +++ b/src/connector/benches/debezium_json_parser.rs @@ -0,0 +1,86 @@ +// 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. + +//! Benchmark for Debezium JSON records with `DebeziumParser`. + +mod json_common; + +use criterion::{criterion_group, criterion_main, BatchSize, Criterion}; +use futures::executor::block_on; +use json_common::*; +use paste::paste; +use rand::Rng; +use risingwave_connector::parser::{DebeziumParser, SourceStreamChunkBuilder}; + +fn generate_debezium_json_row(rng: &mut impl Rng, change_event: &str) -> String { + let source = r#"{"version":"1.7.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1639547113601,"snapshot":"true","db":"inventory","sequence":null,"table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":156,"row":0,"thread":null,"query":null}"#; + let (before, after) = match change_event { + "c" => ("null".to_string(), generate_json_row(rng)), + "r" => ("null".to_string(), generate_json_row(rng)), + "u" => (generate_json_row(rng), generate_json_row(rng)), + "d" => (generate_json_row(rng), "null".to_string()), + _ => unreachable!(), + }; + format!("{{\"before\": {before}, \"after\": {after}, \"source\": {source}, \"op\": \"{change_event}\", \"ts_ms\":1639551564960, \"transaction\":null}}") +} + +macro_rules! create_debezium_bench_helpers { + ($op:ident, $op_sym:expr, $bench_function:expr) => { + paste! { + fn [](c: &mut Criterion) { + let rt = tokio::runtime::Builder::new_multi_thread() + .enable_all() + .build() + .unwrap(); + + // Generate records + let mut rng = rand::thread_rng(); + let mut records = Vec::with_capacity(NUM_RECORDS); + for _ in 0..NUM_RECORDS { + let json_row = generate_debezium_json_row(&mut rng, $op_sym); + records.push(Some(json_row.into_bytes())); + } + + c.bench_function($bench_function, |b| { + b.to_async(&rt).iter_batched( + || (block_on(DebeziumParser::new_for_test(get_descs())).unwrap(), records.clone()) , + | (mut parser, records) | async move { + let mut builder = + SourceStreamChunkBuilder::with_capacity(get_descs(), NUM_RECORDS); + for record in records { + let writer = builder.row_writer(); + parser.parse_inner(None, record, writer).await.unwrap(); + } + }, + BatchSize::SmallInput, + ) + }); + } + } + }; +} + +create_debezium_bench_helpers!(create, "c", "bench_debezium_json_parser_create"); +create_debezium_bench_helpers!(read, "r", "bench_debezium_json_parser_read"); +create_debezium_bench_helpers!(update, "u", "bench_debezium_json_parser_update"); +create_debezium_bench_helpers!(delete, "d", "bench_debezium_json_parser_delete"); + +criterion_group!( + benches, + bench_debezium_json_parser_create, + bench_debezium_json_parser_read, + bench_debezium_json_parser_update, + bench_debezium_json_parser_delete +); +criterion_main!(benches); diff --git a/src/connector/benches/json_common/mod.rs b/src/connector/benches/json_common/mod.rs new file mode 100644 index 0000000000000..cb67c4cb3d547 --- /dev/null +++ b/src/connector/benches/json_common/mod.rs @@ -0,0 +1,57 @@ +// 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. + +//! Common utilities shared by JSON parser benchmarks. + +use rand::distributions::Alphanumeric; +use rand::prelude::*; +use risingwave_common::catalog::ColumnId; +use risingwave_common::types::{DataType, Date, Timestamp}; +use risingwave_connector::source::SourceColumnDesc; + +pub const NUM_RECORDS: usize = 1 << 18; // ~ 250,000 + +pub fn generate_json_row(rng: &mut impl Rng) -> String { + format!("{{\"i32\":{},\"bool\":{},\"i16\":{},\"i64\":{},\"f32\":{},\"f64\":{},\"varchar\":\"{}\",\"date\":\"{}\",\"timestamp\":\"{}\"}}", + rng.gen::(), + rng.gen::(), + rng.gen::(), + rng.gen::(), + rng.gen::(), + rng.gen::(), + rng.sample_iter(&Alphanumeric) + .take(7) + .map(char::from) + .collect::(), + Date::from_num_days_from_ce_uncheck((rng.gen::() % (1 << 20)) as i32).0, + { + let datetime = Timestamp::from_timestamp_uncheck((rng.gen::() % (1u32 << 28)) as i64, 0).0; + format!("{:?} {:?}", datetime.date(), datetime.time()) + } + ) +} + +pub fn get_descs() -> Vec { + vec![ + SourceColumnDesc::simple("i32", DataType::Int32, ColumnId::from(0)), + SourceColumnDesc::simple("bool", DataType::Boolean, ColumnId::from(2)), + SourceColumnDesc::simple("i16", DataType::Int16, ColumnId::from(3)), + SourceColumnDesc::simple("i64", DataType::Int64, ColumnId::from(4)), + SourceColumnDesc::simple("f32", DataType::Float32, ColumnId::from(5)), + SourceColumnDesc::simple("f64", DataType::Float64, ColumnId::from(6)), + SourceColumnDesc::simple("varchar", DataType::Varchar, ColumnId::from(7)), + SourceColumnDesc::simple("date", DataType::Date, ColumnId::from(8)), + SourceColumnDesc::simple("timestamp", DataType::Timestamp, ColumnId::from(9)), + ] +} diff --git a/src/connector/benches/json_parser.rs b/src/connector/benches/json_parser.rs deleted file mode 100644 index 5a12dec735cab..0000000000000 --- a/src/connector/benches/json_parser.rs +++ /dev/null @@ -1,226 +0,0 @@ -// 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::sync::Arc; - -use criterion::{criterion_group, criterion_main, BatchSize, Criterion}; -use futures::executor::block_on; -use paste::paste; -use rand::distributions::Alphanumeric; -use rand::prelude::*; -use risingwave_common::catalog::ColumnId; -use risingwave_common::types::{DataType, Date, Timestamp}; -use risingwave_connector::parser::plain_parser::PlainParser; -use risingwave_connector::parser::{ - DebeziumParser, JsonParser, SourceStreamChunkBuilder, SpecificParserConfig, -}; -use risingwave_connector::source::{SourceColumnDesc, SourceContext}; - -macro_rules! create_debezium_bench_helpers { - ($op:ident, $op_sym:expr, $bench_function:expr) => { - paste! { - fn [](c: &mut Criterion) { - let rt = tokio::runtime::Builder::new_multi_thread() - .enable_all() - .build() - .unwrap(); - - // Generate records - let mut rng = rand::thread_rng(); - let mut records = Vec::with_capacity(NUM_RECORDS); - for _ in 0..NUM_RECORDS { - let json_row = generate_debezium_json_row(&mut rng, $op_sym); - records.push(Some(json_row.into_bytes())); - } - - c.bench_function($bench_function, |b| { - b.to_async(&rt).iter_batched( - || (block_on(DebeziumParser::new_for_test(get_descs())).unwrap(), records.clone()) , - | (mut parser, records) | async move { - let mut builder = - SourceStreamChunkBuilder::with_capacity(get_descs(), NUM_RECORDS); - for record in records { - let writer = builder.row_writer(); - parser.parse_inner(None, record, writer).await.unwrap(); - } - }, - BatchSize::SmallInput, - ) - }); - } - } - }; -} - -create_debezium_bench_helpers!(create, "c", "bench_debezium_json_parser_create"); -create_debezium_bench_helpers!(read, "r", "bench_debezium_json_parser_read"); -create_debezium_bench_helpers!(update, "u", "bench_debezium_json_parser_update"); -create_debezium_bench_helpers!(delete, "d", "bench_debezium_json_parser_delete"); - -const NUM_RECORDS: usize = 1 << 18; // ~ 250,000 - -fn generate_json_row(rng: &mut impl Rng) -> String { - format!("{{\"i32\":{},\"bool\":{},\"i16\":{},\"i64\":{},\"f32\":{},\"f64\":{},\"varchar\":\"{}\",\"date\":\"{}\",\"timestamp\":\"{}\"}}", - rng.gen::(), - rng.gen::(), - rng.gen::(), - rng.gen::(), - rng.gen::(), - rng.gen::(), - rng.sample_iter(&Alphanumeric) - .take(7) - .map(char::from) - .collect::(), - Date::from_num_days_from_ce_uncheck((rng.gen::() % (1 << 20)) as i32).0, - { - let datetime = Timestamp::from_timestamp_uncheck((rng.gen::() % (1u32 << 28)) as i64, 0).0; - format!("{:?} {:?}", datetime.date(), datetime.time()) - } - ) -} - -fn generate_json_rows() -> Vec> { - let mut rng = rand::thread_rng(); - let mut records = Vec::with_capacity(NUM_RECORDS); - for _ in 0..NUM_RECORDS { - records.push(generate_json_row(&mut rng).into_bytes()); - } - records -} - -fn generate_debezium_json_row(rng: &mut impl Rng, change_event: &str) -> String { - let source = r#"{"version":"1.7.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1639547113601,"snapshot":"true","db":"inventory","sequence":null,"table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":156,"row":0,"thread":null,"query":null}"#; - let (before, after) = match change_event { - "c" => ("null".to_string(), generate_json_row(rng)), - "r" => ("null".to_string(), generate_json_row(rng)), - "u" => (generate_json_row(rng), generate_json_row(rng)), - "d" => (generate_json_row(rng), "null".to_string()), - _ => unreachable!(), - }; - format!("{{\"before\": {before}, \"after\": {after}, \"source\": {source}, \"op\": \"{change_event}\", \"ts_ms\":1639551564960, \"transaction\":null}}") -} - -fn get_descs() -> Vec { - vec![ - SourceColumnDesc::simple("i32", DataType::Int32, ColumnId::from(0)), - SourceColumnDesc::simple("bool", DataType::Boolean, ColumnId::from(2)), - SourceColumnDesc::simple("i16", DataType::Int16, ColumnId::from(3)), - SourceColumnDesc::simple("i64", DataType::Int64, ColumnId::from(4)), - SourceColumnDesc::simple("f32", DataType::Float32, ColumnId::from(5)), - SourceColumnDesc::simple("f64", DataType::Float64, ColumnId::from(6)), - SourceColumnDesc::simple("varchar", DataType::Varchar, ColumnId::from(7)), - SourceColumnDesc::simple("date", DataType::Date, ColumnId::from(8)), - SourceColumnDesc::simple("timestamp", DataType::Timestamp, ColumnId::from(9)), - ] -} - -fn bench_json_parser(c: &mut Criterion) { - let descs = get_descs(); - let rt = tokio::runtime::Builder::new_multi_thread() - .enable_all() - .build() - .unwrap(); - let records = generate_json_rows(); - let ctx = Arc::new(SourceContext::dummy()); - c.bench_function("json_parser", |b| { - b.to_async(&rt).iter_batched( - || records.clone(), - |records| async { - let mut parser = block_on(PlainParser::new( - SpecificParserConfig::DEFAULT_PLAIN_JSON, - descs.clone(), - ctx.clone(), - )) - .unwrap(); - let mut builder = - SourceStreamChunkBuilder::with_capacity(descs.clone(), NUM_RECORDS); - for record in records { - let writer = builder.row_writer(); - parser - .parse_inner(None, Some(record), writer) - .await - .unwrap(); - } - }, - BatchSize::SmallInput, - ) - }); -} - -fn bench_plain_parser_and_json_parser(c: &mut Criterion) { - let rt = tokio::runtime::Runtime::new().unwrap(); - let records = generate_json_rows(); - - let mut group = c.benchmark_group("plain parser and json parser comparison"); - - group.bench_function("plain_parser", |b| { - b.to_async(&rt).iter_batched( - || { - let parser = block_on(PlainParser::new( - SpecificParserConfig::DEFAULT_PLAIN_JSON, - get_descs(), - SourceContext::dummy().into(), - )) - .unwrap(); - (parser, records.clone()) - }, - |(mut parser, records)| async move { - let mut builder = SourceStreamChunkBuilder::with_capacity(get_descs(), NUM_RECORDS); - for record in records { - let writer = builder.row_writer(); - parser - .parse_inner(None, Some(record), writer) - .await - .unwrap(); - } - }, - BatchSize::SmallInput, - ) - }); - - group.bench_function("json_parser", |b| { - b.to_async(&rt).iter_batched( - || { - let parser = JsonParser::new( - SpecificParserConfig::DEFAULT_PLAIN_JSON, - get_descs(), - SourceContext::dummy().into(), - ) - .unwrap(); - (parser, records.clone()) - }, - |(parser, records)| async move { - let mut builder = SourceStreamChunkBuilder::with_capacity(get_descs(), NUM_RECORDS); - for record in records { - let writer = builder.row_writer(); - parser.parse_inner(record, writer).await.unwrap(); - } - }, - BatchSize::SmallInput, - ) - }); - - group.finish(); -} - -criterion_group!( - benches, - bench_json_parser, - bench_plain_parser_and_json_parser, - bench_debezium_json_parser_create, - bench_debezium_json_parser_read, - bench_debezium_json_parser_update, - bench_debezium_json_parser_delete -); -criterion_main!(benches); diff --git a/src/connector/benches/parser.rs b/src/connector/benches/json_parser_case_insensitive.rs similarity index 73% rename from src/connector/benches/parser.rs rename to src/connector/benches/json_parser_case_insensitive.rs index 21ce72dd1b2b1..17fd439e6ccc1 100644 --- a/src/connector/benches/parser.rs +++ b/src/connector/benches/json_parser_case_insensitive.rs @@ -12,24 +12,29 @@ // See the License for the specific language governing permissions and // limitations under the License. +//! Benchmarking JSON parsers for scenarios with exact key matches and case-insensitive key matches. + use criterion::{criterion_group, criterion_main, BatchSize, BenchmarkId, Criterion}; +use futures::StreamExt; use maplit::hashmap; use rand::Rng; use risingwave_common::types::DataType; use risingwave_connector::parser::{ - EncodingProperties, JsonParser, JsonProperties, ProtocolProperties, SourceStreamChunkBuilder, - SpecificParserConfig, + ByteStreamSourceParserImpl, CommonParserConfig, ParserConfig, SpecificParserConfig, }; -use risingwave_connector::source::{SourceColumnDesc, SourceContext}; +use risingwave_connector::source::{SourceColumnDesc, SourceMessage}; use serde_json::json; use tokio::runtime::Runtime; -fn gen_input(mode: &str, chunk_size: usize, chunk_num: usize) -> Vec>> { +type Input = Vec>; +type Parser = ByteStreamSourceParserImpl; + +fn gen_input(mode: &str, chunk_size: usize, chunk_num: usize) -> Input { let mut input = Vec::with_capacity(chunk_num); for _ in 0..chunk_num { let mut input_inner = Vec::with_capacity(chunk_size); for _ in 0..chunk_size { - input_inner.push(match mode { + let payload = match mode { "match" => r#"{"alpha": 1, "bravo": 2, "charlie": 3, "delta": 4}"# .as_bytes() .to_vec(), @@ -55,6 +60,10 @@ fn gen_input(mode: &str, chunk_size: usize, chunk_num: usize) -> Vec serde_json::to_string(&value).unwrap().as_bytes().to_vec() } _ => unreachable!(), + }; + input_inner.push(SourceMessage { + payload: Some(payload), + ..SourceMessage::dummy() }); } input.push(input_inner); @@ -62,40 +71,27 @@ fn gen_input(mode: &str, chunk_size: usize, chunk_num: usize) -> Vec input } -fn create_parser( - chunk_size: usize, - chunk_num: usize, - mode: &str, -) -> (JsonParser, Vec, Vec>>) { +fn create_parser(chunk_size: usize, chunk_num: usize, mode: &str) -> (Parser, Input) { let desc = vec![ SourceColumnDesc::simple("alpha", DataType::Int16, 0.into()), SourceColumnDesc::simple("bravo", DataType::Int32, 1.into()), SourceColumnDesc::simple("charlie", DataType::Int64, 2.into()), SourceColumnDesc::simple("delta", DataType::Int64, 3.into()), ]; - let props = SpecificParserConfig { - key_encoding_config: None, - encoding_config: EncodingProperties::Json(JsonProperties { - use_schema_registry: false, - timestamptz_handling: None, - }), - protocol_config: ProtocolProperties::Plain, + let config = ParserConfig { + common: CommonParserConfig { rw_columns: desc }, + specific: SpecificParserConfig::DEFAULT_PLAIN_JSON, }; - let parser = JsonParser::new(props, desc.clone(), SourceContext::dummy().into()).unwrap(); + let parser = ByteStreamSourceParserImpl::create_for_test(config).unwrap(); let input = gen_input(mode, chunk_size, chunk_num); - (parser, desc, input) + (parser, input) } -async fn parse(parser: JsonParser, column_desc: Vec, input: Vec>>) { - for input_inner in input { - let mut builder = - SourceStreamChunkBuilder::with_capacity(column_desc.clone(), input_inner.len()); - for payload in input_inner { - let row_writer = builder.row_writer(); - parser.parse_inner(payload, row_writer).await.unwrap(); - } - builder.finish(); - } +async fn parse(parser: Parser, input: Input) { + parser + .into_stream(futures::stream::iter(input.into_iter().map(Ok)).boxed()) + .count() // consume the stream + .await; } fn do_bench(c: &mut Criterion, mode: &str) { @@ -110,7 +106,7 @@ fn do_bench(c: &mut Criterion, mode: &str) { let chunk_num = TOTAL_SIZE / chunk_size; b.to_async(&rt).iter_batched( || create_parser(chunk_size, chunk_num, mode), - |(parser, column_desc, input)| parse(parser, column_desc, input), + |(parser, input)| parse(parser, input), BatchSize::SmallInput, ); }, diff --git a/src/connector/benches/json_vs_plain_parser.rs b/src/connector/benches/json_vs_plain_parser.rs new file mode 100644 index 0000000000000..a176e3b2b0203 --- /dev/null +++ b/src/connector/benches/json_vs_plain_parser.rs @@ -0,0 +1,173 @@ +// 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. + +//! Benchmark for comparing the performance of parsing JSON records directly +//! through the `JsonParser` versus indirectly through the `PlainParser`. + +mod json_common; + +use criterion::{criterion_group, criterion_main, BatchSize, Criterion}; +use futures::executor::block_on; +use json_common::*; +use old_json_parser::JsonParser; +use risingwave_connector::parser::plain_parser::PlainParser; +use risingwave_connector::parser::{SourceStreamChunkBuilder, SpecificParserConfig}; +use risingwave_connector::source::SourceContext; + +// The original implementation used to parse JSON prior to #13707. +mod old_json_parser { + use anyhow::Context as _; + use itertools::{Either, Itertools as _}; + use risingwave_common::{bail, try_match_expand}; + use risingwave_connector::error::ConnectorResult; + use risingwave_connector::parser::{ + Access as _, EncodingProperties, JsonAccess, SourceStreamChunkRowWriter, + }; + use risingwave_connector::source::{SourceColumnDesc, SourceContextRef}; + + use super::*; + + /// Parser for JSON format + #[derive(Debug)] + pub struct JsonParser { + _rw_columns: Vec, + _source_ctx: SourceContextRef, + // If schema registry is used, the starting index of payload is 5. + payload_start_idx: usize, + } + + impl JsonParser { + pub fn new( + props: SpecificParserConfig, + rw_columns: Vec, + source_ctx: SourceContextRef, + ) -> ConnectorResult { + let json_config = try_match_expand!(props.encoding_config, EncodingProperties::Json)?; + let payload_start_idx = if json_config.use_schema_registry { + 5 + } else { + 0 + }; + Ok(Self { + _rw_columns: rw_columns, + _source_ctx: source_ctx, + payload_start_idx, + }) + } + + #[allow(clippy::unused_async)] + pub async fn parse_inner( + &self, + mut payload: Vec, + mut writer: SourceStreamChunkRowWriter<'_>, + ) -> ConnectorResult<()> { + let value = simd_json::to_borrowed_value(&mut payload[self.payload_start_idx..]) + .context("failed to parse json payload")?; + let values = if let simd_json::BorrowedValue::Array(arr) = value { + Either::Left(arr.into_iter()) + } else { + Either::Right(std::iter::once(value)) + }; + + let mut errors = Vec::new(); + for value in values { + let accessor = JsonAccess::new(value); + match writer + .insert(|column| accessor.access(&[&column.name], Some(&column.data_type))) + { + Ok(_) => {} + Err(err) => errors.push(err), + } + } + + if errors.is_empty() { + Ok(()) + } else { + bail!( + "failed to parse {} row(s) in a single json message: {}", + errors.len(), + errors.iter().format(", ") + ); + } + } + } +} + +fn generate_json_rows() -> Vec> { + let mut rng = rand::thread_rng(); + let mut records = Vec::with_capacity(NUM_RECORDS); + for _ in 0..NUM_RECORDS { + records.push(generate_json_row(&mut rng).into_bytes()); + } + records +} + +fn bench_plain_parser_and_json_parser(c: &mut Criterion) { + let rt = tokio::runtime::Runtime::new().unwrap(); + let records = generate_json_rows(); + + let mut group = c.benchmark_group("plain parser and json parser comparison"); + + group.bench_function("plain_parser", |b| { + b.to_async(&rt).iter_batched( + || { + let parser = block_on(PlainParser::new( + SpecificParserConfig::DEFAULT_PLAIN_JSON, + get_descs(), + SourceContext::dummy().into(), + )) + .unwrap(); + (parser, records.clone()) + }, + |(mut parser, records)| async move { + let mut builder = SourceStreamChunkBuilder::with_capacity(get_descs(), NUM_RECORDS); + for record in records { + let writer = builder.row_writer(); + parser + .parse_inner(None, Some(record), writer) + .await + .unwrap(); + } + }, + BatchSize::SmallInput, + ) + }); + + group.bench_function("json_parser", |b| { + b.to_async(&rt).iter_batched( + || { + let parser = JsonParser::new( + SpecificParserConfig::DEFAULT_PLAIN_JSON, + get_descs(), + SourceContext::dummy().into(), + ) + .unwrap(); + (parser, records.clone()) + }, + |(parser, records)| async move { + let mut builder = SourceStreamChunkBuilder::with_capacity(get_descs(), NUM_RECORDS); + for record in records { + let writer = builder.row_writer(); + parser.parse_inner(record, writer).await.unwrap(); + } + }, + BatchSize::SmallInput, + ) + }); + + group.finish(); +} + +criterion_group!(benches, bench_plain_parser_and_json_parser,); +criterion_main!(benches); diff --git a/src/connector/benches/nexmark_integration.rs b/src/connector/benches/nexmark_integration.rs index 1c05147eeafbb..28596e26eec19 100644 --- a/src/connector/benches/nexmark_integration.rs +++ b/src/connector/benches/nexmark_integration.rs @@ -12,6 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. +//! Integration benchmark for parsing Nexmark events. +//! +//! To cover the code path in real-world scenarios, the parser is created through +//! `ByteStreamSourceParserImpl::create` based on the given configuration, rather +//! than depending on a specific internal implementation. + #![feature(lazy_cell)] use std::sync::LazyLock; @@ -23,11 +29,10 @@ use risingwave_common::array::StreamChunk; use risingwave_common::catalog::ColumnId; use risingwave_common::types::DataType; use risingwave_connector::parser::{ - ByteStreamSourceParser, JsonParser, SourceParserIntoStreamExt, SpecificParserConfig, + ByteStreamSourceParserImpl, CommonParserConfig, ParserConfig, SpecificParserConfig, }; use risingwave_connector::source::{ - BoxChunkSourceStream, BoxSourceStream, SourceColumnDesc, SourceContext, SourceMessage, - SourceMeta, + BoxChunkSourceStream, BoxSourceStream, SourceColumnDesc, SourceMessage, SourceMeta, }; use tracing::Level; use tracing_subscriber::prelude::*; @@ -71,8 +76,8 @@ fn make_data_stream() -> BoxSourceStream { .boxed() } -fn make_parser() -> impl ByteStreamSourceParser { - let columns = [ +fn make_parser() -> ByteStreamSourceParserImpl { + let rw_columns = [ ("auction", DataType::Int64), ("bidder", DataType::Int64), ("price", DataType::Int64), @@ -86,9 +91,12 @@ fn make_parser() -> impl ByteStreamSourceParser { .map(|(i, (n, t))| SourceColumnDesc::simple(n, t, ColumnId::new(i as _))) .collect_vec(); - let props = SpecificParserConfig::DEFAULT_PLAIN_JSON; + let config = ParserConfig { + common: CommonParserConfig { rw_columns }, + specific: SpecificParserConfig::DEFAULT_PLAIN_JSON, + }; - JsonParser::new(props, columns, SourceContext::dummy().into()).unwrap() + ByteStreamSourceParserImpl::create_for_test(config).unwrap() } fn make_stream_iter() -> impl Iterator { diff --git a/src/connector/src/parser/avro/parser.rs b/src/connector/src/parser/avro/parser.rs index 7f498a055ac7e..59bc10b084539 100644 --- a/src/connector/src/parser/avro/parser.rs +++ b/src/connector/src/parser/avro/parser.rs @@ -21,7 +21,7 @@ use apache_avro::{from_avro_datum, Reader, Schema}; use risingwave_common::{bail, try_match_expand}; use risingwave_pb::plan_common::ColumnDesc; -use super::schema_resolver::ConfluentSchemaResolver; +use super::schema_resolver::ConfluentSchemaCache; use super::util::avro_schema_to_column_descs; use crate::error::ConnectorResult; use crate::parser::unified::avro::{AvroAccess, AvroParseOptions}; @@ -36,16 +36,17 @@ use crate::schema::schema_registry::{ #[derive(Debug)] pub struct AvroAccessBuilder { schema: Arc, - pub schema_resolver: Option>, + /// Refer to [`AvroParserConfig::writer_schema_cache`]. + pub writer_schema_cache: Option>, value: Option, } impl AccessBuilder for AvroAccessBuilder { async fn generate_accessor(&mut self, payload: Vec) -> ConnectorResult> { - self.value = self.parse_avro_value(&payload, Some(&*self.schema)).await?; + self.value = self.parse_avro_value(&payload).await?; Ok(AccessImpl::Avro(AvroAccess::new( self.value.as_ref().unwrap(), - AvroParseOptions::default().with_schema(&self.schema), + AvroParseOptions::create(&self.schema), ))) } } @@ -55,7 +56,7 @@ impl AvroAccessBuilder { let AvroParserConfig { schema, key_schema, - schema_resolver, + writer_schema_cache, .. } = config; Ok(Self { @@ -63,35 +64,29 @@ impl AvroAccessBuilder { EncodingType::Key => key_schema.context("Avro with empty key schema")?, EncodingType::Value => schema, }, - schema_resolver, + writer_schema_cache, value: None, }) } - async fn parse_avro_value( - &self, - payload: &[u8], - reader_schema: Option<&Schema>, - ) -> ConnectorResult> { + async fn parse_avro_value(&self, payload: &[u8]) -> ConnectorResult> { // parse payload to avro value // if use confluent schema, get writer schema from confluent schema registry - if let Some(resolver) = &self.schema_resolver { + if let Some(resolver) = &self.writer_schema_cache { let (schema_id, mut raw_payload) = extract_schema_id(payload)?; - let writer_schema = resolver.get(schema_id).await?; + let writer_schema = resolver.get_by_id(schema_id).await?; Ok(Some(from_avro_datum( writer_schema.as_ref(), &mut raw_payload, - reader_schema, + Some(self.schema.as_ref()), )?)) - } else if let Some(schema) = reader_schema { - let mut reader = Reader::with_schema(schema, payload)?; + } else { + let mut reader = Reader::with_schema(self.schema.as_ref(), payload)?; match reader.next() { Some(Ok(v)) => Ok(Some(v)), Some(Err(e)) => Err(e)?, None => bail!("avro parse unexpected eof"), } - } else { - unreachable!("both schema_resolver and reader_schema not exist"); } } } @@ -100,7 +95,9 @@ impl AvroAccessBuilder { pub struct AvroParserConfig { pub schema: Arc, pub key_schema: Option>, - pub schema_resolver: Option>, + /// Writer schema is the schema used to write the data. When parsing Avro data, the exactly same schema + /// must be used to decode the message, and then convert it with the reader schema. + pub writer_schema_cache: Option>, pub map_handling: Option, } @@ -122,7 +119,7 @@ impl AvroParserConfig { let url = handle_sr_list(schema_location.as_str())?; if use_schema_registry { let client = Client::new(url, &client_config)?; - let resolver = ConfluentSchemaResolver::new(client); + let resolver = ConfluentSchemaCache::new(client); let subject_key = if enable_upsert { Some(get_subject_by_strategy( @@ -146,13 +143,13 @@ impl AvroParserConfig { tracing::debug!("infer key subject {subject_key:?}, value subject {subject_value}"); Ok(Self { - schema: resolver.get_by_subject_name(&subject_value).await?, + schema: resolver.get_by_subject(&subject_value).await?, key_schema: if let Some(subject_key) = subject_key { - Some(resolver.get_by_subject_name(&subject_key).await?) + Some(resolver.get_by_subject(&subject_key).await?) } else { None }, - schema_resolver: Some(Arc::new(resolver)), + writer_schema_cache: Some(Arc::new(resolver)), map_handling, }) } else { @@ -166,7 +163,7 @@ impl AvroParserConfig { Ok(Self { schema: Arc::new(schema), key_schema: None, - schema_resolver: None, + writer_schema_cache: None, map_handling, }) } diff --git a/src/connector/src/parser/avro/schema_resolver.rs b/src/connector/src/parser/avro/schema_resolver.rs index cdc52de7accee..058f9bcbf7ea3 100644 --- a/src/connector/src/parser/avro/schema_resolver.rs +++ b/src/connector/src/parser/avro/schema_resolver.rs @@ -21,13 +21,20 @@ use moka::future::Cache; use crate::error::ConnectorResult; use crate::schema::schema_registry::{Client, ConfluentSchema}; +/// Fetch schemas from confluent schema registry and cache them. +/// +/// Background: This is mainly used for Avro **writer schema** (during schema evolution): When decoding an Avro message, +/// we must get the message's schema id, and use the *exactly same schema* to decode the message, and then +/// convert it with the reader schema. (This is also why Avro has to be used with a schema registry instead of a static schema file.) +/// +/// TODO: support protobuf (not sure if it's needed) #[derive(Debug)] -pub struct ConfluentSchemaResolver { +pub struct ConfluentSchemaCache { writer_schemas: Cache>, confluent_client: Client, } -impl ConfluentSchemaResolver { +impl ConfluentSchemaCache { async fn parse_and_cache_schema( &self, raw_schema: ConfluentSchema, @@ -43,29 +50,23 @@ impl ConfluentSchemaResolver { /// Create a new `ConfluentSchemaResolver` pub fn new(client: Client) -> Self { - ConfluentSchemaResolver { + ConfluentSchemaCache { writer_schemas: Cache::new(u64::MAX), confluent_client: client, } } - pub async fn get_by_subject_name(&self, subject_name: &str) -> ConnectorResult> { - let raw_schema = self.get_raw_schema_by_subject_name(subject_name).await?; - self.parse_and_cache_schema(raw_schema).await - } - - pub async fn get_raw_schema_by_subject_name( - &self, - subject_name: &str, - ) -> ConnectorResult { - self.confluent_client + /// Gets the latest schema by subject name, which is used as *reader schema*. + pub async fn get_by_subject(&self, subject_name: &str) -> ConnectorResult> { + let raw_schema = self + .confluent_client .get_schema_by_subject(subject_name) - .await - .map_err(Into::into) + .await?; + self.parse_and_cache_schema(raw_schema).await } - // get the writer schema by id - pub async fn get(&self, schema_id: i32) -> ConnectorResult> { + /// Gets the a specific schema by id, which is used as *writer schema*. + pub async fn get_by_id(&self, schema_id: i32) -> ConnectorResult> { // TODO: use `get_with` if let Some(schema) = self.writer_schemas.get(&schema_id).await { Ok(schema) diff --git a/src/connector/src/parser/avro/util.rs b/src/connector/src/parser/avro/util.rs index 4f36b15e5ce76..ab3a200b513ea 100644 --- a/src/connector/src/parser/avro/util.rs +++ b/src/connector/src/parser/avro/util.rs @@ -147,11 +147,18 @@ fn avro_type_mapping( DataType::List(Box::new(item_type)) } Schema::Union(union_schema) => { - let nested_schema = union_schema - .variants() + // We only support using union to represent nullable fields, not general unions. + let variants = union_schema.variants(); + if variants.len() != 2 || !variants.contains(&Schema::Null) { + bail!( + "unsupported Avro type, only unions like [null, T] is supported: {:?}", + schema + ); + } + let nested_schema = variants .iter() .find_or_first(|s| !matches!(s, Schema::Null)) - .ok_or_else(|| anyhow::format_err!("unsupported Avro type: {:?}", union_schema))?; + .unwrap(); avro_type_mapping(nested_schema, map_handling)? } diff --git a/src/connector/src/parser/debezium/avro_parser.rs b/src/connector/src/parser/debezium/avro_parser.rs index 8d73a789b2669..6f4041ab5d39c 100644 --- a/src/connector/src/parser/debezium/avro_parser.rs +++ b/src/connector/src/parser/debezium/avro_parser.rs @@ -22,7 +22,7 @@ use risingwave_pb::catalog::PbSchemaRegistryNameStrategy; use risingwave_pb::plan_common::ColumnDesc; use crate::error::ConnectorResult; -use crate::parser::avro::schema_resolver::ConfluentSchemaResolver; +use crate::parser::avro::schema_resolver::ConfluentSchemaCache; use crate::parser::avro::util::avro_schema_to_column_descs; use crate::parser::unified::avro::{ avro_extract_field_schema, avro_schema_skip_union, AvroAccess, AvroParseOptions, @@ -41,7 +41,7 @@ const PAYLOAD: &str = "payload"; #[derive(Debug)] pub struct DebeziumAvroAccessBuilder { schema: Schema, - schema_resolver: Arc, + schema_resolver: Arc, key_schema: Option>, value: Option, encoding_type: EncodingType, @@ -51,7 +51,7 @@ pub struct DebeziumAvroAccessBuilder { impl AccessBuilder for DebeziumAvroAccessBuilder { async fn generate_accessor(&mut self, payload: Vec) -> ConnectorResult> { let (schema_id, mut raw_payload) = extract_schema_id(&payload)?; - let schema = self.schema_resolver.get(schema_id).await?; + let schema = self.schema_resolver.get_by_id(schema_id).await?; self.value = Some(from_avro_datum(schema.as_ref(), &mut raw_payload, None)?); self.key_schema = match self.encoding_type { EncodingType::Key => Some(schema), @@ -59,7 +59,7 @@ impl AccessBuilder for DebeziumAvroAccessBuilder { }; Ok(AccessImpl::Avro(AvroAccess::new( self.value.as_mut().unwrap(), - AvroParseOptions::default().with_schema(match self.encoding_type { + AvroParseOptions::create(match self.encoding_type { EncodingType::Key => self.key_schema.as_mut().unwrap(), EncodingType::Value => &self.schema, }), @@ -96,7 +96,7 @@ impl DebeziumAvroAccessBuilder { pub struct DebeziumAvroParserConfig { pub key_schema: Arc, pub outer_schema: Arc, - pub schema_resolver: Arc, + pub schema_resolver: Arc, } impl DebeziumAvroParserConfig { @@ -107,13 +107,13 @@ impl DebeziumAvroParserConfig { let kafka_topic = &avro_config.topic; let url = handle_sr_list(schema_location)?; let client = Client::new(url, client_config)?; - let resolver = ConfluentSchemaResolver::new(client); + let resolver = ConfluentSchemaCache::new(client); let name_strategy = &PbSchemaRegistryNameStrategy::Unspecified; let key_subject = get_subject_by_strategy(name_strategy, kafka_topic, None, true)?; let val_subject = get_subject_by_strategy(name_strategy, kafka_topic, None, false)?; - let key_schema = resolver.get_by_subject_name(&key_subject).await?; - let outer_schema = resolver.get_by_subject_name(&val_subject).await?; + let key_schema = resolver.get_by_subject(&key_subject).await?; + let outer_schema = resolver.get_by_subject(&val_subject).await?; Ok(Self { key_schema, diff --git a/src/connector/src/parser/json_parser.rs b/src/connector/src/parser/json_parser.rs index 3621fbc2724b3..5c511af9efb40 100644 --- a/src/connector/src/parser/json_parser.rs +++ b/src/connector/src/parser/json_parser.rs @@ -12,29 +12,30 @@ // See the License for the specific language governing permissions and // limitations under the License. +// Note on this file: +// +// There's no struct named `JsonParser` anymore since #13707. `ENCODE JSON` will be +// dispatched to `PlainParser` or `UpsertParser` with `JsonAccessBuilder` instead. +// +// This file now only contains utilities and tests for JSON parsing. Also, to avoid +// rely on the internal implementation and allow that to be changed, the tests use +// `ByteStreamSourceParserImpl` to create a parser instance. + use std::collections::HashMap; use anyhow::Context as _; use apache_avro::Schema; -use itertools::{Either, Itertools}; use jst::{convert_avro, Context}; -use risingwave_common::{bail, try_match_expand}; use risingwave_pb::plan_common::ColumnDesc; -use super::avro::schema_resolver::ConfluentSchemaResolver; -use super::unified::Access; use super::util::{bytes_from_url, get_kafka_topic}; -use super::{EncodingProperties, JsonProperties, SchemaRegistryAuth, SpecificParserConfig}; +use super::{JsonProperties, SchemaRegistryAuth}; use crate::error::ConnectorResult; -use crate::only_parse_payload; use crate::parser::avro::util::avro_schema_to_column_descs; use crate::parser::unified::json::{JsonAccess, JsonParseOptions}; use crate::parser::unified::AccessImpl; -use crate::parser::{ - AccessBuilder, ByteStreamSourceParser, ParserFormat, SourceStreamChunkRowWriter, -}; +use crate::parser::AccessBuilder; use crate::schema::schema_registry::{handle_sr_list, Client}; -use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; #[derive(Debug)] pub struct JsonAccessBuilder { @@ -78,80 +79,6 @@ impl JsonAccessBuilder { } } -/// Parser for JSON format -#[derive(Debug)] -pub struct JsonParser { - rw_columns: Vec, - source_ctx: SourceContextRef, - // If schema registry is used, the starting index of payload is 5. - payload_start_idx: usize, -} - -impl JsonParser { - pub fn new( - props: SpecificParserConfig, - rw_columns: Vec, - source_ctx: SourceContextRef, - ) -> ConnectorResult { - let json_config = try_match_expand!(props.encoding_config, EncodingProperties::Json)?; - let payload_start_idx = if json_config.use_schema_registry { - 5 - } else { - 0 - }; - Ok(Self { - rw_columns, - source_ctx, - payload_start_idx, - }) - } - - #[cfg(test)] - pub fn new_for_test(rw_columns: Vec) -> ConnectorResult { - Ok(Self { - rw_columns, - source_ctx: SourceContext::dummy().into(), - payload_start_idx: 0, - }) - } - - #[allow(clippy::unused_async)] - pub async fn parse_inner( - &self, - mut payload: Vec, - mut writer: SourceStreamChunkRowWriter<'_>, - ) -> ConnectorResult<()> { - let value = simd_json::to_borrowed_value(&mut payload[self.payload_start_idx..]) - .context("failed to parse json payload")?; - let values = if let simd_json::BorrowedValue::Array(arr) = value { - Either::Left(arr.into_iter()) - } else { - Either::Right(std::iter::once(value)) - }; - - let mut errors = Vec::new(); - for value in values { - let accessor = JsonAccess::new(value); - match writer.insert(|column| accessor.access(&[&column.name], Some(&column.data_type))) - { - Ok(_) => {} - Err(err) => errors.push(err), - } - } - - if errors.is_empty() { - Ok(()) - } else { - // TODO(error-handling): multiple errors - bail!( - "failed to parse {} row(s) in a single json message: {}", - errors.len(), - errors.iter().format(", ") - ); - } - } -} - pub async fn schema_to_columns( schema_location: &str, schema_registry_auth: Option, @@ -161,12 +88,10 @@ pub async fn schema_to_columns( let json_schema = if let Some(schema_registry_auth) = schema_registry_auth { let client = Client::new(url, &schema_registry_auth)?; let topic = get_kafka_topic(props)?; - let resolver = ConfluentSchemaResolver::new(client); - let content = resolver - .get_raw_schema_by_subject_name(&format!("{}-value", topic)) - .await? - .content; - serde_json::from_str(&content)? + let schema = client + .get_schema_by_subject(&format!("{}-value", topic)) + .await?; + serde_json::from_str(&schema.content)? } else { let url = url.first().unwrap(); let bytes = bytes_from_url(url, None).await?; @@ -179,29 +104,6 @@ pub async fn schema_to_columns( avro_schema_to_column_descs(&schema, None) } -impl ByteStreamSourceParser for JsonParser { - fn columns(&self) -> &[SourceColumnDesc] { - &self.rw_columns - } - - fn source_ctx(&self) -> &SourceContext { - &self.source_ctx - } - - fn parser_format(&self) -> ParserFormat { - ParserFormat::Json - } - - async fn parse_one<'a>( - &'a mut self, - _key: Option>, - payload: Option>, - writer: SourceStreamChunkRowWriter<'a>, - ) -> ConnectorResult<()> { - only_parse_payload!(self, payload, writer) - } -} - #[cfg(test)] mod tests { use std::vec; @@ -215,13 +117,31 @@ mod tests { use risingwave_pb::plan_common::additional_column::ColumnType as AdditionalColumnType; use risingwave_pb::plan_common::{AdditionalColumn, AdditionalColumnKey}; - use super::JsonParser; - use crate::parser::upsert_parser::UpsertParser; + use crate::parser::test_utils::ByteStreamSourceParserImplTestExt as _; use crate::parser::{ - EncodingProperties, JsonProperties, ProtocolProperties, SourceColumnDesc, - SourceStreamChunkBuilder, SpecificParserConfig, + ByteStreamSourceParserImpl, CommonParserConfig, ParserConfig, ProtocolProperties, + SourceColumnDesc, SpecificParserConfig, }; - use crate::source::{SourceColumnType, SourceContext}; + use crate::source::SourceColumnType; + + fn make_parser(rw_columns: Vec) -> ByteStreamSourceParserImpl { + ByteStreamSourceParserImpl::create_for_test(ParserConfig { + common: CommonParserConfig { rw_columns }, + specific: SpecificParserConfig::DEFAULT_PLAIN_JSON, + }) + .unwrap() + } + + fn make_upsert_parser(rw_columns: Vec) -> ByteStreamSourceParserImpl { + ByteStreamSourceParserImpl::create_for_test(ParserConfig { + common: CommonParserConfig { rw_columns }, + specific: SpecificParserConfig { + protocol_config: ProtocolProperties::Upsert, + ..SpecificParserConfig::DEFAULT_PLAIN_JSON + }, + }) + .unwrap() + } fn get_payload() -> Vec> { vec![ @@ -251,21 +171,8 @@ mod tests { SourceColumnDesc::simple("interval", DataType::Interval, 11.into()), ]; - let parser = JsonParser::new( - SpecificParserConfig::DEFAULT_PLAIN_JSON, - descs.clone(), - SourceContext::dummy().into(), - ) - .unwrap(); - - let mut builder = SourceStreamChunkBuilder::with_capacity(descs, 2); - - for payload in get_payload() { - let writer = builder.row_writer(); - parser.parse_inner(payload, writer).await.unwrap(); - } - - let chunk = builder.finish(); + let parser = make_parser(descs); + let chunk = parser.parse(get_payload()).await; let mut rows = chunk.rows(); @@ -361,38 +268,20 @@ mod tests { SourceColumnDesc::simple("v2", DataType::Int16, 1.into()), SourceColumnDesc::simple("v3", DataType::Varchar, 2.into()), ]; - let parser = JsonParser::new( - SpecificParserConfig::DEFAULT_PLAIN_JSON, - descs.clone(), - SourceContext::dummy().into(), - ) - .unwrap(); - let mut builder = SourceStreamChunkBuilder::with_capacity(descs, 3); - - // Parse a correct record. - { - let writer = builder.row_writer(); - let payload = br#"{"v1": 1, "v2": 2, "v3": "3"}"#.to_vec(); - parser.parse_inner(payload, writer).await.unwrap(); - } - // Parse an incorrect record. - { - let writer = builder.row_writer(); + let parser = make_parser(descs); + let payloads = vec![ + // Parse a correct record. + br#"{"v1": 1, "v2": 2, "v3": "3"}"#.to_vec(), + // Parse an incorrect record. // `v2` overflowed. - let payload = br#"{"v1": 1, "v2": 65536, "v3": "3"}"#.to_vec(); // ignored the error, and fill None at v2. - parser.parse_inner(payload, writer).await.unwrap(); - } - - // Parse a correct record. - { - let writer = builder.row_writer(); - let payload = br#"{"v1": 1, "v2": 2, "v3": "3"}"#.to_vec(); - parser.parse_inner(payload, writer).await.unwrap(); - } + br#"{"v1": 1, "v2": 65536, "v3": "3"}"#.to_vec(), + // Parse a correct record. + br#"{"v1": 1, "v2": 2, "v3": "3"}"#.to_vec(), + ]; + let chunk = parser.parse(payloads).await; - let chunk = builder.finish(); assert!(chunk.valid()); assert_eq!(chunk.cardinality(), 3); @@ -432,12 +321,7 @@ mod tests { .map(SourceColumnDesc::from) .collect_vec(); - let parser = JsonParser::new( - SpecificParserConfig::DEFAULT_PLAIN_JSON, - descs.clone(), - SourceContext::dummy().into(), - ) - .unwrap(); + let parser = make_parser(descs); let payload = br#" { "data": { @@ -456,12 +340,8 @@ mod tests { "VarcharCastToI64": "1598197865760800768" } "#.to_vec(); - let mut builder = SourceStreamChunkBuilder::with_capacity(descs, 1); - { - let writer = builder.row_writer(); - parser.parse_inner(payload, writer).await.unwrap(); - } - let chunk = builder.finish(); + let chunk = parser.parse(vec![payload]).await; + let (op, row) = chunk.rows().next().unwrap(); assert_eq!(op, Op::Insert); let row = row.into_owned_row().into_inner(); @@ -504,24 +384,15 @@ mod tests { .map(SourceColumnDesc::from) .collect_vec(); - let parser = JsonParser::new( - SpecificParserConfig::DEFAULT_PLAIN_JSON, - descs.clone(), - SourceContext::dummy().into(), - ) - .unwrap(); + let parser = make_parser(descs); let payload = br#" { "struct": "{\"varchar\": \"varchar\", \"boolean\": true}" } "# .to_vec(); - let mut builder = SourceStreamChunkBuilder::with_capacity(descs, 1); - { - let writer = builder.row_writer(); - parser.parse_inner(payload, writer).await.unwrap(); - } - let chunk = builder.finish(); + let chunk = parser.parse(vec![payload]).await; + let (op, row) = chunk.rows().next().unwrap(); assert_eq!(op, Op::Insert); let row = row.into_owned_row().into_inner(); @@ -550,12 +421,7 @@ mod tests { .map(SourceColumnDesc::from) .collect_vec(); - let parser = JsonParser::new( - SpecificParserConfig::DEFAULT_PLAIN_JSON, - descs.clone(), - SourceContext::dummy().into(), - ) - .unwrap(); + let parser = make_parser(descs); let payload = br#" { "struct": { @@ -564,12 +430,8 @@ mod tests { } "# .to_vec(); - let mut builder = SourceStreamChunkBuilder::with_capacity(descs, 1); - { - let writer = builder.row_writer(); - parser.parse_inner(payload, writer).await.unwrap(); - } - let chunk = builder.finish(); + let chunk = parser.parse(vec![payload]).await; + let (op, row) = chunk.rows().next().unwrap(); assert_eq!(op, Op::Insert); let row = row.into_owned_row().into_inner(); @@ -591,7 +453,10 @@ mod tests { (r#"{"a":2}"#, r#"{"a":2,"b":2}"#), (r#"{"a":2}"#, r#""#), ] - .to_vec(); + .into_iter() + .map(|(k, v)| (k.as_bytes().to_vec(), v.as_bytes().to_vec())) + .collect_vec(); + let key_column_desc = SourceColumnDesc { name: "rw_key".into(), data_type: DataType::Bytea, @@ -609,34 +474,9 @@ mod tests { SourceColumnDesc::simple("b", DataType::Int32, 1.into()), key_column_desc, ]; - let props = SpecificParserConfig { - key_encoding_config: None, - encoding_config: EncodingProperties::Json(JsonProperties { - use_schema_registry: false, - timestamptz_handling: None, - }), - protocol_config: ProtocolProperties::Upsert, - }; - let mut parser = UpsertParser::new(props, descs.clone(), SourceContext::dummy().into()) - .await - .unwrap(); - let mut builder = SourceStreamChunkBuilder::with_capacity(descs, 4); - for item in items { - parser - .parse_inner( - Some(item.0.as_bytes().to_vec()), - if !item.1.is_empty() { - Some(item.1.as_bytes().to_vec()) - } else { - None - }, - builder.row_writer(), - ) - .await - .unwrap(); - } - let chunk = builder.finish(); + let parser = make_upsert_parser(descs); + let chunk = parser.parse_upsert(items).await; // expected chunk // +---+---+---+------------------+ diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index a249807c6a6bb..be697d990a39a 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -45,7 +45,8 @@ pub use self::mysql::mysql_row_to_owned_row; use self::plain_parser::PlainParser; pub use self::postgres::postgres_row_to_owned_row; use self::simd_json_parser::DebeziumJsonAccessBuilder; -pub use self::unified::json::TimestamptzHandling; +pub use self::unified::json::{JsonAccess, TimestamptzHandling}; +pub use self::unified::Access; use self::unified::AccessImpl; use self::upsert_parser::UpsertParser; use self::util::get_kafka_topic; @@ -866,9 +867,8 @@ impl AccessBuilderImpl { /// The entrypoint of parsing. It parses [`SourceMessage`] stream (byte stream) into [`StreamChunk`] stream. /// Used by [`crate::source::into_chunk_stream`]. #[derive(Debug)] -pub(crate) enum ByteStreamSourceParserImpl { +pub enum ByteStreamSourceParserImpl { Csv(CsvParser), - Json(JsonParser), Debezium(DebeziumParser), Plain(PlainParser), Upsert(UpsertParser), @@ -883,7 +883,6 @@ impl ByteStreamSourceParserImpl { #[auto_enum(futures03::Stream)] let stream = match self { Self::Csv(parser) => parser.into_stream(msg_stream), - Self::Json(parser) => parser.into_stream(msg_stream), Self::Debezium(parser) => parser.into_stream(msg_stream), Self::DebeziumMongoJson(parser) => parser.into_stream(msg_stream), Self::Maxwell(parser) => parser.into_stream(msg_stream), @@ -937,6 +936,58 @@ impl ByteStreamSourceParserImpl { _ => unreachable!(), } } + + /// Create a parser for testing purposes. + pub fn create_for_test(parser_config: ParserConfig) -> ConnectorResult { + futures::executor::block_on(Self::create(parser_config, SourceContext::dummy().into())) + } +} + +/// Test utilities for [`ByteStreamSourceParserImpl`]. +#[cfg(test)] +pub mod test_utils { + use futures::StreamExt as _; + use itertools::Itertools as _; + + use super::*; + + #[easy_ext::ext(ByteStreamSourceParserImplTestExt)] + pub(crate) impl ByteStreamSourceParserImpl { + /// Parse the given payloads into a [`StreamChunk`]. + async fn parse(self, payloads: Vec>) -> StreamChunk { + let source_messages = payloads + .into_iter() + .map(|p| SourceMessage { + payload: (!p.is_empty()).then_some(p), + ..SourceMessage::dummy() + }) + .collect_vec(); + + self.into_stream(futures::stream::once(async { Ok(source_messages) }).boxed()) + .next() + .await + .unwrap() + .unwrap() + } + + /// Parse the given key-value pairs into a [`StreamChunk`]. + async fn parse_upsert(self, kvs: Vec<(Vec, Vec)>) -> StreamChunk { + let source_messages = kvs + .into_iter() + .map(|(k, v)| SourceMessage { + key: (!k.is_empty()).then_some(k), + payload: (!v.is_empty()).then_some(v), + ..SourceMessage::dummy() + }) + .collect_vec(); + + self.into_stream(futures::stream::once(async { Ok(source_messages) }).boxed()) + .next() + .await + .unwrap() + .unwrap() + } + } } #[derive(Debug, Clone, Default)] diff --git a/src/connector/src/parser/unified/avro.rs b/src/connector/src/parser/unified/avro.rs index bbab918f5be1d..2c94eb47ccfd1 100644 --- a/src/connector/src/parser/unified/avro.rs +++ b/src/connector/src/parser/unified/avro.rs @@ -34,27 +34,23 @@ use crate::parser::avro::util::avro_to_jsonb; #[derive(Clone)] /// Options for parsing an `AvroValue` into Datum, with an optional avro schema. pub struct AvroParseOptions<'a> { + /// Currently, this schema is only used for decimal pub schema: Option<&'a Schema>, /// Strict Mode /// If strict mode is disabled, an int64 can be parsed from an `AvroInt` (int32) value. pub relax_numeric: bool, } -impl<'a> Default for AvroParseOptions<'a> { - fn default() -> Self { +impl<'a> AvroParseOptions<'a> { + pub fn create(schema: &'a Schema) -> Self { Self { - schema: None, + schema: Some(schema), relax_numeric: true, } } } impl<'a> AvroParseOptions<'a> { - pub fn with_schema(mut self, schema: &'a Schema) -> Self { - self.schema = Some(schema); - self - } - fn extract_inner_schema(&self, key: Option<&'a str>) -> Option<&'a Schema> { self.schema .map(|schema| avro_extract_field_schema(schema, key)) @@ -71,15 +67,23 @@ impl<'a> AvroParseOptions<'a> { } /// Parse an avro value into expected type. - /// 3 kinds of type info are used to parsing things. - /// - `type_expected`. The type that we expect the value is. - /// - value type. The type info together with the value argument. - /// - schema. The `AvroSchema` provided in option. - /// If both `type_expected` and schema are provided, it will check both strictly. - /// If only `type_expected` is provided, it will try to match the value type and the - /// `type_expected`, converting the value if possible. If only value is provided (without - /// schema and `type_expected`), the `DateType` will be inferred. - pub fn parse<'b>(&self, value: &'b Value, type_expected: Option<&'b DataType>) -> AccessResult + /// + /// 3 kinds of type info are used to parsing: + /// - `type_expected`. The type that we expect the value is. + /// - value type. The type info together with the value argument. + /// - schema. The `AvroSchema` provided in option. + /// + /// Cases: (FIXME: Is this precise?) + /// - If both `type_expected` and schema are provided, it will check both strictly. + /// - If only `type_expected` is provided, it will try to match the value type and the + /// `type_expected`, converting the value if possible. + /// - If only value is provided (without schema and `type_expected`), + /// the `DataType` will be inferred. + pub fn convert_to_datum<'b>( + &self, + value: &'b Value, + type_expected: Option<&'b DataType>, + ) -> AccessResult where 'b: 'a, { @@ -97,7 +101,7 @@ impl<'a> AvroParseOptions<'a> { schema, relax_numeric: self.relax_numeric, } - .parse(v, type_expected); + .convert_to_datum(v, type_expected); } // ---- Boolean ----- (Some(DataType::Boolean) | None, Value::Boolean(b)) => (*b).into(), @@ -224,7 +228,7 @@ impl<'a> AvroParseOptions<'a> { schema, relax_numeric: self.relax_numeric, } - .parse(value, Some(field_type))?) + .convert_to_datum(value, Some(field_type))?) } else { Ok(None) } @@ -241,7 +245,7 @@ impl<'a> AvroParseOptions<'a> { schema, relax_numeric: self.relax_numeric, } - .parse(field_value, None) + .convert_to_datum(field_value, None) }) .collect::, AccessError>>()?; ScalarImpl::Struct(StructValue::new(rw_values)) @@ -255,7 +259,7 @@ impl<'a> AvroParseOptions<'a> { schema, relax_numeric: self.relax_numeric, } - .parse(v, Some(item_type))?; + .convert_to_datum(v, Some(item_type))?; builder.append(value); } builder.finish() @@ -325,7 +329,7 @@ where Err(create_error())?; } - options.parse(value, type_expected) + options.convert_to_datum(value, type_expected) } } @@ -484,12 +488,9 @@ mod tests { value_schema: &Schema, shape: &DataType, ) -> crate::error::ConnectorResult { - AvroParseOptions { - schema: Some(value_schema), - relax_numeric: true, - } - .parse(&value, Some(shape)) - .map_err(Into::into) + AvroParseOptions::create(value_schema) + .convert_to_datum(&value, Some(shape)) + .map_err(Into::into) } #[test] @@ -529,8 +530,10 @@ mod tests { .unwrap(); let bytes = vec![0x3f, 0x3f, 0x3f, 0x3f, 0x3f, 0x3f, 0x3f]; let value = Value::Decimal(AvroDecimal::from(bytes)); - let options = AvroParseOptions::default().with_schema(&schema); - let resp = options.parse(&value, Some(&DataType::Decimal)).unwrap(); + let options = AvroParseOptions::create(&schema); + let resp = options + .convert_to_datum(&value, Some(&DataType::Decimal)) + .unwrap(); assert_eq!( resp, Some(ScalarImpl::Decimal(Decimal::Normalized( @@ -566,8 +569,10 @@ mod tests { ("value".to_string(), Value::Bytes(vec![0x01, 0x02, 0x03])), ]); - let options = AvroParseOptions::default().with_schema(&schema); - let resp = options.parse(&value, Some(&DataType::Decimal)).unwrap(); + let options = AvroParseOptions::create(&schema); + let resp = options + .convert_to_datum(&value, Some(&DataType::Decimal)) + .unwrap(); assert_eq!(resp, Some(ScalarImpl::Decimal(Decimal::from(66051)))); } } diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index 9c77382a0143d..a4996eabbf82e 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -316,8 +316,16 @@ pub fn extract_source_struct(info: &PbStreamSourceInfo) -> Result /// Stream of [`SourceMessage`]. pub type BoxSourceStream = BoxStream<'static, crate::error::ConnectorResult>>; -pub trait ChunkSourceStream = - Stream> + Send + 'static; +// Manually expand the trait alias to improve IDE experience. +pub trait ChunkSourceStream: + Stream> + Send + 'static +{ +} +impl ChunkSourceStream for T where + T: Stream> + Send + 'static +{ +} + pub type BoxChunkSourceStream = BoxStream<'static, crate::error::ConnectorResult>; pub type BoxTryStream = BoxStream<'static, crate::error::ConnectorResult>; @@ -543,6 +551,19 @@ pub struct SourceMessage { pub meta: SourceMeta, } +impl SourceMessage { + /// Create a dummy `SourceMessage` with all fields unset for testing purposes. + pub fn dummy() -> Self { + Self { + key: None, + payload: None, + offset: "".to_string(), + split_id: "".into(), + meta: SourceMeta::Empty, + } + } +} + #[derive(Debug, Clone)] pub enum SourceMeta { Kafka(KafkaMeta), diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 4f3b81a20e630..38829a16be11e 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -743,6 +743,7 @@ pub(crate) fn gen_create_table_plan_for_cdc_source( with_version_column: Option, include_column_options: IncludeOption, ) -> Result<(PlanRef, PbTable)> { + // cdc table must have primary key constraint or primary key column if !constraints.iter().any(|c| { matches!( c, @@ -751,6 +752,10 @@ pub(crate) fn gen_create_table_plan_for_cdc_source( .. } ) + }) && !column_defs.iter().any(|col| { + col.options + .iter() + .any(|opt| matches!(opt.option, ColumnOption::Unique { is_primary: true })) }) { return Err(ErrorCode::NotSupported( "CDC table without primary key constraint is not supported".to_owned(), diff --git a/src/frontend/src/handler/declare_cursor.rs b/src/frontend/src/handler/declare_cursor.rs index 6bd4e300ec0fa..25e146fa714ce 100644 --- a/src/frontend/src/handler/declare_cursor.rs +++ b/src/frontend/src/handler/declare_cursor.rs @@ -58,7 +58,6 @@ async fn handle_declare_subscription_cursor( let cursor_from_subscription_name = sub_name.0.last().unwrap().real_value().clone(); let subscription = session.get_subscription_by_name(schema_name, &cursor_from_subscription_name)?; - let table = session.get_table_by_id(&subscription.dependent_table_id)?; // Start the first query of cursor, which includes querying the table and querying the subscription's logstore let start_rw_timestamp = match rw_timestamp { Some(risingwave_sqlparser::ast::Since::TimestampMsNum(start_rw_timestamp)) => { @@ -81,8 +80,8 @@ async fn handle_declare_subscription_cursor( .add_subscription_cursor( cursor_name.clone(), start_rw_timestamp, + subscription.dependent_table_id, subscription, - table, &handle_args, ) .await?; diff --git a/src/frontend/src/optimizer/plan_node/generic/log_scan.rs b/src/frontend/src/optimizer/plan_node/generic/log_scan.rs index cd5ddebdc0724..498d4a44b0fcc 100644 --- a/src/frontend/src/optimizer/plan_node/generic/log_scan.rs +++ b/src/frontend/src/optimizer/plan_node/generic/log_scan.rs @@ -141,6 +141,19 @@ impl LogScan { Schema { fields } } + pub(crate) fn schema_without_table_name(&self) -> Schema { + let mut fields: Vec<_> = self + .output_col_idx + .iter() + .map(|tb_idx| { + let col = &self.table_desc.columns[*tb_idx]; + Field::from(col) + }) + .collect(); + fields.push(Field::with_name(OP_TYPE, OP_NAME)); + Schema { fields } + } + pub(crate) fn ctx(&self) -> OptimizerContextRef { self.ctx.clone() } diff --git a/src/frontend/src/session/cursor_manager.rs b/src/frontend/src/session/cursor_manager.rs index 13eaec03b1663..46eca3beb9966 100644 --- a/src/frontend/src/session/cursor_manager.rs +++ b/src/frontend/src/session/cursor_manager.rs @@ -30,6 +30,7 @@ use risingwave_sqlparser::ast::{Ident, ObjectName, Statement}; use super::SessionImpl; use crate::catalog::subscription_catalog::SubscriptionCatalog; +use crate::catalog::TableId; use crate::error::{ErrorCode, Result}; use crate::handler::declare_cursor::create_stream_for_cursor_stmt; use crate::handler::query::{create_stream, gen_batch_plan_fragmenter, BatchQueryPlanResult}; @@ -136,7 +137,7 @@ enum State { pub struct SubscriptionCursor { cursor_name: String, subscription: Arc, - table: Arc, + dependent_table_id: TableId, cursor_need_drop_time: Instant, state: State, } @@ -146,7 +147,7 @@ impl SubscriptionCursor { cursor_name: String, start_timestamp: Option, subscription: Arc, - table: Arc, + dependent_table_id: TableId, handle_args: &HandlerArgs, ) -> Result { let state = if let Some(start_timestamp) = start_timestamp { @@ -160,7 +161,7 @@ impl SubscriptionCursor { // // TODO: is this the right behavior? Should we delay the query stream initiation till the first fetch? let (row_stream, pg_descs) = - Self::initiate_query(None, &table, handle_args.clone()).await?; + Self::initiate_query(None, &dependent_table_id, handle_args.clone()).await?; let pinned_epoch = handle_args .session .get_pinned_snapshot() @@ -191,15 +192,16 @@ impl SubscriptionCursor { Ok(Self { cursor_name, subscription, - table, + dependent_table_id, cursor_need_drop_time, state, }) } - pub async fn next_row( + async fn next_row( &mut self, - handle_args: HandlerArgs, + handle_args: &HandlerArgs, + expected_pg_descs: &Vec, ) -> Result<(Option, Vec)> { loop { match &mut self.state { @@ -212,7 +214,7 @@ impl SubscriptionCursor { // Initiate a new batch query to continue fetching match Self::get_next_rw_timestamp( *seek_timestamp, - self.table.id.table_id, + self.dependent_table_id.table_id, *expected_timestamp, handle_args.clone(), ) @@ -221,7 +223,7 @@ impl SubscriptionCursor { Ok((Some(rw_timestamp), expected_timestamp)) => { let (mut row_stream, pg_descs) = Self::initiate_query( Some(rw_timestamp), - &self.table, + &self.dependent_table_id, handle_args.clone(), ) .await?; @@ -235,10 +237,15 @@ impl SubscriptionCursor { from_snapshot, rw_timestamp, row_stream, - pg_descs, + pg_descs: pg_descs.clone(), remaining_rows, expected_timestamp, }; + if (!expected_pg_descs.is_empty()) && expected_pg_descs.ne(&pg_descs) { + // If the user alters the table upstream of the sub, there will be different descs here. + // So we should output data for different descs in two separate batches + return Ok((None, vec![])); + } } Ok((None, _)) => return Ok((None, vec![])), Err(e) => { @@ -313,20 +320,25 @@ impl SubscriptionCursor { ) .into()); } - // `FETCH NEXT` is equivalent to `FETCH 1`. - if count != 1 { - Err(crate::error::ErrorCode::InternalError( - "FETCH count with subscription is not supported".to_string(), - ) - .into()) - } else { - let (row, pg_descs) = self.next_row(handle_args).await?; - if let Some(row) = row { - Ok((vec![row], pg_descs)) - } else { - Ok((vec![], pg_descs)) + + let mut ans = Vec::with_capacity(std::cmp::min(100, count) as usize); + let mut cur = 0; + let mut pg_descs_ans = vec![]; + while cur < count { + let (row, descs_ans) = self.next_row(&handle_args, &pg_descs_ans).await?; + match row { + Some(row) => { + pg_descs_ans = descs_ans; + cur += 1; + ans.push(row); + } + None => { + break; + } } } + + Ok((ans, pg_descs_ans)) } async fn get_next_rw_timestamp( @@ -358,16 +370,17 @@ impl SubscriptionCursor { async fn initiate_query( rw_timestamp: Option, - table_catalog: &TableCatalog, + dependent_table_id: &TableId, handle_args: HandlerArgs, ) -> Result<(PgResponseStream, Vec)> { + let session = handle_args.clone().session; + let table_catalog = session.get_table_by_id(dependent_table_id)?; let (row_stream, pg_descs) = if let Some(rw_timestamp) = rw_timestamp { - let context = OptimizerContext::from_handler_args(handle_args.clone()); - let session = handle_args.session; + let context = OptimizerContext::from_handler_args(handle_args); let plan_fragmenter_result = gen_batch_plan_fragmenter( &session, Self::create_batch_plan_for_cursor( - table_catalog, + &table_catalog, &session, context.into(), rw_timestamp, @@ -458,7 +471,11 @@ impl SubscriptionCursor { new_epoch, ); let batch_log_seq_scan = BatchLogSeqScan::new(core); - let out_fields = FixedBitSet::from_iter(0..batch_log_seq_scan.core().schema().len()); + let schema = batch_log_seq_scan + .core() + .schema_without_table_name() + .clone(); + let out_fields = FixedBitSet::from_iter(0..schema.len()); let out_names = batch_log_seq_scan.core().column_names(); // Here we just need a plan_root to call the method, only out_fields and out_names will be used let plan_root = PlanRoot::new_with_batch_plan( @@ -468,7 +485,6 @@ impl SubscriptionCursor { out_fields, out_names, ); - let schema = batch_log_seq_scan.core().schema().clone(); let (batch_log_seq_scan, query_mode) = match session.config().query_mode() { QueryMode::Auto => (plan_root.gen_batch_local_plan()?, QueryMode::Local), QueryMode::Local => (plan_root.gen_batch_local_plan()?, QueryMode::Local), @@ -497,15 +513,15 @@ impl CursorManager { &self, cursor_name: String, start_timestamp: Option, + dependent_table_id: TableId, subscription: Arc, - table: Arc, handle_args: &HandlerArgs, ) -> Result<()> { let cursor = SubscriptionCursor::new( cursor_name.clone(), start_timestamp, subscription, - table, + dependent_table_id, handle_args, ) .await?; diff --git a/src/meta/src/hummock/manager/compaction.rs b/src/meta/src/hummock/manager/compaction.rs index addb416893b08..16ca79a30962d 100644 --- a/src/meta/src/hummock/manager/compaction.rs +++ b/src/meta/src/hummock/manager/compaction.rs @@ -774,6 +774,7 @@ impl HummockManager { target_sub_level_id: compact_task.input.target_sub_level_id, task_type: compact_task.compaction_task_type as i32, split_weight_by_vnode: vnode_partition_count, + max_sub_compaction: group_config.compaction_config.max_sub_compaction, ..Default::default() }; diff --git a/src/risedevtool/src/bin/risedev-compose.rs b/src/risedevtool/src/bin/risedev-compose.rs index ec805a840fa71..5ff56916deca6 100644 --- a/src/risedevtool/src/bin/risedev-compose.rs +++ b/src/risedevtool/src/bin/risedev-compose.rs @@ -219,9 +219,10 @@ fn main() -> Result<()> { volumes.insert(c.id.clone(), ComposeVolume::default()); (c.address.clone(), c.compose(&compose_config)?) } - ServiceConfig::Redis(_) | ServiceConfig::MySql(_) | ServiceConfig::Postgres(_) => { - return Err(anyhow!("not supported")) - } + ServiceConfig::Redis(_) + | ServiceConfig::MySql(_) + | ServiceConfig::Postgres(_) + | ServiceConfig::SchemaRegistry(_) => return Err(anyhow!("not supported")), }; compose.container_name = service.id().to_string(); if opts.deploy { diff --git a/src/risedevtool/src/bin/risedev-dev.rs b/src/risedevtool/src/bin/risedev-dev.rs index 5a7ab843ddae2..8dbe155bcd086 100644 --- a/src/risedevtool/src/bin/risedev-dev.rs +++ b/src/risedevtool/src/bin/risedev-dev.rs @@ -27,8 +27,8 @@ use risedev::{ generate_risedev_env, preflight_check, CompactorService, ComputeNodeService, ConfigExpander, ConfigureTmuxTask, DummyService, EnsureStopService, ExecuteContext, FrontendService, GrafanaService, KafkaService, MetaNodeService, MinioService, MySqlService, PostgresService, - PrometheusService, PubsubService, RedisService, ServiceConfig, SqliteConfig, Task, - TempoService, RISEDEV_NAME, + PrometheusService, PubsubService, RedisService, SchemaRegistryService, ServiceConfig, + SqliteConfig, Task, TempoService, RISEDEV_NAME, }; use tempfile::tempdir; use thiserror_ext::AsReport; @@ -279,6 +279,18 @@ fn task_main( ctx.pb .set_message(format!("kafka {}:{}", c.address, c.port)); } + ServiceConfig::SchemaRegistry(c) => { + let mut ctx = + ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); + let mut service = SchemaRegistryService::new(c.clone()); + service.execute(&mut ctx)?; + let mut task = + risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, c.user_managed)?; + task.execute(&mut ctx)?; + ctx.pb + .set_message(format!("schema registry http://{}:{}", c.address, c.port)); + } + ServiceConfig::Pubsub(c) => { let mut ctx = ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); diff --git a/src/risedevtool/src/config.rs b/src/risedevtool/src/config.rs index 839ebc22486ee..bf768f8e68cd1 100644 --- a/src/risedevtool/src/config.rs +++ b/src/risedevtool/src/config.rs @@ -175,6 +175,9 @@ impl ConfigExpander { "redpanda" => ServiceConfig::RedPanda(serde_yaml::from_str(&out_str)?), "mysql" => ServiceConfig::MySql(serde_yaml::from_str(&out_str)?), "postgres" => ServiceConfig::Postgres(serde_yaml::from_str(&out_str)?), + "schema-registry" => { + ServiceConfig::SchemaRegistry(serde_yaml::from_str(&out_str)?) + } other => return Err(anyhow!("unsupported use type: {}", other)), }; Ok(result) diff --git a/src/risedevtool/src/risedev_env.rs b/src/risedevtool/src/risedev_env.rs index a45864f097854..2b6cc367b2e71 100644 --- a/src/risedevtool/src/risedev_env.rs +++ b/src/risedevtool/src/risedev_env.rs @@ -77,6 +77,15 @@ pub fn generate_risedev_env(services: &Vec) -> String { writeln!(env, r#"RISEDEV_KAFKA_WITH_OPTIONS_COMMON="connector='kafka',properties.bootstrap.server='{brokers}'""#).unwrap(); writeln!(env, r#"RPK_BROKERS="{brokers}""#).unwrap(); } + ServiceConfig::SchemaRegistry(c) => { + let address = &c.address; + let port = &c.port; + writeln!( + env, + r#"RISEDEV_SCHEMA_REGISTRY_URL="http://{address}:{port}""#, + ) + .unwrap(); + } ServiceConfig::MySql(c) => { let host = &c.address; let port = &c.port; diff --git a/src/risedevtool/src/service_config.rs b/src/risedevtool/src/service_config.rs index 88c1594fb1153..71461b0f58bcc 100644 --- a/src/risedevtool/src/service_config.rs +++ b/src/risedevtool/src/service_config.rs @@ -271,12 +271,16 @@ pub struct KafkaConfig { phantom_use: Option, pub id: String, + /// Advertise address pub address: String, #[serde(with = "string")] pub port: u16, + /// Port for other services in docker. They need to connect to `host.docker.internal`, while the host + /// need to connect to `localhost`. + pub docker_port: u16, + #[serde(with = "string")] pub controller_port: u16, - pub listen_address: String, pub image: String, pub persist_data: bool, @@ -284,6 +288,28 @@ pub struct KafkaConfig { pub user_managed: bool, } + +#[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] +#[serde(rename_all = "kebab-case")] +#[serde(deny_unknown_fields)] +pub struct SchemaRegistryConfig { + #[serde(rename = "use")] + phantom_use: Option, + + pub id: String, + + pub address: String, + #[serde(with = "string")] + pub port: u16, + + pub provide_kafka: Option>, + + pub image: String, + /// Redpanda supports schema registry natively. You can configure a `user_managed` schema registry + /// to use with redpanda. + pub user_managed: bool, +} + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] #[serde(rename_all = "kebab-case")] #[serde(deny_unknown_fields)] @@ -380,6 +406,7 @@ pub enum ServiceConfig { Opendal(OpendalConfig), AwsS3(AwsS3Config), Kafka(KafkaConfig), + SchemaRegistry(SchemaRegistryConfig), Pubsub(PubsubConfig), Redis(RedisConfig), RedPanda(RedPandaConfig), @@ -407,10 +434,12 @@ impl ServiceConfig { Self::RedPanda(c) => &c.id, Self::Opendal(c) => &c.id, Self::MySql(c) => &c.id, - ServiceConfig::Postgres(c) => &c.id, + Self::Postgres(c) => &c.id, + Self::SchemaRegistry(c) => &c.id, } } + /// Used to check whether the port is occupied before running the service. pub fn port(&self) -> Option { match self { Self::ComputeNode(c) => Some(c.port), @@ -430,7 +459,8 @@ impl ServiceConfig { Self::RedPanda(_c) => None, Self::Opendal(_) => None, Self::MySql(c) => Some(c.port), - ServiceConfig::Postgres(c) => Some(c.port), + Self::Postgres(c) => Some(c.port), + Self::SchemaRegistry(c) => Some(c.port), } } @@ -454,6 +484,7 @@ impl ServiceConfig { Self::Opendal(_c) => false, Self::MySql(c) => c.user_managed, Self::Postgres(c) => c.user_managed, + Self::SchemaRegistry(c) => c.user_managed, } } } diff --git a/src/risedevtool/src/task.rs b/src/risedevtool/src/task.rs index e34cddd908b7f..21b6f20eec5ee 100644 --- a/src/risedevtool/src/task.rs +++ b/src/risedevtool/src/task.rs @@ -29,6 +29,7 @@ mod postgres_service; mod prometheus_service; mod pubsub_service; mod redis_service; +mod schema_registry_service; mod task_configure_minio; mod task_etcd_ready_check; mod task_kafka_ready_check; @@ -68,6 +69,7 @@ pub use self::postgres_service::*; pub use self::prometheus_service::*; pub use self::pubsub_service::*; pub use self::redis_service::*; +pub use self::schema_registry_service::SchemaRegistryService; pub use self::task_configure_minio::*; pub use self::task_etcd_ready_check::*; pub use self::task_kafka_ready_check::*; diff --git a/src/risedevtool/src/task/docker_service.rs b/src/risedevtool/src/task/docker_service.rs index 58ff2b59648c0..b87ee8a6a8aef 100644 --- a/src/risedevtool/src/task/docker_service.rs +++ b/src/risedevtool/src/task/docker_service.rs @@ -100,7 +100,9 @@ where cmd.arg("run") .arg("--rm") .arg("--name") - .arg(format!("risedev-{}", self.id())); + .arg(format!("risedev-{}", self.id())) + .arg("--add-host") + .arg("host.docker.internal:host-gateway"); for (k, v) in self.config.envs() { cmd.arg("-e").arg(format!("{k}={v}")); diff --git a/src/risedevtool/src/task/kafka_service.rs b/src/risedevtool/src/task/kafka_service.rs index 52bdd227a72a4..7c415b6d9749a 100644 --- a/src/risedevtool/src/task/kafka_service.rs +++ b/src/risedevtool/src/task/kafka_service.rs @@ -37,15 +37,18 @@ impl DockerServiceConfig for KafkaConfig { ), ( "KAFKA_LISTENERS".to_owned(), - "PLAINTEXT://:9092,CONTROLLER://:9093".to_owned(), + "HOST://:9092,CONTROLLER://:9093,DOCKER://:9094".to_owned(), ), ( "KAFKA_ADVERTISED_LISTENERS".to_owned(), - format!("PLAINTEXT://{}:{}", self.address, self.port), + format!( + "HOST://{}:{},DOCKER://host.docker.internal:{}", + self.address, self.port, self.docker_port + ), ), ( "KAFKA_LISTENER_SECURITY_PROTOCOL_MAP".to_owned(), - "PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT".to_owned(), + "HOST:PLAINTEXT,CONTROLLER:PLAINTEXT,DOCKER:PLAINTEXT".to_owned(), ), ( "KAFKA_CONTROLLER_QUORUM_VOTERS".to_owned(), @@ -55,12 +58,19 @@ impl DockerServiceConfig for KafkaConfig { "KAFKA_CONTROLLER_LISTENER_NAMES".to_owned(), "CONTROLLER".to_owned(), ), + ( + "KAFKA_INTER_BROKER_LISTENER_NAME".to_owned(), + "HOST".to_owned(), + ), ("CLUSTER_ID".to_owned(), "RiseDevRiseDevRiseDev1".to_owned()), ] } fn ports(&self) -> Vec<(String, String)> { - vec![(self.port.to_string(), "9092".to_owned())] + vec![ + (self.port.to_string(), "9092".to_owned()), + (self.docker_port.to_string(), "9094".to_owned()), + ] } fn data_path(&self) -> Option { diff --git a/src/risedevtool/src/task/schema_registry_service.rs b/src/risedevtool/src/task/schema_registry_service.rs new file mode 100644 index 0000000000000..5c5eba4fa8f35 --- /dev/null +++ b/src/risedevtool/src/task/schema_registry_service.rs @@ -0,0 +1,65 @@ +// 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 super::docker_service::{DockerService, DockerServiceConfig}; +use crate::SchemaRegistryConfig; + +impl DockerServiceConfig for SchemaRegistryConfig { + fn id(&self) -> String { + self.id.clone() + } + + fn is_user_managed(&self) -> bool { + self.user_managed + } + + fn image(&self) -> String { + self.image.clone() + } + + fn envs(&self) -> Vec<(String, String)> { + // https://docs.confluent.io/platform/current/installation/docker/config-reference.html#sr-long-configuration + // https://docs.confluent.io/platform/current/schema-registry/installation/config.html + let kafka = self + .provide_kafka + .as_ref() + .expect("Kafka is required for Schema Registry"); + if kafka.len() != 1 { + panic!("More than one Kafka is not supported yet"); + } + let kafka = &kafka[0]; + vec![ + ("SCHEMA_REGISTRY_HOST_NAME".to_owned(), self.address.clone()), + ( + "SCHEMA_REGISTRY_LISTENERS".to_owned(), + format!("http://{}:{}", self.address, self.port), + ), + ( + "SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS".to_owned(), + format!("host.docker.internal:{}", kafka.docker_port), + ), + ] + } + + fn ports(&self) -> Vec<(String, String)> { + vec![(self.port.to_string(), "8081".to_owned())] + } + + fn data_path(&self) -> Option { + None + } +} + +/// Docker-backed Schema Registry service. +pub type SchemaRegistryService = DockerService; diff --git a/src/storage/src/hummock/compactor/compaction_utils.rs b/src/storage/src/hummock/compactor/compaction_utils.rs index d0e5fe93c62ee..63b59366195f0 100644 --- a/src/storage/src/hummock/compactor/compaction_utils.rs +++ b/src/storage/src/hummock/compactor/compaction_utils.rs @@ -28,8 +28,7 @@ use risingwave_hummock_sdk::table_stats::TableStatsMap; use risingwave_hummock_sdk::{can_concat, EpochWithGap, KeyComparator}; use risingwave_pb::hummock::compact_task::TaskType; use risingwave_pb::hummock::{ - compact_task, BloomFilterType, CompactTask, KeyRange as KeyRange_vec, LevelType, SstableInfo, - TableSchema, + compact_task, BloomFilterType, CompactTask, LevelType, PbKeyRange, SstableInfo, TableSchema, }; use tokio::time::Instant; @@ -178,7 +177,8 @@ fn generate_splits_fast( sstable_infos: &Vec, compaction_size: u64, context: &CompactorContext, -) -> Vec { + max_sub_compaction: u32, +) -> Vec { let worker_num = context.compaction_executor.worker_num(); let parallel_compact_size = (context.storage_opts.parallel_compact_size_mb as u64) << 20; @@ -186,7 +186,7 @@ fn generate_splits_fast( worker_num, parallel_compact_size, compaction_size, - context.storage_opts.max_sub_compaction, + max_sub_compaction, ); let mut indexes = vec![]; for sst in sstable_infos { @@ -213,13 +213,13 @@ fn generate_splits_fast( } let mut splits = vec![]; - splits.push(KeyRange_vec::new(vec![], vec![])); + splits.push(PbKeyRange::new(vec![], vec![])); let parallel_key_count = indexes.len() / parallelism; let mut last_split_key_count = 0; for key in indexes { if last_split_key_count >= parallel_key_count { splits.last_mut().unwrap().right.clone_from(&key); - splits.push(KeyRange_vec::new(key.clone(), vec![])); + splits.push(PbKeyRange::new(key.clone(), vec![])); last_split_key_count = 0; } last_split_key_count += 1; @@ -232,7 +232,8 @@ pub async fn generate_splits( sstable_infos: &Vec, compaction_size: u64, context: &CompactorContext, -) -> HummockResult> { + max_sub_compaction: u32, +) -> HummockResult> { const MAX_FILE_COUNT: usize = 32; let parallel_compact_size = (context.storage_opts.parallel_compact_size_mb as u64) << 20; if compaction_size > parallel_compact_size { @@ -241,6 +242,7 @@ pub async fn generate_splits( sstable_infos, compaction_size, context, + max_sub_compaction, )); } let mut indexes = vec![]; @@ -269,13 +271,13 @@ pub async fn generate_splits( // sort by key, as for every data block has the same size; indexes.sort_by(|a, b| KeyComparator::compare_encoded_full_key(a.1.as_ref(), b.1.as_ref())); let mut splits = vec![]; - splits.push(KeyRange_vec::new(vec![], vec![])); + splits.push(PbKeyRange::new(vec![], vec![])); let parallelism = calculate_task_parallelism_impl( context.compaction_executor.worker_num(), parallel_compact_size, compaction_size, - context.storage_opts.max_sub_compaction, + max_sub_compaction, ); let sub_compaction_data_size = @@ -291,7 +293,7 @@ pub async fn generate_splits( && remaining_size > parallel_compact_size { splits.last_mut().unwrap().right.clone_from(&key); - splits.push(KeyRange_vec::new(key.clone(), vec![])); + splits.push(PbKeyRange::new(key.clone(), vec![])); last_buffer_size = data_size; } else { last_buffer_size += data_size; @@ -577,7 +579,13 @@ pub async fn generate_splits_for_task( .sum::(); if !optimize_by_copy_block { - let splits = generate_splits(&sstable_infos, compaction_size, context).await?; + let splits = generate_splits( + &sstable_infos, + compaction_size, + context, + compact_task.get_max_sub_compaction(), + ) + .await?; if !splits.is_empty() { compact_task.splits = splits; } @@ -659,7 +667,7 @@ pub fn calculate_task_parallelism(compact_task: &CompactTask, context: &Compacto context.compaction_executor.worker_num(), parallel_compact_size, compaction_size, - context.storage_opts.max_sub_compaction, + compact_task.get_max_sub_compaction(), ) } diff --git a/src/storage/src/opts.rs b/src/storage/src/opts.rs index aa4fd4cbb9630..5a7bca2c30b42 100644 --- a/src/storage/src/opts.rs +++ b/src/storage/src/opts.rs @@ -74,8 +74,6 @@ pub struct StorageOpts { pub sstable_id_remote_fetch_number: u32, /// Whether to enable streaming upload for sstable. pub min_sst_size_for_streaming_upload: u64, - /// Max sub compaction task numbers - pub max_sub_compaction: u32, pub max_concurrent_compaction_task_number: u64, pub max_version_pinning_duration_sec: u64, pub compactor_iter_max_io_retry_times: usize, @@ -176,7 +174,6 @@ impl From<(&RwConfig, &SystemParamsReader, &StorageMemoryConfig)> for StorageOpt compactor_memory_limit_mb: s.compactor_memory_limit_mb, sstable_id_remote_fetch_number: c.storage.sstable_id_remote_fetch_number, min_sst_size_for_streaming_upload: c.storage.min_sst_size_for_streaming_upload, - max_sub_compaction: c.storage.max_sub_compaction, max_concurrent_compaction_task_number: c.storage.max_concurrent_compaction_task_number, max_version_pinning_duration_sec: c.storage.max_version_pinning_duration_sec, data_file_cache_dir: c.storage.data_file_cache.dir.clone(), diff --git a/src/stream/src/common/log_store_impl/kv_log_store/reader.rs b/src/stream/src/common/log_store_impl/kv_log_store/reader.rs index 21ee99ec91d08..e2e767c8d2038 100644 --- a/src/stream/src/common/log_store_impl/kv_log_store/reader.rs +++ b/src/stream/src/common/log_store_impl/kv_log_store/reader.rs @@ -13,12 +13,14 @@ // limitations under the License. use std::future::Future; +use std::ops::Bound; use std::ops::Bound::{Excluded, Included, Unbounded}; use std::pin::Pin; -use std::time::Duration; +use std::time::{Duration, Instant}; use anyhow::anyhow; use await_tree::InstrumentAwait; +use bytes::Bytes; use foyer::CacheContext; use futures::future::{try_join_all, BoxFuture}; use futures::{FutureExt, TryFutureExt}; @@ -31,11 +33,14 @@ use risingwave_common::util::epoch::EpochExt; use risingwave_connector::sink::log_store::{ ChunkId, LogReader, LogStoreReadItem, LogStoreResult, TruncateOffset, }; -use risingwave_hummock_sdk::key::prefixed_range_with_vnode; +use risingwave_hummock_sdk::key::{prefixed_range_with_vnode, FullKey, TableKey, TableKeyRange}; use risingwave_hummock_sdk::HummockEpoch; +use risingwave_storage::error::StorageResult; use risingwave_storage::hummock::CachePolicy; -use risingwave_storage::store::{PrefetchOptions, ReadOptions}; -use risingwave_storage::StateStore; +use risingwave_storage::store::{ + PrefetchOptions, ReadOptions, StateStoreIterItemRef, StateStoreRead, +}; +use risingwave_storage::{StateStore, StateStoreIter}; use tokio::sync::watch; use tokio::time::sleep; use tokio_stream::StreamExt; @@ -113,7 +118,7 @@ pub struct KvLogStoreReader { first_write_epoch: Option, /// `Some` means consuming historical log data - state_store_stream: Option>>>, + state_store_stream: Option>>>>, /// Store the future that attempts to read a flushed stream chunk. /// This is for cancellation safety. Since it is possible that the future of `next_item` @@ -180,12 +185,141 @@ impl KvLogStoreReader { } } +struct AutoRebuildStateStoreReadIter { + state_store: S, + iter: S::Iter, + // call to get whether to rebuild the iter. Once return true, the closure should reset itself. + should_rebuild: F, + end_bound: Bound>, + epoch: HummockEpoch, + options: ReadOptions, +} + +impl bool> AutoRebuildStateStoreReadIter { + async fn new( + state_store: S, + should_rebuild: F, + range: TableKeyRange, + epoch: HummockEpoch, + options: ReadOptions, + ) -> StorageResult { + let (start_bound, end_bound) = range; + let iter = state_store + .iter((start_bound, end_bound.clone()), epoch, options.clone()) + .await?; + Ok(Self { + state_store, + iter, + should_rebuild, + end_bound, + epoch, + options, + }) + } +} + +type TimeoutAutoRebuildIter = + AutoRebuildStateStoreReadIter bool + Send>; + +async fn iter_with_timeout_rebuild( + state_store: S, + range: TableKeyRange, + epoch: HummockEpoch, + options: ReadOptions, + timeout: Duration, +) -> StorageResult> { + const CHECK_TIMEOUT_PERIOD: usize = 100; + // use a struct here to avoid accidental copy instead of move on primitive usize + struct Count(usize); + let mut check_count = Count(0); + let mut total_count = Count(0); + let mut curr_iter_item_count = Count(0); + let mut start_time = Instant::now(); + let initial_start_time = start_time; + AutoRebuildStateStoreReadIter::new( + state_store, + move || { + check_count.0 += 1; + curr_iter_item_count.0 += 1; + total_count.0 += 1; + if check_count.0 == CHECK_TIMEOUT_PERIOD { + check_count.0 = 0; + if start_time.elapsed() > timeout { + let prev_iter_item_count = curr_iter_item_count.0; + curr_iter_item_count.0 = 0; + start_time = Instant::now(); + info!( + table_id = options.table_id.table_id, + iter_exist_time_secs = initial_start_time.elapsed().as_secs(), + prev_iter_item_count, + total_iter_item_count = total_count.0, + "kv log store iter is rebuilt" + ); + true + } else { + false + } + } else { + false + } + }, + range, + epoch, + options, + ) + .await +} + +impl bool + Send> StateStoreIter + for AutoRebuildStateStoreReadIter +{ + async fn try_next(&mut self) -> StorageResult>> { + let should_rebuild = (self.should_rebuild)(); + if should_rebuild { + let Some((key, _value)) = self.iter.try_next().await? else { + return Ok(None); + }; + let key: FullKey<&[u8]> = key; + let range_start = Bytes::copy_from_slice(key.user_key.table_key.as_ref()); + let new_iter = self + .state_store + .iter( + ( + Included(TableKey(range_start.clone())), + self.end_bound.clone(), + ), + self.epoch, + self.options.clone(), + ) + .await?; + self.iter = new_iter; + let item: Option> = self.iter.try_next().await?; + if let Some((key, value)) = item { + assert_eq!( + key.user_key.table_key.0, + range_start.as_ref(), + "the first key should be the previous key" + ); + Ok(Some((key, value))) + } else { + unreachable!( + "the first key should be the previous key {:?}, but get None", + range_start + ) + } + } else { + self.iter.try_next().await + } + } +} + impl KvLogStoreReader { fn read_persisted_log_store( &self, last_persisted_epoch: Option, - ) -> impl Future>>>> + Send - { + ) -> impl Future< + Output = LogStoreResult>>>>, + > + Send { let range_start = if let Some(last_persisted_epoch) = last_persisted_epoch { // start from the next epoch of last_persisted_epoch Included( @@ -210,19 +344,21 @@ impl KvLogStoreReader { ); let state_store = self.state_store.clone(); async move { - state_store - .iter( - key_range, - HummockEpoch::MAX, - ReadOptions { - // This stream lives too long, the connection of prefetch object may break. So use a short connection prefetch. - prefetch_options: PrefetchOptions::prefetch_for_small_range_scan(), - cache_policy: CachePolicy::Fill(CacheContext::LruPriorityLow), - table_id, - ..Default::default() - }, - ) - .await + // rebuild the iter every 10 minutes to avoid pinning hummock version for too long + iter_with_timeout_rebuild( + state_store, + key_range, + HummockEpoch::MAX, + ReadOptions { + // This stream lives too long, the connection of prefetch object may break. So use a short connection prefetch. + prefetch_options: PrefetchOptions::prefetch_for_small_range_scan(), + cache_policy: CachePolicy::Fill(CacheContext::LruPriorityLow), + table_id, + ..Default::default() + }, + Duration::from_secs(10 * 60), + ) + .await } })); @@ -500,3 +636,98 @@ impl LogReader for KvLogStoreReader { Ok((true, Some((**self.serde.vnodes()).clone()))) } } + +#[cfg(test)] +mod tests { + use std::ops::Bound::Unbounded; + + use bytes::Bytes; + use itertools::Itertools; + use risingwave_common::util::epoch::test_epoch; + use risingwave_hummock_sdk::key::TableKey; + use risingwave_storage::hummock::iterator::test_utils::{ + iterator_test_table_key_of, iterator_test_value_of, + }; + use risingwave_storage::memory::MemoryStateStore; + use risingwave_storage::storage_value::StorageValue; + use risingwave_storage::store::{ReadOptions, StateStoreRead, StateStoreWrite, WriteOptions}; + use risingwave_storage::StateStoreIter; + + use crate::common::log_store_impl::kv_log_store::reader::AutoRebuildStateStoreReadIter; + use crate::common::log_store_impl::kv_log_store::test_utils::TEST_TABLE_ID; + + #[tokio::test] + async fn test_auto_rebuild_iter() { + let state_store = MemoryStateStore::new(); + let key_count = 100; + let pairs = (0..key_count) + .map(|i| { + let key = iterator_test_table_key_of(i); + let value = iterator_test_value_of(i); + (TableKey(Bytes::from(key)), StorageValue::new_put(value)) + }) + .collect_vec(); + let epoch = test_epoch(1); + state_store + .ingest_batch( + pairs.clone(), + vec![], + WriteOptions { + epoch, + table_id: TEST_TABLE_ID, + }, + ) + .unwrap(); + + async fn validate( + mut kv_iter: impl Iterator, StorageValue)>, + mut iter: impl StateStoreIter, + ) { + while let Some((key, value)) = iter.try_next().await.unwrap() { + let (k, v) = kv_iter.next().unwrap(); + assert_eq!(key.user_key.table_key, k.to_ref()); + assert_eq!(v.user_value.as_deref(), Some(value)); + } + assert!(kv_iter.next().is_none()); + } + + let read_options = ReadOptions { + table_id: TEST_TABLE_ID, + ..Default::default() + }; + + let kv_iter = pairs.clone().into_iter(); + let iter = state_store + .iter((Unbounded, Unbounded), epoch, read_options.clone()) + .await + .unwrap(); + validate(kv_iter, iter).await; + + let kv_iter = pairs.clone().into_iter(); + let mut count = 0; + let count_mut_ref = &mut count; + let rebuild_period = 8; + let mut rebuild_count = 0; + let rebuild_count_mut_ref = &mut rebuild_count; + let iter = AutoRebuildStateStoreReadIter::new( + state_store, + move || { + *count_mut_ref += 1; + if *count_mut_ref % rebuild_period == 0 { + *rebuild_count_mut_ref += 1; + true + } else { + false + } + }, + (Unbounded, Unbounded), + epoch, + read_options, + ) + .await + .unwrap(); + validate(kv_iter, iter).await; + assert_eq!(count, key_count + 1); // with an extra call on the last None + assert_eq!(rebuild_count, key_count / rebuild_period); + } +} diff --git a/src/stream/src/common/log_store_impl/kv_log_store/serde.rs b/src/stream/src/common/log_store_impl/kv_log_store/serde.rs index 9eb7faf237ead..9871139bafddc 100644 --- a/src/stream/src/common/log_store_impl/kv_log_store/serde.rs +++ b/src/stream/src/common/log_store_impl/kv_log_store/serde.rs @@ -544,7 +544,7 @@ impl LogStoreRowOpStream { } } -pub(crate) type LogStoreItemMergeStream = +pub(crate) type LogStoreItemMergeStream = impl Stream>; pub(crate) fn merge_log_store_item_stream( iters: Vec, diff --git a/src/utils/pgwire/src/pg_field_descriptor.rs b/src/utils/pgwire/src/pg_field_descriptor.rs index 0b33c5743c107..82d75c78f7956 100644 --- a/src/utils/pgwire/src/pg_field_descriptor.rs +++ b/src/utils/pgwire/src/pg_field_descriptor.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -#[derive(Debug, Clone)] +#[derive(Debug, Clone, PartialEq, Eq)] pub struct PgFieldDescriptor { name: String, table_oid: i32,