Aggregating Mobile CI/CD and Kong Gateway Telemetry with Vector for Canary Release Analysis


The mobile canary release process was a black box. We would deploy a new build to a 5% user cohort, and then the frantic, manual correlation would begin. Was the spike in 5xx errors on the Kong gateway related to the new feature-payment-refactor build? Was the surge in crash reports from our iOS client a result of a backend API change or a client-side bug? Answering these questions involved stitching together data from Kibana for Kong logs, Firebase Crashlytics for client crashes, and Jenkins build history. It was a reactive, time-consuming process that often led to inconclusive results, forcing premature rollbacks or, worse, letting subtle bugs slip into production.

The core problem was data silos. Each system produced valuable telemetry, but in its own format, in its own location, and on its own timeline. Our initial concept was to break down these silos by creating a unified, structured, and queryable stream of events that represented the entire lifecycle of a mobile release. We needed to see CI build events, API gateway traffic patterns, and client-side error reports as a single, correlated dataset.

Our technology selection process was driven by pragmatism. We were already using Kong as our API gateway, so leveraging its logging capabilities was a natural starting point. For the data aggregation layer, we evaluated Fluentd, Logstash, and Vector. Vector’s performance, memory efficiency (being written in Rust), and especially its powerful Vector Remodeling Language (VRL) for on-the-fly data transformation, made it the clear winner. VRL would allow us to parse, structure, and enrich disparate log formats into a single, cohesive schema before they ever reached their destination. We chose ClickHouse as our sink for its impressive analytical query performance on large volumes of time-series data, though the principles here apply equally to Elasticsearch or other log analytics platforms.

The goal was to build a feedback loop where the impact of a canary deployment could be measured in near real-time, enabling data-driven decisions on whether to proceed with a full rollout or initiate an automated rollback.

Phase 1: Instrumenting the Kong Gateway

The first step was to make Kong’s logs more intelligent. By default, its access logs are informative but lack context about our mobile builds. We needed to inject the mobile build’s version or commit SHA into the request lifecycle. The CI/CD pipeline, which controls the canary deployment, is the source of this truth.

Our strategy was to have the CI/CD job, when deploying a canary, use the Kong Admin API to add a specific header to requests routed to the canary instance. We used the request-transformer plugin for this.

Here is the declarative configuration for a canary service in Kong. Assume our main service is mobile-backend and the canary is mobile-backend-canary. The route for the canary only matches if a specific X-Canary-Test header is present, allowing our test devices to force the canary path. For the public canary, we rely on weighted load balancing.

# kong.yml - Declarative configuration snippet for Kong
_format_version: "3.0"
services:
- name: mobile-backend-stable
  url: http://stable-service:8080
  routes:
  - name: mobile-backend-stable-route
    paths:
    - /v2/user
    - /v2/profile
- name: mobile-backend-canary
  url: http://canary-service:8081
  plugins:
  # This plugin adds a header identifying traffic that went through the canary service.
  # The value of `X-Build-Version` is set dynamically by the CI/CD pipeline during deployment.
  - name: request-transformer
    config:
      add:
        headers:
        - "X-Build-Version: b4d9f1a" # This value is a placeholder, updated by CI
  routes:
  - name: mobile-backend-canary-route
    paths:
    - /v2/user
    - /v2/profile

upstreams:
- name: mobile-backend-upstream
  targets:
  - target: mobile-backend-stable:8080
    weight: 95
  - target: mobile-backend-canary:8081
    weight: 5

With this in place, every request served by the canary deployment is tagged with a build version. The next step is to get this information into the logs. We configured Kong to output logs in JSON format and ship them over TCP to a Vector agent.

This is done by modifying Kong’s configuration (kong.conf):

# /etc/kong/kong.conf snippet
log_level = info
proxy_access_log = /dev/stdout
# We use a custom nginx template to format logs as JSON.
proxy_access_log = tcp://vector-agent:5166 json
# admin_access_log and admin_error_log can also be forwarded
admin_access_log = tcp://vector-agent:5167 json

The custom json log format needs to be defined within the nginx configuration template that Kong uses. This gives us complete control over the log structure.

# Custom nginx template snippet for kong.conf (nginx_http_log_format)
log_format json escape=json '{'
  '"client_ip": "$remote_addr",'
  '"timestamp": "$time_iso8601",'
  '"request_method": "$request_method",'
  '"request_uri": "$request_uri",'
  '"status": $status,'
  '"latency_ms": $upstream_response_time,'
  '"request_id": "$request_id",'
  '"service_name": "$service_name",'
  '"route_name": "$route_name",'
  '"build_version": "$http_x_build_version",' # <-- The critical field
  '"user_agent": "$http_user_agent"'
'}';

Now, Kong is emitting structured JSON logs over TCP, with each log entry from a canary request clearly marked with the exact build version that served it. This is the foundational link for our correlation efforts.

Phase 2: Building the Vector Aggregation Pipeline

Vector is the heart of this architecture. It will receive data from three distinct sources: Kong’s TCP logs, our CI/CD system’s build events via HTTP, and our mobile client’s crash reports via another HTTP endpoint. It will then parse, enrich, and route this data to ClickHouse.

The vector.toml configuration is substantial, as it defines the entire data transformation logic.

# vector.toml - Central configuration for the observability agent

# ------------------------------------------------------------------------------
# SOURCES - Where data comes from
# ------------------------------------------------------------------------------

# [1] Kong TCP Log Ingest
[sources.kong_tcp_logs]
  type = "socket"
  address = "0.0.0.0:5166"
  mode = "tcp"
  max_length = 102400
  decoding.codec = "json" # Kong is already sending structured JSON

# [2] CI/CD Build Event Ingest (e.g., from GitLab/Jenkins)
[sources.ci_build_events]
  type = "http_server"
  address = "0.0.0.0:8686"
  decoding.codec = "json"
  # In a production setup, add authentication
  # auth.strategy = "basic"
  # auth.user = "ci-user"
  # auth.password = "${CI_VECTOR_TOKEN}"

# [3] Mobile Client Crash/Log Ingest
[sources.mobile_client_logs]
  type = "http_server"
  address = "0.0.0.0:8787"
  decoding.codec = "json"

# ------------------------------------------------------------------------------
# TRANSFORMS - The core logic: parsing, enriching, structuring
# ------------------------------------------------------------------------------

# [Transform 1] Enrich Kong logs
[transforms.enrich_kong_logs]
  type = "remap"
  inputs = ["kong_tcp_logs"]
  source = '''
  # Add a static event type for easier filtering in the sink
  .event_type = "api_access"

  # Parse the user agent string to extract client details.
  # This provides critical context for mobile debugging.
  ua = parse_user_agent(.user_agent)
  if ua != null {
    .client.os.name = ua.os.name
    .client.os.version = ua.os.version
    .client.device.name = ua.device.name
  }

  # Convert latency to a proper float type
  .latency_ms = to_float(.latency_ms) ?? 0.0

  # Normalize the build version. If it's missing (stable traffic), mark it as 'stable'.
  if .build_version == null || .build_version == "" {
    .build_version = "stable"
  }
  '''

# [Transform 2] Process CI Build Events
[transforms.process_ci_events]
  type = "remap"
  inputs = ["ci_build_events"]
  source = '''
  # Add a static event type
  .event_type = "ci_build"

  # Ensure required fields exist and have correct types
  .build_id = to_string(.build_id)
  .commit_sha = to_string(.commit_sha)
  .status = to_string(.status)
  .duration_seconds = to_int(.duration_seconds) ?? 0
  .is_canary = to_boolean(.is_canary) ?? false

  # Basic validation: drop event if essential fields are missing
  if !exists(.commit_sha) || !exists(.status) {
      abort
  }
  '''

# [Transform 3] Process Mobile Client Logs
[transforms.process_mobile_logs]
  type = "remap"
  inputs = ["mobile_client_logs"]
  source = '''
  # The event_type should be sent by the client, e.g., "crash_report" or "client_log"
  if !exists(.event_type) {
    .event_type = "unknown_mobile_event"
  }

  # The mobile client MUST include its build version in every payload
  if !exists(.build_version) {
    .build_version = "unknown"
  }
  
  # Extract and structure nested payload for crashes
  if .event_type == "crash_report" && is_object(.payload) {
    .error.message = .payload.error_message
    .error.stacktrace = .payload.stacktrace
    del(.payload)
  }
  '''

# ------------------------------------------------------------------------------
# SINKS - Where processed data goes
# ------------------------------------------------------------------------------

[sinks.clickhouse_unified]
  type = "clickhouse"
  inputs = ["enrich_kong_logs", "process_ci_events", "process_mobile_logs"]
  endpoint = "http://clickhouse-server:8123"
  database = "observability"
  table = "mobile_events"
  # Buffer settings are crucial for production performance
  batch.max_events = 5000
  batch.timeout_secs = 5
  
  # Define the schema for the target ClickHouse table
  # This ensures data types are correctly handled
  encoding.except_fields = ["timestamp"] # Let ClickHouse handle the timestamp
  schema = {
    "timestamp" = "DateTime64(3, 'UTC')",
    "event_type" = "LowCardinality(String)",
    "build_version" = "LowCardinality(String)",
    # Kong fields
    "client_ip" = "IPv4",
    "request_method" = "LowCardinality(String)",
    "request_uri" = "String",
    "status" = "UInt16",
    "latency_ms" = "Float64",
    "service_name" = "LowCardinality(String)",
    "route_name" = "LowCardinality(String)",
    # Mobile/UA fields
    "client.os.name" = "LowCardinality(String)",
    "client.os.version" = "String",
    "client.device.name" = "String",
    # CI fields
    "commit_sha" = "String",
    "build_id" = "String",
    "status" = "LowCardinality(String)",
    "duration_seconds" = "UInt32",
    "is_canary" = "UInt8",
    # Mobile Error fields
    "error.message" = "String",
    "error.stacktrace" = "String",
  }

The ClickHouse table schema to support this would be:

CREATE TABLE observability.mobile_events
(
    `timestamp` DateTime64(3, 'UTC'),
    `event_type` LowCardinality(String),
    `build_version` LowCardinality(String),

    -- Kong fields
    `client_ip` IPv4,
    `request_method` LowCardinality(String),
    `request_uri` String,
    `status` UInt16,
    `latency_ms` Float64,
    `service_name` LowCardinality(String),
    `route_name` LowCardinality(String),

    -- Mobile/UA fields
    `client.os.name` LowCardinality(String),
    `client.os.version` String,
    `client.device.name` String,

    -- CI fields
    `commit_sha` String,
    `build_id` String,
    `status` LowCardinality(String),
    `duration_seconds` UInt32,
    `is_canary` UInt8,

    -- Mobile Error fields
    `error.message` String,
    `error.stacktrace` String
)
ENGINE = MergeTree()
PARTITION BY toYYYYMM(timestamp)
ORDER BY (event_type, build_version, timestamp);

This Vector configuration creates a robust pipeline. VRL is used not just for parsing but for enforcement of a unified schema, which is critical for effective querying later.

Phase 3: Integrating the CI/CD Pipeline

The CI/CD system (e.g., GitLab CI) becomes an active participant in the observability process. It has two jobs:

  1. Tell the observability system about key build events.
  2. Update Kong to tag traffic for the new canary build.

Here’s a snippet from a .gitlab-ci.yml file illustrating these steps.

stages:
  - build
  - deploy_canary
  - monitor_canary
  - promote_to_stable

variables:
  # The Vector endpoint for CI events
  VECTOR_CI_ENDPOINT: "http://vector-agent:8686"
  # Kong Admin API endpoint
  KONG_ADMIN_URL: "http://kong-admin:8001"

build_android_app:
  stage: build
  script:
    - ./gradlew assembleRelease
    # ... archive artifacts ...
  after_script:
    # Report build result to Vector
    - |
      STATUS="success"
      if [ "$CI_JOB_STATUS" != "success" ]; then
        STATUS="failed"
      fi
      PAYLOAD=$(cat <<EOF
      {
        "build_id": "${CI_JOB_ID}",
        "project": "${CI_PROJECT_NAME}",
        "branch": "${CI_COMMIT_BRANCH}",
        "commit_sha": "${CI_COMMIT_SHA}",
        "status": "${STATUS}",
        "duration_seconds": ${CI_JOB_DURATION},
        "is_canary": true
      }
      EOF
      )
      curl -X POST -H "Content-Type: application/json" --data "${PAYLOAD}" ${VECTOR_CI_ENDPOINT}

deploy_canary_backend:
  stage: deploy_canary
  script:
    # 1. Deploy the new container/service for `mobile-backend-canary`
    - echo "Deploying canary service with commit SHA ${CI_COMMIT_SHA}"
    # ... kubectl apply, docker-compose up, etc. ...
    
    # 2. Update Kong to tag the canary traffic with the new build version
    # The plugin ID for the request-transformer on the canary service must be known.
    # A real implementation would fetch this ID first. Assume it's 'plugin-1234'.
    - |
      curl -X PATCH ${KONG_ADMIN_URL}/services/mobile-backend-canary/plugins/plugin-1234 \
        --data "config.add.headers=X-Build-Version:${CI_COMMIT_SHA}"
    
    # 3. Adjust the traffic weight (e.g., to 5%)
    - |
      curl -X PUT ${KONG_ADMIN_URL}/upstreams/mobile-backend-upstream/targets \
        -d 'target=mobile-backend-stable:8080' -d 'weight=95'
      curl -X POST ${KONG_ADMIN_URL}/upstreams/mobile-backend-upstream/targets \
        -d 'target=mobile-backend-canary:8081' -d 'weight=5'

This closes the loop. The same CI_COMMIT_SHA that is reported to Vector as a build event is also injected into the live traffic logs via Kong, creating the primary key for correlation.

Phase 4: Mobile Client Instrumentation

The final piece is getting data from the client itself. This doesn’t require complex code. The key is that the mobile app must be aware of its own build version (or commit SHA) and include it in every log or crash report payload sent to Vector’s HTTP endpoint.

A conceptual logger on the mobile client (e.g., in Swift):

// Conceptual Swift code for a mobile logger
class ObservabilityLogger {
    static let shared = ObservabilityLogger()
    
    private let vectorEndpoint = URL(string: "http://vector.my-company.com:8787")!
    // This value is baked into the app at compile time by the CI/CD system
    private let buildVersion = "b4d9f1a" // e.g., git SHA

    func logCrash(error: Error, stacktrace: [String]) {
        let payload: [String: Any] = [
            "event_type": "crash_report",
            "build_version": self.buildVersion,
            "client_os": "iOS",
            "os_version": UIDevice.current.systemVersion,
            "payload": [
                "error_message": error.localizedDescription,
                "stacktrace": stacktrace.joined(separator: "\n")
            ]
        ]
        send(payload)
    }

    private func send(_ payload: [String: Any]) {
        guard let data = try? JSONSerialization.data(withJSONObject: payload) else { return }
        var request = URLRequest(url: vectorEndpoint)
        request.httpMethod = "POST"
        request.setValue("application/json", forHTTPHeaderField: "Content-Type")
        request.httpBody = data
        
        URLSession.shared.dataTask(with: request).resume()
    }
}

With this, crash reports are no longer isolated events in a separate system. They become part of the same queryable mobile_events table, directly linkable to a build_version.

The Result: A Unified View

The architecture provides a single plane of glass for mobile release health. The true power is unlocked by the queries we can now run in ClickHouse.

graph TD
    subgraph "CI/CD Pipeline (GitLab)"
        A[Build Job] -->|Sends Build Event| B(Vector HTTP Source);
        A --> C{Deploy Canary};
        C -->|Updates Route via Admin API| D[Kong Gateway];
    end

    subgraph "Live Traffic"
        E[Mobile Client] -->|API Request| D;
        D -->|Serves Request| F[Canary/Stable Backend];
        D -->|Forwards JSON Log over TCP| G(Vector Socket Source);
    end
    
    subgraph "Client-Side Events"
        E -->|Crash/Log Event| H(Vector HTTP Source);
    end

    subgraph "Vector Agent"
        B & G & H --> I{Vector Transforms - VRL};
        I -->|Structured Data| J[Vector ClickHouse Sink];
    end

    subgraph "Analytics & Storage"
        J --> K[(ClickHouse Database)];
        L[Analyst/SRE] -->|SQL Queries| K;
    end

Query 1: Compare canary vs. stable API performance.

SELECT
    build_version,
    avg(latency_ms) AS avg_latency,
    quantile(0.95)(latency_ms) AS p95_latency,
    countIf(status >= 500) / count() * 100 AS error_rate_percent
FROM observability.mobile_events
WHERE
    timestamp >= now() - interval '1 hour'
    AND event_type = 'api_access'
    AND request_uri = '/v2/profile'
GROUP BY build_version;

This single query immediately tells us if canary build b4d9f1a has higher latency or error rates than the stable version.

Query 2: Correlate API errors with client crashes.

SELECT
    crash.build_version,
    crash.client_os_name,
    crash.error_message
FROM observability.mobile_events AS crash
WHERE
    crash.event_type = 'crash_report'
    AND crash.build_version = 'b4d9f1a' -- Our specific canary build
    AND crash.client_ip IN (
        -- Find IPs that experienced 5xx errors from the same build
        SELECT DISTINCT client_ip
        FROM observability.mobile_events
        WHERE build_version = 'b4d9f1a' AND status >= 500
    );

This is the holy grail. We are directly linking backend failures with specific client-side crashes, providing developers with an immediate, actionable starting point for debugging.

Lingering Issues and Future Iterations

This system is a massive improvement, but it is not perfect. The correlation is based on shared metadata (build_version, client_ip) and time, not on a true distributed trace context. A network hiccup on the client side could delay the delivery of a crash report, making time-based correlation less reliable.

The next logical evolution is to embrace OpenTelemetry fully. This would involve instrumenting the mobile client to initiate traces, propagating the trace context through Kong (perhaps with a custom plugin or OpenTelemetry plugin), and into our backend services. Vector has native support for OTLP (OpenTelemetry Protocol), so it could receive traces alongside logs and metrics, providing an even richer context. Another avenue is automation: building a monitoring job that runs these analytical queries every minute after a canary deployment. If the error rate for the new build_version exceeds a predefined SLO, the job could trigger a webhook to the CI/CD pipeline to initiate an automatic rollback by calling the Kong Admin API, creating a fully automated, closed-loop release system.


  TOC