From e28dbab71e58de49835f0f5ce4d7a45c8c918bcd Mon Sep 17 00:00:00 2001 From: Dimitar Dimitrov Date: Thu, 30 May 2024 10:46:26 +0200 Subject: [PATCH] WIP: improve kafka replay speed Signed-off-by: Dimitar Dimitrov --- cmd/mimir/config-descriptor.json | 50 + cmd/mimir/help-all.txt.tmpl | 10 + cmd/mimir/help.txt.tmpl | 10 + .../config/datasource-mimir.yaml | 7 + .../config/grafana-agent.flow | 9 +- .../mimir-ingest-storage/config/mimir.yaml | 11 +- .../mimir-ingest-storage/config/runtime.yaml | 98 +- .../docker-compose.jsonnet | 166 ++- .../mimir-ingest-storage/docker-compose.yml | 284 ++-- .../configuration-parameters/index.md | 24 + .../configure/configure-kafka-backend.md | 35 + go.mod | 4 + go.sum | 6 +- pkg/ingester/ingester_test.go | 4 + pkg/storage/ingest/config.go | 12 +- pkg/storage/ingest/partition_offset_reader.go | 22 +- .../ingest/partition_offset_reader_test.go | 169 ++- pkg/storage/ingest/pusher.go | 519 +++++-- pkg/storage/ingest/pusher_test.go | 510 ++++++- pkg/storage/ingest/reader.go | 734 +++++++++- pkg/storage/ingest/reader_client.go | 7 +- pkg/storage/ingest/reader_test.go | 314 ++++- pkg/storage/ingest/util.go | 9 +- pkg/storage/ingest/writer_test.go | 2 + tools/kafka-dump/main.go | 219 +++ vendor/github.com/go-jose/go-jose/v3/LICENSE | 202 +++ .../go-jose/go-jose/v3/json/LICENSE | 27 + .../go-jose/go-jose/v3/json/README.md | 13 + .../go-jose/go-jose/v3/json/decode.go | 1216 +++++++++++++++++ .../go-jose/go-jose/v3/json/encode.go | 1197 ++++++++++++++++ .../go-jose/go-jose/v3/json/indent.go | 141 ++ .../go-jose/go-jose/v3/json/scanner.go | 623 +++++++++ .../go-jose/go-jose/v3/json/stream.go | 484 +++++++ .../go-jose/go-jose/v3/json/tags.go | 44 + .../twmb/franz-go/pkg/kgo/client.go | 6 +- .../twmb/franz-go/pkg/kgo/compression.go | 2 + .../twmb/franz-go/pkg/kgo/source.go | 136 +- vendor/modules.txt | 6 +- 38 files changed, 6921 insertions(+), 411 deletions(-) create mode 100644 docs/sources/mimir/configure/configure-kafka-backend.md create mode 100644 tools/kafka-dump/main.go create mode 100644 vendor/github.com/go-jose/go-jose/v3/LICENSE create mode 100644 vendor/github.com/go-jose/go-jose/v3/json/LICENSE create mode 100644 vendor/github.com/go-jose/go-jose/v3/json/README.md create mode 100644 vendor/github.com/go-jose/go-jose/v3/json/decode.go create mode 100644 vendor/github.com/go-jose/go-jose/v3/json/encode.go create mode 100644 vendor/github.com/go-jose/go-jose/v3/json/indent.go create mode 100644 vendor/github.com/go-jose/go-jose/v3/json/scanner.go create mode 100644 vendor/github.com/go-jose/go-jose/v3/json/stream.go create mode 100644 vendor/github.com/go-jose/go-jose/v3/json/tags.go diff --git a/cmd/mimir/config-descriptor.json b/cmd/mimir/config-descriptor.json index 9765237a1ba..d03d56b6c30 100644 --- a/cmd/mimir/config-descriptor.json +++ b/cmd/mimir/config-descriptor.json @@ -6647,6 +6647,56 @@ "fieldDefaultValue": 20000000000, "fieldFlag": "ingest-storage.kafka.wait-strong-read-consistency-timeout", "fieldType": "duration" + }, + { + "kind": "field", + "name": "replay_concurrency", + "required": false, + "desc": "The number of concurrent fetch requests that the ingester sends to kafka when catching up during startup.", + "fieldValue": null, + "fieldDefaultValue": 1, + "fieldFlag": "ingest-storage.kafka.replay-concurrency", + "fieldType": "int" + }, + { + "kind": "field", + "name": "replay_shards", + "required": false, + "desc": "The number of concurrent appends to the TSDB head. 0 to disable.", + "fieldValue": null, + "fieldDefaultValue": 0, + "fieldFlag": "ingest-storage.kafka.replay-shards", + "fieldType": "int" + }, + { + "kind": "field", + "name": "batch_size", + "required": false, + "desc": "The number of timeseries to batch together before ingesting into TSDB.", + "fieldValue": null, + "fieldDefaultValue": 128, + "fieldFlag": "ingest-storage.kafka.batch-size", + "fieldType": "int" + }, + { + "kind": "field", + "name": "records_per_fetch", + "required": false, + "desc": "The number of records to fetch from Kafka in a single request.", + "fieldValue": null, + "fieldDefaultValue": 128, + "fieldFlag": "ingest-storage.kafka.records-per-fetch", + "fieldType": "int" + }, + { + "kind": "field", + "name": "use_compressed_bytes_as_fetch_max_bytes", + "required": false, + "desc": "When enabled, the fetch request MaxBytes field is computed using the compressed size of previous records. When disabled, MaxBytes is computed using uncompressed bytes. Different Kafka implementations interpret MaxBytes differently.", + "fieldValue": null, + "fieldDefaultValue": true, + "fieldFlag": "ingest-storage.kafka.use-compressed-bytes-as-fetch-max-bytes", + "fieldType": "boolean" } ], "fieldValue": null, diff --git a/cmd/mimir/help-all.txt.tmpl b/cmd/mimir/help-all.txt.tmpl index a5755b29a85..9ea0f0e006a 100644 --- a/cmd/mimir/help-all.txt.tmpl +++ b/cmd/mimir/help-all.txt.tmpl @@ -1351,6 +1351,8 @@ Usage of ./cmd/mimir/mimir: When auto-creation of Kafka topic is enabled and this value is positive, Kafka's num.partitions configuration option is set on Kafka brokers with this value when Mimir component that uses Kafka starts. This configuration option specifies the default number of partitions that the Kafka broker uses for auto-created topics. Note that this is a Kafka-cluster wide setting, and applies to any auto-created topic. If the setting of num.partitions fails, Mimir proceeds anyways, but auto-created topics could have an incorrect number of partitions. -ingest-storage.kafka.auto-create-topic-enabled Enable auto-creation of Kafka topic if it doesn't exist. (default true) + -ingest-storage.kafka.batch-size int + The number of timeseries to batch together before ingesting into TSDB. (default 128) -ingest-storage.kafka.client-id string The Kafka client ID. -ingest-storage.kafka.consume-from-position-at-startup string @@ -1373,10 +1375,18 @@ Usage of ./cmd/mimir/mimir: The maximum size of (uncompressed) buffered and unacknowledged produced records sent to Kafka. The produce request fails once this limit is reached. This limit is per Kafka client. 0 to disable the limit. (default 1073741824) -ingest-storage.kafka.producer-max-record-size-bytes int The maximum size of a Kafka record data that should be generated by the producer. An incoming write request larger than this size is split into multiple Kafka records. We strongly recommend to not change this setting unless for testing purposes. (default 15983616) + -ingest-storage.kafka.records-per-fetch int + The number of records to fetch from Kafka in a single request. (default 128) + -ingest-storage.kafka.replay-concurrency int + The number of concurrent fetch requests that the ingester sends to kafka when catching up during startup. (default 1) + -ingest-storage.kafka.replay-shards int + The number of concurrent appends to the TSDB head. 0 to disable. -ingest-storage.kafka.target-consumer-lag-at-startup duration The best-effort maximum lag a consumer tries to achieve at startup. Set both -ingest-storage.kafka.target-consumer-lag-at-startup and -ingest-storage.kafka.max-consumer-lag-at-startup to 0 to disable waiting for maximum consumer lag being honored at startup. (default 2s) -ingest-storage.kafka.topic string The Kafka topic name. + -ingest-storage.kafka.use-compressed-bytes-as-fetch-max-bytes + When enabled, the fetch request MaxBytes field is computed using the compressed size of previous records. When disabled, MaxBytes is computed using uncompressed bytes. Different Kafka implementations interpret MaxBytes differently. (default true) -ingest-storage.kafka.wait-strong-read-consistency-timeout duration The maximum allowed for a read requests processed by an ingester to wait until strong read consistency is enforced. 0 to disable the timeout. (default 20s) -ingest-storage.kafka.write-clients int diff --git a/cmd/mimir/help.txt.tmpl b/cmd/mimir/help.txt.tmpl index 365c2c4445b..b5de7dd4a40 100644 --- a/cmd/mimir/help.txt.tmpl +++ b/cmd/mimir/help.txt.tmpl @@ -421,6 +421,8 @@ Usage of ./cmd/mimir/mimir: When auto-creation of Kafka topic is enabled and this value is positive, Kafka's num.partitions configuration option is set on Kafka brokers with this value when Mimir component that uses Kafka starts. This configuration option specifies the default number of partitions that the Kafka broker uses for auto-created topics. Note that this is a Kafka-cluster wide setting, and applies to any auto-created topic. If the setting of num.partitions fails, Mimir proceeds anyways, but auto-created topics could have an incorrect number of partitions. -ingest-storage.kafka.auto-create-topic-enabled Enable auto-creation of Kafka topic if it doesn't exist. (default true) + -ingest-storage.kafka.batch-size int + The number of timeseries to batch together before ingesting into TSDB. (default 128) -ingest-storage.kafka.client-id string The Kafka client ID. -ingest-storage.kafka.consume-from-position-at-startup string @@ -443,10 +445,18 @@ Usage of ./cmd/mimir/mimir: The maximum size of (uncompressed) buffered and unacknowledged produced records sent to Kafka. The produce request fails once this limit is reached. This limit is per Kafka client. 0 to disable the limit. (default 1073741824) -ingest-storage.kafka.producer-max-record-size-bytes int The maximum size of a Kafka record data that should be generated by the producer. An incoming write request larger than this size is split into multiple Kafka records. We strongly recommend to not change this setting unless for testing purposes. (default 15983616) + -ingest-storage.kafka.records-per-fetch int + The number of records to fetch from Kafka in a single request. (default 128) + -ingest-storage.kafka.replay-concurrency int + The number of concurrent fetch requests that the ingester sends to kafka when catching up during startup. (default 1) + -ingest-storage.kafka.replay-shards int + The number of concurrent appends to the TSDB head. 0 to disable. -ingest-storage.kafka.target-consumer-lag-at-startup duration The best-effort maximum lag a consumer tries to achieve at startup. Set both -ingest-storage.kafka.target-consumer-lag-at-startup and -ingest-storage.kafka.max-consumer-lag-at-startup to 0 to disable waiting for maximum consumer lag being honored at startup. (default 2s) -ingest-storage.kafka.topic string The Kafka topic name. + -ingest-storage.kafka.use-compressed-bytes-as-fetch-max-bytes + When enabled, the fetch request MaxBytes field is computed using the compressed size of previous records. When disabled, MaxBytes is computed using uncompressed bytes. Different Kafka implementations interpret MaxBytes differently. (default true) -ingest-storage.kafka.wait-strong-read-consistency-timeout duration The maximum allowed for a read requests processed by an ingester to wait until strong read consistency is enforced. 0 to disable the timeout. (default 20s) -ingest-storage.kafka.write-clients int diff --git a/development/mimir-ingest-storage/config/datasource-mimir.yaml b/development/mimir-ingest-storage/config/datasource-mimir.yaml index c957d58d242..3f614d8d923 100644 --- a/development/mimir-ingest-storage/config/datasource-mimir.yaml +++ b/development/mimir-ingest-storage/config/datasource-mimir.yaml @@ -9,3 +9,10 @@ datasources: isDefault: true jsonData: prometheusType: Mimir + timeInterval: 5s +- name: Jaeger + type: jaeger + access: proxy + uid: jaeger + orgID: 1 + url: http://jaeger:16686/ diff --git a/development/mimir-ingest-storage/config/grafana-agent.flow b/development/mimir-ingest-storage/config/grafana-agent.flow index bdc64537308..1538be40c2a 100644 --- a/development/mimir-ingest-storage/config/grafana-agent.flow +++ b/development/mimir-ingest-storage/config/grafana-agent.flow @@ -36,6 +36,13 @@ prometheus.scrape "metrics_local_mimir_read_write_mode_mimir_write" { container = "mimir-write", namespace = "mimir-read-write-mode", }], + [{ + __address__ = "mimir-write-zone-c-61:8080", + cluster = "docker-compose", + container = "mimir-write", + namespace = "mimir-read-write-mode", + job = "mimir-write-zone-c", + }], ) forward_to = [prometheus.remote_write.metrics_local.receiver] job_name = "mimir-read-write-mode/mimir-write" @@ -97,7 +104,7 @@ prometheus.scrape "metrics_local_mimir_read_write_mode_mimir_backend" { prometheus.remote_write "metrics_local" { endpoint { name = "local" - url = "http://mimir-write-zone-a-1:8080/api/v1/push" + url = "http://mimir-write-zone-a-2:8080/api/v1/push" send_native_histograms = true queue_config { } diff --git a/development/mimir-ingest-storage/config/mimir.yaml b/development/mimir-ingest-storage/config/mimir.yaml index 66a3be5b827..a8806dc382f 100644 --- a/development/mimir-ingest-storage/config/mimir.yaml +++ b/development/mimir-ingest-storage/config/mimir.yaml @@ -12,15 +12,17 @@ common: ingest_storage: enabled: true kafka: - address: kafka:9092 + address: kafka_1:9092 topic: mimir-ingest last_produced_offset_poll_interval: 500ms + replay_concurrency: 3 + replay_shards: 8 ingester: - track_ingester_owned_series: true + track_ingester_owned_series: false # suppress log messages in c-61 about empty ring; doesn't affect testing partition_ring: - min_partition_owners_count: 2 + min_partition_owners_count: 1 min_partition_owners_duration: 10s delete_inactive_partition_after: 1m @@ -99,3 +101,6 @@ limits: runtime_config: file: ./config/runtime.yaml + +server: + log_level: debug diff --git a/development/mimir-ingest-storage/config/runtime.yaml b/development/mimir-ingest-storage/config/runtime.yaml index cf14c302761..2003a1f97a8 100644 --- a/development/mimir-ingest-storage/config/runtime.yaml +++ b/development/mimir-ingest-storage/config/runtime.yaml @@ -1 +1,97 @@ -# This file can be used to set overrides or other runtime config. +distributor_limits: + max_inflight_push_requests: 2000 + max_inflight_push_requests_bytes: 2.147483648e+09 +ingester_limits: + max_inflight_push_requests: 30000 + max_inflight_push_requests_bytes: 5.36870912e+08 + max_ingestion_rate: 0 + max_series: 3e+06 + max_tenants: 500 +overrides: + "17065": + compactor_block_upload_enabled: true + compactor_blocks_retention_period: 10000d + compactor_split_and_merge_shards: 4 + compactor_split_groups: 8 + compactor_tenant_shard_size: 0 + ha_cluster_label: __cluster__ + ingestion_burst_size: 2.25e+07 + ingestion_partitions_tenant_shard_size: 174 + ingestion_rate: 2.25e+06 + ingestion_tenant_shard_size: 522 + max_global_exemplars_per_user: 1e+06 + max_global_metadata_per_metric: 10 + max_global_metadata_per_user: 1e+07 + max_global_series_per_user: 5e+07 + max_labels_query_length: 32d + native_histograms_ingestion_enabled: true + query_sharding_total_shards: 16 + ruler_max_rule_groups_per_tenant: 600 + ruler_max_rules_per_rule_group: 150 + ruler_tenant_shard_size: 8 + store_gateway_tenant_shard_size: 24 + "17923": + compactor_block_upload_enabled: true + compactor_blocks_retention_period: 10000d + compactor_split_and_merge_shards: 4 + compactor_split_groups: 8 + compactor_tenant_shard_size: 0 + ha_cluster_label: __cluster__ + ingestion_burst_size: 2.25e+07 + ingestion_partitions_tenant_shard_size: 174 + ingestion_rate: 2.25e+06 + ingestion_tenant_shard_size: 522 + max_global_exemplars_per_user: 1e+06 + max_global_metadata_per_metric: 10 + max_global_metadata_per_user: 1e+07 + max_global_series_per_user: 5e+07 + max_labels_query_length: 32d + native_histograms_ingestion_enabled: true + otel_metric_suffixes_enabled: true + query_sharding_total_shards: 16 + ruler_max_rule_groups_per_tenant: 600 + ruler_max_rules_per_rule_group: 150 + ruler_tenant_shard_size: 8 + store_gateway_tenant_shard_size: 24 + "230448": + ingestion_burst_size: 1e+06 + ingestion_partitions_tenant_shard_size: 0 + ingestion_rate: 100000 + ingestion_tenant_shard_size: 0 + max_global_metadata_per_metric: 10 + max_global_metadata_per_user: 200000 + max_global_series_per_user: 1e+06 + ruler_max_rule_groups_per_tenant: 140 + ruler_max_rules_per_rule_group: 20 + ruler_tenant_shard_size: 2 + store_gateway_tenant_shard_size: 0 + "295486": + ingestion_burst_size: 1e+06 + ingestion_partitions_tenant_shard_size: 9 + ingestion_rate: 100000 + ingestion_tenant_shard_size: 27 + max_global_exemplars_per_user: 1e+06 + max_global_metadata_per_metric: 10 + max_global_metadata_per_user: 200000 + max_global_series_per_user: 1e+06 + ruler_max_rule_groups_per_tenant: 140 + ruler_max_rules_per_rule_group: 20 + ruler_tenant_shard_size: 2 + store_gateway_tenant_shard_size: 6 + load-generator-1: + ingestion_burst_size: 1e+06 + ingestion_partitions_tenant_shard_size: 9 + ingestion_rate: 100000 + ingestion_tenant_shard_size: 27 + max_global_metadata_per_metric: 10 + max_global_metadata_per_user: 200000 + max_global_series_per_user: 1e+06 + ruler_max_rule_groups_per_tenant: 140 + ruler_max_rules_per_rule_group: 20 + ruler_tenant_shard_size: 2 + store_gateway_tenant_shard_size: 6 + ruler_limits: + ruler_max_rule_groups_per_tenant_by_namespace: + asserts: 300 + ruler_max_rules_per_rule_group_by_namespace: + asserts: 50 diff --git a/development/mimir-ingest-storage/docker-compose.jsonnet b/development/mimir-ingest-storage/docker-compose.jsonnet index 28344d323ca..2d9c4c39d03 100644 --- a/development/mimir-ingest-storage/docker-compose.jsonnet +++ b/development/mimir-ingest-storage/docker-compose.jsonnet @@ -9,7 +9,10 @@ std.manifestYamlDoc({ self.grafana + self.grafana_agent + self.memcached + - self.kafka + + self.kafka_1 + + self.kafka_2 + + self.kafka_3 + + self.jaeger + {}, write:: { @@ -35,28 +38,24 @@ std.manifestYamlDoc({ extraArguments: ['-ingester.ring.instance-availability-zone=zone-a'], extraVolumes: ['.data-mimir-write-zone-a-3:/data:delegated'], }), - - // Zone-b. - 'mimir-write-zone-b-1': mimirService({ - name: 'mimir-write-zone-b-1', - target: 'write', - publishedHttpPort: 8011, - extraArguments: ['-ingester.ring.instance-availability-zone=zone-b'], - extraVolumes: ['.data-mimir-write-zone-b-1:/data:delegated'], - }), - 'mimir-write-zone-b-2': mimirService({ - name: 'mimir-write-zone-b-2', - target: 'write', - publishedHttpPort: 8012, - extraArguments: ['-ingester.ring.instance-availability-zone=zone-b'], - extraVolumes: ['.data-mimir-write-zone-b-2:/data:delegated'], - }), - 'mimir-write-zone-b-3': mimirService({ - name: 'mimir-write-zone-b-3', - target: 'write', - publishedHttpPort: 8013, - extraArguments: ['-ingester.ring.instance-availability-zone=zone-b'], - extraVolumes: ['.data-mimir-write-zone-b-3:/data:delegated'], + 'mimir-write-zone-c-61': mimirService({ + name: 'mimir-write-zone-c-61', + target: 'ingester', + debug: false, + publishedHttpPort: 8064, + extraArguments: [ + '-ingester.ring.instance-availability-zone=zone-c', + '-ingester.ring.instance-id=ingester-zone-c-61', + '-ingester.partition-ring.prefix=exclusive-prefix', + '-ingester.ring.prefix=exclusive-prefix', + '-ingest-storage.kafka.consume-from-position-at-startup=start', + '-ingest-storage.kafka.consume-from-timestamp-at-startup=0', + '-ingest-storage.kafka.replay-shards=2', + '-ingest-storage.kafka.batch-size=150', + '-ingest-storage.kafka.replay-concurrency=4', + '-ingest-storage.kafka.records-per-fetch=6000', + ], + extraVolumes: ['.data-mimir-write-zone-c-61:/data:delegated'], }), }, @@ -116,27 +115,31 @@ std.manifestYamlDoc({ }, }, - kafka:: { - kafka: { + local commonKafkaEnvVars = [ + 'CLUSTER_ID=zH1GDqcNTzGMDCXm5VZQdg', // Cluster ID is required in KRaft mode; the value is random UUID. + 'KAFKA_NUM_PARTITIONS=100', // Default number of partitions for auto-created topics. + 'KAFKA_PROCESS_ROLES=broker,controller', + 'KAFKA_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT,PLAINTEXT_HOST:PLAINTEXT', + 'KAFKA_INTER_BROKER_LISTENER_NAME=PLAINTEXT', + 'KAFKA_CONTROLLER_LISTENER_NAMES=CONTROLLER', + 'KAFKA_CONTROLLER_QUORUM_VOTERS=1@kafka_1:9093,2@kafka_2:9093,3@kafka_3:9093', + 'KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR=2', + 'KAFKA_DEFAULT_REPLICATION_FACTOR=2', + 'KAFKA_LOG_RETENTION_CHECK_INTERVAL_MS=10000', + + // Decomment the following config to keep a short retention of records in Kafka. + // This is useful to test the behaviour when Kafka records are deleted. + // 'KAFKA_LOG_RETENTION_MINUTES=1', + // 'KAFKA_LOG_SEGMENT_BYTES=1000000', + ], + + kafka_1:: { + kafka_1: { image: 'confluentinc/cp-kafka:latest', - environment: [ - 'CLUSTER_ID=zH1GDqcNTzGMDCXm5VZQdg', // Cluster ID is required in KRaft mode; the value is random UUID. + environment: commonKafkaEnvVars + [ 'KAFKA_BROKER_ID=1', - 'KAFKA_NUM_PARTITIONS=100', // Default number of partitions for auto-created topics. - 'KAFKA_PROCESS_ROLES=broker,controller', 'KAFKA_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093,PLAINTEXT_HOST://:29092', - 'KAFKA_ADVERTISED_LISTENERS=PLAINTEXT://kafka:9092,PLAINTEXT_HOST://localhost:29092', - 'KAFKA_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT,PLAINTEXT_HOST: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_LOG_RETENTION_CHECK_INTERVAL_MS=10000', - - // Decomment the following config to keep a short retention of records in Kafka. - // This is useful to test the behaviour when Kafka records are deleted. - // 'KAFKA_LOG_RETENTION_MINUTES=1', - // 'KAFKA_LOG_SEGMENT_BYTES=1000000', + 'KAFKA_ADVERTISED_LISTENERS=PLAINTEXT://kafka_1:9092,PLAINTEXT_HOST://localhost:29092', ], ports: [ '29092:29092', @@ -151,6 +154,48 @@ std.manifestYamlDoc({ }, }, + + kafka_2:: { + kafka_2: { + image: 'confluentinc/cp-kafka:latest', + environment: commonKafkaEnvVars + [ + 'KAFKA_BROKER_ID=2', + 'KAFKA_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093,PLAINTEXT_HOST://:29093', + 'KAFKA_ADVERTISED_LISTENERS=PLAINTEXT://kafka_2:9092,PLAINTEXT_HOST://localhost:29093', + ], + ports: [ + '29093:29093', + ], + healthcheck: { + test: 'nc -z localhost 9092 || exit -1', + start_period: '1s', + interval: '1s', + timeout: '1s', + retries: '30', + }, + }, + }, + kafka_3:: { + kafka_3: { + image: 'confluentinc/cp-kafka:latest', + environment: commonKafkaEnvVars + [ + 'KAFKA_BROKER_ID=3', + 'KAFKA_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093,PLAINTEXT_HOST://:29094', + 'KAFKA_ADVERTISED_LISTENERS=PLAINTEXT://kafka_3:9092,PLAINTEXT_HOST://localhost:29094', + ], + ports: [ + '29094:29094', + ], + healthcheck: { + test: 'nc -z localhost 9092 || exit -1', + start_period: '1s', + interval: '1s', + timeout: '1s', + retries: '30', + }, + }, + }, + memcached:: { memcached: { image: 'memcached:1.6.19-alpine', @@ -187,6 +232,22 @@ std.manifestYamlDoc({ }, }, + jaeger:: { + jaeger: { + image: 'jaegertracing/all-in-one', + ports: ['16686:16686', '14268'], + }, + }, + + local jaegerEnv(appName) = { + JAEGER_AGENT_HOST: 'jaeger', + JAEGER_AGENT_PORT: 6831, + JAEGER_SAMPLER_TYPE: 'const', + JAEGER_SAMPLER_PARAM: 1, + JAEGER_TAGS: 'app=%s' % appName, + JAEGER_REPORTER_MAX_QUEUE_SIZE: 1000, + }, + // This function builds docker-compose declaration for Mimir service. local mimirService(serviceOptions) = { local defaultOptions = { @@ -196,10 +257,13 @@ std.manifestYamlDoc({ publishedHttpPort: error 'missing publishedHttpPort', dependsOn: { minio: { condition: 'service_started' }, - kafka: { condition: 'service_healthy' }, + kafka_1: { condition: 'service_healthy' }, + kafka_2: { condition: 'service_healthy' }, }, - env: {}, + env: jaegerEnv(self.target), extraArguments: [], + debug: true, + debugPort: self.publishedHttpPort + 3000, extraVolumes: [], memberlistBindPort: self.publishedHttpPort + 2000, }, @@ -212,11 +276,15 @@ std.manifestYamlDoc({ }, image: 'mimir', command: [ - './mimir', - '-config.file=./config/mimir.yaml' % options, - '-target=%(target)s' % options, - '-activity-tracker.filepath=/activity/%(name)s' % options, - ] + options.extraArguments, + 'sh', + '-c', + std.join(' ', [ + (if options.debug then 'exec ./dlv exec ./mimir --listen=:%(debugPort)d --headless=true --api-version=2 --accept-multiclient --continue -- ' % options else 'exec ./mimir'), + '-config.file=./config/mimir.yaml' % options, + '-target=%(target)s' % options, + '-activity-tracker.filepath=/activity/%(name)s' % options, + ] + options.extraArguments), + ], environment: [ '%s=%s' % [key, options.env[key]] for key in std.objectFields(options.env) @@ -224,7 +292,7 @@ std.manifestYamlDoc({ ], hostname: options.name, // Only publish HTTP port, but not gRPC one. - ports: ['%d:8080' % options.publishedHttpPort], + ports: ['%d:8080' % options.publishedHttpPort, '%(debugPort)d:%(debugPort)d' % options], depends_on: options.dependsOn, volumes: ['./config:/mimir/config', './activity:/activity'] + options.extraVolumes, }, diff --git a/development/mimir-ingest-storage/docker-compose.yml b/development/mimir-ingest-storage/docker-compose.yml index 404edac14ea..71cefefab81 100644 --- a/development/mimir-ingest-storage/docker-compose.yml +++ b/development/mimir-ingest-storage/docker-compose.yml @@ -23,20 +23,26 @@ - "9091:9091" "volumes": - "./config:/etc/agent-config" - "kafka": + "jaeger": + "image": "jaegertracing/all-in-one" + "ports": + - "16686:16686" + - "14268" + "kafka_1": "environment": - "CLUSTER_ID=zH1GDqcNTzGMDCXm5VZQdg" - - "KAFKA_BROKER_ID=1" - "KAFKA_NUM_PARTITIONS=100" - "KAFKA_PROCESS_ROLES=broker,controller" - - "KAFKA_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093,PLAINTEXT_HOST://:29092" - - "KAFKA_ADVERTISED_LISTENERS=PLAINTEXT://kafka:9092,PLAINTEXT_HOST://localhost:29092" - "KAFKA_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT,PLAINTEXT_HOST: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_CONTROLLER_QUORUM_VOTERS=1@kafka_1:9093,2@kafka_2:9093,3@kafka_3:9093" + - "KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR=2" + - "KAFKA_DEFAULT_REPLICATION_FACTOR=2" - "KAFKA_LOG_RETENTION_CHECK_INTERVAL_MS=10000" + - "KAFKA_BROKER_ID=1" + - "KAFKA_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093,PLAINTEXT_HOST://:29092" + - "KAFKA_ADVERTISED_LISTENERS=PLAINTEXT://kafka_1:9092,PLAINTEXT_HOST://localhost:29092" "healthcheck": "interval": "1s" "retries": "30" @@ -46,6 +52,54 @@ "image": "confluentinc/cp-kafka:latest" "ports": - "29092:29092" + "kafka_2": + "environment": + - "CLUSTER_ID=zH1GDqcNTzGMDCXm5VZQdg" + - "KAFKA_NUM_PARTITIONS=100" + - "KAFKA_PROCESS_ROLES=broker,controller" + - "KAFKA_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT,PLAINTEXT_HOST:PLAINTEXT" + - "KAFKA_INTER_BROKER_LISTENER_NAME=PLAINTEXT" + - "KAFKA_CONTROLLER_LISTENER_NAMES=CONTROLLER" + - "KAFKA_CONTROLLER_QUORUM_VOTERS=1@kafka_1:9093,2@kafka_2:9093,3@kafka_3:9093" + - "KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR=2" + - "KAFKA_DEFAULT_REPLICATION_FACTOR=2" + - "KAFKA_LOG_RETENTION_CHECK_INTERVAL_MS=10000" + - "KAFKA_BROKER_ID=2" + - "KAFKA_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093,PLAINTEXT_HOST://:29093" + - "KAFKA_ADVERTISED_LISTENERS=PLAINTEXT://kafka_2:9092,PLAINTEXT_HOST://localhost:29093" + "healthcheck": + "interval": "1s" + "retries": "30" + "start_period": "1s" + "test": "nc -z localhost 9092 || exit -1" + "timeout": "1s" + "image": "confluentinc/cp-kafka:latest" + "ports": + - "29093:29093" + "kafka_3": + "environment": + - "CLUSTER_ID=zH1GDqcNTzGMDCXm5VZQdg" + - "KAFKA_NUM_PARTITIONS=100" + - "KAFKA_PROCESS_ROLES=broker,controller" + - "KAFKA_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT,PLAINTEXT_HOST:PLAINTEXT" + - "KAFKA_INTER_BROKER_LISTENER_NAME=PLAINTEXT" + - "KAFKA_CONTROLLER_LISTENER_NAMES=CONTROLLER" + - "KAFKA_CONTROLLER_QUORUM_VOTERS=1@kafka_1:9093,2@kafka_2:9093,3@kafka_3:9093" + - "KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR=2" + - "KAFKA_DEFAULT_REPLICATION_FACTOR=2" + - "KAFKA_LOG_RETENTION_CHECK_INTERVAL_MS=10000" + - "KAFKA_BROKER_ID=3" + - "KAFKA_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093,PLAINTEXT_HOST://:29094" + - "KAFKA_ADVERTISED_LISTENERS=PLAINTEXT://kafka_3:9092,PLAINTEXT_HOST://localhost:29094" + "healthcheck": + "interval": "1s" + "retries": "30" + "start_period": "1s" + "test": "nc -z localhost 9092 || exit -1" + "timeout": "1s" + "image": "confluentinc/cp-kafka:latest" + "ports": + - "29094:29094" "memcached": "image": "memcached:1.6.19-alpine" "mimir-backend-1": @@ -53,20 +107,28 @@ "context": "." "dockerfile": "dev.dockerfile" "command": - - "./mimir" - - "-config.file=./config/mimir.yaml" - - "-target=backend" - - "-activity-tracker.filepath=/activity/mimir-backend-1" + - "sh" + - "-c" + - "exec ./dlv exec ./mimir --listen=:11006 --headless=true --api-version=2 --accept-multiclient --continue -- -config.file=./config/mimir.yaml -target=backend -activity-tracker.filepath=/activity/mimir-backend-1" "depends_on": - "kafka": + "kafka_1": + "condition": "service_healthy" + "kafka_2": "condition": "service_healthy" "minio": "condition": "service_started" - "environment": [] + "environment": + - "JAEGER_AGENT_HOST=jaeger" + - "JAEGER_AGENT_PORT=6831" + - "JAEGER_REPORTER_MAX_QUEUE_SIZE=1000" + - "JAEGER_SAMPLER_PARAM=1" + - "JAEGER_SAMPLER_TYPE=const" + - "JAEGER_TAGS=app=backend" "hostname": "mimir-backend-1" "image": "mimir" "ports": - "8006:8080" + - "11006:11006" "volumes": - "./config:/mimir/config" - "./activity:/activity" @@ -75,20 +137,28 @@ "context": "." "dockerfile": "dev.dockerfile" "command": - - "./mimir" - - "-config.file=./config/mimir.yaml" - - "-target=backend" - - "-activity-tracker.filepath=/activity/mimir-backend-2" + - "sh" + - "-c" + - "exec ./dlv exec ./mimir --listen=:11007 --headless=true --api-version=2 --accept-multiclient --continue -- -config.file=./config/mimir.yaml -target=backend -activity-tracker.filepath=/activity/mimir-backend-2" "depends_on": - "kafka": + "kafka_1": + "condition": "service_healthy" + "kafka_2": "condition": "service_healthy" "minio": "condition": "service_started" - "environment": [] + "environment": + - "JAEGER_AGENT_HOST=jaeger" + - "JAEGER_AGENT_PORT=6831" + - "JAEGER_REPORTER_MAX_QUEUE_SIZE=1000" + - "JAEGER_SAMPLER_PARAM=1" + - "JAEGER_SAMPLER_TYPE=const" + - "JAEGER_TAGS=app=backend" "hostname": "mimir-backend-2" "image": "mimir" "ports": - "8007:8080" + - "11007:11007" "volumes": - "./config:/mimir/config" - "./activity:/activity" @@ -97,20 +167,28 @@ "context": "." "dockerfile": "dev.dockerfile" "command": - - "./mimir" - - "-config.file=./config/mimir.yaml" - - "-target=read" - - "-activity-tracker.filepath=/activity/mimir-read-1" + - "sh" + - "-c" + - "exec ./dlv exec ./mimir --listen=:11004 --headless=true --api-version=2 --accept-multiclient --continue -- -config.file=./config/mimir.yaml -target=read -activity-tracker.filepath=/activity/mimir-read-1" "depends_on": - "kafka": + "kafka_1": + "condition": "service_healthy" + "kafka_2": "condition": "service_healthy" "minio": "condition": "service_started" - "environment": [] + "environment": + - "JAEGER_AGENT_HOST=jaeger" + - "JAEGER_AGENT_PORT=6831" + - "JAEGER_REPORTER_MAX_QUEUE_SIZE=1000" + - "JAEGER_SAMPLER_PARAM=1" + - "JAEGER_SAMPLER_TYPE=const" + - "JAEGER_TAGS=app=read" "hostname": "mimir-read-1" "image": "mimir" "ports": - "8004:8080" + - "11004:11004" "volumes": - "./config:/mimir/config" - "./activity:/activity" @@ -119,20 +197,28 @@ "context": "." "dockerfile": "dev.dockerfile" "command": - - "./mimir" - - "-config.file=./config/mimir.yaml" - - "-target=read" - - "-activity-tracker.filepath=/activity/mimir-read-2" + - "sh" + - "-c" + - "exec ./dlv exec ./mimir --listen=:11005 --headless=true --api-version=2 --accept-multiclient --continue -- -config.file=./config/mimir.yaml -target=read -activity-tracker.filepath=/activity/mimir-read-2" "depends_on": - "kafka": + "kafka_1": + "condition": "service_healthy" + "kafka_2": "condition": "service_healthy" "minio": "condition": "service_started" - "environment": [] + "environment": + - "JAEGER_AGENT_HOST=jaeger" + - "JAEGER_AGENT_PORT=6831" + - "JAEGER_REPORTER_MAX_QUEUE_SIZE=1000" + - "JAEGER_SAMPLER_PARAM=1" + - "JAEGER_SAMPLER_TYPE=const" + - "JAEGER_TAGS=app=read" "hostname": "mimir-read-2" "image": "mimir" "ports": - "8005:8080" + - "11005:11005" "volumes": - "./config:/mimir/config" - "./activity:/activity" @@ -141,21 +227,28 @@ "context": "." "dockerfile": "dev.dockerfile" "command": - - "./mimir" - - "-config.file=./config/mimir.yaml" - - "-target=write" - - "-activity-tracker.filepath=/activity/mimir-write-zone-a-1" - - "-ingester.ring.instance-availability-zone=zone-a" + - "sh" + - "-c" + - "exec ./dlv exec ./mimir --listen=:11001 --headless=true --api-version=2 --accept-multiclient --continue -- -config.file=./config/mimir.yaml -target=write -activity-tracker.filepath=/activity/mimir-write-zone-a-1 -ingester.ring.instance-availability-zone=zone-a" "depends_on": - "kafka": + "kafka_1": + "condition": "service_healthy" + "kafka_2": "condition": "service_healthy" "minio": "condition": "service_started" - "environment": [] + "environment": + - "JAEGER_AGENT_HOST=jaeger" + - "JAEGER_AGENT_PORT=6831" + - "JAEGER_REPORTER_MAX_QUEUE_SIZE=1000" + - "JAEGER_SAMPLER_PARAM=1" + - "JAEGER_SAMPLER_TYPE=const" + - "JAEGER_TAGS=app=write" "hostname": "mimir-write-zone-a-1" "image": "mimir" "ports": - "8001:8080" + - "11001:11001" "volumes": - "./config:/mimir/config" - "./activity:/activity" @@ -165,21 +258,28 @@ "context": "." "dockerfile": "dev.dockerfile" "command": - - "./mimir" - - "-config.file=./config/mimir.yaml" - - "-target=write" - - "-activity-tracker.filepath=/activity/mimir-write-zone-a-2" - - "-ingester.ring.instance-availability-zone=zone-a" + - "sh" + - "-c" + - "exec ./dlv exec ./mimir --listen=:11002 --headless=true --api-version=2 --accept-multiclient --continue -- -config.file=./config/mimir.yaml -target=write -activity-tracker.filepath=/activity/mimir-write-zone-a-2 -ingester.ring.instance-availability-zone=zone-a" "depends_on": - "kafka": + "kafka_1": + "condition": "service_healthy" + "kafka_2": "condition": "service_healthy" "minio": "condition": "service_started" - "environment": [] + "environment": + - "JAEGER_AGENT_HOST=jaeger" + - "JAEGER_AGENT_PORT=6831" + - "JAEGER_REPORTER_MAX_QUEUE_SIZE=1000" + - "JAEGER_SAMPLER_PARAM=1" + - "JAEGER_SAMPLER_TYPE=const" + - "JAEGER_TAGS=app=write" "hostname": "mimir-write-zone-a-2" "image": "mimir" "ports": - "8002:8080" + - "11002:11002" "volumes": - "./config:/mimir/config" - "./activity:/activity" @@ -189,97 +289,63 @@ "context": "." "dockerfile": "dev.dockerfile" "command": - - "./mimir" - - "-config.file=./config/mimir.yaml" - - "-target=write" - - "-activity-tracker.filepath=/activity/mimir-write-zone-a-3" - - "-ingester.ring.instance-availability-zone=zone-a" + - "sh" + - "-c" + - "exec ./dlv exec ./mimir --listen=:11003 --headless=true --api-version=2 --accept-multiclient --continue -- -config.file=./config/mimir.yaml -target=write -activity-tracker.filepath=/activity/mimir-write-zone-a-3 -ingester.ring.instance-availability-zone=zone-a" "depends_on": - "kafka": + "kafka_1": + "condition": "service_healthy" + "kafka_2": "condition": "service_healthy" "minio": "condition": "service_started" - "environment": [] + "environment": + - "JAEGER_AGENT_HOST=jaeger" + - "JAEGER_AGENT_PORT=6831" + - "JAEGER_REPORTER_MAX_QUEUE_SIZE=1000" + - "JAEGER_SAMPLER_PARAM=1" + - "JAEGER_SAMPLER_TYPE=const" + - "JAEGER_TAGS=app=write" "hostname": "mimir-write-zone-a-3" "image": "mimir" "ports": - "8003:8080" + - "11003:11003" "volumes": - "./config:/mimir/config" - "./activity:/activity" - ".data-mimir-write-zone-a-3:/data:delegated" - "mimir-write-zone-b-1": - "build": - "context": "." - "dockerfile": "dev.dockerfile" - "command": - - "./mimir" - - "-config.file=./config/mimir.yaml" - - "-target=write" - - "-activity-tracker.filepath=/activity/mimir-write-zone-b-1" - - "-ingester.ring.instance-availability-zone=zone-b" - "depends_on": - "kafka": - "condition": "service_healthy" - "minio": - "condition": "service_started" - "environment": [] - "hostname": "mimir-write-zone-b-1" - "image": "mimir" - "ports": - - "8011:8080" - "volumes": - - "./config:/mimir/config" - - "./activity:/activity" - - ".data-mimir-write-zone-b-1:/data:delegated" - "mimir-write-zone-b-2": + "mimir-write-zone-c-61": "build": "context": "." "dockerfile": "dev.dockerfile" "command": - - "./mimir" - - "-config.file=./config/mimir.yaml" - - "-target=write" - - "-activity-tracker.filepath=/activity/mimir-write-zone-b-2" - - "-ingester.ring.instance-availability-zone=zone-b" + - "sh" + - "-c" + - "exec ./mimir -config.file=./config/mimir.yaml -target=ingester -activity-tracker.filepath=/activity/mimir-write-zone-c-61 -ingester.ring.instance-availability-zone=zone-c -ingester.ring.instance-id=ingester-zone-c-61 -ingester.partition-ring.prefix=exclusive-prefix -ingester.ring.prefix=exclusive-prefix -ingest-storage.kafka.consume-from-position-at-startup=start -ingest-storage.kafka.consume-from-timestamp-at-startup=0 -ingest-storage.kafka.replay-shards=2 -ingest-storage.kafka.batch-size=150 -ingest-storage.kafka.replay-concurrency=4 -ingest-storage.kafka.records-per-fetch=6000" "depends_on": - "kafka": + "kafka_1": "condition": "service_healthy" - "minio": - "condition": "service_started" - "environment": [] - "hostname": "mimir-write-zone-b-2" - "image": "mimir" - "ports": - - "8012:8080" - "volumes": - - "./config:/mimir/config" - - "./activity:/activity" - - ".data-mimir-write-zone-b-2:/data:delegated" - "mimir-write-zone-b-3": - "build": - "context": "." - "dockerfile": "dev.dockerfile" - "command": - - "./mimir" - - "-config.file=./config/mimir.yaml" - - "-target=write" - - "-activity-tracker.filepath=/activity/mimir-write-zone-b-3" - - "-ingester.ring.instance-availability-zone=zone-b" - "depends_on": - "kafka": + "kafka_2": "condition": "service_healthy" "minio": "condition": "service_started" - "environment": [] - "hostname": "mimir-write-zone-b-3" + "environment": + - "JAEGER_AGENT_HOST=jaeger" + - "JAEGER_AGENT_PORT=6831" + - "JAEGER_REPORTER_MAX_QUEUE_SIZE=1000" + - "JAEGER_SAMPLER_PARAM=1" + - "JAEGER_SAMPLER_TYPE=const" + - "JAEGER_TAGS=app=ingester" + "hostname": "mimir-write-zone-c-61" "image": "mimir" "ports": - - "8013:8080" + - "8064:8080" + - "11064:11064" "volumes": - "./config:/mimir/config" - "./activity:/activity" - - ".data-mimir-write-zone-b-3:/data:delegated" + - ".data-mimir-write-zone-c-61:/data:delegated" "minio": "command": - "server" diff --git a/docs/sources/mimir/configure/configuration-parameters/index.md b/docs/sources/mimir/configure/configuration-parameters/index.md index 9420968be78..e84ac8ce205 100644 --- a/docs/sources/mimir/configure/configuration-parameters/index.md +++ b/docs/sources/mimir/configure/configuration-parameters/index.md @@ -3851,6 +3851,30 @@ kafka: # CLI flag: -ingest-storage.kafka.wait-strong-read-consistency-timeout [wait_strong_read_consistency_timeout: | default = 20s] + # The number of concurrent fetch requests that the ingester sends to kafka + # when catching up during startup. + # CLI flag: -ingest-storage.kafka.replay-concurrency + [replay_concurrency: | default = 1] + + # The number of concurrent appends to the TSDB head. 0 to disable. + # CLI flag: -ingest-storage.kafka.replay-shards + [replay_shards: | default = 0] + + # The number of timeseries to batch together before ingesting into TSDB. + # CLI flag: -ingest-storage.kafka.batch-size + [batch_size: | default = 128] + + # The number of records to fetch from Kafka in a single request. + # CLI flag: -ingest-storage.kafka.records-per-fetch + [records_per_fetch: | default = 128] + + # When enabled, the fetch request MaxBytes field is computed using the + # compressed size of previous records. When disabled, MaxBytes is computed + # using uncompressed bytes. Different Kafka implementations interpret MaxBytes + # differently. + # CLI flag: -ingest-storage.kafka.use-compressed-bytes-as-fetch-max-bytes + [use_compressed_bytes_as_fetch_max_bytes: | default = true] + migration: # When both this option and ingest storage are enabled, distributors write to # both Kafka and ingesters. A write request is considered successful only when diff --git a/docs/sources/mimir/configure/configure-kafka-backend.md b/docs/sources/mimir/configure/configure-kafka-backend.md new file mode 100644 index 00000000000..017b5bc2eba --- /dev/null +++ b/docs/sources/mimir/configure/configure-kafka-backend.md @@ -0,0 +1,35 @@ +--- +aliases: + - ../operators-guide/configure/configure-kafka-backend/ +description: Learn how to configure Grafana Mimir to use Kafka for ingest storage. +menuTitle: Kafka +title: Configure Grafana Mimir kafka backend +weight: 66 +--- + +# Configure the Grafana Mimir Kafka backend + +Grafana Mimir supports using Kafka for the first layer of ingestion. This is an experimental feature released in Mimir 2.14. +This page is incomplete. It will be updated as the ingest storage feature matures and moves out of the experimental phase. + +## Different Kafka backend implementations + +Some Kafka-compatible implementations have different behavior for the Kafka API. +To set up Mimir to work with different Kafka backends, you need to configure some parameters. +Here are the Kafka flavors and additional configurations needed to set them up in Mimir. + +### Apache Kafka + +Use the default options with Apache Kafka. No additional configuration is needed. + +### Confluent Kafka + +Use the default options with Confluent Kafka. No additional configuration is needed. + +### Warpstream + +Configure the following CLI flags or their YAML equivalent. + +``` +-ingest-storage.kafka.use-compressed-bytes-as-fetch-max-bytes=false +``` diff --git a/go.mod b/go.mod index 1f0490a47d3..e25921a87bc 100644 --- a/go.mod +++ b/go.mod @@ -181,6 +181,7 @@ require ( github.com/felixge/httpsnoop v1.0.4 // indirect github.com/fsnotify/fsnotify v1.7.0 // indirect github.com/go-errors/errors v1.4.2 // indirect + github.com/go-jose/go-jose/v3 v3.0.3 github.com/go-logfmt/logfmt v0.6.0 // indirect github.com/go-logr/logr v1.4.2 // indirect github.com/go-logr/stdr v1.2.2 // indirect @@ -312,6 +313,9 @@ replace github.com/opentracing-contrib/go-grpc => github.com/charleskorn/go-grpc // Replacing prometheus/alertmanager with our fork. replace github.com/prometheus/alertmanager => github.com/grafana/prometheus-alertmanager v0.25.1-0.20240625192351-66ec17e3aa45 +// Replacing with a fork commit based on v1.17.1 with https://github.com/twmb/franz-go/pull/803 cherry-picked. +replace github.com/twmb/franz-go => github.com/dimitarvdimitrov/franz-go v0.0.0-20240904145554-ceadc28d3bd9 + // Pin Google GRPC to v1.65.0 as v1.66.0 has API changes and also potentially performance regressions. // Following https://github.com/grafana/dskit/pull/581 replace google.golang.org/grpc => google.golang.org/grpc v1.65.0 diff --git a/go.sum b/go.sum index d2c066f7466..9d1ddeae89a 100644 --- a/go.sum +++ b/go.sum @@ -950,6 +950,8 @@ github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f h1:lO4WD4F/r github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f/go.mod h1:cuUVRXasLTGF7a8hSLbxyZXjz+1KgoB3wDUb6vlszIc= github.com/digitalocean/godo v1.121.0 h1:ilXiHuEnhbJs2fmFEPX0r/QQ6KfiOIMAhJN3f8NiCfI= github.com/digitalocean/godo v1.121.0/go.mod h1:WQVH83OHUy6gC4gXpEVQKtxTd4L5oCp+5OialidkPLY= +github.com/dimitarvdimitrov/franz-go v0.0.0-20240904145554-ceadc28d3bd9 h1:jszPVGeTr25QTJ/jWiT7eXnabc4R4itChxUVFSCLjRQ= +github.com/dimitarvdimitrov/franz-go v0.0.0-20240904145554-ceadc28d3bd9/go.mod h1:NreRdJ2F7dziDY/m6VyspWd6sNxHKXdMZI42UfQ3GXM= github.com/distribution/reference v0.5.0 h1:/FUIFXtfc/x2gpa5/VGfiGLuOIdYa1t65IKK2OFGvA0= github.com/distribution/reference v0.5.0/go.mod h1:BbU0aIcezP1/5jX/8MP0YiH4SdvB5Y4f/wlDRiLyi3E= github.com/dlclark/regexp2 v1.11.0 h1:G/nrcoOa7ZXlpoa/91N3X7mM3r8eIlMBBJZvsz/mxKI= @@ -1023,6 +1025,8 @@ github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2 github.com/go-ini/ini v1.67.0 h1:z6ZrTEZqSWOTyH2FlglNbNgARyHG8oLW9gMELqKr06A= github.com/go-ini/ini v1.67.0/go.mod h1:ByCAeIL28uOIIG0E3PJtZPDL8WnHpFKFOtgjp+3Ies8= github.com/go-jose/go-jose/v3 v3.0.0/go.mod h1:RNkWWRld676jZEYoV3+XK8L2ZnNSvIsxFMht0mSX+u8= +github.com/go-jose/go-jose/v3 v3.0.3 h1:fFKWeig/irsp7XD2zBxvnmA/XaRWp5V3CBsZXJF7G7k= +github.com/go-jose/go-jose/v3 v3.0.3/go.mod h1:5b+7YgP7ZICgJDBdfjZaIt+H/9L9T/YQrVfLAMboGkQ= github.com/go-jose/go-jose/v4 v4.0.1 h1:QVEPDE3OluqXBQZDcnNvQrInro2h0e4eqNbnZSWqS6U= github.com/go-jose/go-jose/v4 v4.0.1/go.mod h1:WVf9LFMHh/QVrmqrOfqun0C45tMe3RoiKJMPvgWwLfY= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= @@ -1690,8 +1694,6 @@ github.com/tklauser/go-sysconf v0.3.12/go.mod h1:Ho14jnntGE1fpdOqQEEaiKRpvIavV0h github.com/tklauser/numcpus v0.6.1 h1:ng9scYS7az0Bk4OZLvrNXNSAO2Pxr1XXRAPyjhIx+Fk= github.com/tklauser/numcpus v0.6.1/go.mod h1:1XfjsgE2zo8GVw7POkMbHENHzVg3GzmoZ9fESEdAacY= github.com/tv42/httpunix v0.0.0-20150427012821-b75d8614f926/go.mod h1:9ESjWnEqriFuLhtthL60Sar/7RFoluCcXsuvEwTV5KM= -github.com/twmb/franz-go v1.17.1 h1:0LwPsbbJeJ9R91DPUHSEd4su82WJWcTY1Zzbgbg4CeQ= -github.com/twmb/franz-go v1.17.1/go.mod h1:NreRdJ2F7dziDY/m6VyspWd6sNxHKXdMZI42UfQ3GXM= github.com/twmb/franz-go/pkg/kadm v1.13.0 h1:bJq4C2ZikUE2jh/wl9MtMTQ/kpmnBgVFh8XMQBEC+60= github.com/twmb/franz-go/pkg/kadm v1.13.0/go.mod h1:VMvpfjz/szpH9WB+vGM+rteTzVv0djyHFimci9qm2C0= github.com/twmb/franz-go/pkg/kfake v0.0.0-20240821035758-b77dd13e2bfa h1:OmQ4DJhqeOPdIH60Psut1vYU8A6LGyxJbF09w5RAa2w= diff --git a/pkg/ingester/ingester_test.go b/pkg/ingester/ingester_test.go index 737b0693af1..154dcd3d9d3 100644 --- a/pkg/ingester/ingester_test.go +++ b/pkg/ingester/ingester_test.go @@ -8383,10 +8383,14 @@ func matrixToLables(m model.Matrix) [][]mimirpb.LabelAdapter { } func runTestQuery(ctx context.Context, t *testing.T, ing *Ingester, ty labels.MatchType, n, v string) (model.Matrix, *client.QueryRequest, error) { + t.Helper() + return runTestQueryTimes(ctx, t, ing, ty, n, v, model.Earliest, model.Latest) } func runTestQueryTimes(ctx context.Context, t *testing.T, ing *Ingester, ty labels.MatchType, n, v string, start, end model.Time) (model.Matrix, *client.QueryRequest, error) { + t.Helper() + matcher, err := labels.NewMatcher(ty, n, v) if err != nil { return nil, nil, err diff --git a/pkg/storage/ingest/config.go b/pkg/storage/ingest/config.go index 2ea46e57c1b..38f81eb6e67 100644 --- a/pkg/storage/ingest/config.go +++ b/pkg/storage/ingest/config.go @@ -91,7 +91,12 @@ type KafkaConfig struct { WaitStrongReadConsistencyTimeout time.Duration `yaml:"wait_strong_read_consistency_timeout"` // Used when logging unsampled client errors. Set from ingester's ErrorSampleRate. - FallbackClientErrorSampleRate int64 `yaml:"-"` + FallbackClientErrorSampleRate int64 `yaml:"-"` + ReplayConcurrency int `yaml:"replay_concurrency"` + ReplayShards int `yaml:"replay_shards"` + BatchSize int `yaml:"batch_size"` + RecordsPerFetch int `yaml:"records_per_fetch"` + UseCompressedBytesAsFetchMaxBytes bool `yaml:"use_compressed_bytes_as_fetch_max_bytes"` } func (cfg *KafkaConfig) RegisterFlags(f *flag.FlagSet) { @@ -126,6 +131,11 @@ func (cfg *KafkaConfig) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) f.Int64Var(&cfg.ProducerMaxBufferedBytes, prefix+".producer-max-buffered-bytes", 1024*1024*1024, "The maximum size of (uncompressed) buffered and unacknowledged produced records sent to Kafka. The produce request fails once this limit is reached. This limit is per Kafka client. 0 to disable the limit.") f.DurationVar(&cfg.WaitStrongReadConsistencyTimeout, prefix+".wait-strong-read-consistency-timeout", 20*time.Second, "The maximum allowed for a read requests processed by an ingester to wait until strong read consistency is enforced. 0 to disable the timeout.") + f.IntVar(&cfg.ReplayConcurrency, prefix+".replay-concurrency", 1, "The number of concurrent fetch requests that the ingester sends to kafka when catching up during startup.") + f.IntVar(&cfg.ReplayShards, prefix+".replay-shards", 0, "The number of concurrent appends to the TSDB head. 0 to disable.") + f.IntVar(&cfg.BatchSize, prefix+".batch-size", 128, "The number of timeseries to batch together before ingesting into TSDB.") + f.IntVar(&cfg.RecordsPerFetch, prefix+".records-per-fetch", 128, "The number of records to fetch from Kafka in a single request.") + f.BoolVar(&cfg.UseCompressedBytesAsFetchMaxBytes, prefix+".use-compressed-bytes-as-fetch-max-bytes", true, "When enabled, the fetch request MaxBytes field is computed using the compressed size of previous records. When disabled, MaxBytes is computed using uncompressed bytes. Different Kafka implementations interpret MaxBytes differently.") } func (cfg *KafkaConfig) Validate() error { diff --git a/pkg/storage/ingest/partition_offset_reader.go b/pkg/storage/ingest/partition_offset_reader.go index 943d1c15129..9a3f43112a8 100644 --- a/pkg/storage/ingest/partition_offset_reader.go +++ b/pkg/storage/ingest/partition_offset_reader.go @@ -13,6 +13,7 @@ import ( "github.com/grafana/dskit/services" "github.com/prometheus/client_golang/prometheus" "github.com/twmb/franz-go/pkg/kgo" + "go.uber.org/atomic" ) var ( @@ -32,6 +33,9 @@ type genericOffsetReader[O any] struct { // request that will be issued (not the current in-flight one, if any). nextResultPromiseMx sync.RWMutex nextResultPromise *resultPromise[O] + + // lastResultPromise is the last returned offset. + lastResultPromise *atomic.Pointer[resultPromise[O]] } func newGenericOffsetReader[O any](fetchLastProducedOffset func(context.Context) (O, error), pollInterval time.Duration, logger log.Logger) *genericOffsetReader[O] { @@ -39,9 +43,11 @@ func newGenericOffsetReader[O any](fetchLastProducedOffset func(context.Context) logger: logger, fetchLastProducedOffset: fetchLastProducedOffset, nextResultPromise: newResultPromise[O](), + lastResultPromise: atomic.NewPointer(newResultPromise[O]()), } - p.Service = services.NewTimerService(pollInterval, nil, p.onPollInterval, p.stopping) + // Run the poll interval once at startup so we can cache the offset. + p.Service = services.NewTimerService(pollInterval, p.onPollInterval, p.onPollInterval, p.stopping) return p } @@ -86,6 +92,7 @@ func (r *genericOffsetReader[O]) getAndNotifyLastProducedOffset(ctx context.Cont // Notify whoever was waiting for it. promise.notify(offset, err) + r.lastResultPromise.Store(promise) } // WaitNextFetchLastProducedOffset returns the result of the *next* "last produced offset" request @@ -102,6 +109,12 @@ func (r *genericOffsetReader[O]) WaitNextFetchLastProducedOffset(ctx context.Con return promise.wait(ctx) } +// CachedOffset returns the last result of fetching the offset. This is likely outdated, but it's useful to get a directionally correct value quickly. +func (r *genericOffsetReader[O]) CachedOffset() (O, error) { + c := r.lastResultPromise.Load() + return c.resultValue, c.resultErr +} + // partitionOffsetReader is responsible to read the offsets of a single partition. type partitionOffsetReader struct { *genericOffsetReader[int64] @@ -112,8 +125,13 @@ type partitionOffsetReader struct { } func newPartitionOffsetReader(client *kgo.Client, topic string, partitionID int32, pollInterval time.Duration, reg prometheus.Registerer, logger log.Logger) *partitionOffsetReader { + offsetClient := newPartitionOffsetClient(client, topic, reg, logger) + return newPartitionOffsetReaderWithOffsetClient(offsetClient, partitionID, pollInterval, logger) +} + +func newPartitionOffsetReaderWithOffsetClient(offsetClient *partitionOffsetClient, partitionID int32, pollInterval time.Duration, logger log.Logger) *partitionOffsetReader { r := &partitionOffsetReader{ - client: newPartitionOffsetClient(client, topic, reg, logger), + client: offsetClient, partitionID: partitionID, logger: logger, // Do not wrap with partition ID because it's already done by the caller. } diff --git a/pkg/storage/ingest/partition_offset_reader_test.go b/pkg/storage/ingest/partition_offset_reader_test.go index 00a097fdb3d..34bc4a1ea04 100644 --- a/pkg/storage/ingest/partition_offset_reader_test.go +++ b/pkg/storage/ingest/partition_offset_reader_test.go @@ -4,6 +4,7 @@ package ingest import ( "context" + "fmt" "sync" "testing" "time" @@ -81,15 +82,19 @@ func TestPartitionOffsetReader_WaitNextFetchLastProducedOffset(t *testing.T) { client = createTestKafkaClient(t, kafkaCfg) reader = newPartitionOffsetReader(client, topicName, partitionID, pollInterval, nil, logger) - lastOffset = atomic.NewInt64(1) - firstRequestReceived = make(chan struct{}) + lastOffset = atomic.NewInt64(1) + firstRequestReceived = make(chan struct{}) + secondRequestReceived = make(chan struct{}) ) cluster.ControlKey(int16(kmsg.ListOffsets), func(kreq kmsg.Request) (kmsg.Response, error, bool) { cluster.KeepControl() - if lastOffset.Load() == 1 { + switch lastOffset.Load() { + case 1: close(firstRequestReceived) + case 2: + close(secondRequestReceived) } // Mock the response so that we can increase the offset each time. @@ -109,20 +114,22 @@ func TestPartitionOffsetReader_WaitNextFetchLastProducedOffset(t *testing.T) { wg := sync.WaitGroup{} - // The 1st WaitNextFetchLastProducedOffset() is called before the service starts, so it's expected - // to wait the result of the 1st request. - runAsync(&wg, func() { + // The 1st WaitNextFetchLastProducedOffset() is called before the service starts. + // The service fetches the offset once at startup, so it's expected that the first wait + // to wait the result of the 2nd request. + // If we don't do synchronisation, then it's also possible that we fit in the first request, but we synchronise to avoid flaky tests + runAsyncAfter(&wg, firstRequestReceived, func() { actual, err := reader.WaitNextFetchLastProducedOffset(ctx) require.NoError(t, err) - assert.Equal(t, int64(1), actual) + assert.Equal(t, int64(2), actual) }) - // The 2nd WaitNextFetchLastProducedOffset() is called while the 1st request is running, so it's expected - // to wait the result of the 2nd request. - runAsyncAfter(&wg, firstRequestReceived, func() { + // The 2nd WaitNextFetchLastProducedOffset() is called while the 1st is running, so it's expected + // to wait the result of the 3rd request. + runAsyncAfter(&wg, secondRequestReceived, func() { actual, err := reader.WaitNextFetchLastProducedOffset(ctx) require.NoError(t, err) - assert.Equal(t, int64(2), actual) + assert.Equal(t, int64(3), actual) }) // Now we can start the service. @@ -215,15 +222,19 @@ func TestTopicOffsetsReader_WaitNextFetchLastProducedOffset(t *testing.T) { client = createTestKafkaClient(t, kafkaCfg) reader = NewTopicOffsetsReader(client, topicName, allPartitionIDs, pollInterval, nil, logger) - lastOffset = atomic.NewInt64(1) - firstRequestReceived = make(chan struct{}) + lastOffset = atomic.NewInt64(1) + firstRequestReceived = make(chan struct{}) + secondRequestReceived = make(chan struct{}) ) cluster.ControlKey(int16(kmsg.ListOffsets), func(kreq kmsg.Request) (kmsg.Response, error, bool) { cluster.KeepControl() - if lastOffset.Load() == 1 { + switch lastOffset.Load() { + case 1: close(firstRequestReceived) + case 3: + close(secondRequestReceived) } // Mock the response so that we can increase the offset each time. @@ -247,20 +258,22 @@ func TestTopicOffsetsReader_WaitNextFetchLastProducedOffset(t *testing.T) { wg := sync.WaitGroup{} - // The 1st WaitNextFetchLastProducedOffset() is called before the service starts, so it's expected - // to wait the result of the 1st request. - runAsync(&wg, func() { + // The 1st WaitNextFetchLastProducedOffset() is called before the service starts. + // The service fetches the offset once at startup, so it's expected that the first wait + // to wait the result of the 2nd request. + // If we don't do synchronisation, then it's also possible that we fit in the first request, but we synchronise to avoid flaky tests + runAsyncAfter(&wg, firstRequestReceived, func() { actual, err := reader.WaitNextFetchLastProducedOffset(ctx) require.NoError(t, err) - assert.Equal(t, map[int32]int64{0: int64(1), 1: int64(2)}, actual) + assert.Equal(t, map[int32]int64{0: int64(3), 1: int64(4)}, actual) }) - // The 2nd WaitNextFetchLastProducedOffset() is called while the 1st request is running, so it's expected - // to wait the result of the 2nd request. - runAsyncAfter(&wg, firstRequestReceived, func() { + // The 2nd WaitNextFetchLastProducedOffset() is called while the 1st is running, so it's expected + // to wait the result of the 3rd request. + runAsyncAfter(&wg, secondRequestReceived, func() { actual, err := reader.WaitNextFetchLastProducedOffset(ctx) require.NoError(t, err) - assert.Equal(t, map[int32]int64{0: int64(3), 1: int64(4)}, actual) + assert.Equal(t, map[int32]int64{0: int64(5), 1: int64(6)}, actual) }) // Now we can start the service. @@ -289,3 +302,115 @@ func TestTopicOffsetsReader_WaitNextFetchLastProducedOffset(t *testing.T) { assert.ErrorIs(t, err, context.Canceled) }) } + +func TestGenericPartitionReader_Caching(t *testing.T) { + logger := log.NewNopLogger() + + t.Run("should initialize with fetched offset", func(t *testing.T) { + ctx := context.Background() + mockFetch := func(context.Context) (int64, error) { + return 42, nil + } + + reader := newGenericOffsetReader[int64](mockFetch, time.Second, logger) + require.NoError(t, services.StartAndAwaitRunning(ctx, reader)) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(ctx, reader)) + }) + + offset, err := reader.CachedOffset() + assert.NoError(t, err) + assert.Equal(t, int64(42), offset) + }) + + t.Run("should cache error from initial fetch", func(t *testing.T) { + ctx := context.Background() + expectedErr := fmt.Errorf("fetch error") + mockFetch := func(context.Context) (int64, error) { + return 0, expectedErr + } + + reader := newGenericOffsetReader[int64](mockFetch, time.Second, logger) + require.NoError(t, services.StartAndAwaitRunning(ctx, reader)) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(ctx, reader)) + }) + + offset, err := reader.CachedOffset() + assert.ErrorIs(t, err, expectedErr) + assert.Equal(t, int64(0), offset) + }) + + t.Run("should update cache on poll interval", func(t *testing.T) { + ctx := context.Background() + fetchCount := 0 + fetchChan := make(chan struct{}, 3) // Buffer size of 3 to allow multiple fetches + mockFetch := func(ctx context.Context) (int64, error) { + fetchCount++ + select { + case <-ctx.Done(): + case fetchChan <- struct{}{}: + } + return int64(fetchCount), nil + } + + reader := newGenericOffsetReader[int64](mockFetch, 10*time.Millisecond, logger) + require.NoError(t, services.StartAndAwaitRunning(ctx, reader)) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(ctx, reader)) + }) + + // Wait for at least two fetches to complete and have their results cached. + <-fetchChan + <-fetchChan + <-fetchChan + + offset, err := reader.CachedOffset() + assert.NoError(t, err) + assert.GreaterOrEqual(t, offset, int64(2), "Offset should have been updated at least once") + }) + + t.Run("should handle context cancellation", func(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + mockFetch := func(context.Context) (int64, error) { + return 42, nil + } + + reader := newGenericOffsetReader[int64](mockFetch, time.Second, logger) + require.NoError(t, services.StartAndAwaitRunning(ctx, reader)) + t.Cleanup(func() { + cancel() + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), reader)) + }) + + // The cached offset should be available + offset, err := reader.CachedOffset() + assert.NoError(t, err) + assert.Equal(t, int64(42), offset) + }) + + t.Run("should handle concurrent access", func(t *testing.T) { + ctx := context.Background() + mockFetch := func(context.Context) (int64, error) { + return 42, nil + } + + reader := newGenericOffsetReader[int64](mockFetch, time.Second, logger) + require.NoError(t, services.StartAndAwaitRunning(ctx, reader)) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(ctx, reader)) + }) + + var wg sync.WaitGroup + for i := 0; i < 100; i++ { + wg.Add(1) + go func() { + defer wg.Done() + offset, err := reader.CachedOffset() + assert.NoError(t, err) + assert.Equal(t, int64(42), offset) + }() + } + wg.Wait() + }) +} diff --git a/pkg/storage/ingest/pusher.go b/pkg/storage/ingest/pusher.go index 46751241427..be186166ade 100644 --- a/pkg/storage/ingest/pusher.go +++ b/pkg/storage/ingest/pusher.go @@ -6,55 +6,69 @@ import ( "context" "errors" "fmt" + "sync" "time" "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/grafana/dskit/cancellation" "github.com/grafana/dskit/middleware" + "github.com/grafana/dskit/multierror" "github.com/grafana/dskit/user" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" + "github.com/prometheus/prometheus/model/labels" "github.com/grafana/mimir/pkg/mimirpb" util_log "github.com/grafana/mimir/pkg/util/log" "github.com/grafana/mimir/pkg/util/spanlogger" ) +const shardForSeriesBuffer = 2000 // TODO dimitarvdimitrov 2000 is arbitrary; the idea is that we don't block the goroutine calling PushToStorage while we're flushing. A linked list with a sync.Cond or something different would also work + type Pusher interface { PushToStorage(context.Context, *mimirpb.WriteRequest) error } +type PusherCloser interface { + PushToStorage(context.Context, *mimirpb.WriteRequest) error + // Calls to close are safe and will not be called concurrenctly. + Close() []error +} + +// pusherConsumer receivers records from Kafka and pushes them to the storage. +// Each time a batch of records is received from Kafka, we instantiate a new pusherConsumer, this is to ensure we can retry if necessary and know whether we have completed that batch or not. type pusherConsumer struct { + fallbackClientErrSampler *util_log.Sampler + metrics *pusherConsumerMetrics + logger log.Logger + + kafkaConfig KafkaConfig + pusher Pusher +} +type pusherConsumerMetrics struct { + numTimeSeriesPerFlush prometheus.Histogram processingTimeSeconds prometheus.Observer clientErrRequests prometheus.Counter serverErrRequests prometheus.Counter totalRequests prometheus.Counter - - fallbackClientErrSampler *util_log.Sampler // Fallback log message sampler client errors that are not sampled yet. - logger log.Logger } -type parsedRecord struct { - *mimirpb.WriteRequest - // Context holds the tracing and cancellation data for this record/request. - ctx context.Context - tenantID string - err error -} - -func newPusherConsumer(p Pusher, fallbackClientErrSampler *util_log.Sampler, reg prometheus.Registerer, l log.Logger) *pusherConsumer { +// newPusherConsumerMetrics creates a new pusherConsumerMetrics instance. +func newPusherConsumerMetrics(reg prometheus.Registerer) *pusherConsumerMetrics { errRequestsCounter := promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ Name: "cortex_ingest_storage_reader_records_failed_total", Help: "Number of records (write requests) which caused errors while processing. Client errors are errors such as tenant limits and samples out of bounds. Server errors indicate internal recoverable errors.", }, []string{"cause"}) - return &pusherConsumer{ - pusher: p, - logger: l, - fallbackClientErrSampler: fallbackClientErrSampler, + return &pusherConsumerMetrics{ + numTimeSeriesPerFlush: promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ + Name: "cortex_ingester_pusher_num_timeseries_per_flush", + Help: "Number of time series per flush", + NativeHistogramBucketFactor: 1.1, + }), processingTimeSeconds: promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ Name: "cortex_ingest_storage_reader_processing_time_seconds", Help: "Time taken to process a single record (write request).", @@ -72,35 +86,112 @@ func newPusherConsumer(p Pusher, fallbackClientErrSampler *util_log.Sampler, reg } } -func (c pusherConsumer) consume(ctx context.Context, records []record) error { +// newPusherConsumer creates a new pusherConsumer instance. +func newPusherConsumer(pusher Pusher, kafkaCfg KafkaConfig, metrics *pusherConsumerMetrics, logger log.Logger) *pusherConsumer { + return &pusherConsumer{ + pusher: pusher, + kafkaConfig: kafkaCfg, + metrics: metrics, + logger: logger, + fallbackClientErrSampler: util_log.NewSampler(kafkaCfg.FallbackClientErrorSampleRate), + } +} + +// Consume implements the recordConsumer interface. +// It'll use a separate goroutine to unmarshal the next record while we push the current record to storage. +func (c pusherConsumer) Consume(ctx context.Context, records []record) error { + type parsedRecord struct { + *mimirpb.WriteRequest + // ctx holds the tracing baggage for this record/request. + ctx context.Context + tenantID string + err error + index int + } + + recordsChannel := make(chan parsedRecord) + + // Create a cancellable context to let the unmarshalling goroutine know when to stop. ctx, cancel := context.WithCancelCause(ctx) - defer cancel(cancellation.NewErrorf("done consuming records")) - recC := make(chan parsedRecord) + // Now, unmarshal the records into the channel. + go func(unmarshalCtx context.Context, records []record, ch chan<- parsedRecord) { + defer close(ch) - // Speed up consumption by unmarhsalling the next request while the previous one is being pushed. - go c.unmarshalRequests(ctx, records, recC) - return c.pushRequests(recC) -} + for index, r := range records { + // Before we being unmarshalling the write request check if the context was cancelled. + select { + case <-unmarshalCtx.Done(): + // No more processing is needed, so we need to abort. + return + default: + } -func (c pusherConsumer) pushRequests(reqC <-chan parsedRecord) error { - recordIdx := -1 - for wr := range reqC { - recordIdx++ - if wr.err != nil { - level.Error(c.logger).Log("msg", "failed to parse write request; skipping", "err", wr.err) + parsed := parsedRecord{ + ctx: r.ctx, + tenantID: r.tenantID, + WriteRequest: &mimirpb.WriteRequest{}, + index: index, + } + + // We don't free the WriteRequest slices because they are being freed by a level below. + err := parsed.WriteRequest.Unmarshal(r.content) + if err != nil { + parsed.err = fmt.Errorf("parsing ingest consumer write request: %w", err) + } + + // Now that we're done, check again before we send it to the channel. + select { + case <-unmarshalCtx.Done(): + return + case ch <- parsed: + } + } + }(ctx, records, recordsChannel) + + writer := c.newStorageWriter() + for r := range recordsChannel { + if r.err != nil { + level.Error(spanlogger.FromContext(ctx, c.logger)).Log("msg", "failed to parse write request; skipping", "err", r.err) continue } - err := c.pushToStorage(wr.ctx, wr.tenantID, wr.WriteRequest) + // If we get an error at any point, we need to stop processing the records. They will be retried at some point. + err := c.pushToStorage(r.ctx, r.tenantID, r.WriteRequest, writer) if err != nil { - return fmt.Errorf("consuming record at index %d for tenant %s: %w", recordIdx, wr.tenantID, err) + cancel(cancellation.NewErrorf("error while pushing to storage")) // Stop the unmarshalling goroutine. + return fmt.Errorf("consuming record at index %d for tenant %s: %w", r.index, r.tenantID, err) } } - return nil + + cancel(cancellation.NewErrorf("done unmarshalling records")) + + // We need to tell the storage writer that we're done and no more records are coming. + // err := c.close(ctx, writer) + spanLog := spanlogger.FromContext(ctx, log.NewNopLogger()) + errs := writer.Close() + for eIdx := 0; eIdx < len(errs); eIdx++ { + err := errs[eIdx] + isServerErr := c.handlePushErr(ctx, "TODO", err, spanLog) + if !isServerErr { + errs[len(errs)-1], errs[eIdx] = errs[eIdx], errs[len(errs)-1] + errs = errs[:len(errs)-1] + eIdx-- + } + } + + return multierror.New(errs...).Err() } -func (c pusherConsumer) pushToStorage(ctx context.Context, tenantID string, req *mimirpb.WriteRequest) error { +func (c pusherConsumer) newStorageWriter() PusherCloser { + if c.kafkaConfig.ReplayShards == 0 { + return newSequentialStoragePusher(c.metrics, c.pusher) + } + + return newParallelStoragePusher(c.metrics, c.pusher, c.kafkaConfig.ReplayShards, c.kafkaConfig.BatchSize, c.logger) +} + +func (c pusherConsumer) pushToStorage(ctx context.Context, tenantID string, req *mimirpb.WriteRequest, writer PusherCloser) error { spanLog, ctx := spanlogger.NewWithLogger(ctx, c.logger, "pusherConsumer.pushToStorage") defer spanLog.Finish() @@ -108,31 +199,42 @@ func (c pusherConsumer) pushToStorage(ctx context.Context, tenantID string, req // Note that the implementation of the Pusher expects the tenantID to be in the context. ctx = user.InjectOrgID(ctx, tenantID) - err := c.pusher.PushToStorage(ctx, req) + err := writer.PushToStorage(ctx, req) - c.processingTimeSeconds.Observe(time.Since(processingStart).Seconds()) - c.totalRequests.Inc() + // TODO dimitarvdimitrov processing time is flawed because it's only counting enqueuing time, not processing time. + c.metrics.processingTimeSeconds.Observe(time.Since(processingStart).Seconds()) + c.metrics.totalRequests.Inc() - if err != nil { - // Only return non-client errors; these will stop the processing of the current Kafka fetches and retry (possibly). - if !mimirpb.IsClientError(err) { - c.serverErrRequests.Inc() - return spanLog.Error(err) - } + isServerErr := c.handlePushErr(ctx, tenantID, err, spanLog) + if isServerErr { + return err + } + return nil +} - c.clientErrRequests.Inc() +func (c pusherConsumer) handlePushErr(ctx context.Context, tenantID string, err error, spanLog *spanlogger.SpanLogger) bool { + if err == nil { + return false + } + // Only return non-client errors; these will stop the processing of the current Kafka fetches and retry (possibly). + if !mimirpb.IsClientError(err) { + c.metrics.serverErrRequests.Inc() + _ = spanLog.Error(err) + return true + } - // The error could be sampled or marked to be skipped in logs, so we check whether it should be - // logged before doing it. - if keep, reason := c.shouldLogClientError(ctx, err); keep { - if reason != "" { - err = fmt.Errorf("%w (%s)", err, reason) - } - // This error message is consistent with error message in Prometheus remote-write and OTLP handlers in distributors. - level.Warn(spanLog).Log("msg", "detected a client error while ingesting write request (the request may have been partially ingested)", "user", tenantID, "insight", true, "err", err) + c.metrics.clientErrRequests.Inc() + + // The error could be sampled or marked to be skipped in logs, so we check whether it should be + // logged before doing it. + if keep, reason := c.shouldLogClientError(ctx, err); keep { + if reason != "" { + err = fmt.Errorf("%w (%s)", err, reason) } + // This error message is consistent with error message in Prometheus remote-write and OTLP handlers in distributors. + level.Warn(spanLog).Log("msg", "detected a client error while ingesting write request (the request may have been partially ingested)", "user", tenantID, "insight", true, "err", err) } - return nil + return false } // shouldLogClientError returns whether err should be logged. @@ -150,34 +252,299 @@ func (c pusherConsumer) shouldLogClientError(ctx context.Context, err error) (bo return optional.ShouldLog(ctx) } -// The passed context is expected to be cancelled after all items in records were fully processed and are ready -// to be released. This so to guaranty the release of resources associated with each parsedRecord context. -func (c pusherConsumer) unmarshalRequests(ctx context.Context, records []record, recC chan<- parsedRecord) { - defer close(recC) - done := ctx.Done() - - for _, rec := range records { - // rec.ctx contains the tracing baggage for this record, which we propagate down the call tree. - // Since rec.ctx cancellation is disjointed from the context passed to unmarshalRequests(), the context.AfterFunc below, - // fuses the two lifetimes together. - recCtx, cancelRecCtx := context.WithCancelCause(rec.ctx) - context.AfterFunc(ctx, func() { - cancelRecCtx(context.Cause(ctx)) - }) - pRecord := parsedRecord{ - ctx: recCtx, - tenantID: rec.tenantID, - WriteRequest: &mimirpb.WriteRequest{}, +// sequentialStoragePusher receives mimirpb.WriteRequest which are then pushed to the storage one by one. +type sequentialStoragePusher struct { + metrics *pusherConsumerMetrics + + pusher Pusher +} + +// newSequentialStoragePusher creates a new sequentialStoragePusher instance. +func newSequentialStoragePusher(metrics *pusherConsumerMetrics, pusher Pusher) sequentialStoragePusher { + return sequentialStoragePusher{ + metrics: metrics, + pusher: pusher, + } +} + +// PushToStorage implements the PusherCloser interface. +func (ssp sequentialStoragePusher) PushToStorage(ctx context.Context, wr *mimirpb.WriteRequest) error { + // TODO: What about time?? + ssp.metrics.numTimeSeriesPerFlush.Observe(float64(len(wr.Timeseries))) + return ssp.pusher.PushToStorage(ctx, wr) +} + +// Close implements the PusherCloser interface. +func (ssp sequentialStoragePusher) Close() []error { + return nil +} + +// parallelStoragePusher receives WriteRequest which are then pushed to the storage in parallel. +// The parallelism is two-tiered which means that we first parallelize by tenantID and then by series. +type parallelStoragePusher struct { + metrics *pusherConsumerMetrics + logger log.Logger + + pushers map[string]*parallelStorageShards + upstreamPusher Pusher + numShards int + batchSize int +} + +// newParallelStoragePusher creates a new parallelStoragePusher instance. +func newParallelStoragePusher(metrics *pusherConsumerMetrics, pusher Pusher, numShards int, batchSize int, logger log.Logger) *parallelStoragePusher { + return ¶llelStoragePusher{ + logger: log.With(logger, "component", "parallel-storage-pusher"), + pushers: make(map[string]*parallelStorageShards), + upstreamPusher: pusher, + numShards: numShards, + batchSize: batchSize, + metrics: metrics, + } +} + +// PushToStorage implements the PusherCloser interface. +func (c parallelStoragePusher) PushToStorage(ctx context.Context, wr *mimirpb.WriteRequest) error { + userID, err := user.ExtractOrgID(ctx) + if err != nil { + level.Error(c.logger).Log("msg", "failed to extract tenant ID from context", "err", err) + } + + shards := c.shardsFor(userID) + return shards.ShardWriteRequest(ctx, wr) +} + +// Close implements the PusherCloser interface. +func (c parallelStoragePusher) Close() []error { + var errs multierror.MultiError + for _, p := range c.pushers { + errs.Add(p.Stop()) + } + clear(c.pushers) + return errs +} + +// shardsFor returns the parallelStorageShards for the given userID. Once created the same shards are re-used for the same userID. +// We create a shard for each tenantID to parallelize the writes. +func (c parallelStoragePusher) shardsFor(userID string) *parallelStorageShards { + if p := c.pushers[userID]; p != nil { + return p + } + // Use the same hashing function that's used for stripes in the TSDB. That way we make use of the low-contention property of stripes. + hashLabels := labels.Labels.Hash + p := newParallelStorageShards(c.metrics.numTimeSeriesPerFlush, c.numShards, c.batchSize, shardForSeriesBuffer, c.upstreamPusher, hashLabels) + c.pushers[userID] = p + return p +} + +type labelsHashFunc func(labels.Labels) uint64 + +// parallelStorageShards is a collection of shards that are used to parallelize the writes to the storage by series. +// Each series is hashed to a shard that contains its own batchingQueue. +type parallelStorageShards struct { + numTimeSeriesPerFlush prometheus.Histogram + + pusher Pusher + hashLabels labelsHashFunc + + numShards int + batchSize int + capacity int + + wg *sync.WaitGroup + shards []*batchingQueue +} + +type flushableWriteRequest struct { + *mimirpb.WriteRequest + context.Context +} + +// newParallelStorageShards creates a new parallelStorageShards instance. +func newParallelStorageShards(numTimeSeriesPerFlush prometheus.Histogram, numShards int, batchSize int, capacity int, pusher Pusher, hashLabels labelsHashFunc) *parallelStorageShards { + p := ¶llelStorageShards{ + numShards: numShards, + pusher: pusher, + hashLabels: hashLabels, + capacity: capacity, + numTimeSeriesPerFlush: numTimeSeriesPerFlush, + batchSize: batchSize, + wg: &sync.WaitGroup{}, + } + + p.start() + + return p +} + +// ShardWriteRequest hashes each time series in the write requests and sends them to the appropriate shard which is then handled by the current batchingQueue in that shard. +func (p *parallelStorageShards) ShardWriteRequest(ctx context.Context, request *mimirpb.WriteRequest) error { + var ( + builder labels.ScratchBuilder + nonCopiedLabels labels.Labels + errs multierror.MultiError + ) + + for _, ts := range request.Timeseries { + mimirpb.FromLabelAdaptersOverwriteLabels(&builder, ts.Labels, &nonCopiedLabels) + shard := nonCopiedLabels.Hash() % uint64(p.numShards) + + // TODO: Add metrics to measure how long are items sitting in the queue before they are flushed. + // TODO dimitarvdimitrov support metadata and the rest of the fields; perhaps cut a new request for different values of SkipLabelNameValidation? + if err := p.shards[shard].AddToBatch(ctx, ts); err != nil { + // TODO: Technically, we should determine at this point what type of error it is and abort the whole push if it's a server error. + // We'll do that in the next PR as otherwise it's too many changes right now. + if !mimirpb.IsClientError(err) { + return err + } + + errs.Add(err) } - // We don't free the WriteRequest slices because they are being freed by the Pusher. - err := pRecord.WriteRequest.Unmarshal(rec.content) + } + + // We might some data left in some of the queues in the shards, but they will be flushed eventually once Stop is called, and we're certain that no more data is coming. + // Return whatever errors we have now, we'll call stop eventually and collect the rest. + return errs.Err() +} + +// Stop stops all the shards and waits for them to finish. +func (p *parallelStorageShards) Stop() error { + var errs multierror.MultiError + + for _, shard := range p.shards { + errs.Add(shard.Close()) + } + + p.wg.Wait() + + return errs.Err() +} + +// start starts the shards, each in its own goroutine. +func (p *parallelStorageShards) start() { + shards := make([]*batchingQueue, p.numShards) + p.wg.Add(p.numShards) + + for i := range shards { + shards[i] = newBatchingQueue(p.capacity, p.batchSize) + go p.run(shards[i]) + } + + p.shards = shards +} + +// run runs the batchingQueue for the shard. +func (p *parallelStorageShards) run(queue *batchingQueue) { + defer p.wg.Done() + defer queue.Done() + + for wr := range queue.Channel() { + p.numTimeSeriesPerFlush.Observe(float64(len(wr.WriteRequest.Timeseries))) + err := p.pusher.PushToStorage(wr.Context, wr.WriteRequest) if err != nil { - pRecord.err = fmt.Errorf("parsing ingest consumer write request: %w", err) + queue.ErrorChannel() <- err } + } +} + +// batchingQueue is a queue that batches the incoming time series according to the batch size. +// Once the batch size is reached, the batch is pushed to a channel which can be accessed through the Channel() method. +type batchingQueue struct { + ch chan flushableWriteRequest + errCh chan error + done chan struct{} + + currentBatch flushableWriteRequest + batchSize int +} + +// newBatchingQueue creates a new batchingQueue instance. +func newBatchingQueue(capacity int, batchSize int) *batchingQueue { + return &batchingQueue{ + ch: make(chan flushableWriteRequest, capacity), + errCh: make(chan error, capacity), + done: make(chan struct{}), + currentBatch: flushableWriteRequest{WriteRequest: &mimirpb.WriteRequest{Timeseries: mimirpb.PreallocTimeseriesSliceFromPool()}}, + batchSize: batchSize, + } +} + +// AddToBatch adds a time series to the current batch. If the batch size is reached, the batch is pushed to the Channel(). +// If an error occurs while pushing the batch, it returns the error and ensures the batch is pushed. +func (q *batchingQueue) AddToBatch(ctx context.Context, ts mimirpb.PreallocTimeseries) error { + q.currentBatch.Timeseries = append(q.currentBatch.Timeseries, ts) + q.currentBatch.Context = ctx + + if len(q.currentBatch.Timeseries) >= q.batchSize { + if err := q.push(); err != nil { + return err + } + } + + return nil +} + +// Close closes the batchingQueue, it'll push the current branch to the channel if it's not empty. +// and then close the channel. +func (q *batchingQueue) Close() error { + var errs multierror.MultiError + if len(q.currentBatch.Timeseries) > 0 { + if err := q.push(); err != nil { + errs.Add(err) + } + } + + close(q.ch) + <-q.done + + errs = append(errs, q.collectErrors()...) + close(q.errCh) + return errs.Err() +} + +// Channel returns the channel where the batches are pushed. +func (q *batchingQueue) Channel() <-chan flushableWriteRequest { + return q.ch +} + +// ErrorChannel returns the channel where errors are pushed. +func (q *batchingQueue) ErrorChannel() chan<- error { + return q.errCh +} + +// Done signals the queue that there is no more data coming for both the channel and the error channel. +// It is necessary to ensure we don't close the channel before all the data is flushed. +func (q *batchingQueue) Done() { + close(q.done) +} + +// push pushes the current batch to the channel and resets the current batch. +// It also collects any errors that might have occurred while pushing the batch. +func (q *batchingQueue) push() error { + errs := q.collectErrors() + + q.ch <- q.currentBatch + q.resetCurrentBatch() + + return errs.Err() +} + +// resetCurrentBatch resets the current batch to an empty state. +func (q *batchingQueue) resetCurrentBatch() { + q.currentBatch = flushableWriteRequest{ + WriteRequest: &mimirpb.WriteRequest{Timeseries: mimirpb.PreallocTimeseriesSliceFromPool()}, + } +} + +func (q *batchingQueue) collectErrors() multierror.MultiError { + var errs multierror.MultiError + + for { select { - case <-done: - return - case recC <- pRecord: + case err := <-q.errCh: + errs.Add(err) + default: + return errs } } } diff --git a/pkg/storage/ingest/pusher_test.go b/pkg/storage/ingest/pusher_test.go index 626cc9a1d0b..c23f2f445f2 100644 --- a/pkg/storage/ingest/pusher_test.go +++ b/pkg/storage/ingest/pusher_test.go @@ -7,18 +7,22 @@ import ( "fmt" "strings" "testing" + "time" "github.com/go-kit/log" "github.com/gogo/status" - "github.com/grafana/dskit/cancellation" "github.com/grafana/dskit/concurrency" "github.com/grafana/dskit/middleware" "github.com/grafana/dskit/tenant" "github.com/grafana/regexp" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" "github.com/prometheus/client_golang/prometheus/testutil" + "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" + "go.uber.org/atomic" "google.golang.org/grpc/codes" "github.com/grafana/mimir/pkg/mimirpb" @@ -27,6 +31,10 @@ import ( type pusherFunc func(context.Context, *mimirpb.WriteRequest) error +func (p pusherFunc) Close() []error { + return nil +} + func (p pusherFunc) PushToStorage(ctx context.Context, request *mimirpb.WriteRequest) error { return p(ctx, request) } @@ -34,7 +42,7 @@ func (p pusherFunc) PushToStorage(ctx context.Context, request *mimirpb.WriteReq func TestPusherConsumer(t *testing.T) { const tenantID = "t1" writeReqs := []*mimirpb.WriteRequest{ - {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_1"), mockPreallocTimeseries("series_2")}}, + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_1")}}, {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_3")}}, {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_4")}}, {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_5")}}, @@ -181,12 +189,12 @@ func TestPusherConsumer(t *testing.T) { for name, tc := range testCases { t.Run(name, func(t *testing.T) { - receivedReqs := 0 + receivedReqs := atomic.NewInt64(0) pusher := pusherFunc(func(ctx context.Context, request *mimirpb.WriteRequest) error { - defer func() { receivedReqs++ }() - require.GreaterOrEqualf(t, len(tc.expectedWRs), receivedReqs+1, "received more requests (%d) than expected (%d)", receivedReqs+1, len(tc.expectedWRs)) + reqIdx := int(receivedReqs.Inc() - 1) + require.GreaterOrEqualf(t, len(tc.expectedWRs), reqIdx+1, "received more requests (%d) than expected (%d)", reqIdx+1, len(tc.expectedWRs)) - expectedWR := tc.expectedWRs[receivedReqs] + expectedWR := tc.expectedWRs[reqIdx] for i, ts := range request.Timeseries { assert.Truef(t, ts.Equal(expectedWR.Timeseries[i].TimeSeries), "timeseries %d not equal; got %v, expected %v", i, ts, expectedWR.Timeseries[i].TimeSeries) } @@ -195,12 +203,12 @@ func TestPusherConsumer(t *testing.T) { assert.NoError(t, err) assert.Equal(t, tenantID, actualTenantID) - return tc.responses[receivedReqs].err + return tc.responses[reqIdx].err }) logs := &concurrency.SyncBuffer{} - c := newPusherConsumer(pusher, nil, prometheus.NewPedanticRegistry(), log.NewLogfmtLogger(logs)) - err := c.consume(context.Background(), tc.records) + c := newPusherConsumer(pusher, KafkaConfig{}, newPusherConsumerMetrics(prometheus.NewPedanticRegistry()), log.NewLogfmtLogger(logs)) + err := c.Consume(context.Background(), tc.records) if tc.expErr == "" { assert.NoError(t, err) } else { @@ -217,13 +225,13 @@ func TestPusherConsumer(t *testing.T) { } } -var unimportantLogFieldsPattern = regexp.MustCompile(`\scaller=\S+\.go:\d+\s`) +var unimportantLogFieldsPattern = regexp.MustCompile(`(\s?)caller=\S+\.go:\d+\s`) func removeUnimportantLogFields(lines []string) []string { // The 'caller' field is not important to these tests (we just care about the message and other information), // and can change as we refactor code, making these tests brittle. So we remove it before making assertions about the log lines. for i, line := range lines { - lines[i] = unimportantLogFieldsPattern.ReplaceAllString(line, " ") + lines[i] = unimportantLogFieldsPattern.ReplaceAllString(line, "$1") } return lines @@ -269,7 +277,8 @@ func TestPusherConsumer_clientErrorSampling(t *testing.T) { }, } { t.Run(name, func(t *testing.T) { - c := newPusherConsumer(nil, tc.sampler, prometheus.NewPedanticRegistry(), log.NewNopLogger()) + c := newPusherConsumer(nil, KafkaConfig{}, newPusherConsumerMetrics(prometheus.NewPedanticRegistry()), log.NewNopLogger()) + c.fallbackClientErrSampler = tc.sampler sampled, reason := c.shouldLogClientError(context.Background(), tc.err) assert.Equal(t, tc.expectedSampled, sampled) @@ -298,7 +307,7 @@ func TestPusherConsumer_consume_ShouldLogErrorsHonoringOptionalLogging(t *testin reg := prometheus.NewPedanticRegistry() logs := &concurrency.SyncBuffer{} - consumer := newPusherConsumer(pusher, nil, reg, log.NewLogfmtLogger(logs)) + consumer := newPusherConsumer(pusher, KafkaConfig{}, newPusherConsumerMetrics(reg), log.NewLogfmtLogger(logs)) return consumer, logs, reg } @@ -308,7 +317,7 @@ func TestPusherConsumer_consume_ShouldLogErrorsHonoringOptionalLogging(t *testin consumer, logs, reg := setupTest(pusherErr) // Should return no error on client errors. - require.NoError(t, consumer.consume(context.Background(), []record{reqRecord})) + require.NoError(t, consumer.Consume(context.Background(), []record{reqRecord})) assert.Contains(t, logs.String(), pusherErr.Error()) assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` @@ -330,7 +339,7 @@ func TestPusherConsumer_consume_ShouldLogErrorsHonoringOptionalLogging(t *testin consumer, logs, reg := setupTest(pusherErr) // Should return no error on client errors. - require.NoError(t, consumer.consume(context.Background(), []record{reqRecord})) + require.NoError(t, consumer.Consume(context.Background(), []record{reqRecord})) assert.Contains(t, logs.String(), fmt.Sprintf("%s (sampled 1/100)", pusherErr.Error())) assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` @@ -351,7 +360,7 @@ func TestPusherConsumer_consume_ShouldLogErrorsHonoringOptionalLogging(t *testin consumer, logs, reg := setupTest(pusherErr) // Should return no error on client errors. - require.NoError(t, consumer.consume(context.Background(), []record{reqRecord})) + require.NoError(t, consumer.Consume(context.Background(), []record{reqRecord})) assert.Empty(t, logs.String()) assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` @@ -364,48 +373,449 @@ func TestPusherConsumer_consume_ShouldLogErrorsHonoringOptionalLogging(t *testin } -func TestPusherConsumer_consume_ShouldHonorContextCancellation(t *testing.T) { - // Create a request that will be used in this test; the content doesn't matter, - // since we only test errors. - req := &mimirpb.WriteRequest{Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_1")}} - reqBytes, err := req.Marshal() - require.NoError(t, err) +// ingesterError mimics how the ingester construct errors +func ingesterError(cause mimirpb.ErrorCause, statusCode codes.Code, message string) error { + errorDetails := &mimirpb.ErrorDetails{Cause: cause} + statWithDetails, err := status.New(statusCode, message).WithDetails(errorDetails) + if err != nil { + panic(err) + } + return statWithDetails.Err() +} - reqRecord := record{ - ctx: context.Background(), // The record's context isn't important for the test. - tenantID: "user-1", - content: reqBytes, +type mockPusher struct { + mock.Mock +} + +func (m *mockPusher) PushToStorage(ctx context.Context, request *mimirpb.WriteRequest) error { + args := m.Called(ctx, request) + return args.Error(0) +} + +func TestParallelStorageShards_ShardWriteRequest(t *testing.T) { + noopHistogram := promauto.With(prometheus.NewRegistry()).NewHistogram(prometheus.HistogramOpts{Name: "noop", NativeHistogramBucketFactor: 1.1}) + + testCases := map[string]struct { + shardCount int + batchSize int + requests []*mimirpb.WriteRequest + expectedErrs []error + expectedErrsCount int + + expectedUpstreamPushes []*mimirpb.WriteRequest + upstreamPushErrs []error + expectedCloseErr error + }{ + "push to a single shard and fill exactly capacity": { + shardCount: 1, + batchSize: 2, + requests: []*mimirpb.WriteRequest{ + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_1_1")}}, + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_2_1")}}, + }, + expectedErrs: []error{nil, nil}, + + expectedUpstreamPushes: []*mimirpb.WriteRequest{ + {Timeseries: []mimirpb.PreallocTimeseries{ + mockPreallocTimeseries("series_1_1"), + mockPreallocTimeseries("series_2_1"), + }}, + }, + upstreamPushErrs: []error{nil}, + expectedCloseErr: nil, + }, + "push to multiple shards and fill exact capacity": { + shardCount: 2, + batchSize: 2, + requests: []*mimirpb.WriteRequest{ + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_1_2")}}, + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_2_2")}}, + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_3_2")}}, + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_4_2")}}, + }, + expectedErrs: []error{nil, nil, nil, nil}, + + expectedUpstreamPushes: []*mimirpb.WriteRequest{ + {Timeseries: []mimirpb.PreallocTimeseries{ + mockPreallocTimeseries("series_1_2"), + mockPreallocTimeseries("series_3_2"), + }}, + {Timeseries: []mimirpb.PreallocTimeseries{ + mockPreallocTimeseries("series_2_2"), + mockPreallocTimeseries("series_4_2"), + }}, + }, + upstreamPushErrs: []error{nil, nil}, + expectedCloseErr: nil, + }, + "push to single shard and underfill capacity": { + shardCount: 1, + batchSize: 2, + requests: []*mimirpb.WriteRequest{ + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_1_3")}}, + }, + expectedErrs: []error{nil}, + + expectedUpstreamPushes: []*mimirpb.WriteRequest{ + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_1_3")}}, + }, + upstreamPushErrs: []error{nil}, + expectedCloseErr: nil, + }, + "push to single shard and overfill capacity": { + shardCount: 1, + batchSize: 2, + requests: []*mimirpb.WriteRequest{ + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_1_4")}}, + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_2_4")}}, + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_3_4")}}, + }, + expectedErrs: []error{nil, nil, nil}, + + expectedUpstreamPushes: []*mimirpb.WriteRequest{ + {Timeseries: []mimirpb.PreallocTimeseries{ + mockPreallocTimeseries("series_1_4"), + mockPreallocTimeseries("series_2_4"), + }}, + {Timeseries: []mimirpb.PreallocTimeseries{ + mockPreallocTimeseries("series_3_4"), + }}, + }, + upstreamPushErrs: []error{nil, nil}, + expectedCloseErr: nil, + }, + "push to single shard and overfill only with the series of a singe request": { + shardCount: 1, + batchSize: 2, + requests: []*mimirpb.WriteRequest{ + {Timeseries: []mimirpb.PreallocTimeseries{ + mockPreallocTimeseries("series_1_5"), + mockPreallocTimeseries("series_2_5"), + mockPreallocTimeseries("series_3_5"), + }}, + }, + expectedErrs: []error{nil}, + + expectedUpstreamPushes: []*mimirpb.WriteRequest{ + {Timeseries: []mimirpb.PreallocTimeseries{ + mockPreallocTimeseries("series_1_5"), + mockPreallocTimeseries("series_2_5"), + }}, + {Timeseries: []mimirpb.PreallocTimeseries{ + mockPreallocTimeseries("series_3_5"), + }}, + }, + upstreamPushErrs: []error{nil, nil}, + expectedCloseErr: nil, + }, + "push to multiple shards and overfill capacity on one shard and underfill on another": { + shardCount: 2, + batchSize: 2, + requests: []*mimirpb.WriteRequest{ + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_1_6")}}, + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_2_6")}}, + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_3_6")}}, + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_3_6")}}, + }, + + expectedErrs: []error{nil, nil, nil, nil}, + expectedUpstreamPushes: []*mimirpb.WriteRequest{ + {Timeseries: []mimirpb.PreallocTimeseries{ + mockPreallocTimeseries("series_1_6"), + mockPreallocTimeseries("series_3_6"), + }}, + {Timeseries: []mimirpb.PreallocTimeseries{ + mockPreallocTimeseries("series_2_6"), + }}, + {Timeseries: []mimirpb.PreallocTimeseries{ + mockPreallocTimeseries("series_3_6"), + }}, + }, + upstreamPushErrs: []error{nil, nil, nil}, + expectedCloseErr: nil, + }, + "push to single shard and get an error with an underfilled shard (i.e. when calling Close() on the Pusher)": { + shardCount: 1, + batchSize: 2, + requests: []*mimirpb.WriteRequest{ + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_1_7")}}, + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_2_7")}}, + }, + expectedErrs: []error{nil, nil}, + + expectedUpstreamPushes: []*mimirpb.WriteRequest{ + {Timeseries: []mimirpb.PreallocTimeseries{ + mockPreallocTimeseries("series_1_7"), + mockPreallocTimeseries("series_2_7"), + }}, + }, + upstreamPushErrs: []error{assert.AnError}, + expectedCloseErr: assert.AnError, + }, + "push to single shard and get an error with an overfilled shard (i.e. during some of the pushes)": { + shardCount: 1, + batchSize: 2, + requests: []*mimirpb.WriteRequest{ + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_1_8")}}, + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_2_8")}}, + + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_3_8")}}, + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_3_8")}}, + + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_3_8")}}, + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_3_8")}}, + + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_3_8")}}, + {Timeseries: []mimirpb.PreallocTimeseries{mockPreallocTimeseries("series_3_8")}}, + }, + expectedErrsCount: 1, // at least one of those should fail because the first flush failed + + expectedUpstreamPushes: []*mimirpb.WriteRequest{ + {Timeseries: []mimirpb.PreallocTimeseries{ + mockPreallocTimeseries("series_1_8"), + mockPreallocTimeseries("series_2_8"), + }}, + {Timeseries: []mimirpb.PreallocTimeseries{ + mockPreallocTimeseries("series_3_8"), + mockPreallocTimeseries("series_3_8"), + }}, + {Timeseries: []mimirpb.PreallocTimeseries{ + mockPreallocTimeseries("series_3_8"), + mockPreallocTimeseries("series_3_8"), + }}, + {Timeseries: []mimirpb.PreallocTimeseries{ + mockPreallocTimeseries("series_3_8"), + mockPreallocTimeseries("series_3_8"), + }}, + }, + upstreamPushErrs: []error{assert.AnError, nil, nil, nil}, + expectedCloseErr: nil, + }, } - // didPush signals that the testing record was pushed to the pusher. - didPush := make(chan struct{}, 1) - pusher := pusherFunc(func(ctx context.Context, _ *mimirpb.WriteRequest) error { - close(didPush) - <-ctx.Done() - return context.Cause(ctx) + for name, tc := range testCases { + tc := tc + t.Run(name, func(t *testing.T) { + require.Equal(t, len(tc.expectedUpstreamPushes), len(tc.upstreamPushErrs)) + if len(tc.expectedErrs) != 0 && tc.expectedErrsCount > 0 { + require.Fail(t, "expectedErrs and expectedErrsCount are mutually exclusive") + } + if len(tc.expectedErrs) != 0 { + require.Equal(t, len(tc.expectedErrs), len(tc.requests)) + } + + pusher := &mockPusher{} + // run with a buffer of one, so some of the tests can fill the buffer and test the error handling + const buffer = 1 + shardingP := newParallelStorageShards(noopHistogram, tc.shardCount, tc.batchSize, buffer, pusher, labels.StableHash) + + for i, req := range tc.expectedUpstreamPushes { + pusher.On("PushToStorage", mock.Anything, req).Return(tc.upstreamPushErrs[i]) + } + var actualPushErrs []error + for _, req := range tc.requests { + err := shardingP.ShardWriteRequest(context.Background(), req) + actualPushErrs = append(actualPushErrs, err) + } + + if len(tc.expectedErrs) > 0 { + require.Equal(t, tc.expectedErrs, actualPushErrs) + } else { + receivedErrs := 0 + for _, err := range actualPushErrs { + if err != nil { + receivedErrs++ + } + } + require.Equalf(t, tc.expectedErrsCount, receivedErrs, "received %d errors instead of %d: %v", receivedErrs, tc.expectedErrsCount, actualPushErrs) + } + + closeErr := shardingP.Stop() + require.ErrorIs(t, closeErr, tc.expectedCloseErr) + pusher.AssertNumberOfCalls(t, "PushToStorage", len(tc.expectedUpstreamPushes)) + pusher.AssertExpectations(t) + }) + } +} + +func TestBatchingQueue(t *testing.T) { + capacity := 5 + batchSize := 3 + + series1 := mockPreallocTimeseries("series_1") + series2 := mockPreallocTimeseries("series_2") + + series := []mimirpb.PreallocTimeseries{series1, series2} + + t.Run("batch not flushed because batch size is 3 and we have 2 items in the queue", func(t *testing.T) { + queue := setupQueue(t, capacity, batchSize, series) + + select { + case <-queue.Channel(): + t.Fatal("expected batch to not be flushed") + case <-time.After(100 * time.Millisecond): + } + }) + + t.Run("batch flushed because batch size is 3 and we have 3 items in the queue", func(t *testing.T) { + queue := setupQueue(t, capacity, batchSize, series) + + series3 := mockPreallocTimeseries("series_3") + require.NoError(t, queue.AddToBatch(context.Background(), series3)) + + select { + case batch := <-queue.Channel(): + require.Len(t, batch.WriteRequest.Timeseries, 3) + require.Equal(t, series1, batch.WriteRequest.Timeseries[0]) + require.Equal(t, series2, batch.WriteRequest.Timeseries[1]) + require.Equal(t, series3, batch.WriteRequest.Timeseries[2]) + case <-time.After(time.Second): + t.Fatal("expected batch to be flushed") + } + + // after the batch is flushed, the queue should be empty. + require.Len(t, queue.currentBatch.Timeseries, 0) }) - consumer := newPusherConsumer(pusher, nil, prometheus.NewPedanticRegistry(), log.NewNopLogger()) - wantCancelErr := cancellation.NewErrorf("stop") + t.Run("if you close the queue with items in the queue, the queue should flush the items", func(t *testing.T) { + queue := setupQueue(t, capacity, batchSize, series) + + // Channel is empty. + select { + case <-queue.Channel(): + t.Fatal("expected batch to not be flushed") + case <-time.After(100 * time.Millisecond): + } + + // Read in a separate goroutine as when we close the queue, the channel will be closed. + var batch flushableWriteRequest + go func() { + defer queue.Done() + for b := range queue.Channel() { + batch = b + queue.ErrorChannel() <- nil + } + }() + + // Close the queue, and the items should be flushed. + require.NoError(t, queue.Close()) + + require.Len(t, batch.WriteRequest.Timeseries, 2) + require.Equal(t, series1, batch.WriteRequest.Timeseries[0]) + require.Equal(t, series2, batch.WriteRequest.Timeseries[1]) + }) + + t.Run("test queue capacity", func(t *testing.T) { + queue := setupQueue(t, capacity, batchSize, nil) + + // Queue channel is empty because there are only 2 items in the current currentBatch. + require.Len(t, queue.ch, 0) + require.Len(t, queue.currentBatch.Timeseries, 0) - // For this test, cancelling the top-most context must cancel an in-flight call to push, - // to prevent pusher from hanging forever. - canceledCtx, cancel := context.WithCancelCause(context.Background()) - go func() { - <-didPush - cancel(wantCancelErr) - }() + // Add items to the queue until it's full. + for i := 0; i < capacity*batchSize; i++ { + s := mockPreallocTimeseries(fmt.Sprintf("series_%d", i)) + require.NoError(t, queue.AddToBatch(context.Background(), s)) + } - err = consumer.consume(canceledCtx, []record{reqRecord}) - require.ErrorIs(t, err, wantCancelErr) + // We should have 5 items in the queue channel and 0 items in the currentBatch. + require.Len(t, queue.ch, 5) + require.Len(t, queue.currentBatch.Timeseries, 0) + + // Read one item to free up a queue space. + batch := <-queue.Channel() + require.Len(t, batch.WriteRequest.Timeseries, 3) + + // Queue should have 4 items now and the currentBatch remains the same. + require.Len(t, queue.ch, 4) + require.Len(t, queue.currentBatch.Timeseries, 0) + + // Add three more items to fill up the queue again, this shouldn't block. + s := mockPreallocTimeseries("series_100") + require.NoError(t, queue.AddToBatch(context.Background(), s)) + require.NoError(t, queue.AddToBatch(context.Background(), s)) + require.NoError(t, queue.AddToBatch(context.Background(), s)) + + require.Len(t, queue.ch, 5) + require.Len(t, queue.currentBatch.Timeseries, 0) + }) } -// ingesterError mimics how the ingester construct errors -func ingesterError(cause mimirpb.ErrorCause, statusCode codes.Code, message string) error { - errorDetails := &mimirpb.ErrorDetails{Cause: cause} - statWithDetails, err := status.New(statusCode, message).WithDetails(errorDetails) - if err != nil { - panic(err) +func TestBatchingQueue_ErrorHandling(t *testing.T) { + capacity := 2 + batchSize := 2 + series1 := mockPreallocTimeseries("series_1") + series2 := mockPreallocTimeseries("series_2") + + t.Run("AddToBatch returns all errors and it pushes the batch when the batch is filled ", func(t *testing.T) { + queue := setupQueue(t, capacity, batchSize, nil) + ctx := context.Background() + + // Push 1 series so that the next push will complete the batch. + require.NoError(t, queue.AddToBatch(ctx, series2)) + + // Push an error to fill the error channel. + queue.ErrorChannel() <- fmt.Errorf("mock error 1") + queue.ErrorChannel() <- fmt.Errorf("mock error 2") + + // AddToBatch should return an error now. + err := queue.AddToBatch(ctx, series2) + assert.Equal(t, "2 errors: mock error 1; mock error 2", err.Error()) + // Also the batch was pushed. + select { + case batch := <-queue.Channel(): + require.Equal(t, series2, batch.WriteRequest.Timeseries[0]) + require.Equal(t, series2, batch.WriteRequest.Timeseries[1]) + default: + t.Fatal("expected batch to be flushed") + } + + // AddToBatch should work again. + require.NoError(t, queue.AddToBatch(ctx, series2)) + require.NoError(t, queue.AddToBatch(ctx, series2)) + }) + + t.Run("Any errors pushed after last AddToBatch call are received on Close", func(t *testing.T) { + queue := setupQueue(t, capacity, batchSize, nil) + ctx := context.Background() + + // Add a batch to a batch but make sure nothing is pushed., + require.NoError(t, queue.AddToBatch(ctx, series1)) + + select { + case <-queue.Channel(): + t.Fatal("expected batch to not be flushed") + default: + } + + // Push multiple errors + queue.ErrorChannel() <- fmt.Errorf("mock error 1") + queue.ErrorChannel() <- fmt.Errorf("mock error 2") + + // Close and Done on the queue. + queue.Done() + err := queue.Close() + require.Error(t, err) + assert.Equal(t, "2 errors: mock error 1; mock error 2", err.Error()) + + // Batch is also pushed. + select { + case batch := <-queue.Channel(): + require.Equal(t, series1, batch.WriteRequest.Timeseries[0]) + default: + t.Fatal("expected batch to be flushed") + } + }) +} + +func setupQueue(t *testing.T, capacity, batchSize int, series []mimirpb.PreallocTimeseries) *batchingQueue { + t.Helper() + + queue := newBatchingQueue(capacity, batchSize) + + for _, s := range series { + require.NoError(t, queue.AddToBatch(context.Background(), s)) } - return statWithDetails.Err() + + return queue } diff --git a/pkg/storage/ingest/reader.go b/pkg/storage/ingest/reader.go index 4cbd9df43b0..329022785a5 100644 --- a/pkg/storage/ingest/reader.go +++ b/pkg/storage/ingest/reader.go @@ -3,10 +3,13 @@ package ingest import ( + "container/list" "context" "fmt" "math" + "os" "strconv" + "sync" "time" "github.com/go-kit/log" @@ -14,16 +17,18 @@ import ( "github.com/grafana/dskit/backoff" "github.com/grafana/dskit/multierror" "github.com/grafana/dskit/services" + "github.com/opentracing/opentracing-go" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" "github.com/twmb/franz-go/pkg/kadm" "github.com/twmb/franz-go/pkg/kerr" "github.com/twmb/franz-go/pkg/kgo" + "github.com/twmb/franz-go/pkg/kmsg" + "github.com/twmb/franz-go/plugin/kotel" "github.com/twmb/franz-go/plugin/kprom" "go.uber.org/atomic" - util_log "github.com/grafana/mimir/pkg/util/log" "github.com/grafana/mimir/pkg/util/spanlogger" ) @@ -33,11 +38,16 @@ const ( // kafkaOffsetEnd is a special offset value that means the end of the partition. kafkaOffsetEnd = int64(-1) + + // defaultMinBytesWaitTime is the time the Kafka broker can wait for MinBytes to be filled. + // This is usually used when there aren't enough records available to fulfil MinBytes, so the broker waits for more records to be produced. + defaultMinBytesWaitTime = 10 * time.Second ) var ( errWaitStrongReadConsistencyTimeoutExceeded = errors.Wrap(context.DeadlineExceeded, "wait strong read consistency timeout exceeded") errWaitTargetLagDeadlineExceeded = errors.Wrap(context.DeadlineExceeded, "target lag deadline exceeded") + errUnknownPartitionLeader = fmt.Errorf("unknown partition leader") ) type record struct { @@ -48,22 +58,30 @@ type record struct { } type recordConsumer interface { - // consume should return an error only if there is a recoverable error. Returning an error will cause consumption to slow down. - consume(context.Context, []record) error + // Consume consumes the given records in the order they are provided. We need this as samples that will be ingested, + // are also needed to be in order to avoid ingesting samples out of order. + // The function is expected to be idempotent and incremental, meaning that it can be called multiple times with the same records, and it won't respond to context cancellation. + Consume(context.Context, []record) error +} + +type fetcher interface { + pollFetches(context.Context) (kgo.Fetches, context.Context) } type PartitionReader struct { services.Service dependencies *services.Manager - kafkaCfg KafkaConfig - partitionID int32 - consumerGroup string + kafkaCfg KafkaConfig + partitionID int32 + consumerGroup string + concurrentFetchersMinBytesMaxWaitTime time.Duration - client *kgo.Client + client *kgo.Client + fetcher fetcher - consumer recordConsumer - metrics readerMetrics + newConsumer consumerFactory + metrics readerMetrics committer *partitionCommitter @@ -76,21 +94,35 @@ type PartitionReader struct { reg prometheus.Registerer } +type consumerFactoryFunc func() recordConsumer + +func (c consumerFactoryFunc) consumer() recordConsumer { + return c() +} + func NewPartitionReaderForPusher(kafkaCfg KafkaConfig, partitionID int32, instanceID string, pusher Pusher, logger log.Logger, reg prometheus.Registerer) (*PartitionReader, error) { - consumer := newPusherConsumer(pusher, util_log.NewSampler(kafkaCfg.FallbackClientErrorSampleRate), reg, logger) - return newPartitionReader(kafkaCfg, partitionID, instanceID, consumer, logger, reg) + metrics := newPusherConsumerMetrics(reg) + factory := consumerFactoryFunc(func() recordConsumer { + return newPusherConsumer(pusher, kafkaCfg, metrics, logger) + }) + return newPartitionReader(kafkaCfg, partitionID, instanceID, factory, logger, reg) +} + +type consumerFactory interface { + consumer() recordConsumer } -func newPartitionReader(kafkaCfg KafkaConfig, partitionID int32, instanceID string, consumer recordConsumer, logger log.Logger, reg prometheus.Registerer) (*PartitionReader, error) { +func newPartitionReader(kafkaCfg KafkaConfig, partitionID int32, instanceID string, consumer consumerFactory, logger log.Logger, reg prometheus.Registerer) (*PartitionReader, error) { r := &PartitionReader{ - kafkaCfg: kafkaCfg, - partitionID: partitionID, - consumer: consumer, - consumerGroup: kafkaCfg.GetConsumerGroup(instanceID, partitionID), - metrics: newReaderMetrics(partitionID, reg), - consumedOffsetWatcher: newPartitionOffsetWatcher(), - logger: log.With(logger, "partition", partitionID), - reg: reg, + kafkaCfg: kafkaCfg, + partitionID: partitionID, + newConsumer: consumer, + consumerGroup: kafkaCfg.GetConsumerGroup(instanceID, partitionID), + metrics: newReaderMetrics(partitionID, reg), + consumedOffsetWatcher: newPartitionOffsetWatcher(), + concurrentFetchersMinBytesMaxWaitTime: defaultMinBytesWaitTime, + logger: log.With(logger, "partition", partitionID), + reg: reg, } r.Service = services.NewBasicService(r.start, r.run, r.stop) @@ -113,7 +145,6 @@ func (r *PartitionReader) start(ctx context.Context) (returnErr error) { if err != nil { return err } - // Initialise the last consumed offset only if we've got an actual offset from the consumer group. if lastConsumedOffset >= 0 { r.consumedOffsetWatcher.Notify(lastConsumedOffset) @@ -125,9 +156,21 @@ func (r *PartitionReader) start(ctx context.Context) (returnErr error) { } r.committer = newPartitionCommitter(r.kafkaCfg, kadm.NewClient(r.client), r.partitionID, r.consumerGroup, r.logger, r.reg) - r.offsetReader = newPartitionOffsetReader(r.client, r.kafkaCfg.Topic, r.partitionID, r.kafkaCfg.LastProducedOffsetPollInterval, r.reg, r.logger) + offsetsClient := newPartitionOffsetClient(r.client, r.kafkaCfg.Topic, r.reg, r.logger) + + // It's ok to have the start offset slightly outdated. + // We only need this offset accurate if we fall behind or if we start and the log gets truncated from beneath us. + // In both cases we should recover after receiving one updated value. + // In the more common case where this offset is used when we're fetching from after the end, there we don't need an accurate value. + const startOffsetReaderRefreshDuration = 10 * time.Second + getPartitionStart := func(ctx context.Context) (int64, error) { + return offsetsClient.FetchPartitionStartOffset(ctx, r.partitionID) + } + startOffsetReader := newGenericOffsetReader(getPartitionStart, startOffsetReaderRefreshDuration, r.logger) + + r.offsetReader = newPartitionOffsetReaderWithOffsetClient(offsetsClient, r.partitionID, r.kafkaCfg.LastProducedOffsetPollInterval, r.logger) - r.dependencies, err = services.NewManager(r.committer, r.offsetReader, r.consumedOffsetWatcher) + r.dependencies, err = services.NewManager(r.committer, r.offsetReader, r.consumedOffsetWatcher, startOffsetReader) if err != nil { return errors.Wrap(err, "creating service manager") } @@ -136,6 +179,15 @@ func (r *PartitionReader) start(ctx context.Context) (returnErr error) { return errors.Wrap(err, "starting service manager") } + if r.kafkaCfg.ReplayConcurrency > 1 { + r.fetcher, err = newConcurrentFetchers(ctx, r.client, r.logger, r.kafkaCfg.Topic, r.partitionID, startOffset, r.kafkaCfg.ReplayConcurrency, r.kafkaCfg.RecordsPerFetch, r.kafkaCfg.UseCompressedBytesAsFetchMaxBytes, r.concurrentFetchersMinBytesMaxWaitTime, offsetsClient, startOffsetReader, &r.metrics) + if err != nil { + return errors.Wrap(err, "creating concurrent fetchers") + } + } else { + r.fetcher = r + } + // Enforce the max consumer lag (if enabled). if targetLag, maxLag := r.kafkaCfg.TargetConsumerLagAtStartup, r.kafkaCfg.MaxConsumerLagAtStartup; targetLag > 0 && maxLag > 0 { if startOffset != kafkaOffsetEnd { @@ -183,7 +235,9 @@ func (r *PartitionReader) run(ctx context.Context) error { } func (r *PartitionReader) processNextFetches(ctx context.Context, delayObserver prometheus.Observer) error { - fetches := r.pollFetches(ctx) + fetches, fetchCtx := r.fetcher.pollFetches(ctx) + // Propagate the fetching span to consuming the records. + ctx = opentracing.ContextWithSpan(ctx, opentracing.SpanFromContext(fetchCtx)) r.recordFetchesMetrics(fetches, delayObserver) r.logFetchErrors(fetches) fetches = filterOutErrFetches(fetches) @@ -260,6 +314,10 @@ func (r *PartitionReader) processNextFetchesUntilTargetOrMaxLagHonored(ctx conte } func (r *PartitionReader) processNextFetchesUntilLagHonored(ctx context.Context, maxLag time.Duration, logger log.Logger) (currLag time.Duration, _ error) { + // clean-up resources spun up from this function + ctx, cancel := context.WithCancelCause(ctx) + defer cancel(fmt.Errorf("partition reader stopped consuming partition until max consumer lag is honored")) + boff := backoff.New(ctx, backoff.Config{ MinBackoff: 100 * time.Millisecond, MaxBackoff: time.Second, @@ -297,14 +355,12 @@ func (r *PartitionReader) processNextFetchesUntilLagHonored(ctx context.Context, // This message is NOT expected to be logged with a very high rate. In this log we display the last measured // lag. If we don't have it (lag is zero value), then it will not be logged. level.Info(loggerWithCurrentLagIfSet(logger, currLag)).Log("msg", "partition reader is consuming records to honor target and max consumer lag", "partition_start_offset", partitionStartOffset, "last_produced_offset", lastProducedOffset) - for boff.Ongoing() { // Continue reading until we reached the desired offset. lastConsumedOffset := r.consumedOffsetWatcher.LastConsumedOffset() if lastProducedOffset <= lastConsumedOffset { break } - err := r.processNextFetches(ctx, r.metrics.receiveDelayWhenStarting) if err != nil { return 0, err @@ -388,6 +444,9 @@ func (r *PartitionReader) enqueueCommit(fetches kgo.Fetches) { } func (r *PartitionReader) consumeFetches(ctx context.Context, fetches kgo.Fetches) error { + span, ctx := opentracing.StartSpanFromContext(ctx, "PartitionReader.consumeFetches") + defer span.Finish() + if fetches.NumRecords() == 0 { return nil } @@ -414,20 +473,27 @@ func (r *PartitionReader) consumeFetches(ctx context.Context, fetches kgo.Fetche MaxBackoff: 2 * time.Second, MaxRetries: 0, // retry forever }) + defer func(consumeStart time.Time) { + r.metrics.consumeLatency.Observe(time.Since(consumeStart).Seconds()) + }(time.Now()) + + logger := spanlogger.FromContext(ctx, r.logger) + for boff.Ongoing() { + // We instantiate the consumer on each iteration because it is stateful, and we can't reuse it after closing. + consumer := r.newConsumer.consumer() // If the PartitionReader is stopping and the ctx was cancelled, we don't want to interrupt the in-flight // processing midway. Instead, we let it finish, assuming it'll succeed. // If the processing fails while stopping, we log the error and let the backoff stop and bail out. // There is an edge-case when the processing gets stuck and doesn't let the stopping process. In such a case, // we expect the infrastructure (e.g. k8s) to eventually kill the process. consumeCtx := context.WithoutCancel(ctx) - consumeStart := time.Now() - err := r.consumer.consume(consumeCtx, records) - r.metrics.consumeLatency.Observe(time.Since(consumeStart).Seconds()) + err := consumer.Consume(consumeCtx, records) if err == nil { - return nil + level.Debug(logger).Log("msg", "closing consumer after successful consumption") + break } - level.Error(r.logger).Log( + level.Error(logger).Log( "msg", "encountered error while ingesting data from Kafka; should retry", "err", err, "record_min_offset", minOffset, @@ -648,11 +714,608 @@ func (r *PartitionReader) waitReadConsistency(ctx context.Context, withOffset bo return err } -func (r *PartitionReader) pollFetches(ctx context.Context) kgo.Fetches { +func (r *PartitionReader) pollFetches(ctx context.Context) (result kgo.Fetches, ctx2 context.Context) { defer func(start time.Time) { r.metrics.fetchWaitDuration.Observe(time.Since(start).Seconds()) }(time.Now()) - return r.client.PollFetches(ctx) + return r.client.PollFetches(ctx), ctx +} + +// fetchWant represents a range of offsets to fetch. +// Based on a given number of records, it tries to estimate how many bytes we need to fetch, given there's no support for fetching offsets directly. +// fetchWant also contains the channel on which to send the fetched records for the offset range. +type fetchWant struct { + startOffset int64 // inclusive + endOffset int64 // exclusive + bytesPerRecord int + + // result should be closed when there are no more fetches for this partition. It is ok to send multiple times on the channel. + result chan fetchResult +} + +func fetchWantFrom(offset int64, recordsPerFetch int) fetchWant { + return fetchWant{ + startOffset: offset, + endOffset: offset + int64(recordsPerFetch), + result: make(chan fetchResult, 1), // buffer of 1 so we can do secondary attempt requests in the background + } +} + +// Next returns the fetchWant for the next numRecords starting from the last known offset. +func (w fetchWant) Next(numRecords int) fetchWant { + n := fetchWantFrom(w.endOffset, numRecords) + n.bytesPerRecord = w.bytesPerRecord + return n.trimIfTooBig() +} + +// MaxBytes returns the maximum number of bytes we can fetch in a single request. +// It's capped at math.MaxInt32 to avoid overflow, and it'll always fetch a minimum of 1MB. +func (w fetchWant) MaxBytes() int32 { + fetchBytes := w.expectedBytes() + if fetchBytes > math.MaxInt32 { + // This shouldn't happen because w should have been trimmed before sending the request. + // But we definitely don't want to request negative bytes by casting to int32, so add this safeguard. + return math.MaxInt32 + } + fetchBytes = max(1_000_000, fetchBytes) // when we're fetching few records, we can afford to over-fetch to avoid more requests. + return int32(fetchBytes) +} + +// UpdateBytesPerRecord updates the expected bytes per record based on the results of the last fetch and trims the fetchWant if MaxBytes() would now exceed math.MaxInt32. +func (w fetchWant) UpdateBytesPerRecord(lastFetchBytes int, lastFetchNumberOfRecords int) fetchWant { + // Smooth over the estimation to avoid having outlier fetches from throwing off the estimation. + // We don't want a fetch of 5 records to determine how we fetch the next fetch of 6000 records. + // Ideally we weigh the estimation on the number of records observed, but it's simpler to smooth it over with a constant factor. + const currentEstimateWeight = 0.8 + + actualBytesPerRecord := float64(lastFetchBytes) / float64(lastFetchNumberOfRecords) + w.bytesPerRecord = int(currentEstimateWeight*float64(w.bytesPerRecord) + (1-currentEstimateWeight)*actualBytesPerRecord) + + return w.trimIfTooBig() +} + +// expectedBytes returns how many bytes we'd need to accommodate the range of offsets using bytesPerRecord. +// They may be more than the kafka protocol supports (> MaxInt32). Use MaxBytes. +func (w fetchWant) expectedBytes() int { + // We over-fetch bytes to reduce the likelihood of under-fetching and having to run another request. + // Based on some testing 65% of under-estimations are by less than 5%. So we account for that. + const overFetchBytesFactor = 1.05 + return int(overFetchBytesFactor * float64(w.bytesPerRecord*int(w.endOffset-w.startOffset))) +} + +// trimIfTooBig adjusts the end offset if we expect to fetch too many bytes. +// It's capped at math.MaxInt32 bytes. +func (w fetchWant) trimIfTooBig() fetchWant { + if w.expectedBytes() <= math.MaxInt32 { + return w + } + // We are overflowing, so we need to trim the end offset. + // We do this by calculating how many records we can fetch with the max bytes, and then setting the end offset to that. + w.endOffset = w.startOffset + int64(math.MaxInt32/w.bytesPerRecord) + return w +} + +type fetchResult struct { + kgo.FetchPartition + ctx context.Context + fetchedBytes int +} + +func (fr fetchResult) logCompletedFetch(fetchStartTime time.Time, w fetchWant) { + var logger log.Logger = spanlogger.FromContext(fr.ctx, log.NewLogfmtLogger(os.Stderr)) + + msg := "fetched records" + if fr.Err != nil { + msg = "received an error while fetching records; will retry after processing received records (if any)" + } + var ( + gotRecords = int64(len(fr.Records)) + askedRecords = w.endOffset - w.startOffset + ) + switch { + case fr.Err == nil, errors.Is(fr.Err, kerr.OffsetOutOfRange): + logger = level.Debug(logger) + default: + logger = level.Error(logger) + } + logger.Log( + "msg", msg, + "duration", time.Since(fetchStartTime), + "start_offset", w.startOffset, + "end_offset", w.endOffset, + "asked_records", askedRecords, + "got_records", gotRecords, + "diff_records", askedRecords-gotRecords, + "asked_bytes", w.MaxBytes(), + "got_bytes", fr.fetchedBytes, + "diff_bytes", int(w.MaxBytes())-fr.fetchedBytes, + "hwm", fr.HighWatermark, + "lso", fr.LogStartOffset, + "err", fr.Err, + ) +} + +func (fr fetchResult) logOrderedFetch() { + if fr.ctx == nil { + return + } + spanlogger.FromContext(fr.ctx, log.NewNopLogger()).DebugLog("msg", "fetch result is enqueued for consuming") +} + +func newEmptyFetchResult(ctx context.Context, err error) fetchResult { + return fetchResult{ + ctx: ctx, + fetchedBytes: 0, + FetchPartition: kgo.FetchPartition{Err: err}, + } +} + +type concurrentFetchers struct { + client *kgo.Client + logger log.Logger + partitionID int32 + topicID [16]byte + topicName string + metrics *readerMetrics + tracer *kotel.Tracer + + concurrency int + recordsPerFetch int + minBytesWaitTime time.Duration + + orderedFetches chan fetchResult + lastReturnedRecord int64 + startOffsets *genericOffsetReader[int64] + + // trackCompressedBytes controls whether to calculate MaxBytes for fetch requests based on previous responses' compressed or uncompressed bytes. + trackCompressedBytes bool +} + +// newConcurrentFetchers creates a new concurrentFetchers. startOffset can be kafkaOffsetStart, kafkaOffsetEnd or a specific offset. +func newConcurrentFetchers( + ctx context.Context, + client *kgo.Client, + logger log.Logger, + topic string, + partition int32, + startOffset int64, + concurrency int, + recordsPerFetch int, + trackCompressedBytes bool, + minBytesWaitTime time.Duration, + offsetReader *partitionOffsetClient, + startOffsetsReader *genericOffsetReader[int64], + metrics *readerMetrics, +) (*concurrentFetchers, error) { + + const noReturnedRecords = -1 // we still haven't returned the 0 offset. + f := &concurrentFetchers{ + client: client, + logger: logger, + concurrency: concurrency, + topicName: topic, + partitionID: partition, + metrics: metrics, + recordsPerFetch: recordsPerFetch, + minBytesWaitTime: minBytesWaitTime, + lastReturnedRecord: noReturnedRecords, + startOffsets: startOffsetsReader, + trackCompressedBytes: trackCompressedBytes, + tracer: recordsTracer(), + orderedFetches: make(chan fetchResult), + } + + var err error + switch startOffset { + case kafkaOffsetStart: + startOffset, err = offsetReader.FetchPartitionStartOffset(ctx, partition) + case kafkaOffsetEnd: + startOffset, err = offsetReader.FetchPartitionLastProducedOffset(ctx, partition) + // End (-1) means "ignore all existing records". FetchPartitionLastProducedOffset returns the offset of an existing record. + // We need to start from the next one, which is still not produced. + startOffset++ + } + if err != nil { + return nil, fmt.Errorf("resolving offset to start consuming from: %w", err) + } + + topics, err := kadm.NewClient(client).ListTopics(ctx, topic) + if err != nil { + return nil, fmt.Errorf("failed to find topic ID: %w", err) + } + if !topics.Has(topic) { + return nil, fmt.Errorf("failed to find topic ID: topic not found") + } + f.topicID = topics[topic].ID + + go f.runFetchers(ctx, startOffset) + + return f, nil +} + +func (r *concurrentFetchers) pollFetches(ctx context.Context) (kgo.Fetches, context.Context) { + waitStartTime := time.Now() + select { + case <-ctx.Done(): + return kgo.Fetches{}, ctx + case f := <-r.orderedFetches: + firstUnreturnedRecordIdx := recordIndexAfterOffset(f.Records, r.lastReturnedRecord) + r.recordOrderedFetchTelemetry(f, firstUnreturnedRecordIdx, waitStartTime) + + r.lastReturnedRecord = f.Records[len(f.Records)-1].Offset + f.Records = f.Records[firstUnreturnedRecordIdx:] + + return kgo.Fetches{{ + Topics: []kgo.FetchTopic{ + { + Topic: r.topicName, + Partitions: []kgo.FetchPartition{f.FetchPartition}, + }, + }, + }}, f.ctx + } +} + +func recordIndexAfterOffset(records []*kgo.Record, offset int64) int { + for i, r := range records { + if r.Offset > offset { + return i + } + } + return len(records) - 1 +} + +func (r *concurrentFetchers) recordOrderedFetchTelemetry(f fetchResult, firstReturnedRecordIndex int, waitStartTime time.Time) { + waitDuration := time.Since(waitStartTime) + level.Debug(r.logger).Log("msg", "received ordered fetch", "num_records", len(f.Records), "wait_duration", waitDuration) + r.metrics.fetchWaitDuration.Observe(waitDuration.Seconds()) + + doubleFetchedBytes := 0 + for i, record := range f.Records { + if i < firstReturnedRecordIndex { + doubleFetchedBytes += len(record.Value) + spanlogger.FromContext(record.Context, r.logger).DebugLog("msg", "skipping record because it has already been returned", "offset", record.Offset) + } + r.tracer.OnFetchRecordUnbuffered(record, true) + } + r.metrics.fetchedDiscardedRecordBytes.Add(float64(doubleFetchedBytes)) +} + +// fetchSingle attempts to find out the leader leader Kafka broker for a partition and then sends a fetch request to the leader of the fetchWant request and parses the responses +// fetchSingle returns a fetchResult which may or may not fulfil the entire fetchWant. +// If ctx is cancelled, fetchSingle will return an empty fetchResult without an error. +func (r *concurrentFetchers) fetchSingle(ctx context.Context, fw fetchWant) (fr fetchResult) { + defer func(fetchStartTime time.Time) { + fr.logCompletedFetch(fetchStartTime, fw) + }(time.Now()) + + leaderID, leaderEpoch, err := r.client.PartitionLeader(r.topicName, r.partitionID) + if err != nil || (leaderID == -1 && leaderEpoch == -1) { + if err != nil { + return newEmptyFetchResult(ctx, fmt.Errorf("finding leader for partition: %w", err)) + } + return newEmptyFetchResult(ctx, errUnknownPartitionLeader) + } + + req := r.buildFetchRequest(fw, leaderEpoch) + + resp, err := req.RequestWith(ctx, r.client.Broker(int(leaderID))) + if err != nil { + if errors.Is(err, context.Canceled) { + return newEmptyFetchResult(ctx, nil) + } + return newEmptyFetchResult(ctx, fmt.Errorf("fetching from kafka: %w", err)) + } + + return r.parseFetchResponse(ctx, fw.startOffset, resp) +} + +func (r *concurrentFetchers) buildFetchRequest(fw fetchWant, leaderEpoch int32) kmsg.FetchRequest { + req := kmsg.NewFetchRequest() + req.MinBytes = 1 + req.Version = 13 + req.MaxWaitMillis = int32(r.minBytesWaitTime / time.Millisecond) + req.MaxBytes = fw.MaxBytes() + + reqTopic := kmsg.NewFetchRequestTopic() + reqTopic.Topic = r.topicName + reqTopic.TopicID = r.topicID + + reqPartition := kmsg.NewFetchRequestTopicPartition() + reqPartition.Partition = r.partitionID + reqPartition.FetchOffset = fw.startOffset + reqPartition.PartitionMaxBytes = req.MaxBytes + reqPartition.CurrentLeaderEpoch = leaderEpoch + + reqTopic.Partitions = append(reqTopic.Partitions, reqPartition) + req.Topics = append(req.Topics, reqTopic) + return req +} + +func (r *concurrentFetchers) parseFetchResponse(ctx context.Context, startOffset int64, resp *kmsg.FetchResponse) fetchResult { + // Here we ignore resp.ErrorCode. That error code was added for support for KIP-227 and is only set if we're using fetch sessions. We don't use fetch sessions. + // We also ignore rawPartitionResp.PreferredReadReplica to keep the code simpler. We don't provide any rack in the FetchRequest, so the broker _probably_ doesn't have a recommended replica for us. + + // Sanity check for the response we get. + // If we get something we didn't expect, maybe we're sending the wrong request or there's a bug in the kafka implementation. + // Even in case of errors we get the topic partition. + err := assertResponseContainsPartition(resp, r.topicID, r.partitionID) + if err != nil { + return newEmptyFetchResult(ctx, err) + } + + parseOptions := kgo.ProcessFetchPartitionOptions{ + KeepControlRecords: false, + Offset: startOffset, + IsolationLevel: kgo.ReadUncommitted(), // we don't produce in transactions, but leaving this here so it's explicit. + Topic: r.topicName, + Partition: r.partitionID, + } + + observeMetrics := func(m kgo.FetchBatchMetrics) { + brokerMeta := kgo.BrokerMetadata{} // leave it empty because kprom doesn't use it, and we don't exactly have all the metadata + r.metrics.kprom.OnFetchBatchRead(brokerMeta, r.topicName, r.partitionID, m) + } + rawPartitionResp := resp.Topics[0].Partitions[0] + partition, _ := kgo.ProcessRespPartition(parseOptions, &rawPartitionResp, observeMetrics) + partition.EachRecord(r.tracer.OnFetchRecordBuffered) + + fetchedBytes := len(rawPartitionResp.RecordBatches) + if !r.trackCompressedBytes { + fetchedBytes = sumRecordLengths(partition.Records) + } + + return fetchResult{ + ctx: ctx, + FetchPartition: partition, + fetchedBytes: fetchedBytes, + } +} + +func assertResponseContainsPartition(resp *kmsg.FetchResponse, topicID kadm.TopicID, partitionID int32) error { + if topics := resp.Topics; len(topics) < 1 || topics[0].TopicID != topicID { + receivedTopicID := kadm.TopicID{} + if len(topics) > 0 { + receivedTopicID = topics[0].TopicID + } + return fmt.Errorf("didn't find expected topic %s in fetch response; received topic %s", topicID, receivedTopicID) + } + if partitions := resp.Topics[0].Partitions; len(partitions) < 1 || partitions[0].Partition != partitionID { + receivedPartitionID := int32(-1) + if len(partitions) > 0 { + receivedPartitionID = partitions[0].Partition + } + return fmt.Errorf("didn't find expected partition %d in fetch response; received partition %d", partitionID, receivedPartitionID) + } + return nil +} + +func sumRecordLengths(records []*kgo.Record) (sum int) { + for _, r := range records { + sum += len(r.Value) + } + return sum +} + +func (r *concurrentFetchers) runFetcher(ctx context.Context, fetchersWg *sync.WaitGroup, wants chan fetchWant, logger log.Logger) { + defer fetchersWg.Done() + errBackoff := backoff.New(ctx, backoff.Config{ + MinBackoff: 250 * time.Millisecond, + MaxBackoff: 2 * time.Second, + MaxRetries: 0, // retry forever + }) + + // more aggressive backoff when we're waiting for records to be produced. + // It's likely there's already some records produced by the time we get back the response and send another request. + newRecordsProducedBackoff := backoff.New(ctx, backoff.Config{ + MinBackoff: 10 * time.Millisecond, + MaxBackoff: time.Second, + MaxRetries: 0, // retry forever + }) + + for w := range wants { + // Start new span for each fetchWant. We want to record the lifecycle of a single record from being fetched to being ingested. + wantSpan, ctx := opentracing.StartSpanFromContext(ctx, "concurrentFetcher.fetch") + wantSpan.SetTag("start_offset", w.startOffset) + wantSpan.SetTag("end_offset", w.endOffset) + + for attempt := 0; errBackoff.Ongoing() && w.endOffset > w.startOffset; attempt++ { + attemptSpan, ctx := opentracing.StartSpanFromContext(ctx, "concurrentFetcher.fetch.attempt") + attemptSpan.SetTag("attempt", attempt) + + f := r.fetchSingle(ctx, w) + if f.Err != nil { + w = handleKafkaFetchErr(f.Err, w, errBackoff, newRecordsProducedBackoff, r.startOffsets, r.client, spanlogger.FromContext(ctx, logger)) + } + if len(f.Records) == 0 { + // Typically if we had an error, then there wouldn't be any records. + // But it's hard to verify this for all errors from the Kafka API docs, so just to be sure, we process any records we might have received. + attemptSpan.Finish() + continue + } + // Next attempt will be from the last record onwards. + w.startOffset = f.Records[len(f.Records)-1].Offset + 1 + + // We reset the backoff if we received any records whatsoever. A received record means _some_ success. + // We don't want to slow down until we hit a larger error. + errBackoff.Reset() + newRecordsProducedBackoff.Reset() + + // Propagate the span context to consuming the records. + f.ctx = ctx + select { + case w.result <- f: + case <-ctx.Done(): + } + attemptSpan.Finish() + } + wantSpan.Finish() + close(w.result) + } +} + +func (r *concurrentFetchers) runFetchers(ctx context.Context, startOffset int64) { + fetchersWg := &sync.WaitGroup{} + fetchersWg.Add(r.concurrency) + defer fetchersWg.Wait() + + wants := make(chan fetchWant) + defer close(wants) + for i := 0; i < r.concurrency; i++ { + logger := log.With(r.logger, "fetcher", i) + go r.runFetcher(ctx, fetchersWg, wants, logger) + } + + var ( + nextFetch = fetchWantFrom(startOffset, r.recordsPerFetch) + nextResult chan fetchResult + pendingResults = list.New() + + bufferedResult fetchResult + readyBufferedResults chan fetchResult // this is non-nil when bufferedResult is non-empty + ) + nextFetch.bytesPerRecord = 10_000 // start with an estimation, we will update it as we consume + + for { + refillBufferedResult := nextResult + if readyBufferedResults != nil { + // We have a single result that's still not consumed. + // So we don't try to get new results from the fetchers. + refillBufferedResult = nil + } + select { + case <-ctx.Done(): + return + + case wants <- nextFetch: + pendingResults.PushBack(nextFetch.result) + if nextResult == nil { + // In case we previously exhausted pendingResults, we just created + nextResult = pendingResults.Front().Value.(chan fetchResult) + pendingResults.Remove(pendingResults.Front()) + } + nextFetch = nextFetch.Next(r.recordsPerFetch) + + case result, moreLeft := <-refillBufferedResult: + if !moreLeft { + if pendingResults.Len() > 0 { + nextResult = pendingResults.Front().Value.(chan fetchResult) + pendingResults.Remove(pendingResults.Front()) + } else { + nextResult = nil + } + continue + } + nextFetch = nextFetch.UpdateBytesPerRecord(result.fetchedBytes, len(result.Records)) + bufferedResult = result + bufferedResult.logOrderedFetch() + readyBufferedResults = r.orderedFetches + + case readyBufferedResults <- bufferedResult: + readyBufferedResults = nil + bufferedResult = fetchResult{} + } + } +} + +type waiter interface { + Wait() +} + +type metadataRefresher interface { + ForceMetadataRefresh() +} + +// handleKafkaFetchErr handles all the errors listed in the franz-go documentation as possible errors when fetching records. +// For most of them we just apply a backoff. They are listed here so we can be explicit in what we're handling and how. +// It may also return an adjusted fetchWant in case the error indicated, we were consuming not yet produced records or records already deleted due to retention. +func handleKafkaFetchErr(err error, fw fetchWant, shortBackoff, longBackoff waiter, partitionStartOffset *genericOffsetReader[int64], refresher metadataRefresher, logger log.Logger) fetchWant { + // Typically franz-go will update its own metadata when it detects a change in brokers. But it's hard to verify this. + // So we force a metadata refresh here to be sure. + // It's ok to call this from multiple fetchers concurrently. franz-go will only be sending one metadata request at a time (whether automatic, periodic, or forced). + // + // Metadata refresh is asynchronous. So even after forcing the refresh we might have outdated metadata. + // Hopefully the backoff that will follow is enough to get the latest metadata. + // If not, the fetcher will end up here again on the next attempt. + triggerMetadataRefresh := refresher.ForceMetadataRefresh + + switch { + case err == nil: + case errors.Is(err, kerr.OffsetOutOfRange): + // We're either consuming from before the first offset or after the last offset. + partitionStart, err := partitionStartOffset.CachedOffset() + if err != nil { + level.Error(logger).Log("msg", "failed to find start offset to readjust on OffsetOutOfRange; retrying same records range", "err", err) + break + } + + if fw.startOffset < partitionStart { + // We're too far behind. + if partitionStart >= fw.endOffset { + // The next fetch want is responsible for this range. We set startOffset=endOffset to effectively mark this fetch as complete. + fw.startOffset = fw.endOffset + level.Debug(logger).Log("msg", "we're too far behind aborting fetch", "log_start_offset", partitionStart, "start_offset", fw.startOffset, "end_offset", fw.endOffset) + break + } + // Only some of the offsets of our want are out of range, so let's fast-forward. + fw.startOffset = partitionStart + level.Debug(logger).Log("msg", "part of fetch want is outside of available offsets, adjusted start offset", "log_start_offset", partitionStart, "start_offset", fw.startOffset, "end_offset", fw.endOffset) + } else { + // If the broker is behind or if we are requesting offsets which have not yet been produced, we end up here. + // We set a MaxWaitMillis on fetch requests, but even then there may be no records for some time. + // Wait for a short time to allow the broker to catch up or for new records to be produced. + level.Debug(logger).Log("msg", "offset out of range; waiting for new records to be produced") + shortBackoff.Wait() + } + case errors.Is(err, kerr.TopicAuthorizationFailed): + longBackoff.Wait() + case errors.Is(err, kerr.UnknownTopicOrPartition): + longBackoff.Wait() + case errors.Is(err, kerr.UnsupportedCompressionType): + level.Error(logger).Log("msg", "received UNSUPPORTED_COMPRESSION_TYPE from kafka; this shouldn't happen; please report this as a bug", "err", err) + longBackoff.Wait() // this shouldn't happen - only happens when the request version was under 10, but we always use 13 - log error and backoff - we can't afford to lose records + case errors.Is(err, kerr.UnsupportedVersion): + level.Error(logger).Log("msg", "received UNSUPPORTED_VERSION from kafka; the Kafka cluster is probably too old", "err", err) + longBackoff.Wait() // in this case our client is too old, not much we can do. This will probably continue logging the error until someone upgrades their Kafka cluster. + case errors.Is(err, kerr.KafkaStorageError): + longBackoff.Wait() // server-side error, effectively same as HTTP 500 + case errors.Is(err, kerr.UnknownTopicID): + longBackoff.Wait() // Maybe it wasn't created by the producers yet. + case errors.Is(err, kerr.OffsetMovedToTieredStorage): + level.Error(logger).Log("msg", "received OFFSET_MOVED_TO_TIERED_STORAGE from kafka; this shouldn't happen; please report this as a bug", "err", err) + longBackoff.Wait() // This should be only intra-broker error, and we shouldn't get it. + case errors.Is(err, kerr.NotLeaderForPartition): + // We're asking a broker which is no longer the leader. For a partition. We should refresh our metadata and try again. + triggerMetadataRefresh() + longBackoff.Wait() + case errors.Is(err, kerr.ReplicaNotAvailable): + // Maybe the replica hasn't replicated the log yet, or it is no longer a replica for this partition. + // We should refresh and try again with a leader or replica which is up to date. + triggerMetadataRefresh() + longBackoff.Wait() + case errors.Is(err, kerr.UnknownLeaderEpoch): + // Maybe there's an ongoing election. We should refresh our metadata and try again with a leader in the current epoch. + triggerMetadataRefresh() + longBackoff.Wait() + case errors.Is(err, kerr.FencedLeaderEpoch): + // We missed a new epoch (leader election). We should refresh our metadata and try again with a leader in the current epoch. + triggerMetadataRefresh() + longBackoff.Wait() + case errors.Is(err, kerr.LeaderNotAvailable): + // This isn't listed in the possible errors in franz-go, but Apache Kafka returns it when the partition has no leader. + triggerMetadataRefresh() + longBackoff.Wait() + case errors.Is(err, errUnknownPartitionLeader): + triggerMetadataRefresh() + longBackoff.Wait() + case errors.Is(err, &kgo.ErrFirstReadEOF{}): + longBackoff.Wait() + + default: + level.Error(logger).Log("msg", "received an error we're not prepared to handle; this shouldn't happen; please report this as a bug", "err", err) + longBackoff.Wait() + } + return fw } type partitionCommitter struct { @@ -792,6 +1455,7 @@ type readerMetrics struct { fetchesErrors prometheus.Counter fetchesTotal prometheus.Counter fetchWaitDuration prometheus.Histogram + fetchedDiscardedRecordBytes prometheus.Counter strongConsistencyInstrumentation *StrongReadConsistencyInstrumentation[struct{}] lastConsumedOffset prometheus.Gauge consumeLatency prometheus.Histogram @@ -841,6 +1505,10 @@ func newReaderMetrics(partitionID int32, reg prometheus.Registerer) readerMetric Help: "How long a consumer spent waiting for a batch of records from the Kafka client. If fetching is faster than processing, then this will be close to 0.", NativeHistogramBucketFactor: 1.1, }), + fetchedDiscardedRecordBytes: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "cortex_ingest_storage_reader_fetched_discarded_bytes_total", + Help: "Total number of uncompressed bytes of records discarded from because they were already consumed. A higher rate means that the concurrent fetching estimations are less accurate.", + }), consumeLatency: promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ Name: "cortex_ingest_storage_reader_records_batch_process_duration_seconds", Help: "How long a consumer spent processing a batch of records from Kafka.", diff --git a/pkg/storage/ingest/reader_client.go b/pkg/storage/ingest/reader_client.go index 65331aa8ab0..24bb1061190 100644 --- a/pkg/storage/ingest/reader_client.go +++ b/pkg/storage/ingest/reader_client.go @@ -19,14 +19,15 @@ func NewKafkaReaderClient(cfg KafkaConfig, metrics *kprom.Metrics, logger log.Lo opts = append(opts, commonKafkaClientOptions(cfg, metrics, logger)...) opts = append(opts, kgo.FetchMinBytes(1), - kgo.FetchMaxBytes(fetchMaxBytes), + kgo.FetchMaxBytes(fetchMaxBytes), // these are unused by concurrent fetchers kgo.FetchMaxWait(5*time.Second), - kgo.FetchMaxPartitionBytes(50_000_000), + kgo.FetchMaxPartitionBytes(50_000_000), // these are unused by concurrent fetchers // BrokerMaxReadBytes sets the maximum response size that can be read from // Kafka. This is a safety measure to avoid OOMing on invalid responses. // franz-go recommendation is to set it 2x FetchMaxBytes. - kgo.BrokerMaxReadBytes(2*fetchMaxBytes), + // With concurrent fetchers we set FetchMaxBytes and FetchMaxPartitionBytes on a per-request basis, so here we put a high enough limit that should work for those requests. + kgo.BrokerMaxReadBytes(1_000_000_000), ) client, err := kgo.NewClient(opts...) if err != nil { diff --git a/pkg/storage/ingest/reader_test.go b/pkg/storage/ingest/reader_test.go index 14e012486de..ebf1de7f0fb 100644 --- a/pkg/storage/ingest/reader_test.go +++ b/pkg/storage/ingest/reader_test.go @@ -121,7 +121,7 @@ func TestPartitionReader_ConsumerError(t *testing.T) { consumer := consumerFunc(func(ctx context.Context, records []record) error { invocations.Inc() if !returnErrors.Load() { - return trackingConsumer.consume(ctx, records) + return trackingConsumer.Consume(ctx, records) } // There may be more records, but we only care that the one we failed to consume in the first place is still there. assert.Equal(t, "1", string(records[0].content)) @@ -876,6 +876,122 @@ func TestPartitionReader_ConsumeAtStartup(t *testing.T) { } }) + t.Run("should consume partition from start if position=start, and wait until target lag is honored, and then consume some records after lag is honored", func(t *testing.T) { + t.Parallel() + + var ( + cluster, clusterAddr = testkafka.CreateCluster(t, partitionID+1, topicName) + fetchRequestsCount = atomic.NewInt64(0) + fetchShouldFail = atomic.NewBool(false) + consumedRecordsMx sync.Mutex + consumedRecords []string + ) + + consumer := consumerFunc(func(_ context.Context, records []record) error { + consumedRecordsMx.Lock() + defer consumedRecordsMx.Unlock() + + for _, r := range records { + consumedRecords = append(consumedRecords, string(r.content)) + } + return nil + }) + + cluster.ControlKey(int16(kmsg.Fetch), func(kmsg.Request) (kmsg.Response, error, bool) { + cluster.KeepControl() + fetchRequestsCount.Inc() + + if fetchShouldFail.Load() { + return nil, errors.New("mocked error"), true + } + + return nil, nil, false + }) + + // Produce some records. + writeClient := newKafkaProduceClient(t, clusterAddr) + produceRecord(ctx, t, writeClient, topicName, partitionID, []byte("record-1")) + produceRecord(ctx, t, writeClient, topicName, partitionID, []byte("record-2")) + t.Log("produced 2 records") + + // Run the test twice with the same Kafka cluster to show that second time it consumes all records again. + // Reset the test. + fetchShouldFail.Store(true) + fetchRequestsCount.Store(0) + consumedRecordsMx.Lock() + consumedRecords = nil + consumedRecordsMx.Unlock() + + // Create and start the reader. + reg := prometheus.NewPedanticRegistry() + logs := &concurrency.SyncBuffer{} + reader := createReader(t, clusterAddr, topicName, partitionID, consumer, + withConsumeFromPositionAtStartup(consumeFromStart), + withTargetAndMaxConsumerLagAtStartup(time.Second, 2*time.Second), + withRegistry(reg), + withLogger(log.NewLogfmtLogger(logs))) + require.NoError(t, reader.StartAsync(ctx)) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(ctx, reader)) + }) + + // Wait until the Kafka cluster received few Fetch requests. + test.Poll(t, 5*time.Second, true, func() interface{} { + return fetchRequestsCount.Load() > 2 + }) + + // Since the mocked Kafka cluster is configured to fail any Fetch we expect the reader hasn't + // catched up yet, and it's still in Starting state. + assert.Equal(t, services.Starting, reader.State()) + + // Unblock the Fetch requests. Now they will succeed. + fetchShouldFail.Store(false) + + // We expect the reader to catch up, and then switch to Running state. + test.Poll(t, 5*time.Second, services.Running, func() interface{} { + return reader.State() + }) + + // We expect the reader to have switched to running because target consumer lag has been honored. + assert.Contains(t, logs.String(), "partition reader consumed partition and current lag is lower than configured target consumer lag") + + // We expect the reader to have consumed the partition from start. + test.Poll(t, time.Second, []string{"record-1", "record-2"}, func() interface{} { + consumedRecordsMx.Lock() + defer consumedRecordsMx.Unlock() + return slices.Clone(consumedRecords) + }) + + // We expect the last consumed offset to be tracked in a metric. + test.Poll(t, time.Second, nil, func() interface{} { + return promtest.GatherAndCompare(reg, strings.NewReader(` + # HELP cortex_ingest_storage_reader_last_consumed_offset The last offset successfully consumed by the partition reader. Set to -1 if not offset has been consumed yet. + # TYPE cortex_ingest_storage_reader_last_consumed_offset gauge + cortex_ingest_storage_reader_last_consumed_offset{partition="1"} 1 + `), "cortex_ingest_storage_reader_last_consumed_offset") + }) + + produceRecord(ctx, t, writeClient, topicName, partitionID, []byte("record-3")) + produceRecord(ctx, t, writeClient, topicName, partitionID, []byte("record-4")) + t.Log("produced 2 records") + + // We expect the reader to have consumed the partition from start. + test.Poll(t, time.Second, []string{"record-1", "record-2", "record-3", "record-4"}, func() interface{} { + consumedRecordsMx.Lock() + defer consumedRecordsMx.Unlock() + return slices.Clone(consumedRecords) + }) + + // We expect the last consumed offset to be tracked in a metric. + test.Poll(t, time.Second, nil, func() interface{} { + return promtest.GatherAndCompare(reg, strings.NewReader(` + # HELP cortex_ingest_storage_reader_last_consumed_offset The last offset successfully consumed by the partition reader. Set to -1 if not offset has been consumed yet. + # TYPE cortex_ingest_storage_reader_last_consumed_offset gauge + cortex_ingest_storage_reader_last_consumed_offset{partition="1"} 3 + `), "cortex_ingest_storage_reader_last_consumed_offset") + }) + }) + t.Run("should consume partition from the timestamp if position=timestamp, and wait until target lag is honored", func(t *testing.T) { t.Parallel() @@ -1654,7 +1770,7 @@ func produceRecord(ctx context.Context, t *testing.T, writeClient *kgo.Client, t type readerTestCfg struct { kafka KafkaConfig partitionID int32 - consumer recordConsumer + consumer consumerFactory registry *prometheus.Registry logger log.Logger } @@ -1717,7 +1833,9 @@ func defaultReaderTestConfig(t *testing.T, addr string, topicName string, partit logger: mimirtest.NewTestingLogger(t), kafka: createTestKafkaConfig(addr, topicName), partitionID: partitionID, - consumer: consumer, + consumer: consumerFactoryFunc(func() recordConsumer { + return consumer + }), } } @@ -1733,6 +1851,9 @@ func createReader(t *testing.T, addr string, topicName string, partitionID int32 reader, err := newPartitionReader(cfg.kafka, cfg.partitionID, "test-group", cfg.consumer, cfg.logger, cfg.registry) require.NoError(t, err) + // Reduce the time the fake kafka would wait for new records. Sometimes this blocks startup. + reader.concurrentFetchersMinBytesMaxWaitTime = time.Second + return reader } @@ -1850,6 +1971,189 @@ func TestPartitionReader_Commit(t *testing.T) { }) } +func TestHandleKafkaFetchErr(t *testing.T) { + logger := log.NewNopLogger() + + tests := map[string]struct { + err error + lso int64 + fw fetchWant + + expectedFw fetchWant + expectedShortBackoff bool + expectedLongBackoff bool + expectedMetadataRefresh bool + }{ + "no error": { + err: nil, + lso: 1, + fw: fetchWant{ + startOffset: 1, + endOffset: 5, + }, + expectedFw: fetchWant{ + startOffset: 1, + endOffset: 5, + }, + }, + "offset out of range - fetching slightly before start": { + err: kerr.OffsetOutOfRange, + lso: 5, + fw: fetchWant{ + startOffset: 4, + endOffset: 10, + }, + expectedFw: fetchWant{ + startOffset: 5, + endOffset: 10, + }, + }, + "offset out of range - fetching completely outside of available offsets": { + err: kerr.OffsetOutOfRange, + lso: 5, + fw: fetchWant{ + startOffset: 1, + endOffset: 3, + }, + expectedFw: fetchWant{ + startOffset: 3, + endOffset: 3, + }, + }, + "recoverable error": { + err: kerr.KafkaStorageError, + lso: -1, // unknown + fw: fetchWant{ + startOffset: 11, + endOffset: 15, + }, + expectedFw: fetchWant{ + startOffset: 11, + endOffset: 15, + }, + expectedLongBackoff: true, + }, + "NotLeaderForPartition": { + err: kerr.NotLeaderForPartition, + lso: 5, + fw: fetchWant{ + startOffset: 11, + endOffset: 15, + }, + expectedFw: fetchWant{ + startOffset: 11, + endOffset: 15, + }, + expectedLongBackoff: true, + expectedMetadataRefresh: true, + }, + "ReplicaNotAvailable": { + err: kerr.ReplicaNotAvailable, + lso: 5, + fw: fetchWant{ + startOffset: 11, + endOffset: 15, + }, + expectedFw: fetchWant{ + startOffset: 11, + endOffset: 15, + }, + expectedLongBackoff: true, + expectedMetadataRefresh: true, + }, + "UnknownLeaderEpoch": { + err: kerr.UnknownLeaderEpoch, + lso: 5, + fw: fetchWant{ + startOffset: 11, + endOffset: 15, + }, + expectedFw: fetchWant{ + startOffset: 11, + endOffset: 15, + }, + expectedLongBackoff: true, + expectedMetadataRefresh: true, + }, + "FencedLeaderEpoch": { + err: kerr.FencedLeaderEpoch, + lso: 5, + fw: fetchWant{ + startOffset: 11, + endOffset: 15, + }, + expectedFw: fetchWant{ + startOffset: 11, + endOffset: 15, + }, + expectedLongBackoff: true, + expectedMetadataRefresh: true, + }, + "LeaderNotAvailable": { + err: kerr.LeaderNotAvailable, + lso: 5, + fw: fetchWant{ + startOffset: 11, + endOffset: 15, + }, + expectedFw: fetchWant{ + startOffset: 11, + endOffset: 15, + }, + expectedLongBackoff: true, + expectedMetadataRefresh: true, + }, + "errUnknownPartitionLeader": { + err: errUnknownPartitionLeader, + lso: 5, + fw: fetchWant{ + startOffset: 11, + endOffset: 15, + }, + expectedFw: fetchWant{ + startOffset: 11, + endOffset: 15, + }, + expectedLongBackoff: true, + expectedMetadataRefresh: true, + }, + } + + for testName, testCase := range tests { + t.Run(testName, func(t *testing.T) { + require.False(t, testCase.expectedShortBackoff && testCase.expectedLongBackoff, "set either long or short backoff") + waitedShort := false + shortBackOff := waiterFunc(func() { waitedShort = true }) + waitedLong := false + longBackOff := waiterFunc(func() { waitedLong = true }) + refreshed := false + refresher := refresherFunc(func() { refreshed = true }) + + offsetR := newGenericOffsetReader(func(_ context.Context) (int64, error) { + return testCase.lso, nil + }, time.Millisecond, logger) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), offsetR)) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), offsetR)) + }) + + actualFw := handleKafkaFetchErr(testCase.err, testCase.fw, shortBackOff, longBackOff, offsetR, refresher, logger) + assert.Equal(t, testCase.expectedFw, actualFw) + assert.Equal(t, testCase.expectedShortBackoff, waitedShort) + assert.Equal(t, testCase.expectedLongBackoff, waitedLong) + assert.Equal(t, testCase.expectedMetadataRefresh, refreshed) + }) + } +} + +type waiterFunc func() + +func (w waiterFunc) Wait() { w() } + +type refresherFunc func() + +func (r refresherFunc) ForceMetadataRefresh() { r() } + type testConsumer struct { records chan []byte } @@ -1860,7 +2164,7 @@ func newTestConsumer(capacity int) testConsumer { } } -func (t testConsumer) consume(ctx context.Context, records []record) error { +func (t testConsumer) Consume(ctx context.Context, records []record) error { for _, r := range records { select { case <-ctx.Done(): @@ -1901,7 +2205,7 @@ func (t testConsumer) waitRecords(numRecords int, waitTimeout, drainPeriod time. type consumerFunc func(ctx context.Context, records []record) error -func (c consumerFunc) consume(ctx context.Context, records []record) error { +func (c consumerFunc) Consume(ctx context.Context, records []record) error { return c(ctx, records) } diff --git a/pkg/storage/ingest/util.go b/pkg/storage/ingest/util.go index b17a433aeb4..6abde06f73d 100644 --- a/pkg/storage/ingest/util.go +++ b/pkg/storage/ingest/util.go @@ -97,10 +97,7 @@ func commonKafkaClientOptions(cfg KafkaConfig, metrics *kprom.Metrics, logger lo opts = append(opts, kgo.AllowAutoTopicCreation()) } - tracer := kotel.NewTracer( - kotel.TracerPropagator(propagation.NewCompositeTextMapPropagator(onlySampledTraces{propagation.TraceContext{}})), - ) - opts = append(opts, kgo.WithHooks(kotel.NewKotel(kotel.WithTracer(tracer)).Hooks()...)) + opts = append(opts, kgo.WithHooks(kotel.NewKotel(kotel.WithTracer(recordsTracer())).Hooks()...)) if metrics != nil { opts = append(opts, kgo.WithHooks(metrics)) @@ -109,6 +106,10 @@ func commonKafkaClientOptions(cfg KafkaConfig, metrics *kprom.Metrics, logger lo return opts } +func recordsTracer() *kotel.Tracer { + return kotel.NewTracer(kotel.TracerPropagator(propagation.NewCompositeTextMapPropagator(onlySampledTraces{propagation.TraceContext{}}))) +} + // resultPromise is a simple utility to have multiple goroutines waiting for a result from another one. type resultPromise[T any] struct { // done is a channel used to wait the result. Once the channel is closed diff --git a/pkg/storage/ingest/writer_test.go b/pkg/storage/ingest/writer_test.go index 6156a9b4913..f36dec78144 100644 --- a/pkg/storage/ingest/writer_test.go +++ b/pkg/storage/ingest/writer_test.go @@ -1078,6 +1078,8 @@ func createTestKafkaConfig(clusterAddr, topicName string) KafkaConfig { cfg.Address = clusterAddr cfg.Topic = topicName cfg.WriteTimeout = 2 * time.Second + cfg.ReplayConcurrency = 2 + cfg.RecordsPerFetch = 2 return cfg } diff --git a/tools/kafka-dump/main.go b/tools/kafka-dump/main.go new file mode 100644 index 00000000000..23a6aedcce4 --- /dev/null +++ b/tools/kafka-dump/main.go @@ -0,0 +1,219 @@ +// SPDX-License-Identifier: AGPL-3.0-only + +package main + +import ( + "bufio" + "context" + "errors" + "flag" + "fmt" + "io" + "os" + "time" + + "github.com/go-jose/go-jose/v3/json" + "github.com/grafana/dskit/flagext" + "github.com/prometheus/client_golang/prometheus" + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kgo" + "github.com/twmb/franz-go/plugin/kprom" + "go.uber.org/atomic" +) + +type key int + +const ( + originalOffsetKey key = iota +) + +func main() { + // Clean up all flags registered via init() methods of 3rd-party libraries. + flag.CommandLine = flag.NewFlagSet(os.Args[0], flag.ExitOnError) + + cfg := struct { + topic string + brokers string + partition int + skipFirst int + mode string + exportOffsetStart int64 + exportMaxRecords int + fileName string + }{} + + flag.StringVar(&cfg.topic, "topic", "mimir", "Kafka topic to dump") + flag.StringVar(&cfg.brokers, "brokers", "localhost:9092", "Kafka brokers") + flag.IntVar(&cfg.partition, "partition", 0, "Kafka partition to dump or import into") + flag.IntVar(&cfg.skipFirst, "skipFirst", 0, "Skip until input record with offset N") + flag.StringVar(&cfg.mode, "mode", "import", "Mode to run in: import or export") + flag.Int64Var(&cfg.exportOffsetStart, "exportOffsetStart", 0, "Offset to start exporting from") + flag.IntVar(&cfg.exportMaxRecords, "exportMaxRecords", 1_000_000, "Maximum number of records to export") + flag.StringVar(&cfg.fileName, "file", "-", "File to read from or write to. - for stdin/stdout") + + // Parse the CLI arguments. + if err := flagext.ParseFlagsWithoutArguments(flag.CommandLine); err != nil { + fmt.Fprintln(os.Stderr, err.Error()) + os.Exit(1) + } + + _, _ = fmt.Fprintf(os.Stderr, "Importing to topic %q via brokers %q partition %d\n", cfg.topic, cfg.brokers, cfg.partition) + var file io.ReadWriter + if cfg.fileName == "-" { + switch cfg.mode { + case "export": + file = os.Stdout + case "import": + file = os.Stdin + } + } else { + explicitFile, err := os.Open(cfg.fileName) + if err != nil { + _, _ = fmt.Fprintf(os.Stderr, "failed to open file %s: %v\n", cfg.fileName, err) + return + } + defer explicitFile.Close() + file = explicitFile + } + + if cfg.mode == "import" { + if err := doImport(file, cfg.topic, cfg.brokers, cfg.partition, cfg.skipFirst); err != nil { + _, _ = fmt.Fprintf(os.Stderr, "failed to import: %v\n", err) + return + } + } else if cfg.mode == "export" { + if err := doExport(file, cfg.topic, cfg.brokers, cfg.partition, cfg.exportOffsetStart, cfg.exportMaxRecords); err != nil { + _, _ = fmt.Fprintf(os.Stderr, "failed to export: %v\n", err) + return + } + } else { + _, _ = fmt.Fprintf(os.Stderr, "unknown mode %q\n", cfg.mode) + } +} + +func doExport(output io.Writer, topicName string, broker string, partition int, offset int64, maxRecords int) error { + client, err := kgo.NewClient( + kgo.WithHooks(kprom.NewMetrics("franz_go", kprom.Registerer(prometheus.DefaultRegisterer))), + kgo.SeedBrokers(broker), + kgo.ConsumePartitions(map[string]map[int32]kgo.Offset{ + topicName: {int32(partition): kgo.NewOffset().At(offset)}}, + ), + ) + if err != nil { + return fmt.Errorf("failed to create kafka client: %w", err) + } + defer client.Close() + err = testClient(client) + if err != nil { + return fmt.Errorf("failed to test client: %w", err) + } + go func() { + for { + time.Sleep(time.Second) + fmt.Fprintf(os.Stderr, "produced records: %d, offset %d\n", recordCount.Load(), consumedOffset.Load()) + } + }() + + encoder := json.NewEncoder(output) + + for recordCount.Load() < int64(maxRecords) { + fetches := client.PollFetches(context.Background()) + if err != nil { + return fmt.Errorf("failed to fetch records: %w", err) + } + fetches.EachRecord(func(record *kgo.Record) { + if recordCount.Inc() > int64(maxRecords) { + return + } + consumedOffset.Store(record.Offset) + err = encoder.Encode(record) + if err != nil { + panic(fmt.Sprintf("encoding offset %d: %v", record.Offset, err)) + } + }) + } + return nil +} + +var ( + recordCount = &atomic.Int64{} + consumedOffset = &atomic.Int64{} + recordsTooLarge = &atomic.Int64{} + corruptedJSONRecords = &atomic.Int64{} +) + +func doImport(from io.Reader, topic, broker string, partition, skipUntil int) error { + // create kafka client + client, err := kgo.NewClient( + kgo.WithHooks(kprom.NewMetrics("franz_go", kprom.Registerer(prometheus.DefaultRegisterer))), + kgo.SeedBrokers(broker), + kgo.RecordPartitioner(kgo.ManualPartitioner()), + kgo.DisableIdempotentWrite(), + kgo.BrokerMaxWriteBytes(268_435_456), + kgo.MaxBufferedBytes(268_435_456), + ) + if err != nil { + return fmt.Errorf("failed to create kafka client: %w", err) + } + + err = testClient(client) + if err != nil { + return fmt.Errorf("failed to test client: %w", err) + } + go func() { + for { + time.Sleep(time.Second) + fmt.Printf("produced items: %d, of those skipped because too large: %d, buffered records: %d, buffered bytes: %d\n", recordCount.Load(), recordsTooLarge.Load(), client.BufferedProduceRecords(), client.BufferedProduceBytes()) + } + }() + + separator := bufio.NewScanner(from) + separator.Buffer(make([]byte, 10_000_000), 10_000_000) // 10MB buffer because we can have large records + + for recordsIdx := 0; separator.Scan(); recordsIdx++ { + item := separator.Bytes() + record := &kgo.Record{} + err = json.Unmarshal(item, record) + if err != nil { + corruptedJSONRecords.Inc() + _, _ = fmt.Fprintf(os.Stderr, "corrupted JSON record %d: %v\n", recordsIdx, err) + continue + } + if record.Offset < int64(skipUntil) { + continue + } + record.Topic = topic + record.Partition = int32(partition) + record.Context = context.WithValue(context.Background(), originalOffsetKey, record.Offset) + + client.Produce(context.Background(), record, func(record *kgo.Record, err error) { + recordCount.Inc() + if errors.Is(err, kerr.MessageTooLarge) { + recordsTooLarge.Inc() + return + } + if err != nil { + panic(fmt.Sprintf("failed to produce record with offset %d: %v", record.Context.Value(originalOffsetKey), err)) + } + }) + } + fmt.Println("waiting for produce to finish") + err = client.Flush(context.Background()) + if err != nil { + return fmt.Errorf("failed to flush records: %w", err) + } + if separator.Err() != nil { + return fmt.Errorf("separator scan failed: %w", separator.Err()) + } + return nil +} + +func testClient(client *kgo.Client) error { + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + defer cancel() + err := client.Ping(ctx) + if err != nil { + return fmt.Errorf("failed to ping kafka: %w", err) + } + return nil +} diff --git a/vendor/github.com/go-jose/go-jose/v3/LICENSE b/vendor/github.com/go-jose/go-jose/v3/LICENSE new file mode 100644 index 00000000000..d6456956733 --- /dev/null +++ b/vendor/github.com/go-jose/go-jose/v3/LICENSE @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + 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. diff --git a/vendor/github.com/go-jose/go-jose/v3/json/LICENSE b/vendor/github.com/go-jose/go-jose/v3/json/LICENSE new file mode 100644 index 00000000000..74487567632 --- /dev/null +++ b/vendor/github.com/go-jose/go-jose/v3/json/LICENSE @@ -0,0 +1,27 @@ +Copyright (c) 2012 The Go Authors. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/vendor/github.com/go-jose/go-jose/v3/json/README.md b/vendor/github.com/go-jose/go-jose/v3/json/README.md new file mode 100644 index 00000000000..86de5e5581f --- /dev/null +++ b/vendor/github.com/go-jose/go-jose/v3/json/README.md @@ -0,0 +1,13 @@ +# Safe JSON + +This repository contains a fork of the `encoding/json` package from Go 1.6. + +The following changes were made: + +* Object deserialization uses case-sensitive member name matching instead of + [case-insensitive matching](https://www.ietf.org/mail-archive/web/json/current/msg03763.html). + This is to avoid differences in the interpretation of JOSE messages between + go-jose and libraries written in other languages. +* When deserializing a JSON object, we check for duplicate keys and reject the + input whenever we detect a duplicate. Rather than trying to work with malformed + data, we prefer to reject it right away. diff --git a/vendor/github.com/go-jose/go-jose/v3/json/decode.go b/vendor/github.com/go-jose/go-jose/v3/json/decode.go new file mode 100644 index 00000000000..50634dd8478 --- /dev/null +++ b/vendor/github.com/go-jose/go-jose/v3/json/decode.go @@ -0,0 +1,1216 @@ +// Copyright 2010 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Represents JSON data structure using native Go types: booleans, floats, +// strings, arrays, and maps. + +package json + +import ( + "bytes" + "encoding" + "encoding/base64" + "errors" + "fmt" + "math" + "reflect" + "runtime" + "strconv" + "unicode" + "unicode/utf16" + "unicode/utf8" +) + +// Unmarshal parses the JSON-encoded data and stores the result +// in the value pointed to by v. +// +// Unmarshal uses the inverse of the encodings that +// Marshal uses, allocating maps, slices, and pointers as necessary, +// with the following additional rules: +// +// To unmarshal JSON into a pointer, Unmarshal first handles the case of +// the JSON being the JSON literal null. In that case, Unmarshal sets +// the pointer to nil. Otherwise, Unmarshal unmarshals the JSON into +// the value pointed at by the pointer. If the pointer is nil, Unmarshal +// allocates a new value for it to point to. +// +// To unmarshal JSON into a struct, Unmarshal matches incoming object +// keys to the keys used by Marshal (either the struct field name or its tag), +// preferring an exact match but also accepting a case-insensitive match. +// Unmarshal will only set exported fields of the struct. +// +// To unmarshal JSON into an interface value, +// Unmarshal stores one of these in the interface value: +// +// bool, for JSON booleans +// float64, for JSON numbers +// string, for JSON strings +// []interface{}, for JSON arrays +// map[string]interface{}, for JSON objects +// nil for JSON null +// +// To unmarshal a JSON array into a slice, Unmarshal resets the slice length +// to zero and then appends each element to the slice. +// As a special case, to unmarshal an empty JSON array into a slice, +// Unmarshal replaces the slice with a new empty slice. +// +// To unmarshal a JSON array into a Go array, Unmarshal decodes +// JSON array elements into corresponding Go array elements. +// If the Go array is smaller than the JSON array, +// the additional JSON array elements are discarded. +// If the JSON array is smaller than the Go array, +// the additional Go array elements are set to zero values. +// +// To unmarshal a JSON object into a string-keyed map, Unmarshal first +// establishes a map to use, If the map is nil, Unmarshal allocates a new map. +// Otherwise Unmarshal reuses the existing map, keeping existing entries. +// Unmarshal then stores key-value pairs from the JSON object into the map. +// +// If a JSON value is not appropriate for a given target type, +// or if a JSON number overflows the target type, Unmarshal +// skips that field and completes the unmarshaling as best it can. +// If no more serious errors are encountered, Unmarshal returns +// an UnmarshalTypeError describing the earliest such error. +// +// The JSON null value unmarshals into an interface, map, pointer, or slice +// by setting that Go value to nil. Because null is often used in JSON to mean +// “not present,” unmarshaling a JSON null into any other Go type has no effect +// on the value and produces no error. +// +// When unmarshaling quoted strings, invalid UTF-8 or +// invalid UTF-16 surrogate pairs are not treated as an error. +// Instead, they are replaced by the Unicode replacement +// character U+FFFD. +func Unmarshal(data []byte, v interface{}) error { + // Check for well-formedness. + // Avoids filling out half a data structure + // before discovering a JSON syntax error. + var d decodeState + err := checkValid(data, &d.scan) + if err != nil { + return err + } + + d.init(data) + return d.unmarshal(v) +} + +// Unmarshaler is the interface implemented by objects +// that can unmarshal a JSON description of themselves. +// The input can be assumed to be a valid encoding of +// a JSON value. UnmarshalJSON must copy the JSON data +// if it wishes to retain the data after returning. +type Unmarshaler interface { + UnmarshalJSON([]byte) error +} + +// An UnmarshalTypeError describes a JSON value that was +// not appropriate for a value of a specific Go type. +type UnmarshalTypeError struct { + Value string // description of JSON value - "bool", "array", "number -5" + Type reflect.Type // type of Go value it could not be assigned to + Offset int64 // error occurred after reading Offset bytes +} + +func (e *UnmarshalTypeError) Error() string { + return "json: cannot unmarshal " + e.Value + " into Go value of type " + e.Type.String() +} + +// An UnmarshalFieldError describes a JSON object key that +// led to an unexported (and therefore unwritable) struct field. +// (No longer used; kept for compatibility.) +type UnmarshalFieldError struct { + Key string + Type reflect.Type + Field reflect.StructField +} + +func (e *UnmarshalFieldError) Error() string { + return "json: cannot unmarshal object key " + strconv.Quote(e.Key) + " into unexported field " + e.Field.Name + " of type " + e.Type.String() +} + +// An InvalidUnmarshalError describes an invalid argument passed to Unmarshal. +// (The argument to Unmarshal must be a non-nil pointer.) +type InvalidUnmarshalError struct { + Type reflect.Type +} + +func (e *InvalidUnmarshalError) Error() string { + if e.Type == nil { + return "json: Unmarshal(nil)" + } + + if e.Type.Kind() != reflect.Ptr { + return "json: Unmarshal(non-pointer " + e.Type.String() + ")" + } + return "json: Unmarshal(nil " + e.Type.String() + ")" +} + +func (d *decodeState) unmarshal(v interface{}) (err error) { + defer func() { + if r := recover(); r != nil { + if _, ok := r.(runtime.Error); ok { + panic(r) + } + err = r.(error) + } + }() + + rv := reflect.ValueOf(v) + if rv.Kind() != reflect.Ptr || rv.IsNil() { + return &InvalidUnmarshalError{reflect.TypeOf(v)} + } + + d.scan.reset() + // We decode rv not rv.Elem because the Unmarshaler interface + // test must be applied at the top level of the value. + d.value(rv) + return d.savedError +} + +// A Number represents a JSON number literal. +type Number string + +// String returns the literal text of the number. +func (n Number) String() string { return string(n) } + +// Float64 returns the number as a float64. +func (n Number) Float64() (float64, error) { + return strconv.ParseFloat(string(n), 64) +} + +// Int64 returns the number as an int64. +func (n Number) Int64() (int64, error) { + return strconv.ParseInt(string(n), 10, 64) +} + +// isValidNumber reports whether s is a valid JSON number literal. +func isValidNumber(s string) bool { + // This function implements the JSON numbers grammar. + // See https://tools.ietf.org/html/rfc7159#section-6 + // and http://json.org/number.gif + + if s == "" { + return false + } + + // Optional - + if s[0] == '-' { + s = s[1:] + if s == "" { + return false + } + } + + // Digits + switch { + default: + return false + + case s[0] == '0': + s = s[1:] + + case '1' <= s[0] && s[0] <= '9': + s = s[1:] + for len(s) > 0 && '0' <= s[0] && s[0] <= '9' { + s = s[1:] + } + } + + // . followed by 1 or more digits. + if len(s) >= 2 && s[0] == '.' && '0' <= s[1] && s[1] <= '9' { + s = s[2:] + for len(s) > 0 && '0' <= s[0] && s[0] <= '9' { + s = s[1:] + } + } + + // e or E followed by an optional - or + and + // 1 or more digits. + if len(s) >= 2 && (s[0] == 'e' || s[0] == 'E') { + s = s[1:] + if s[0] == '+' || s[0] == '-' { + s = s[1:] + if s == "" { + return false + } + } + for len(s) > 0 && '0' <= s[0] && s[0] <= '9' { + s = s[1:] + } + } + + // Make sure we are at the end. + return s == "" +} + +type NumberUnmarshalType int + +const ( + // unmarshal a JSON number into an interface{} as a float64 + UnmarshalFloat NumberUnmarshalType = iota + // unmarshal a JSON number into an interface{} as a `json.Number` + UnmarshalJSONNumber + // unmarshal a JSON number into an interface{} as a int64 + // if value is an integer otherwise float64 + UnmarshalIntOrFloat +) + +// decodeState represents the state while decoding a JSON value. +type decodeState struct { + data []byte + off int // read offset in data + scan scanner + nextscan scanner // for calls to nextValue + savedError error + numberType NumberUnmarshalType +} + +// errPhase is used for errors that should not happen unless +// there is a bug in the JSON decoder or something is editing +// the data slice while the decoder executes. +var errPhase = errors.New("JSON decoder out of sync - data changing underfoot?") + +func (d *decodeState) init(data []byte) *decodeState { + d.data = data + d.off = 0 + d.savedError = nil + return d +} + +// error aborts the decoding by panicking with err. +func (d *decodeState) error(err error) { + panic(err) +} + +// saveError saves the first err it is called with, +// for reporting at the end of the unmarshal. +func (d *decodeState) saveError(err error) { + if d.savedError == nil { + d.savedError = err + } +} + +// next cuts off and returns the next full JSON value in d.data[d.off:]. +// The next value is known to be an object or array, not a literal. +func (d *decodeState) next() []byte { + c := d.data[d.off] + item, rest, err := nextValue(d.data[d.off:], &d.nextscan) + if err != nil { + d.error(err) + } + d.off = len(d.data) - len(rest) + + // Our scanner has seen the opening brace/bracket + // and thinks we're still in the middle of the object. + // invent a closing brace/bracket to get it out. + if c == '{' { + d.scan.step(&d.scan, '}') + } else { + d.scan.step(&d.scan, ']') + } + + return item +} + +// scanWhile processes bytes in d.data[d.off:] until it +// receives a scan code not equal to op. +// It updates d.off and returns the new scan code. +func (d *decodeState) scanWhile(op int) int { + var newOp int + for { + if d.off >= len(d.data) { + newOp = d.scan.eof() + d.off = len(d.data) + 1 // mark processed EOF with len+1 + } else { + c := d.data[d.off] + d.off++ + newOp = d.scan.step(&d.scan, c) + } + if newOp != op { + break + } + } + return newOp +} + +// value decodes a JSON value from d.data[d.off:] into the value. +// it updates d.off to point past the decoded value. +func (d *decodeState) value(v reflect.Value) { + if !v.IsValid() { + _, rest, err := nextValue(d.data[d.off:], &d.nextscan) + if err != nil { + d.error(err) + } + d.off = len(d.data) - len(rest) + + // d.scan thinks we're still at the beginning of the item. + // Feed in an empty string - the shortest, simplest value - + // so that it knows we got to the end of the value. + if d.scan.redo { + // rewind. + d.scan.redo = false + d.scan.step = stateBeginValue + } + d.scan.step(&d.scan, '"') + d.scan.step(&d.scan, '"') + + n := len(d.scan.parseState) + if n > 0 && d.scan.parseState[n-1] == parseObjectKey { + // d.scan thinks we just read an object key; finish the object + d.scan.step(&d.scan, ':') + d.scan.step(&d.scan, '"') + d.scan.step(&d.scan, '"') + d.scan.step(&d.scan, '}') + } + + return + } + + switch op := d.scanWhile(scanSkipSpace); op { + default: + d.error(errPhase) + + case scanBeginArray: + d.array(v) + + case scanBeginObject: + d.object(v) + + case scanBeginLiteral: + d.literal(v) + } +} + +type unquotedValue struct{} + +// valueQuoted is like value but decodes a +// quoted string literal or literal null into an interface value. +// If it finds anything other than a quoted string literal or null, +// valueQuoted returns unquotedValue{}. +func (d *decodeState) valueQuoted() interface{} { + switch op := d.scanWhile(scanSkipSpace); op { + default: + d.error(errPhase) + + case scanBeginArray: + d.array(reflect.Value{}) + + case scanBeginObject: + d.object(reflect.Value{}) + + case scanBeginLiteral: + switch v := d.literalInterface().(type) { + case nil, string: + return v + } + } + return unquotedValue{} +} + +// indirect walks down v allocating pointers as needed, +// until it gets to a non-pointer. +// if it encounters an Unmarshaler, indirect stops and returns that. +// if decodingNull is true, indirect stops at the last pointer so it can be set to nil. +func (d *decodeState) indirect(v reflect.Value, decodingNull bool) (Unmarshaler, encoding.TextUnmarshaler, reflect.Value) { + // If v is a named type and is addressable, + // start with its address, so that if the type has pointer methods, + // we find them. + if v.Kind() != reflect.Ptr && v.Type().Name() != "" && v.CanAddr() { + v = v.Addr() + } + for { + // Load value from interface, but only if the result will be + // usefully addressable. + if v.Kind() == reflect.Interface && !v.IsNil() { + e := v.Elem() + if e.Kind() == reflect.Ptr && !e.IsNil() && (!decodingNull || e.Elem().Kind() == reflect.Ptr) { + v = e + continue + } + } + + if v.Kind() != reflect.Ptr { + break + } + + if v.Elem().Kind() != reflect.Ptr && decodingNull && v.CanSet() { + break + } + if v.IsNil() { + v.Set(reflect.New(v.Type().Elem())) + } + if v.Type().NumMethod() > 0 { + if u, ok := v.Interface().(Unmarshaler); ok { + return u, nil, reflect.Value{} + } + if u, ok := v.Interface().(encoding.TextUnmarshaler); ok { + return nil, u, reflect.Value{} + } + } + v = v.Elem() + } + return nil, nil, v +} + +// array consumes an array from d.data[d.off-1:], decoding into the value v. +// the first byte of the array ('[') has been read already. +func (d *decodeState) array(v reflect.Value) { + // Check for unmarshaler. + u, ut, pv := d.indirect(v, false) + if u != nil { + d.off-- + err := u.UnmarshalJSON(d.next()) + if err != nil { + d.error(err) + } + return + } + if ut != nil { + d.saveError(&UnmarshalTypeError{"array", v.Type(), int64(d.off)}) + d.off-- + d.next() + return + } + + v = pv + + // Check type of target. + switch v.Kind() { + case reflect.Interface: + if v.NumMethod() == 0 { + // Decoding into nil interface? Switch to non-reflect code. + v.Set(reflect.ValueOf(d.arrayInterface())) + return + } + // Otherwise it's invalid. + fallthrough + default: + d.saveError(&UnmarshalTypeError{"array", v.Type(), int64(d.off)}) + d.off-- + d.next() + return + case reflect.Array: + case reflect.Slice: + break + } + + i := 0 + for { + // Look ahead for ] - can only happen on first iteration. + op := d.scanWhile(scanSkipSpace) + if op == scanEndArray { + break + } + + // Back up so d.value can have the byte we just read. + d.off-- + d.scan.undo(op) + + // Get element of array, growing if necessary. + if v.Kind() == reflect.Slice { + // Grow slice if necessary + if i >= v.Cap() { + newcap := v.Cap() + v.Cap()/2 + if newcap < 4 { + newcap = 4 + } + newv := reflect.MakeSlice(v.Type(), v.Len(), newcap) + reflect.Copy(newv, v) + v.Set(newv) + } + if i >= v.Len() { + v.SetLen(i + 1) + } + } + + if i < v.Len() { + // Decode into element. + d.value(v.Index(i)) + } else { + // Ran out of fixed array: skip. + d.value(reflect.Value{}) + } + i++ + + // Next token must be , or ]. + op = d.scanWhile(scanSkipSpace) + if op == scanEndArray { + break + } + if op != scanArrayValue { + d.error(errPhase) + } + } + + if i < v.Len() { + if v.Kind() == reflect.Array { + // Array. Zero the rest. + z := reflect.Zero(v.Type().Elem()) + for ; i < v.Len(); i++ { + v.Index(i).Set(z) + } + } else { + v.SetLen(i) + } + } + if i == 0 && v.Kind() == reflect.Slice { + v.Set(reflect.MakeSlice(v.Type(), 0, 0)) + } +} + +var nullLiteral = []byte("null") + +// object consumes an object from d.data[d.off-1:], decoding into the value v. +// the first byte ('{') of the object has been read already. +func (d *decodeState) object(v reflect.Value) { + // Check for unmarshaler. + u, ut, pv := d.indirect(v, false) + if u != nil { + d.off-- + err := u.UnmarshalJSON(d.next()) + if err != nil { + d.error(err) + } + return + } + if ut != nil { + d.saveError(&UnmarshalTypeError{"object", v.Type(), int64(d.off)}) + d.off-- + d.next() // skip over { } in input + return + } + v = pv + + // Decoding into nil interface? Switch to non-reflect code. + if v.Kind() == reflect.Interface && v.NumMethod() == 0 { + v.Set(reflect.ValueOf(d.objectInterface())) + return + } + + // Check type of target: struct or map[string]T + switch v.Kind() { + case reflect.Map: + // map must have string kind + t := v.Type() + if t.Key().Kind() != reflect.String { + d.saveError(&UnmarshalTypeError{"object", v.Type(), int64(d.off)}) + d.off-- + d.next() // skip over { } in input + return + } + if v.IsNil() { + v.Set(reflect.MakeMap(t)) + } + case reflect.Struct: + + default: + d.saveError(&UnmarshalTypeError{"object", v.Type(), int64(d.off)}) + d.off-- + d.next() // skip over { } in input + return + } + + var mapElem reflect.Value + keys := map[string]bool{} + + for { + // Read opening " of string key or closing }. + op := d.scanWhile(scanSkipSpace) + if op == scanEndObject { + // closing } - can only happen on first iteration. + break + } + if op != scanBeginLiteral { + d.error(errPhase) + } + + // Read key. + start := d.off - 1 + op = d.scanWhile(scanContinue) + item := d.data[start : d.off-1] + key, ok := unquote(item) + if !ok { + d.error(errPhase) + } + + // Check for duplicate keys. + _, ok = keys[key] + if !ok { + keys[key] = true + } else { + d.error(fmt.Errorf("json: duplicate key '%s' in object", key)) + } + + // Figure out field corresponding to key. + var subv reflect.Value + destring := false // whether the value is wrapped in a string to be decoded first + + if v.Kind() == reflect.Map { + elemType := v.Type().Elem() + if !mapElem.IsValid() { + mapElem = reflect.New(elemType).Elem() + } else { + mapElem.Set(reflect.Zero(elemType)) + } + subv = mapElem + } else { + var f *field + fields := cachedTypeFields(v.Type()) + for i := range fields { + ff := &fields[i] + if bytes.Equal(ff.nameBytes, []byte(key)) { + f = ff + break + } + } + if f != nil { + subv = v + destring = f.quoted + for _, i := range f.index { + if subv.Kind() == reflect.Ptr { + if subv.IsNil() { + subv.Set(reflect.New(subv.Type().Elem())) + } + subv = subv.Elem() + } + subv = subv.Field(i) + } + } + } + + // Read : before value. + if op == scanSkipSpace { + op = d.scanWhile(scanSkipSpace) + } + if op != scanObjectKey { + d.error(errPhase) + } + + // Read value. + if destring { + switch qv := d.valueQuoted().(type) { + case nil: + d.literalStore(nullLiteral, subv, false) + case string: + d.literalStore([]byte(qv), subv, true) + default: + d.saveError(fmt.Errorf("json: invalid use of ,string struct tag, trying to unmarshal unquoted value into %v", subv.Type())) + } + } else { + d.value(subv) + } + + // Write value back to map; + // if using struct, subv points into struct already. + if v.Kind() == reflect.Map { + kv := reflect.ValueOf(key).Convert(v.Type().Key()) + v.SetMapIndex(kv, subv) + } + + // Next token must be , or }. + op = d.scanWhile(scanSkipSpace) + if op == scanEndObject { + break + } + if op != scanObjectValue { + d.error(errPhase) + } + } +} + +// literal consumes a literal from d.data[d.off-1:], decoding into the value v. +// The first byte of the literal has been read already +// (that's how the caller knows it's a literal). +func (d *decodeState) literal(v reflect.Value) { + // All bytes inside literal return scanContinue op code. + start := d.off - 1 + op := d.scanWhile(scanContinue) + + // Scan read one byte too far; back up. + d.off-- + d.scan.undo(op) + + d.literalStore(d.data[start:d.off], v, false) +} + +// convertNumber converts the number literal s to a float64, int64 or a Number +// depending on d.numberDecodeType. +func (d *decodeState) convertNumber(s string) (interface{}, error) { + switch d.numberType { + + case UnmarshalJSONNumber: + return Number(s), nil + case UnmarshalIntOrFloat: + v, err := strconv.ParseInt(s, 10, 64) + if err == nil { + return v, nil + } + + // tries to parse integer number in scientific notation + f, err := strconv.ParseFloat(s, 64) + if err != nil { + return nil, &UnmarshalTypeError{"number " + s, reflect.TypeOf(0.0), int64(d.off)} + } + + // if it has no decimal value use int64 + if fi, fd := math.Modf(f); fd == 0.0 { + return int64(fi), nil + } + return f, nil + default: + f, err := strconv.ParseFloat(s, 64) + if err != nil { + return nil, &UnmarshalTypeError{"number " + s, reflect.TypeOf(0.0), int64(d.off)} + } + return f, nil + } + +} + +var numberType = reflect.TypeOf(Number("")) + +// literalStore decodes a literal stored in item into v. +// +// fromQuoted indicates whether this literal came from unwrapping a +// string from the ",string" struct tag option. this is used only to +// produce more helpful error messages. +func (d *decodeState) literalStore(item []byte, v reflect.Value, fromQuoted bool) { + // Check for unmarshaler. + if len(item) == 0 { + //Empty string given + d.saveError(fmt.Errorf("json: invalid use of ,string struct tag, trying to unmarshal %q into %v", item, v.Type())) + return + } + wantptr := item[0] == 'n' // null + u, ut, pv := d.indirect(v, wantptr) + if u != nil { + err := u.UnmarshalJSON(item) + if err != nil { + d.error(err) + } + return + } + if ut != nil { + if item[0] != '"' { + if fromQuoted { + d.saveError(fmt.Errorf("json: invalid use of ,string struct tag, trying to unmarshal %q into %v", item, v.Type())) + } else { + d.saveError(&UnmarshalTypeError{"string", v.Type(), int64(d.off)}) + } + return + } + s, ok := unquoteBytes(item) + if !ok { + if fromQuoted { + d.error(fmt.Errorf("json: invalid use of ,string struct tag, trying to unmarshal %q into %v", item, v.Type())) + } else { + d.error(errPhase) + } + } + err := ut.UnmarshalText(s) + if err != nil { + d.error(err) + } + return + } + + v = pv + + switch c := item[0]; c { + case 'n': // null + switch v.Kind() { + case reflect.Interface, reflect.Ptr, reflect.Map, reflect.Slice: + v.Set(reflect.Zero(v.Type())) + // otherwise, ignore null for primitives/string + } + case 't', 'f': // true, false + value := c == 't' + switch v.Kind() { + default: + if fromQuoted { + d.saveError(fmt.Errorf("json: invalid use of ,string struct tag, trying to unmarshal %q into %v", item, v.Type())) + } else { + d.saveError(&UnmarshalTypeError{"bool", v.Type(), int64(d.off)}) + } + case reflect.Bool: + v.SetBool(value) + case reflect.Interface: + if v.NumMethod() == 0 { + v.Set(reflect.ValueOf(value)) + } else { + d.saveError(&UnmarshalTypeError{"bool", v.Type(), int64(d.off)}) + } + } + + case '"': // string + s, ok := unquoteBytes(item) + if !ok { + if fromQuoted { + d.error(fmt.Errorf("json: invalid use of ,string struct tag, trying to unmarshal %q into %v", item, v.Type())) + } else { + d.error(errPhase) + } + } + switch v.Kind() { + default: + d.saveError(&UnmarshalTypeError{"string", v.Type(), int64(d.off)}) + case reflect.Slice: + if v.Type().Elem().Kind() != reflect.Uint8 { + d.saveError(&UnmarshalTypeError{"string", v.Type(), int64(d.off)}) + break + } + b := make([]byte, base64.StdEncoding.DecodedLen(len(s))) + n, err := base64.StdEncoding.Decode(b, s) + if err != nil { + d.saveError(err) + break + } + v.SetBytes(b[:n]) + case reflect.String: + v.SetString(string(s)) + case reflect.Interface: + if v.NumMethod() == 0 { + v.Set(reflect.ValueOf(string(s))) + } else { + d.saveError(&UnmarshalTypeError{"string", v.Type(), int64(d.off)}) + } + } + + default: // number + if c != '-' && (c < '0' || c > '9') { + if fromQuoted { + d.error(fmt.Errorf("json: invalid use of ,string struct tag, trying to unmarshal %q into %v", item, v.Type())) + } else { + d.error(errPhase) + } + } + s := string(item) + switch v.Kind() { + default: + if v.Kind() == reflect.String && v.Type() == numberType { + v.SetString(s) + if !isValidNumber(s) { + d.error(fmt.Errorf("json: invalid number literal, trying to unmarshal %q into Number", item)) + } + break + } + if fromQuoted { + d.error(fmt.Errorf("json: invalid use of ,string struct tag, trying to unmarshal %q into %v", item, v.Type())) + } else { + d.error(&UnmarshalTypeError{"number", v.Type(), int64(d.off)}) + } + case reflect.Interface: + n, err := d.convertNumber(s) + if err != nil { + d.saveError(err) + break + } + if v.NumMethod() != 0 { + d.saveError(&UnmarshalTypeError{"number", v.Type(), int64(d.off)}) + break + } + v.Set(reflect.ValueOf(n)) + + case reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64: + n, err := strconv.ParseInt(s, 10, 64) + if err != nil || v.OverflowInt(n) { + d.saveError(&UnmarshalTypeError{"number " + s, v.Type(), int64(d.off)}) + break + } + v.SetInt(n) + + case reflect.Uint, reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64, reflect.Uintptr: + n, err := strconv.ParseUint(s, 10, 64) + if err != nil || v.OverflowUint(n) { + d.saveError(&UnmarshalTypeError{"number " + s, v.Type(), int64(d.off)}) + break + } + v.SetUint(n) + + case reflect.Float32, reflect.Float64: + n, err := strconv.ParseFloat(s, v.Type().Bits()) + if err != nil || v.OverflowFloat(n) { + d.saveError(&UnmarshalTypeError{"number " + s, v.Type(), int64(d.off)}) + break + } + v.SetFloat(n) + } + } +} + +// The xxxInterface routines build up a value to be stored +// in an empty interface. They are not strictly necessary, +// but they avoid the weight of reflection in this common case. + +// valueInterface is like value but returns interface{} +func (d *decodeState) valueInterface() interface{} { + switch d.scanWhile(scanSkipSpace) { + default: + d.error(errPhase) + panic("unreachable") + case scanBeginArray: + return d.arrayInterface() + case scanBeginObject: + return d.objectInterface() + case scanBeginLiteral: + return d.literalInterface() + } +} + +// arrayInterface is like array but returns []interface{}. +func (d *decodeState) arrayInterface() []interface{} { + var v = make([]interface{}, 0) + for { + // Look ahead for ] - can only happen on first iteration. + op := d.scanWhile(scanSkipSpace) + if op == scanEndArray { + break + } + + // Back up so d.value can have the byte we just read. + d.off-- + d.scan.undo(op) + + v = append(v, d.valueInterface()) + + // Next token must be , or ]. + op = d.scanWhile(scanSkipSpace) + if op == scanEndArray { + break + } + if op != scanArrayValue { + d.error(errPhase) + } + } + return v +} + +// objectInterface is like object but returns map[string]interface{}. +func (d *decodeState) objectInterface() map[string]interface{} { + m := make(map[string]interface{}) + keys := map[string]bool{} + + for { + // Read opening " of string key or closing }. + op := d.scanWhile(scanSkipSpace) + if op == scanEndObject { + // closing } - can only happen on first iteration. + break + } + if op != scanBeginLiteral { + d.error(errPhase) + } + + // Read string key. + start := d.off - 1 + op = d.scanWhile(scanContinue) + item := d.data[start : d.off-1] + key, ok := unquote(item) + if !ok { + d.error(errPhase) + } + + // Check for duplicate keys. + _, ok = keys[key] + if !ok { + keys[key] = true + } else { + d.error(fmt.Errorf("json: duplicate key '%s' in object", key)) + } + + // Read : before value. + if op == scanSkipSpace { + op = d.scanWhile(scanSkipSpace) + } + if op != scanObjectKey { + d.error(errPhase) + } + + // Read value. + m[key] = d.valueInterface() + + // Next token must be , or }. + op = d.scanWhile(scanSkipSpace) + if op == scanEndObject { + break + } + if op != scanObjectValue { + d.error(errPhase) + } + } + return m +} + +// literalInterface is like literal but returns an interface value. +func (d *decodeState) literalInterface() interface{} { + // All bytes inside literal return scanContinue op code. + start := d.off - 1 + op := d.scanWhile(scanContinue) + + // Scan read one byte too far; back up. + d.off-- + d.scan.undo(op) + item := d.data[start:d.off] + + switch c := item[0]; c { + case 'n': // null + return nil + + case 't', 'f': // true, false + return c == 't' + + case '"': // string + s, ok := unquote(item) + if !ok { + d.error(errPhase) + } + return s + + default: // number + if c != '-' && (c < '0' || c > '9') { + d.error(errPhase) + } + n, err := d.convertNumber(string(item)) + if err != nil { + d.saveError(err) + } + return n + } +} + +// getu4 decodes \uXXXX from the beginning of s, returning the hex value, +// or it returns -1. +func getu4(s []byte) rune { + if len(s) < 6 || s[0] != '\\' || s[1] != 'u' { + return -1 + } + r, err := strconv.ParseUint(string(s[2:6]), 16, 64) + if err != nil { + return -1 + } + return rune(r) +} + +// unquote converts a quoted JSON string literal s into an actual string t. +// The rules are different than for Go, so cannot use strconv.Unquote. +func unquote(s []byte) (t string, ok bool) { + s, ok = unquoteBytes(s) + t = string(s) + return +} + +func unquoteBytes(s []byte) (t []byte, ok bool) { + if len(s) < 2 || s[0] != '"' || s[len(s)-1] != '"' { + return + } + s = s[1 : len(s)-1] + + // Check for unusual characters. If there are none, + // then no unquoting is needed, so return a slice of the + // original bytes. + r := 0 + for r < len(s) { + c := s[r] + if c == '\\' || c == '"' || c < ' ' { + break + } + if c < utf8.RuneSelf { + r++ + continue + } + rr, size := utf8.DecodeRune(s[r:]) + if rr == utf8.RuneError && size == 1 { + break + } + r += size + } + if r == len(s) { + return s, true + } + + b := make([]byte, len(s)+2*utf8.UTFMax) + w := copy(b, s[0:r]) + for r < len(s) { + // Out of room? Can only happen if s is full of + // malformed UTF-8 and we're replacing each + // byte with RuneError. + if w >= len(b)-2*utf8.UTFMax { + nb := make([]byte, (len(b)+utf8.UTFMax)*2) + copy(nb, b[0:w]) + b = nb + } + switch c := s[r]; { + case c == '\\': + r++ + if r >= len(s) { + return + } + switch s[r] { + default: + return + case '"', '\\', '/', '\'': + b[w] = s[r] + r++ + w++ + case 'b': + b[w] = '\b' + r++ + w++ + case 'f': + b[w] = '\f' + r++ + w++ + case 'n': + b[w] = '\n' + r++ + w++ + case 'r': + b[w] = '\r' + r++ + w++ + case 't': + b[w] = '\t' + r++ + w++ + case 'u': + r-- + rr := getu4(s[r:]) + if rr < 0 { + return + } + r += 6 + if utf16.IsSurrogate(rr) { + rr1 := getu4(s[r:]) + if dec := utf16.DecodeRune(rr, rr1); dec != unicode.ReplacementChar { + // A valid pair; consume. + r += 6 + w += utf8.EncodeRune(b[w:], dec) + break + } + // Invalid surrogate; fall back to replacement rune. + rr = unicode.ReplacementChar + } + w += utf8.EncodeRune(b[w:], rr) + } + + // Quote, control characters are invalid. + case c == '"', c < ' ': + return + + // ASCII + case c < utf8.RuneSelf: + b[w] = c + r++ + w++ + + // Coerce to well-formed UTF-8. + default: + rr, size := utf8.DecodeRune(s[r:]) + r += size + w += utf8.EncodeRune(b[w:], rr) + } + } + return b[0:w], true +} diff --git a/vendor/github.com/go-jose/go-jose/v3/json/encode.go b/vendor/github.com/go-jose/go-jose/v3/json/encode.go new file mode 100644 index 00000000000..98de68ce1e9 --- /dev/null +++ b/vendor/github.com/go-jose/go-jose/v3/json/encode.go @@ -0,0 +1,1197 @@ +// Copyright 2010 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Package json implements encoding and decoding of JSON objects as defined in +// RFC 4627. The mapping between JSON objects and Go values is described +// in the documentation for the Marshal and Unmarshal functions. +// +// See "JSON and Go" for an introduction to this package: +// https://golang.org/doc/articles/json_and_go.html +package json + +import ( + "bytes" + "encoding" + "encoding/base64" + "fmt" + "math" + "reflect" + "runtime" + "sort" + "strconv" + "strings" + "sync" + "unicode" + "unicode/utf8" +) + +// Marshal returns the JSON encoding of v. +// +// Marshal traverses the value v recursively. +// If an encountered value implements the Marshaler interface +// and is not a nil pointer, Marshal calls its MarshalJSON method +// to produce JSON. If no MarshalJSON method is present but the +// value implements encoding.TextMarshaler instead, Marshal calls +// its MarshalText method. +// The nil pointer exception is not strictly necessary +// but mimics a similar, necessary exception in the behavior of +// UnmarshalJSON. +// +// Otherwise, Marshal uses the following type-dependent default encodings: +// +// Boolean values encode as JSON booleans. +// +// Floating point, integer, and Number values encode as JSON numbers. +// +// String values encode as JSON strings coerced to valid UTF-8, +// replacing invalid bytes with the Unicode replacement rune. +// The angle brackets "<" and ">" are escaped to "\u003c" and "\u003e" +// to keep some browsers from misinterpreting JSON output as HTML. +// Ampersand "&" is also escaped to "\u0026" for the same reason. +// +// Array and slice values encode as JSON arrays, except that +// []byte encodes as a base64-encoded string, and a nil slice +// encodes as the null JSON object. +// +// Struct values encode as JSON objects. Each exported struct field +// becomes a member of the object unless +// - the field's tag is "-", or +// - the field is empty and its tag specifies the "omitempty" option. +// +// The empty values are false, 0, any +// nil pointer or interface value, and any array, slice, map, or string of +// length zero. The object's default key string is the struct field name +// but can be specified in the struct field's tag value. The "json" key in +// the struct field's tag value is the key name, followed by an optional comma +// and options. Examples: +// +// // Field is ignored by this package. +// Field int `json:"-"` +// +// // Field appears in JSON as key "myName". +// Field int `json:"myName"` +// +// // Field appears in JSON as key "myName" and +// // the field is omitted from the object if its value is empty, +// // as defined above. +// Field int `json:"myName,omitempty"` +// +// // Field appears in JSON as key "Field" (the default), but +// // the field is skipped if empty. +// // Note the leading comma. +// Field int `json:",omitempty"` +// +// The "string" option signals that a field is stored as JSON inside a +// JSON-encoded string. It applies only to fields of string, floating point, +// integer, or boolean types. This extra level of encoding is sometimes used +// when communicating with JavaScript programs: +// +// Int64String int64 `json:",string"` +// +// The key name will be used if it's a non-empty string consisting of +// only Unicode letters, digits, dollar signs, percent signs, hyphens, +// underscores and slashes. +// +// Anonymous struct fields are usually marshaled as if their inner exported fields +// were fields in the outer struct, subject to the usual Go visibility rules amended +// as described in the next paragraph. +// An anonymous struct field with a name given in its JSON tag is treated as +// having that name, rather than being anonymous. +// An anonymous struct field of interface type is treated the same as having +// that type as its name, rather than being anonymous. +// +// The Go visibility rules for struct fields are amended for JSON when +// deciding which field to marshal or unmarshal. If there are +// multiple fields at the same level, and that level is the least +// nested (and would therefore be the nesting level selected by the +// usual Go rules), the following extra rules apply: +// +// 1) Of those fields, if any are JSON-tagged, only tagged fields are considered, +// even if there are multiple untagged fields that would otherwise conflict. +// 2) If there is exactly one field (tagged or not according to the first rule), that is selected. +// 3) Otherwise there are multiple fields, and all are ignored; no error occurs. +// +// Handling of anonymous struct fields is new in Go 1.1. +// Prior to Go 1.1, anonymous struct fields were ignored. To force ignoring of +// an anonymous struct field in both current and earlier versions, give the field +// a JSON tag of "-". +// +// Map values encode as JSON objects. +// The map's key type must be string; the map keys are used as JSON object +// keys, subject to the UTF-8 coercion described for string values above. +// +// Pointer values encode as the value pointed to. +// A nil pointer encodes as the null JSON object. +// +// Interface values encode as the value contained in the interface. +// A nil interface value encodes as the null JSON object. +// +// Channel, complex, and function values cannot be encoded in JSON. +// Attempting to encode such a value causes Marshal to return +// an UnsupportedTypeError. +// +// JSON cannot represent cyclic data structures and Marshal does not +// handle them. Passing cyclic structures to Marshal will result in +// an infinite recursion. +func Marshal(v interface{}) ([]byte, error) { + e := &encodeState{} + err := e.marshal(v) + if err != nil { + return nil, err + } + return e.Bytes(), nil +} + +// MarshalIndent is like Marshal but applies Indent to format the output. +func MarshalIndent(v interface{}, prefix, indent string) ([]byte, error) { + b, err := Marshal(v) + if err != nil { + return nil, err + } + var buf bytes.Buffer + err = Indent(&buf, b, prefix, indent) + if err != nil { + return nil, err + } + return buf.Bytes(), nil +} + +// HTMLEscape appends to dst the JSON-encoded src with <, >, &, U+2028 and U+2029 +// characters inside string literals changed to \u003c, \u003e, \u0026, \u2028, \u2029 +// so that the JSON will be safe to embed inside HTML