diff --git a/.gitignore b/.gitignore index 8bdfcd613..dd88239fc 100644 --- a/.gitignore +++ b/.gitignore @@ -92,6 +92,9 @@ agents bin conf lib +# Exception: docker demo config files are tracked +!docker/cdc-demo/conf/ +!docker/cdc-demo/conf/** # Local gradle cache .gradle diff --git a/CHANGES.txt b/CHANGES.txt index 45666644d..85a0ae018 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,5 +1,6 @@ 0.4.0 ----- + * Add Docker Compose setup for local CDC demo (Cassandra → Sidecar → Kafka) (CASSSIDECAR-419) * Add JDK11_OPTIONS to the startup script (CASSSIDECAR-416) * Add safety check to Live Migration data copy task endpoint (CASSSIDECAR-409) * Define common operational job tracking interface and refactor current operational job tracker (CASSSIDECAR-372) diff --git a/docker/cdc-demo/Dockerfile.sidecar b/docker/cdc-demo/Dockerfile.sidecar new file mode 100644 index 000000000..29445c63a --- /dev/null +++ b/docker/cdc-demo/Dockerfile.sidecar @@ -0,0 +1,59 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +# +# Two-stage build: Stage 1 compiles via Gradle installDist; Stage 2 copies +# the assembled distribution into a lean JRE image. +# Usage: docker build -f docker/cdc-demo/Dockerfile.sidecar -t cassandra-sidecar:dev . + +# ── Stage 1: build ─────────────────────────────────────────────────────────── +FROM eclipse-temurin:11-jdk-jammy AS builder + +WORKDIR /build + +# Build descriptors first — keeps the dependency-download layer cached separately. +COPY gradlew gradlew +COPY gradle/ gradle/ +COPY gradle.properties settings.gradle build.gradle ./ +COPY adapters/ adapters/ +COPY client/ client/ +COPY client-common/ client-common/ +COPY docs/ docs/ +COPY server-common/ server-common/ +COPY vertx-auth-mtls/ vertx-auth-mtls/ +COPY vertx-client/ vertx-client/ +COPY vertx-client-shaded/ vertx-client-shaded/ +COPY server/ server/ +COPY conf/ conf/ + +# Output: build/install/apache-cassandra-sidecar/{bin,lib,agents,conf} +RUN --mount=type=cache,target=/root/.gradle \ + ./gradlew installDist -x test -x integrationTest -x containerTest \ + --no-daemon --parallel --quiet + +# ── Stage 2: runtime ───────────────────────────────────────────────────────── +FROM eclipse-temurin:11-jre-jammy + +WORKDIR /app + +COPY --from=builder /build/build/install/apache-cassandra-sidecar/ ./ + +# Bake demo config; docker-compose overrides at runtime via volume mount. +COPY docker/cdc-demo/conf/sidecar.yaml /app/conf/sidecar.yaml + +EXPOSE 9043 + +ENTRYPOINT ["/app/bin/cassandra-sidecar"] diff --git a/docker/cdc-demo/README.md b/docker/cdc-demo/README.md new file mode 100644 index 000000000..0631c7e7b --- /dev/null +++ b/docker/cdc-demo/README.md @@ -0,0 +1,208 @@ + +# CDC Demo — Docker Compose Setup + +End-to-end demo that boots Cassandra, Cassandra Sidecar, and Kafka. +Writes to a CDC-enabled Cassandra table are captured by the sidecar and +published as Avro messages to a Kafka topic. + +## Architecture + +``` +┌──────────────┐ cdc_raw/commitlog ┌──────────────────┐ +│ Cassandra │ ─────────────────────► │ Cassandra │──► Kafka (cdc-mutations) +│ (port 9042) │ (shared volume) │ Sidecar │ +└──────────────┘ │ (port 9043) │ + └──────────────────┘ + │ + ▼ + ┌──────────────────┐ + │ Kafka UI │ + │ localhost:8080 │ + └──────────────────┘ +``` + +**Services:** +- `kafka` — KRaft Kafka broker (no ZooKeeper) +- `cassandra` — Cassandra with CDC enabled +- `cassandra-init` — one-shot container that seeds the sidecar schema and CDC/Kafka configs (reuses the already-pulled Cassandra image) +- `sidecar` — Cassandra Sidecar; starts after schema is ready +- `kafka-ui` — Web UI to browse Kafka topics at http://localhost:8080 + +## Prerequisites + +| Tool | Version | +|----------------|--------------| +| Docker | 24+ | +| Docker Compose | v2 (plugin) | +| Java | 11 | +| Gradle | via wrapper | + +## Exposed ports + +| Port | Service | +|--------|-------------------| +| `9042` | Cassandra CQL | +| `9043` | Cassandra Sidecar | +| `8080` | Kafka UI | + +## Quick Start + +### Step 1 — Build and start the stack + +From `docker/cdc-demo/`, run the start script — it builds the sidecar image and +starts all services: + +```bash +cd docker/cdc-demo +./scripts/start.sh +``` + +To wipe all data and start completely fresh: + +```bash +./scripts/start.sh --clean +``` + +> **Manual build** (if you prefer): from the repository root, run +> `DOCKER_BUILDKIT=1 docker build -f docker/cdc-demo/Dockerfile.sidecar -t cassandra-sidecar:dev .` +> (`DOCKER_BUILDKIT=1` is required for the Gradle dependency cache mount in the Dockerfile) +> then update `docker-compose.yml` to use `image: cassandra-sidecar:dev` instead of +> the `build:` block, and run `docker compose up -d`. + +### Step 2 — Wait for CDC to be ready + +The `start.sh` script automatically waits until the sidecar is up and CDC +iterators have started, then prints a **Setup complete** message. No manual +log watching is needed. + +To follow progress in another terminal while waiting: + +```bash +docker compose logs -f cassandra-init sidecar +``` + +### Step 3 — Write mutations to the CDC-enabled table + +```bash +docker exec -it cdc-demo-cassandra-1 cqlsh -e " + INSERT INTO cdc_demo.events (id, msg, ts) + VALUES (uuid(), 'hello from CDC', toTimestamp(now())); +" +``` + +### Step 4 — View messages in Kafka UI + +The **Setup complete** message printed by `start.sh` includes a direct link to +the `cdc-mutations` topic. You can also open it directly: + +``` +http://localhost:8080/ui/clusters/local/all-topics/cdc-mutations/messages +``` + +Click the **Messages** tab to see one Avro-encoded message per Cassandra mutation. + +### Step 5 — Consume events from Kafka (CLI) + +```bash +docker exec -it cdc-demo-kafka-1 \ + kafka-console-consumer \ + --bootstrap-server kafka:9092 \ + --topic cdc-mutations \ + --from-beginning \ + --max-messages 5 +``` + +## Supported Cassandra Versions + +CDC is supported for **4.0, 4.1, 5.0, 5.1**. To use a different version: + +```bash +CASSANDRA_VERSION=4.1 docker compose up -d +``` + +The default is `5.0`. Note: the `cassandra:4.0` Docker image is `linux/amd64` only — on Apple Silicon it runs under Rosetta emulation and may be slow to start. Use `4.1` or later for ARM64 support. + +## Configuration + +`conf/sidecar.yaml` is volume-mounted into the sidecar container and can be +edited without rebuilding the image. Restart the sidecar to pick up changes: + +```bash +docker compose restart sidecar +``` + +CDC and Kafka properties read by the sidecar are stored in Cassandra and seeded +automatically on first boot. To update them after the cluster is running: + +```bash +docker exec -it cdc-demo-cassandra-1 cqlsh -e " + UPDATE sidecar_internal.configs + SET config = config + {'micro_batch_delay_millis': '500'} + WHERE service = 'cdc'; +" +``` + +## Persistence + +All data is stored in named Docker volumes and survives `docker compose down`. + +| Volume | Contents | +|--------------------|--------------------------------------------------| +| `cassandra-varlib` | All Cassandra data (data/, commitlog/, cdc_raw/) | +| `kafka-data` | Topic partitions + consumer offsets | + +**To wipe everything and start fresh:** + +```bash +docker compose down -v +``` + +## Stopping + +```bash +docker compose down # stop containers, keep volumes +docker compose down -v # stop containers AND delete all data +``` + +## Troubleshooting + +**Sidecar keeps restarting** +The sidecar waits for `cassandra-init` to complete successfully. Check its logs: + +```bash +docker compose logs cassandra-init +``` + +**CDC events not arriving in Kafka** +1. Verify configs were seeded: `docker exec cdc-demo-cassandra-1 cqlsh -e "SELECT * FROM sidecar_internal.configs;"` +2. Check sidecar logs for `CDC iterators started successfully` +3. Confirm CDC is enabled on the table: `docker exec cdc-demo-cassandra-1 cqlsh -e "DESCRIBE TABLE cdc_demo.events;"` + +**JMX connection refused** +Remote JMX is enabled by the `LOCAL_JMX=no` env var on the Cassandra service in +`docker-compose.yml`, which causes the stock Cassandra Docker entrypoint to set +`jmxremote.local.only=false`. The `JVM_EXTRA_OPTS` env var additionally sets +`-Djava.rmi.server.hostname=cassandra` so RMI binds to the right interface. +If the sidecar still can't connect, verify the flags are active: + +```bash +docker exec cdc-demo-cassandra-1 ps aux | grep jmxremote +``` diff --git a/docker/cdc-demo/conf/sidecar.yaml b/docker/cdc-demo/conf/sidecar.yaml new file mode 100644 index 000000000..525a12030 --- /dev/null +++ b/docker/cdc-demo/conf/sidecar.yaml @@ -0,0 +1,78 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +# +cassandra_instances: + - id: 1 + host: cassandra + port: 9042 + storage_dir: /var/lib/cassandra + cdc_dir: /var/lib/cassandra/cdc_raw + commitlog_dir: /var/lib/cassandra/commitlog + staging_dir: /var/lib/cassandra/sstable-staging + jmx_host: 127.0.0.1 + jmx_port: 7199 + jmx_ssl_enabled: false + +sidecar: + host: 0.0.0.0 + port: 9043 + cdc: + enabled: true + segment_hardlink_cache_expiry: 5m + table_schema_refresh_time: 5s + config_refresh_time: 5s + worker_pools: + service: + name: "sidecar-worker-pool" + size: 20 + max_execution_time: 1m + internal: + name: "sidecar-internal-worker-pool" + size: 20 + max_execution_time: 15m + jmx: + max_retries: 10 + retry_delay: 3s + # Required: sidecar creates sidecar_internal keyspace and configs/cdc_states tables + schema: + is_enabled: true + keyspace: sidecar_internal + replication_strategy: NetworkTopologyStrategy + replication_factor: 1 + lease_schema_ttl: 5m + coordination: + cluster_lease_claim: + electorate_membership_strategy: MostReplicatedKeyspaceTokenZeroElectorateMembership + enabled: true + initial_delay: 1s + initial_delay_random_delta: 5s + execute_interval: 100s + +driver_parameters: + contact_points: + - "cassandra:9042" + num_connections: 6 + local_dc: datacenter1 + +healthcheck: + initial_delay: 0ms + execute_interval: 30s + +cluster_topology_monitor: + enabled: true + initial_delay: 0s + execute_interval: 1000ms diff --git a/docker/cdc-demo/docker-compose.yml b/docker/cdc-demo/docker-compose.yml new file mode 100644 index 000000000..514edb4be --- /dev/null +++ b/docker/cdc-demo/docker-compose.yml @@ -0,0 +1,155 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +# +# Cassandra CDC demo stack +# +# Startup order: +# kafka +# cassandra ──► cassandra-init (seeds schema + configs, then exits) +# └──► sidecar +# kafka-ui +# +# Wipe all data: docker compose down -v + +services: + + # ── Kafka (KRaft — no ZooKeeper) ──────────────────────────────────────────── + kafka: + image: confluentinc/cp-kafka:7.6.0 + networks: + - demo-network + environment: + KAFKA_NODE_ID: 1 + KAFKA_PROCESS_ROLES: broker,controller + KAFKA_LISTENERS: PLAINTEXT://0.0.0.0:9092,CONTROLLER://0.0.0.0:9093 + KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://kafka:9092 + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT + KAFKA_INTER_BROKER_LISTENER_NAME: PLAINTEXT + KAFKA_CONTROLLER_LISTENER_NAMES: CONTROLLER + KAFKA_CONTROLLER_QUORUM_VOTERS: 1@kafka:9093 + KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 + KAFKA_AUTO_CREATE_TOPICS_ENABLE: "true" + # Must not change across restarts — required for volume persistence. + CLUSTER_ID: MkU3OEVBNTcwNTJENDM2Qk + volumes: + - kafka-data:/var/lib/kafka/data + healthcheck: + test: ["CMD", "kafka-topics", "--bootstrap-server", "kafka:9092", "--list"] + interval: 10s + timeout: 10s + retries: 20 + + # ── Cassandra ─────────────────────────────────────────────────────────────── + cassandra: + image: cassandra:${CASSANDRA_VERSION:-5.0} + # Patches CDC settings into cassandra.yaml, then delegates to the stock entrypoint. + entrypoint: ["/bin/bash", "/scripts/cassandra-entrypoint.sh"] + command: ["cassandra", "-f"] + networks: + - demo-network + environment: + CASSANDRA_CLUSTER_NAME: "CDC Demo Cluster" + CASSANDRA_DC: datacenter1 + CASSANDRA_RACK: rack1 + CASSANDRA_ENDPOINT_SNITCH: GossipingPropertyFileSnitch + # Required: lets SidecarLoadBalancingPolicy match this node by hostname. + CASSANDRA_BROADCAST_RPC_ADDRESS: cassandra + # Enables remote JMX so the sidecar can connect. + LOCAL_JMX: "no" + JVM_EXTRA_OPTS: "-Djava.rmi.server.hostname=cassandra -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false" + MAX_HEAP_SIZE: "512M" + HEAP_NEWSIZE: "128M" + volumes: + - ./scripts:/scripts:ro + # commitlog/ and cdc_raw/ must share a filesystem for CDC hard-links. + - cassandra-varlib:/var/lib/cassandra + ports: + - "9042:9042" + # 9043 is published here because the sidecar shares this container's network namespace. + - "9043:9043" + healthcheck: + test: ["CMD-SHELL", "cqlsh -e 'SELECT now() FROM system.local' 2>/dev/null || exit 1"] + interval: 10s + timeout: 10s + retries: 20 + start_period: 60s + + # ── Cassandra init (one-shot) ─────────────────────────────────────────────── + # Seeds sidecar_internal schema and CDC/Kafka configs after Cassandra is healthy. + cassandra-init: + image: cassandra:${CASSANDRA_VERSION:-5.0} + depends_on: + cassandra: + condition: service_healthy + networks: + - demo-network + entrypoint: ["/bin/bash", "-c", + "bash /scripts/init-cdc-schema.sh && bash /scripts/seed-cdc-configs.sh"] + environment: + CASSANDRA_HOST: cassandra + KAFKA_BOOTSTRAP_SERVERS: kafka:9092 + CDC_TOPIC: cdc-mutations + CDC_DATACENTER: datacenter1 + volumes: + - ./scripts:/scripts:ro + restart: "no" + + # ── Kafka UI ───────────────────────────────────────────────────────────────── + # Browse topics and messages at http://localhost:8080 + kafka-ui: + image: provectuslabs/kafka-ui:latest + depends_on: + kafka: + condition: service_healthy + networks: + - demo-network + environment: + KAFKA_CLUSTERS_0_NAME: local + KAFKA_CLUSTERS_0_BOOTSTRAPSERVERS: kafka:9092 + ports: + - "8080:8080" + + # ── Sidecar ───────────────────────────────────────────────────────────────── + # Shares Cassandra's network namespace — sidecar is reachable at cassandra:9043. + sidecar: + build: + context: ../../ + dockerfile: docker/cdc-demo/Dockerfile.sidecar + depends_on: + cassandra-init: + condition: service_completed_successfully + network_mode: "service:cassandra" + volumes: + # Overrides the baked-in config; edit without rebuilding the image. + - ./conf/sidecar.yaml:/app/conf/sidecar.yaml:ro + # Shared with Cassandra so the sidecar can read cdc_raw/ and commitlog/. + - cassandra-varlib:/var/lib/cassandra + healthcheck: + test: ["CMD-SHELL", "curl -sf http://localhost:9043/api/v1/__health || exit 1"] + interval: 10s + timeout: 5s + retries: 20 + start_period: 30s + +volumes: + kafka-data: + # commitlog/ and cdc_raw/ must share a filesystem for CDC hard-links. + cassandra-varlib: + +networks: + demo-network: + driver: bridge diff --git a/docker/cdc-demo/scripts/cassandra-entrypoint.sh b/docker/cdc-demo/scripts/cassandra-entrypoint.sh new file mode 100644 index 000000000..1a6720b00 --- /dev/null +++ b/docker/cdc-demo/scripts/cassandra-entrypoint.sh @@ -0,0 +1,49 @@ +#!/bin/bash +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +# +# Patches CDC settings into the stock cassandra.yaml, +# then hands off to the original Docker entrypoint. +set -e + +YAML="/etc/cassandra/cassandra.yaml" + +patch_yaml() { + local key="$1" value="$2" + if grep -q "^${key}:" "$YAML"; then + sed -i "s|^${key}:.*|${key}: ${value}|" "$YAML" + elif grep -q "^#.*${key}:" "$YAML"; then + sed -i "s|^#.*${key}:.*|${key}: ${value}|" "$YAML" + else + echo "${key}: ${value}" >> "$YAML" + fi +} + +# commitlog and cdc_raw must share the same filesystem for CDC hard-links. +patch_yaml "commitlog_directory" "/var/lib/cassandra/commitlog" +patch_yaml "cdc_enabled" "true" +patch_yaml "cdc_raw_directory" "/var/lib/cassandra/cdc_raw" +patch_yaml "cdc_on_repair_enabled" "false" + +# Cassandra 4.x uses cdc_total_space_in_mb; 5.x uses cdc_total_space. +if grep -q "cdc_total_space_in_mb" "$YAML"; then + patch_yaml "cdc_total_space_in_mb" "4096" +else + patch_yaml "cdc_total_space" "4096MiB" +fi + +exec /usr/local/bin/docker-entrypoint.sh "$@" diff --git a/docker/cdc-demo/scripts/init-cdc-schema.sh b/docker/cdc-demo/scripts/init-cdc-schema.sh new file mode 100644 index 000000000..2100d5cc9 --- /dev/null +++ b/docker/cdc-demo/scripts/init-cdc-schema.sh @@ -0,0 +1,44 @@ +#!/usr/bin/env bash +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +# +# Creates the sidecar_internal schema and CDC demo keyspace/table. +set -euo pipefail + +CASSANDRA_HOST=${CASSANDRA_HOST:-cassandra} + +cqlsh "${CASSANDRA_HOST}" <<'CQL' +CREATE KEYSPACE IF NOT EXISTS sidecar_internal + WITH replication = {'class': 'NetworkTopologyStrategy', 'datacenter1': 1}; + +CREATE TABLE IF NOT EXISTS sidecar_internal.configs ( + service text, + config map, + PRIMARY KEY (service) +); + +CREATE KEYSPACE IF NOT EXISTS cdc_demo + WITH replication = {'class': 'NetworkTopologyStrategy', 'datacenter1': 1}; + +CREATE TABLE IF NOT EXISTS cdc_demo.events ( + id uuid PRIMARY KEY, + msg text, + ts timestamp +) WITH cdc = true; +CQL + +echo "Schema initialised." diff --git a/docker/cdc-demo/scripts/seed-cdc-configs.sh b/docker/cdc-demo/scripts/seed-cdc-configs.sh new file mode 100644 index 000000000..733ec31e8 --- /dev/null +++ b/docker/cdc-demo/scripts/seed-cdc-configs.sh @@ -0,0 +1,55 @@ +#!/usr/bin/env bash +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +# +# Seeds CDC and Kafka configuration into sidecar_internal.configs. +# IF NOT EXISTS makes each insert idempotent across restarts. +set -euo pipefail + +CASSANDRA_HOST=${CASSANDRA_HOST:-cassandra} +KAFKA_BOOTSTRAP=${KAFKA_BOOTSTRAP_SERVERS:-kafka:9092} +CDC_TOPIC=${CDC_TOPIC:-cdc-mutations} +CDC_DATACENTER=${CDC_DATACENTER:-datacenter1} + +cqlsh "${CASSANDRA_HOST}" <&2; exit 1 ;; + esac +done + +# Run all docker compose commands from the demo directory so no -f flag is needed. +cd "$DEMO_DIR" + +echo "==> Stopping stack..." +if $CLEAN; then + docker compose down -v --remove-orphans +else + docker compose down --remove-orphans +fi + +echo "==> Building sidecar image..." +DOCKER_BUILDKIT=1 docker build \ + -f "$REPO_ROOT/docker/cdc-demo/Dockerfile.sidecar" \ + -t cassandra-sidecar:dev \ + "$REPO_ROOT" + +echo "==> Starting stack..." +docker compose up -d + +echo "" +echo "Waiting for sidecar to be ready (to follow progress: docker compose logs -f cassandra-init sidecar)..." +until curl -sf http://localhost:9043/api/v1/__health > /dev/null 2>&1; do + sleep 5 +done + +echo "Sidecar is up. Waiting for CDC iterators to start..." +docker compose logs -f sidecar 2>&1 | grep -m 1 "CDC iterators started successfully" > /dev/null || true + +echo "" +printf "${GREEN}${BOLD}╔══════════════════════════════════════════════╗${RESET}\n" +printf "${GREEN}${BOLD}║ Setup complete. CDC pipeline is running. ║${RESET}\n" +printf "${GREEN}${BOLD}╚══════════════════════════════════════════════╝${RESET}\n" +echo "" +printf " ${BOLD}Step 1 — Insert a test mutation:${RESET}\n" +printf " ${CYAN}\$ docker exec -it cdc-demo-cassandra-1 cqlsh -e \"INSERT INTO cdc_demo.events (id, msg, ts) VALUES (uuid(), 'hello', toTimestamp(now()));\"${RESET}\n" +echo "" +printf " ${BOLD}Step 2 — View mutations arriving in Kafka UI:${RESET}\n" +printf " ${UNDERLINE}http://localhost:8080/ui/clusters/local/all-topics/cdc-mutations/messages${RESET}\n" +echo "" diff --git a/scripts/build-shaded-dtest-jar-local.sh b/scripts/build-shaded-dtest-jar-local.sh index ac7fe7749..e2b7a2f03 100755 --- a/scripts/build-shaded-dtest-jar-local.sh +++ b/scripts/build-shaded-dtest-jar-local.sh @@ -35,6 +35,9 @@ echo "${JAVA_HOME}" # The container that runs the script has jdk11 installed only. # Setting the env var to build with jdk11. export CASSANDRA_USE_JDK11=true +# Trunk (5.1-SNAPSHOT) compiles 2700+ source files; without an explicit heap +# limit ant hits the JVM default (~2GB) and gets OOM-killed on large executors. +export ANT_OPTS="${ANT_OPTS:-} -Xmx4g" ant realclean ant dtest-jar -Dno-checkstyle=true