From 64306a0e74913a8c9b00dc87b7a221b995f2f9fb Mon Sep 17 00:00:00 2001 From: George Robinson Date: Tue, 18 Jun 2024 14:48:50 +0100 Subject: [PATCH] Ingester RF-1 This commit is a work-in-progress of Ingester RF-1 (replication factor = 1). It is disabled by default. --- cmd/loki/loki-local-config.yaml | 4 + docs/sources/shared/configuration.md | 358 +++++- pkg/ingester-rf1/clientpool/client.go | 104 ++ .../clientpool/ingester_client_pool.go | 46 + pkg/ingester-rf1/flush.go | 186 +++ pkg/ingester-rf1/ingester.go | 1072 +++++++++++++++++ pkg/ingester-rf1/instance.go | 299 +++++ pkg/ingester-rf1/limiter.go | 226 ++++ pkg/ingester-rf1/mapper.go | 152 +++ pkg/ingester-rf1/metrics.go | 297 +++++ pkg/ingester-rf1/owned_streams.go | 74 ++ pkg/ingester-rf1/ring_client.go | 77 ++ pkg/ingester-rf1/stream.go | 325 +++++ pkg/ingester-rf1/stream_rate_calculator.go | 131 ++ pkg/ingester-rf1/streams_map.go | 149 +++ pkg/ingester-rf1/tee.go | 88 ++ pkg/ingester/flush_test.go | 5 + pkg/ingester/ingester_test.go | 7 +- pkg/logproto/ingester-rf1.pb.go | 130 ++ pkg/logproto/ingester-rf1.proto | 14 + pkg/loki/config_wrapper.go | 16 + pkg/loki/loki.go | 31 +- pkg/loki/modules.go | 68 ++ pkg/push/push-rf1.pb.go | 128 ++ pkg/push/push-rf1.proto | 13 + pkg/querier/querier_mock_test.go | 4 + .../client/aws/dynamodb_storage_client.go | 5 + .../chunk/client/cassandra/storage_client.go | 5 + pkg/storage/chunk/client/client.go | 2 + .../client/gcp/bigtable_object_client.go | 5 + .../chunk/client/grpc/storage_client.go | 5 + pkg/storage/chunk/client/metrics.go | 5 + pkg/storage/chunk/client/object_client.go | 10 + pkg/storage/store.go | 5 + pkg/storage/stores/composite_store.go | 13 + pkg/storage/stores/composite_store_test.go | 5 + pkg/storage/stores/series_store_write.go | 5 + pkg/storage/stores/series_store_write_test.go | 5 + pkg/storage/util_test.go | 5 +- pkg/storage/wal/segment.go | 75 +- pkg/storage/wal/segment_test.go | 163 ++- .../grafana/loki/pkg/push/push-rf1.pb.go | 128 ++ .../grafana/loki/pkg/push/push-rf1.proto | 13 + 43 files changed, 4375 insertions(+), 83 deletions(-) create mode 100644 pkg/ingester-rf1/clientpool/client.go create mode 100644 pkg/ingester-rf1/clientpool/ingester_client_pool.go create mode 100644 pkg/ingester-rf1/flush.go create mode 100644 pkg/ingester-rf1/ingester.go create mode 100644 pkg/ingester-rf1/instance.go create mode 100644 pkg/ingester-rf1/limiter.go create mode 100644 pkg/ingester-rf1/mapper.go create mode 100644 pkg/ingester-rf1/metrics.go create mode 100644 pkg/ingester-rf1/owned_streams.go create mode 100644 pkg/ingester-rf1/ring_client.go create mode 100644 pkg/ingester-rf1/stream.go create mode 100644 pkg/ingester-rf1/stream_rate_calculator.go create mode 100644 pkg/ingester-rf1/streams_map.go create mode 100644 pkg/ingester-rf1/tee.go create mode 100644 pkg/logproto/ingester-rf1.pb.go create mode 100644 pkg/logproto/ingester-rf1.proto create mode 100644 pkg/push/push-rf1.pb.go create mode 100644 pkg/push/push-rf1.proto create mode 100644 vendor/github.com/grafana/loki/pkg/push/push-rf1.pb.go create mode 100644 vendor/github.com/grafana/loki/pkg/push/push-rf1.proto diff --git a/cmd/loki/loki-local-config.yaml b/cmd/loki/loki-local-config.yaml index e2c54d5452790..d7e256c45a70c 100644 --- a/cmd/loki/loki-local-config.yaml +++ b/cmd/loki/loki-local-config.yaml @@ -4,6 +4,7 @@ server: http_listen_port: 3100 grpc_listen_port: 9096 log_level: debug + grpc_server_max_concurrent_streams: 1000 common: instance_addr: 127.0.0.1 @@ -17,6 +18,9 @@ common: kvstore: store: inmemory +ingester_rf1: + enabled: true + query_range: results_cache: cache: diff --git a/docs/sources/shared/configuration.md b/docs/sources/shared/configuration.md index 145ab85144a06..1420e4ad373c2 100644 --- a/docs/sources/shared/configuration.md +++ b/docs/sources/shared/configuration.md @@ -133,10 +133,279 @@ Pass the `-config.expand-env` flag at the command line to enable this way of set # the querier. [ingester_client: ] +# The ingester_client block configures how the distributor will connect to +# ingesters. Only appropriate when running all components, the distributor, or +# the querier. +[ingester_rf1_client: ] + # The ingester block configures the ingester and how the ingester will register # itself to a key value store. [ingester: ] +ingester_rf1: + # Whether the ingester is enabled. + # CLI flag: -ingester-rf1.enabled + [enabled: | default = false] + + # Configures how the lifecycle of the ingester will operate and where it will + # register for discovery. + lifecycler: + ring: + kvstore: + # Backend storage to use for the ring. Supported values are: consul, + # etcd, inmemory, memberlist, multi. + # CLI flag: -ingester-rf1.store + [store: | default = "consul"] + + # The prefix for the keys in the store. Should end with a /. + # CLI flag: -ingester-rf1.prefix + [prefix: | default = "collectors/"] + + # Configuration for a Consul client. Only applies if the selected + # kvstore is consul. + # The CLI flags prefix for this block configuration is: ingester-rf1 + [consul: ] + + # Configuration for an ETCD v3 client. Only applies if the selected + # kvstore is etcd. + # The CLI flags prefix for this block configuration is: ingester-rf1 + [etcd: ] + + multi: + # Primary backend storage used by multi-client. + # CLI flag: -ingester-rf1.multi.primary + [primary: | default = ""] + + # Secondary backend storage used by multi-client. + # CLI flag: -ingester-rf1.multi.secondary + [secondary: | default = ""] + + # Mirror writes to secondary store. + # CLI flag: -ingester-rf1.multi.mirror-enabled + [mirror_enabled: | default = false] + + # Timeout for storing value to secondary store. + # CLI flag: -ingester-rf1.multi.mirror-timeout + [mirror_timeout: | default = 2s] + + # The heartbeat timeout after which ingesters are skipped for + # reads/writes. 0 = never (timeout disabled). + # CLI flag: -ingester-rf1.ring.heartbeat-timeout + [heartbeat_timeout: | default = 1m] + + # The number of ingesters to write to and read from. + # CLI flag: -ingester-rf1.distributor.replication-factor + [replication_factor: | default = 3] + + # True to enable the zone-awareness and replicate ingested samples across + # different availability zones. + # CLI flag: -ingester-rf1.distributor.zone-awareness-enabled + [zone_awareness_enabled: | default = false] + + # Comma-separated list of zones to exclude from the ring. Instances in + # excluded zones will be filtered out from the ring. + # CLI flag: -ingester-rf1.distributor.excluded-zones + [excluded_zones: | default = ""] + + # Number of tokens for each ingester. + # CLI flag: -ingester-rf1.num-tokens + [num_tokens: | default = 128] + + # Period at which to heartbeat to consul. 0 = disabled. + # CLI flag: -ingester-rf1.heartbeat-period + [heartbeat_period: | default = 5s] + + # Heartbeat timeout after which instance is assumed to be unhealthy. 0 = + # disabled. + # CLI flag: -ingester-rf1.heartbeat-timeout + [heartbeat_timeout: | default = 1m] + + # Observe tokens after generating to resolve collisions. Useful when using + # gossiping ring. + # CLI flag: -ingester-rf1.observe-period + [observe_period: | default = 0s] + + # Period to wait for a claim from another member; will join automatically + # after this. + # CLI flag: -ingester-rf1.join-after + [join_after: | default = 0s] + + # Minimum duration to wait after the internal readiness checks have passed + # but before succeeding the readiness endpoint. This is used to slowdown + # deployment controllers (eg. Kubernetes) after an instance is ready and + # before they proceed with a rolling update, to give the rest of the cluster + # instances enough time to receive ring updates. + # CLI flag: -ingester-rf1.min-ready-duration + [min_ready_duration: | default = 15s] + + # Name of network interface to read address from. + # CLI flag: -ingester-rf1.lifecycler.interface + [interface_names: | default = []] + + # Enable IPv6 support. Required to make use of IP addresses from IPv6 + # interfaces. + # CLI flag: -ingester-rf1.enable-inet6 + [enable_inet6: | default = false] + + # Duration to sleep for before exiting, to ensure metrics are scraped. + # CLI flag: -ingester-rf1.final-sleep + [final_sleep: | default = 0s] + + # File path where tokens are stored. If empty, tokens are not stored at + # shutdown and restored at startup. + # CLI flag: -ingester-rf1.tokens-file-path + [tokens_file_path: | default = ""] + + # The availability zone where this instance is running. + # CLI flag: -ingester-rf1.availability-zone + [availability_zone: | default = ""] + + # Unregister from the ring upon clean shutdown. It can be useful to disable + # for rolling restarts with consistent naming in conjunction with + # -distributor.extend-writes=false. + # CLI flag: -ingester-rf1.unregister-on-shutdown + [unregister_on_shutdown: | default = true] + + # When enabled the readiness probe succeeds only after all instances are + # ACTIVE and healthy in the ring, otherwise only the instance itself is + # checked. This option should be disabled if in your cluster multiple + # instances can be rolled out simultaneously, otherwise rolling updates may + # be slowed down. + # CLI flag: -ingester-rf1.readiness-check-ring-health + [readiness_check_ring_health: | default = true] + + # IP address to advertise in the ring. + # CLI flag: -ingester-rf1.lifecycler.addr + [address: | default = ""] + + # port to advertise in consul (defaults to server.grpc-listen-port). + # CLI flag: -ingester-rf1.lifecycler.port + [port: | default = 0] + + # ID to register in the ring. + # CLI flag: -ingester-rf1.lifecycler.ID + [id: | default = ""] + + # How many flushes can happen concurrently from each stream. + # CLI flag: -ingester-rf1.concurrent-flushes + [concurrent_flushes: | default = 32] + + # How often should the ingester see if there are any blocks to flush. The + # first flush check is delayed by a random time up to 0.8x the flush check + # period. Additionally, there is +/- 1% jitter added to the interval. + # CLI flag: -ingester-rf1.flush-check-period + [flush_check_period: | default = 500ms] + + flush_op_backoff: + # Minimum backoff period when a flush fails. Each concurrent flush has its + # own backoff, see `ingester.concurrent-flushes`. + # CLI flag: -ingester-rf1.flush-op-backoff-min-period + [min_period: | default = 100ms] + + # Maximum backoff period when a flush fails. Each concurrent flush has its + # own backoff, see `ingester.concurrent-flushes`. + # CLI flag: -ingester-rf1.flush-op-backoff-max-period + [max_period: | default = 1m] + + # Maximum retries for failed flushes. + # CLI flag: -ingester-rf1.flush-op-backoff-retries + [max_retries: | default = 10] + + # The timeout for an individual flush. Will be retried up to + # `flush-op-backoff-retries` times. + # CLI flag: -ingester-rf1.flush-op-timeout + [flush_op_timeout: | default = 10m] + + # How long chunks should be retained in-memory after they've been flushed. + # CLI flag: -ingester-rf1.chunks-retain-period + [chunk_retain_period: | default = 0s] + + [chunk_idle_period: ] + + # The targeted _uncompressed_ size in bytes of a chunk block When this + # threshold is exceeded the head block will be cut and compressed inside the + # chunk. + # CLI flag: -ingester-rf1.chunks-block-size + [chunk_block_size: | default = 262144] + + # A target _compressed_ size in bytes for chunks. This is a desired size not + # an exact size, chunks may be slightly bigger or significantly smaller if + # they get flushed for other reasons (e.g. chunk_idle_period). A value of 0 + # creates chunks with a fixed 10 blocks, a non zero value will create chunks + # with a variable number of blocks to meet the target size. + # CLI flag: -ingester-rf1.chunk-target-size + [chunk_target_size: | default = 1572864] + + # The algorithm to use for compressing chunk. (none, gzip, lz4-64k, snappy, + # lz4-256k, lz4-1M, lz4, flate, zstd) + # CLI flag: -ingester-rf1.chunk-encoding + [chunk_encoding: | default = "gzip"] + + # The maximum duration of a timeseries chunk in memory. If a timeseries runs + # for longer than this, the current chunk will be flushed to the store and a + # new chunk created. + # CLI flag: -ingester-rf1.max-chunk-age + [max_chunk_age: | default = 2h] + + # Forget about ingesters having heartbeat timestamps older than + # `ring.kvstore.heartbeat_timeout`. This is equivalent to clicking on the + # `/ring` `forget` button in the UI: the ingester is removed from the ring. + # This is a useful setting when you are sure that an unhealthy node won't + # return. An example is when not using stateful sets or the equivalent. Use + # `memberlist.rejoin_interval` > 0 to handle network partition cases when + # using a memberlist. + # CLI flag: -ingester-rf1.autoforget-unhealthy + [autoforget_unhealthy: | default = false] + + # The maximum number of errors a stream will report to the user when a push + # fails. 0 to make unlimited. + # CLI flag: -ingester-rf1.max-ignored-stream-errors + [max_returned_stream_errors: | default = 10] + + # Shard factor used in the ingesters for the in process reverse index. This + # MUST be evenly divisible by ALL schema shard factors or Loki will not start. + # CLI flag: -ingester-rf1.index-shards + [index_shards: | default = 32] + + # Maximum number of dropped streams to keep in memory during tailing. + # CLI flag: -ingester-rf1.tailer.max-dropped-streams + [max_dropped_streams: | default = 10] + + # Path where the shutdown marker file is stored. If not set and + # common.path_prefix is set then common.path_prefix will be used. + # CLI flag: -ingester-rf1.shutdown-marker-path + [shutdown_marker_path: | default = ""] + + # Interval at which the ingester ownedStreamService checks for changes in the + # ring to recalculate owned streams. + # CLI flag: -ingester-rf1.owned-streams-check-interval + [owned_streams_check_interval: | default = 30s] + + # Configures how the pattern ingester will connect to the ingesters. + client_config: + # Configures how connections are pooled. + pool_config: + # How frequently to clean up clients for ingesters that have gone away. + # CLI flag: -ingester-rf1.client-cleanup-period + [client_cleanup_period: | default = 15s] + + # Run a health check on each ingester client during periodic cleanup. + # CLI flag: -ingester-rf1.health-check-ingesters + [health_check_ingesters: | default = true] + + # Timeout for the health check. + # CLI flag: -ingester-rf1.remote-timeout + [remote_timeout: | default = 1s] + + # The remote request timeout on the client side. + # CLI flag: -ingester-rf1.client.timeout + [remote_timeout: | default = 5s] + + # Configures how the gRPC connection to ingesters work as a client. + # The CLI flags prefix for this block configuration is: + # pattern-ingester.client + [grpc_client_config: ] + pattern_ingester: # Whether the pattern ingester is enabled. # CLI flag: -pattern-ingester.enabled @@ -303,7 +572,7 @@ pattern_ingester: # Configures how the gRPC connection to ingesters work as a client. # The CLI flags prefix for this block configuration is: - # pattern-ingester.client + # bloom-build.builder.grpc [grpc_client_config: ] # How many flushes can happen concurrently from each stream. @@ -370,7 +639,7 @@ bloom_build: # The grpc_client block configures the gRPC client used to communicate # between a client and server component in Loki. # The CLI flags prefix for this block configuration is: - # bloom-build.builder.grpc + # bloom-gateway-client.grpc [grpc_config: ] # Hostname (and port) of the bloom planner @@ -1120,8 +1389,7 @@ client: # The grpc_client block configures the gRPC client used to communicate between # a client and server component in Loki. - # The CLI flags prefix for this block configuration is: - # bloom-gateway-client.grpc + # The CLI flags prefix for this block configuration is: bigtable [grpc_client_config: ] results_cache: @@ -1867,6 +2135,7 @@ Configuration for a Consul client. Only applies if the selected kvstore is `cons - `compactor.ring` - `distributor.ring` - `index-gateway.ring` +- `ingester-rf1` - `pattern-ingester` - `query-scheduler.ring` - `ruler.ring` @@ -2087,6 +2356,7 @@ Configuration for an ETCD v3 client. Only applies if the selected kvstore is `et - `compactor.ring` - `distributor.ring` - `index-gateway.ring` +- `ingester-rf1` - `pattern-ingester` - `query-scheduler.ring` - `ruler.ring` @@ -2305,20 +2575,18 @@ The `frontend_worker` configures the worker - running within the Loki querier - # Configures the querier gRPC client used to communicate with the # query-frontend. This can't be used in conjunction with 'grpc_client_config'. -# The CLI flags prefix for this block configuration is: -# querier.frontend-grpc-client +# The CLI flags prefix for this block configuration is: querier.frontend-client [query_frontend_grpc_client: ] # Configures the querier gRPC client used to communicate with the query-frontend # and with the query-scheduler. This can't be used in conjunction with # 'query_frontend_grpc_client' or 'query_scheduler_grpc_client'. -# The CLI flags prefix for this block configuration is: querier.frontend-client +# The CLI flags prefix for this block configuration is: +# querier.scheduler-grpc-client [grpc_client_config: ] # Configures the querier gRPC client used to communicate with the # query-scheduler. This can't be used in conjunction with 'grpc_client_config'. -# The CLI flags prefix for this block configuration is: -# querier.scheduler-grpc-client [query_scheduler_grpc_client: ] ``` @@ -2375,6 +2643,7 @@ The `grpc_client` block configures the gRPC client used to communicate between a - `bloom-gateway-client.grpc` - `boltdb.shipper.index-gateway-client.grpc` - `frontend.grpc-client-config` +- `ingester-rf1.client` - `ingester.client` - `pattern-ingester.client` - `querier.frontend-client` @@ -2388,82 +2657,82 @@ The `grpc_client` block configures the gRPC client used to communicate between a ```yaml # gRPC client max receive message size (bytes). -# CLI flag: -.grpc-max-recv-msg-size +# CLI flag: -boltdb.shipper.index-gateway-client.grpc.grpc-max-recv-msg-size [max_recv_msg_size: | default = 104857600] # gRPC client max send message size (bytes). -# CLI flag: -.grpc-max-send-msg-size +# CLI flag: -boltdb.shipper.index-gateway-client.grpc.grpc-max-send-msg-size [max_send_msg_size: | default = 104857600] # Use compression when sending messages. Supported values are: 'gzip', 'snappy' # and '' (disable compression) -# CLI flag: -.grpc-compression +# CLI flag: -boltdb.shipper.index-gateway-client.grpc.grpc-compression [grpc_compression: | default = ""] # Rate limit for gRPC client; 0 means disabled. -# CLI flag: -.grpc-client-rate-limit +# CLI flag: -boltdb.shipper.index-gateway-client.grpc.grpc-client-rate-limit [rate_limit: | default = 0] # Rate limit burst for gRPC client. -# CLI flag: -.grpc-client-rate-limit-burst +# CLI flag: -boltdb.shipper.index-gateway-client.grpc.grpc-client-rate-limit-burst [rate_limit_burst: | default = 0] # Enable backoff and retry when we hit rate limits. -# CLI flag: -.backoff-on-ratelimits +# CLI flag: -boltdb.shipper.index-gateway-client.grpc.backoff-on-ratelimits [backoff_on_ratelimits: | default = false] backoff_config: # Minimum delay when backing off. - # CLI flag: -.backoff-min-period + # CLI flag: -boltdb.shipper.index-gateway-client.grpc.backoff-min-period [min_period: | default = 100ms] # Maximum delay when backing off. - # CLI flag: -.backoff-max-period + # CLI flag: -boltdb.shipper.index-gateway-client.grpc.backoff-max-period [max_period: | default = 10s] # Number of times to backoff and retry before failing. - # CLI flag: -.backoff-retries + # CLI flag: -boltdb.shipper.index-gateway-client.grpc.backoff-retries [max_retries: | default = 10] # Initial stream window size. Values less than the default are not supported and # are ignored. Setting this to a value other than the default disables the BDP # estimator. -# CLI flag: -.initial-stream-window-size +# CLI flag: -boltdb.shipper.index-gateway-client.grpc.initial-stream-window-size [initial_stream_window_size: | default = 63KiB1023B] # Initial connection window size. Values less than the default are not supported # and are ignored. Setting this to a value other than the default disables the # BDP estimator. -# CLI flag: -.initial-connection-window-size +# CLI flag: -boltdb.shipper.index-gateway-client.grpc.initial-connection-window-size [initial_connection_window_size: | default = 63KiB1023B] # Enable TLS in the gRPC client. This flag needs to be enabled when any other # TLS flag is set. If set to false, insecure connection to gRPC server will be # used. -# CLI flag: -.tls-enabled +# CLI flag: -boltdb.shipper.index-gateway-client.grpc.tls-enabled [tls_enabled: | default = false] # Path to the client certificate, which will be used for authenticating with the # server. Also requires the key path to be configured. -# CLI flag: -.tls-cert-path +# CLI flag: -boltdb.shipper.index-gateway-client.grpc.tls-cert-path [tls_cert_path: | default = ""] # Path to the key for the client certificate. Also requires the client # certificate to be configured. -# CLI flag: -.tls-key-path +# CLI flag: -boltdb.shipper.index-gateway-client.grpc.tls-key-path [tls_key_path: | default = ""] # Path to the CA certificates to validate server certificate against. If not # set, the host's root CA certificates are used. -# CLI flag: -.tls-ca-path +# CLI flag: -boltdb.shipper.index-gateway-client.grpc.tls-ca-path [tls_ca_path: | default = ""] # Override the expected name on the server certificate. -# CLI flag: -.tls-server-name +# CLI flag: -boltdb.shipper.index-gateway-client.grpc.tls-server-name [tls_server_name: | default = ""] # Skip validating server certificate. -# CLI flag: -.tls-insecure-skip-verify +# CLI flag: -boltdb.shipper.index-gateway-client.grpc.tls-insecure-skip-verify [tls_insecure_skip_verify: | default = false] # Override the default cipher suite list (separated by commas). Allowed values: @@ -2496,27 +2765,27 @@ backoff_config: # - TLS_ECDHE_RSA_WITH_3DES_EDE_CBC_SHA # - TLS_ECDHE_ECDSA_WITH_AES_128_CBC_SHA256 # - TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256 -# CLI flag: -.tls-cipher-suites +# CLI flag: -boltdb.shipper.index-gateway-client.grpc.tls-cipher-suites [tls_cipher_suites: | default = ""] # Override the default minimum TLS version. Allowed values: VersionTLS10, # VersionTLS11, VersionTLS12, VersionTLS13 -# CLI flag: -.tls-min-version +# CLI flag: -boltdb.shipper.index-gateway-client.grpc.tls-min-version [tls_min_version: | default = ""] # The maximum amount of time to establish a connection. A value of 0 means # default gRPC client connect timeout and backoff. -# CLI flag: -.connect-timeout +# CLI flag: -boltdb.shipper.index-gateway-client.grpc.connect-timeout [connect_timeout: | default = 5s] # Initial backoff delay after first connection failure. Only relevant if # ConnectTimeout > 0. -# CLI flag: -.connect-backoff-base-delay +# CLI flag: -boltdb.shipper.index-gateway-client.grpc.connect-backoff-base-delay [connect_backoff_base_delay: | default = 1s] # Maximum backoff delay when establishing a connection. Only relevant if # ConnectTimeout > 0. -# CLI flag: -.connect-backoff-max-delay +# CLI flag: -boltdb.shipper.index-gateway-client.grpc.connect-backoff-max-delay [connect_backoff_max_delay: | default = 5s] ``` @@ -2920,26 +3189,16 @@ The `ingester_client` block configures how the distributor will connect to inges ```yaml # Configures how connections are pooled. pool_config: - # How frequently to clean up clients for ingesters that have gone away. - # CLI flag: -distributor.client-cleanup-period - [client_cleanup_period: | default = 15s] + [client_cleanup_period: ] - # Run a health check on each ingester client during periodic cleanup. - # CLI flag: -distributor.health-check-ingesters - [health_check_ingesters: | default = true] + [health_check_ingesters: ] - # How quickly a dead client will be removed after it has been detected to - # disappear. Set this to a value to allow time for a secondary health check to - # recover the missing client. - # CLI flag: -ingester.client.healthcheck-timeout - [remote_timeout: | default = 1s] + [remote_timeout: ] -# The remote request timeout on the client side. -# CLI flag: -ingester.client.timeout -[remote_timeout: | default = 5s] +[remote_timeout: ] # Configures how the gRPC connection to ingesters work as a client. -# The CLI flags prefix for this block configuration is: ingester.client +# The CLI flags prefix for this block configuration is: ingester-rf1.client [grpc_client_config: ] ``` @@ -5073,7 +5332,8 @@ bigtable: # The grpc_client block configures the gRPC client used to communicate between # a client and server component in Loki. - # The CLI flags prefix for this block configuration is: bigtable + # The CLI flags prefix for this block configuration is: + # boltdb.shipper.index-gateway-client.grpc [grpc_client_config: ] # If enabled, once a tables info is fetched, it is cached. @@ -5365,7 +5625,7 @@ boltdb_shipper: # The grpc_client block configures the gRPC client used to communicate # between a client and server component in Loki. # The CLI flags prefix for this block configuration is: - # boltdb.shipper.index-gateway-client.grpc + # tsdb.shipper.index-gateway-client.grpc [grpc_client_config: ] # Hostname or IP of the Index Gateway gRPC server running in simple mode. @@ -5420,7 +5680,7 @@ tsdb_shipper: # The grpc_client block configures the gRPC client used to communicate # between a client and server component in Loki. # The CLI flags prefix for this block configuration is: - # tsdb.shipper.index-gateway-client.grpc + # querier.frontend-grpc-client [grpc_client_config: ] # Hostname or IP of the Index Gateway gRPC server running in simple mode. diff --git a/pkg/ingester-rf1/clientpool/client.go b/pkg/ingester-rf1/clientpool/client.go new file mode 100644 index 0000000000000..4407a5856dd6f --- /dev/null +++ b/pkg/ingester-rf1/clientpool/client.go @@ -0,0 +1,104 @@ +package clientpool + +import ( + "flag" + "io" + "time" + + "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/util/server" + + "github.com/grafana/dskit/grpcclient" + "github.com/grafana/dskit/middleware" + "github.com/grpc-ecosystem/grpc-opentracing/go/otgrpc" + "github.com/opentracing/opentracing-go" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "google.golang.org/grpc" + "google.golang.org/grpc/health/grpc_health_v1" +) + +var ingesterClientRequestDuration = promauto.NewHistogramVec(prometheus.HistogramOpts{ + Name: "loki_ingester_rf1_client_request_duration_seconds", + Help: "Time spent doing Ingester RF1 requests.", + Buckets: prometheus.ExponentialBuckets(0.001, 4, 6), +}, []string{"operation", "status_code"}) + +type HealthAndIngesterClient interface { + grpc_health_v1.HealthClient + Close() error +} + +type ClosableHealthAndIngesterClient struct { + logproto.PusherRF1Client + grpc_health_v1.HealthClient + io.Closer +} + +// Config for an ingester client. +type Config struct { + PoolConfig PoolConfig `yaml:"pool_config,omitempty" doc:"description=Configures how connections are pooled."` + RemoteTimeout time.Duration `yaml:"remote_timeout,omitempty"` + GRPCClientConfig grpcclient.Config `yaml:"grpc_client_config" doc:"description=Configures how the gRPC connection to ingesters work as a client."` + GRPCUnaryClientInterceptors []grpc.UnaryClientInterceptor `yaml:"-"` + GRCPStreamClientInterceptors []grpc.StreamClientInterceptor `yaml:"-"` + + // Internal is used to indicate that this client communicates on behalf of + // a machine and not a user. When Internal = true, the client won't attempt + // to inject an userid into the context. + Internal bool `yaml:"-"` +} + +// RegisterFlags registers flags. +func (cfg *Config) RegisterFlags(f *flag.FlagSet) { + cfg.GRPCClientConfig.RegisterFlagsWithPrefix("ingester-rf1.client", f) + cfg.PoolConfig.RegisterFlagsWithPrefix("ingester-rf1.", f) + + f.DurationVar(&cfg.PoolConfig.RemoteTimeout, "ingester-rf1.client.healthcheck-timeout", 1*time.Second, "How quickly a dead client will be removed after it has been detected to disappear. Set this to a value to allow time for a secondary health check to recover the missing client.") + f.DurationVar(&cfg.RemoteTimeout, "ingester-rf1.client.timeout", 5*time.Second, "The remote request timeout on the client side.") +} + +// New returns a new ingester client. +func NewClient(cfg Config, addr string) (HealthAndIngesterClient, error) { + opts := []grpc.DialOption{ + grpc.WithDefaultCallOptions(cfg.GRPCClientConfig.CallOptions()...), + } + + dialOpts, err := cfg.GRPCClientConfig.DialOption(instrumentation(&cfg)) + if err != nil { + return nil, err + } + + opts = append(opts, dialOpts...) + conn, err := grpc.Dial(addr, opts...) + if err != nil { + return nil, err + } + return ClosableHealthAndIngesterClient{ + PusherRF1Client: logproto.NewPusherRF1Client(conn), + HealthClient: grpc_health_v1.NewHealthClient(conn), + Closer: conn, + }, nil +} + +func instrumentation(cfg *Config) ([]grpc.UnaryClientInterceptor, []grpc.StreamClientInterceptor) { + var unaryInterceptors []grpc.UnaryClientInterceptor + unaryInterceptors = append(unaryInterceptors, cfg.GRPCUnaryClientInterceptors...) + unaryInterceptors = append(unaryInterceptors, server.UnaryClientQueryTagsInterceptor) + unaryInterceptors = append(unaryInterceptors, otgrpc.OpenTracingClientInterceptor(opentracing.GlobalTracer())) + if !cfg.Internal { + unaryInterceptors = append(unaryInterceptors, middleware.ClientUserHeaderInterceptor) + } + unaryInterceptors = append(unaryInterceptors, middleware.UnaryClientInstrumentInterceptor(ingesterClientRequestDuration)) + + var streamInterceptors []grpc.StreamClientInterceptor + streamInterceptors = append(streamInterceptors, cfg.GRCPStreamClientInterceptors...) + streamInterceptors = append(streamInterceptors, server.StreamClientQueryTagsInterceptor) + streamInterceptors = append(streamInterceptors, otgrpc.OpenTracingStreamClientInterceptor(opentracing.GlobalTracer())) + if !cfg.Internal { + streamInterceptors = append(streamInterceptors, middleware.StreamClientUserHeaderInterceptor) + } + streamInterceptors = append(streamInterceptors, middleware.StreamClientInstrumentInterceptor(ingesterClientRequestDuration)) + + return unaryInterceptors, streamInterceptors +} diff --git a/pkg/ingester-rf1/clientpool/ingester_client_pool.go b/pkg/ingester-rf1/clientpool/ingester_client_pool.go new file mode 100644 index 0000000000000..7c84f6aa69b1b --- /dev/null +++ b/pkg/ingester-rf1/clientpool/ingester_client_pool.go @@ -0,0 +1,46 @@ +package clientpool + +import ( + "flag" + "time" + + "github.com/go-kit/log" + "github.com/grafana/dskit/ring" + ring_client "github.com/grafana/dskit/ring/client" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" +) + +var clients prometheus.Gauge + +// PoolConfig is config for creating a Pool. +type PoolConfig struct { + ClientCleanupPeriod time.Duration `yaml:"client_cleanup_period"` + HealthCheckIngesters bool `yaml:"health_check_ingesters"` + RemoteTimeout time.Duration `yaml:"remote_timeout"` +} + +// RegisterFlags adds the flags required to config this to the given FlagSet. +func (cfg *PoolConfig) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { + f.DurationVar(&cfg.ClientCleanupPeriod, prefix+"client-cleanup-period", 15*time.Second, "How frequently to clean up clients for ingesters that have gone away.") + f.BoolVar(&cfg.HealthCheckIngesters, prefix+"health-check-ingesters", true, "Run a health check on each ingester client during periodic cleanup.") + f.DurationVar(&cfg.RemoteTimeout, prefix+"remote-timeout", 1*time.Second, "Timeout for the health check.") +} + +func NewPool(name string, cfg PoolConfig, ring ring.ReadRing, factory ring_client.PoolFactory, logger log.Logger, metricsNamespace string) *ring_client.Pool { + poolCfg := ring_client.PoolConfig{ + CheckInterval: cfg.ClientCleanupPeriod, + HealthCheckEnabled: cfg.HealthCheckIngesters, + HealthCheckTimeout: cfg.RemoteTimeout, + } + + if clients == nil { + clients = promauto.NewGauge(prometheus.GaugeOpts{ + Namespace: metricsNamespace, + Name: "ingester_rf1_clients", + Help: "The current number of RF1 ingester clients.", + }) + } + // TODO(chaudum): Allow configuration of metric name by the caller. + return ring_client.NewPool(name, poolCfg, ring_client.NewRingServiceDiscovery(ring), factory, clients, logger) +} diff --git a/pkg/ingester-rf1/flush.go b/pkg/ingester-rf1/flush.go new file mode 100644 index 0000000000000..d46619575eeae --- /dev/null +++ b/pkg/ingester-rf1/flush.go @@ -0,0 +1,186 @@ +package ingesterrf1 + +import ( + "fmt" + "net/http" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/grafana/dskit/backoff" + "github.com/grafana/dskit/ring" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/common/model" + "golang.org/x/net/context" + + "github.com/grafana/loki/v3/pkg/chunkenc" + "github.com/grafana/loki/v3/pkg/storage/chunk" + "github.com/grafana/loki/v3/pkg/storage/wal" + "github.com/grafana/loki/v3/pkg/util" +) + +const ( + // Backoff for retrying 'immediate' flushes. Only counts for queue + // position, not wallclock time. + flushBackoff = 1 * time.Second + + nameLabel = "__name__" + logsValue = "logs" + + flushReasonIdle = "idle" + flushReasonMaxAge = "max_age" + flushReasonForced = "forced" + flushReasonFull = "full" + flushReasonSynced = "synced" +) + +// Note: this is called both during the WAL replay (zero or more times) +// and then after replay as well. +func (i *Ingester) InitFlushQueues() { + i.flushQueuesDone.Add(i.cfg.ConcurrentFlushes) + for j := 0; j < i.cfg.ConcurrentFlushes; j++ { + i.flushQueues[j] = util.NewPriorityQueue(i.metrics.flushQueueLength) + go i.flushLoop(j) + } +} + +// Flush implements ring.FlushTransferer +// Flush triggers a flush of all the chunks and closes the flush queues. +// Called from the Lifecycler as part of the ingester shutdown. +func (i *Ingester) Flush() { + i.flush() +} + +// TransferOut implements ring.FlushTransferer +// Noop implemenetation because ingesters have a WAL now that does not require transferring chunks any more. +// We return ErrTransferDisabled to indicate that we don't support transfers, and therefore we may flush on shutdown if configured to do so. +func (i *Ingester) TransferOut(_ context.Context) error { + return ring.ErrTransferDisabled +} + +func (i *Ingester) flush() { + // TODO: Flush the last chunks + // Close the flush queues, to unblock waiting workers. + for _, flushQueue := range i.flushQueues { + flushQueue.Close() + } + + i.flushQueuesDone.Wait() + level.Debug(i.logger).Log("msg", "flush queues have drained") +} + +// FlushHandler triggers a flush of all in memory chunks. Mainly used for +// local testing. +func (i *Ingester) FlushHandler(w http.ResponseWriter, _ *http.Request) { + w.WriteHeader(http.StatusNoContent) +} + +type flushOp struct { + from model.Time + userID string + fp model.Fingerprint + immediate bool +} + +func (o *flushOp) Key() string { + return fmt.Sprintf("%s-%s-%v", o.userID, o.fp, o.immediate) +} + +func (o *flushOp) Priority() int64 { + return -int64(o.from) +} + +func (i *Ingester) flushLoop(j int) { + l := log.With(i.logger, "loop", j) + defer func() { + level.Debug(l).Log("msg", "Ingester.flushLoop() exited") + i.flushQueuesDone.Done() + }() + + for { + o := i.flushQueues[j].Dequeue() + if o == nil { + return + } + op := o.(*flushCtx) + + err := i.flushOp(l, op) + if err != nil { + level.Error(l).Log("msg", "failed to flush", "err", err) + // Immediately re-queue another attempt at flushing this segment. + // TODO: Add some backoff or something? + i.flushQueues[j].Enqueue(op) + } else { + // Close the channel and trigger all waiting listeners to return + // TODO: Figure out how to return an error if we want to? + close(op.flushDone) + } + } +} + +func (i *Ingester) flushOp(l log.Logger, flushCtx *flushCtx) error { + ctx, cancelFunc := context.WithCancel(context.Background()) + defer cancelFunc() + + b := backoff.New(ctx, i.cfg.FlushOpBackoff) + for b.Ongoing() { + err := i.flushSegment(ctx, flushCtx.segmentWriter) + if err == nil { + break + } + level.Error(l).Log("msg", "failed to flush", "retries", b.NumRetries(), "err", err) + b.Wait() + } + return b.Err() +} + +// flushChunk flushes the given chunk to the store. +// +// If the flush is successful, metrics for this flush are to be reported. +// If the flush isn't successful, the operation for this userID is requeued allowing this and all other unflushed +// segments to have another opportunity to be flushed. +func (i *Ingester) flushSegment(ctx context.Context, ch *wal.SegmentWriter) error { + if err := i.store.PutWal(ctx, ch); err != nil { + i.metrics.chunksFlushFailures.Inc() + return fmt.Errorf("store put chunk: %w", err) + } + i.metrics.flushedChunksStats.Inc(1) + // TODO: report some flush metrics + return nil +} + +// reportFlushedChunkStatistics calculate overall statistics of flushed chunks without compromising the flush process. +func (i *Ingester) reportFlushedChunkStatistics(ch *chunk.Chunk, desc *chunkDesc, sizePerTenant prometheus.Counter, countPerTenant prometheus.Counter, reason string) { + byt, err := ch.Encoded() + if err != nil { + level.Error(i.logger).Log("msg", "failed to encode flushed wire chunk", "err", err) + return + } + + i.metrics.chunksFlushedPerReason.WithLabelValues(reason).Add(1) + + compressedSize := float64(len(byt)) + uncompressedSize, ok := chunkenc.UncompressedSize(ch.Data) + + if ok && compressedSize > 0 { + i.metrics.chunkCompressionRatio.Observe(float64(uncompressedSize) / compressedSize) + } + + utilization := ch.Data.Utilization() + i.metrics.chunkUtilization.Observe(utilization) + numEntries := desc.chunk.Size() + i.metrics.chunkEntries.Observe(float64(numEntries)) + i.metrics.chunkSize.Observe(compressedSize) + sizePerTenant.Add(compressedSize) + countPerTenant.Inc() + + boundsFrom, boundsTo := desc.chunk.Bounds() + i.metrics.chunkAge.Observe(time.Since(boundsFrom).Seconds()) + i.metrics.chunkLifespan.Observe(boundsTo.Sub(boundsFrom).Hours()) + + i.metrics.flushedChunksBytesStats.Record(compressedSize) + i.metrics.flushedChunksLinesStats.Record(float64(numEntries)) + i.metrics.flushedChunksUtilizationStats.Record(utilization) + i.metrics.flushedChunksAgeStats.Record(time.Since(boundsFrom).Seconds()) + i.metrics.flushedChunksLifespanStats.Record(boundsTo.Sub(boundsFrom).Seconds()) +} diff --git a/pkg/ingester-rf1/ingester.go b/pkg/ingester-rf1/ingester.go new file mode 100644 index 0000000000000..6fcdcee504ad4 --- /dev/null +++ b/pkg/ingester-rf1/ingester.go @@ -0,0 +1,1072 @@ +package ingesterrf1 + +import ( + "context" + "flag" + "fmt" + "math/rand" + "net/http" + "os" + "path" + "runtime/pprof" + "sync" + "time" + + ring_client "github.com/grafana/dskit/ring/client" + "github.com/opentracing/opentracing-go" + + "github.com/grafana/loki/v3/pkg/ingester-rf1/clientpool" + "github.com/grafana/loki/v3/pkg/ingester/index" + "github.com/grafana/loki/v3/pkg/loghttp/push" + lokilog "github.com/grafana/loki/v3/pkg/logql/log" + "github.com/grafana/loki/v3/pkg/storage/types" + "github.com/grafana/loki/v3/pkg/storage/wal" + util_log "github.com/grafana/loki/v3/pkg/util/log" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/grafana/dskit/backoff" + "github.com/grafana/dskit/modules" + "github.com/grafana/dskit/multierror" + "github.com/grafana/dskit/ring" + "github.com/grafana/dskit/services" + "github.com/grafana/dskit/tenant" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "google.golang.org/grpc/health/grpc_health_v1" + + server_util "github.com/grafana/loki/v3/pkg/util/server" + + "github.com/grafana/loki/v3/pkg/analytics" + "github.com/grafana/loki/v3/pkg/chunkenc" + "github.com/grafana/loki/v3/pkg/distributor/writefailures" + "github.com/grafana/loki/v3/pkg/ingester/client" + "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/logql/syntax" + "github.com/grafana/loki/v3/pkg/runtime" + "github.com/grafana/loki/v3/pkg/storage" + "github.com/grafana/loki/v3/pkg/storage/chunk" + "github.com/grafana/loki/v3/pkg/storage/config" + "github.com/grafana/loki/v3/pkg/storage/stores" + indexstore "github.com/grafana/loki/v3/pkg/storage/stores/index" + "github.com/grafana/loki/v3/pkg/util" +) + +const ( + // RingKey is the key under which we store the ingesters ring in the KVStore. + RingKey = "ring" + + shutdownMarkerFilename = "shutdown-requested.txt" +) + +// ErrReadOnly is returned when the ingester is shutting down and a push was +// attempted. +var ( + ErrReadOnly = errors.New("Ingester is shutting down") + + compressionStats = analytics.NewString("ingester_compression") + targetSizeStats = analytics.NewInt("ingester_target_size_bytes") + activeTenantsStats = analytics.NewInt("ingester_active_tenants") +) + +// Config for an ingester. +type Config struct { + Enabled bool `yaml:"enabled" doc:"description=Whether the ingester is enabled."` + + LifecyclerConfig ring.LifecyclerConfig `yaml:"lifecycler,omitempty" doc:"description=Configures how the lifecycle of the ingester will operate and where it will register for discovery."` + + ConcurrentFlushes int `yaml:"concurrent_flushes"` + FlushCheckPeriod time.Duration `yaml:"flush_check_period"` + FlushOpBackoff backoff.Config `yaml:"flush_op_backoff"` + FlushOpTimeout time.Duration `yaml:"flush_op_timeout"` + RetainPeriod time.Duration `yaml:"chunk_retain_period"` + MaxChunkIdle time.Duration `yaml:"chunk_idle_period"` + BlockSize int `yaml:"chunk_block_size"` + TargetChunkSize int `yaml:"chunk_target_size"` + ChunkEncoding string `yaml:"chunk_encoding"` + parsedEncoding chunkenc.Encoding `yaml:"-"` // placeholder for validated encoding + MaxChunkAge time.Duration `yaml:"max_chunk_age"` + AutoForgetUnhealthy bool `yaml:"autoforget_unhealthy"` + + MaxReturnedErrors int `yaml:"max_returned_stream_errors"` + + // For testing, you can override the address and ID of this ingester. + ingesterClientFactory func(cfg client.Config, addr string) (client.HealthAndIngesterClient, error) + + // Optional wrapper that can be used to modify the behaviour of the ingester + Wrapper Wrapper `yaml:"-"` + + IndexShards int `yaml:"index_shards"` + + MaxDroppedStreams int `yaml:"max_dropped_streams"` + + ShutdownMarkerPath string `yaml:"shutdown_marker_path"` + + OwnedStreamsCheckInterval time.Duration `yaml:"owned_streams_check_interval" doc:"description=Interval at which the ingester ownedStreamService checks for changes in the ring to recalculate owned streams."` + + // Tee configs + ClientConfig clientpool.Config `yaml:"client_config,omitempty" doc:"description=Configures how the pattern ingester will connect to the ingesters."` + factory ring_client.PoolFactory `yaml:"-"` +} + +// RegisterFlags registers the flags. +func (cfg *Config) RegisterFlags(f *flag.FlagSet) { + cfg.LifecyclerConfig.RegisterFlagsWithPrefix("ingester-rf1.", f, util_log.Logger) + cfg.ClientConfig.RegisterFlags(f) + + f.IntVar(&cfg.ConcurrentFlushes, "ingester-rf1.concurrent-flushes", 32, "How many flushes can happen concurrently from each stream.") + f.DurationVar(&cfg.FlushCheckPeriod, "ingester-rf1.flush-check-period", 500*time.Millisecond, "How often should the ingester see if there are any blocks to flush. The first flush check is delayed by a random time up to 0.8x the flush check period. Additionally, there is +/- 1% jitter added to the interval.") + f.DurationVar(&cfg.FlushOpBackoff.MinBackoff, "ingester-rf1.flush-op-backoff-min-period", 100*time.Millisecond, "Minimum backoff period when a flush fails. Each concurrent flush has its own backoff, see `ingester.concurrent-flushes`.") + f.DurationVar(&cfg.FlushOpBackoff.MaxBackoff, "ingester-rf1.flush-op-backoff-max-period", time.Minute, "Maximum backoff period when a flush fails. Each concurrent flush has its own backoff, see `ingester.concurrent-flushes`.") + f.IntVar(&cfg.FlushOpBackoff.MaxRetries, "ingester-rf1.flush-op-backoff-retries", 10, "Maximum retries for failed flushes.") + f.DurationVar(&cfg.FlushOpTimeout, "ingester-rf1.flush-op-timeout", 10*time.Minute, "The timeout for an individual flush. Will be retried up to `flush-op-backoff-retries` times.") + f.DurationVar(&cfg.RetainPeriod, "ingester-rf1.chunks-retain-period", 0, "How long chunks should be retained in-memory after they've been flushed.") + //f.DurationVar(&cfg.MaxChunkIdle, "ingester-rf1.chunks-idle-period", 30*time.Minute, "How long chunks should sit in-memory with no updates before being flushed if they don't hit the max block size. This means that half-empty chunks will still be flushed after a certain period as long as they receive no further activity.") + f.IntVar(&cfg.BlockSize, "ingester-rf1.chunks-block-size", 256*1024, "The targeted _uncompressed_ size in bytes of a chunk block When this threshold is exceeded the head block will be cut and compressed inside the chunk.") + f.IntVar(&cfg.TargetChunkSize, "ingester-rf1.chunk-target-size", 1572864, "A target _compressed_ size in bytes for chunks. This is a desired size not an exact size, chunks may be slightly bigger or significantly smaller if they get flushed for other reasons (e.g. chunk_idle_period). A value of 0 creates chunks with a fixed 10 blocks, a non zero value will create chunks with a variable number of blocks to meet the target size.") // 1.5 MB + f.StringVar(&cfg.ChunkEncoding, "ingester-rf1.chunk-encoding", chunkenc.EncGZIP.String(), fmt.Sprintf("The algorithm to use for compressing chunk. (%s)", chunkenc.SupportedEncoding())) + f.IntVar(&cfg.MaxReturnedErrors, "ingester-rf1.max-ignored-stream-errors", 10, "The maximum number of errors a stream will report to the user when a push fails. 0 to make unlimited.") + f.DurationVar(&cfg.MaxChunkAge, "ingester-rf1.max-chunk-age", 2*time.Hour, "The maximum duration of a timeseries chunk in memory. If a timeseries runs for longer than this, the current chunk will be flushed to the store and a new chunk created.") + f.BoolVar(&cfg.AutoForgetUnhealthy, "ingester-rf1.autoforget-unhealthy", false, "Forget about ingesters having heartbeat timestamps older than `ring.kvstore.heartbeat_timeout`. This is equivalent to clicking on the `/ring` `forget` button in the UI: the ingester is removed from the ring. This is a useful setting when you are sure that an unhealthy node won't return. An example is when not using stateful sets or the equivalent. Use `memberlist.rejoin_interval` > 0 to handle network partition cases when using a memberlist.") + f.IntVar(&cfg.IndexShards, "ingester-rf1.index-shards", index.DefaultIndexShards, "Shard factor used in the ingesters for the in process reverse index. This MUST be evenly divisible by ALL schema shard factors or Loki will not start.") + f.IntVar(&cfg.MaxDroppedStreams, "ingester-rf1.tailer.max-dropped-streams", 10, "Maximum number of dropped streams to keep in memory during tailing.") + f.StringVar(&cfg.ShutdownMarkerPath, "ingester-rf1.shutdown-marker-path", "", "Path where the shutdown marker file is stored. If not set and common.path_prefix is set then common.path_prefix will be used.") + f.DurationVar(&cfg.OwnedStreamsCheckInterval, "ingester-rf1.owned-streams-check-interval", 30*time.Second, "Interval at which the ingester ownedStreamService checks for changes in the ring to recalculate owned streams.") + f.BoolVar(&cfg.Enabled, "ingester-rf1.enabled", false, "Flag to enable or disable the usage of the ingester-rf1 component.") +} + +func (cfg *Config) Validate() error { + enc, err := chunkenc.ParseEncoding(cfg.ChunkEncoding) + if err != nil { + return err + } + cfg.parsedEncoding = enc + + if cfg.FlushOpBackoff.MinBackoff > cfg.FlushOpBackoff.MaxBackoff { + return errors.New("invalid flush op min backoff: cannot be larger than max backoff") + } + if cfg.FlushOpBackoff.MaxRetries <= 0 { + return fmt.Errorf("invalid flush op max retries: %d", cfg.FlushOpBackoff.MaxRetries) + } + if cfg.FlushOpTimeout <= 0 { + return fmt.Errorf("invalid flush op timeout: %s", cfg.FlushOpTimeout) + } + + return nil +} + +type Wrapper interface { + Wrap(wrapped Interface) Interface +} + +// Store is the store interface we need on the ingester. +type Store interface { + stores.ChunkWriter + stores.ChunkFetcher + storage.SelectStore + storage.SchemaConfigProvider + indexstore.StatsReader +} + +// Interface is an interface for the Ingester +type Interface interface { + services.Service + http.Handler + + logproto.PusherServer + //logproto.QuerierServer + //logproto.StreamDataServer + + CheckReady(ctx context.Context) error + FlushHandler(w http.ResponseWriter, _ *http.Request) + GetOrCreateInstance(instanceID string) (*instance, error) + ShutdownHandler(w http.ResponseWriter, r *http.Request) + PrepareShutdown(w http.ResponseWriter, r *http.Request) +} + +type flushCtx struct { + lock *sync.RWMutex + flushDone chan struct{} + newCtxAvailable chan struct{} + segmentWriter *wal.SegmentWriter + creationTime time.Time +} + +func (o *flushCtx) Key() string { + return fmt.Sprintf("%d", o.creationTime.UnixNano()) +} + +func (o *flushCtx) Priority() int64 { + return -o.creationTime.UnixNano() +} + +// Ingester builds chunks for incoming log streams. +type Ingester struct { + services.Service + + cfg Config + logger log.Logger + + clientConfig client.Config + tenantConfigs *runtime.TenantConfigs + + shutdownMtx sync.Mutex // Allows processes to grab a lock and prevent a shutdown + instancesMtx sync.RWMutex + instances map[string]*instance + readonly bool + + lifecycler *ring.Lifecycler + lifecyclerWatcher *services.FailureWatcher + + store Store + periodicConfigs []config.PeriodConfig + + loopDone sync.WaitGroup + loopQuit chan struct{} + tailersQuit chan struct{} + + // One queue per flush thread. Fingerprint is used to + // pick a queue. + flushQueues []*util.PriorityQueue + flushQueuesDone sync.WaitGroup + + flushCtx *flushCtx + + limiter *Limiter + + // Flag for whether stopping the ingester service should also terminate the + // loki process. + // This is set when calling the shutdown handler. + terminateOnShutdown bool + + // Only used by WAL & flusher to coordinate backpressure during replay. + //replayController *replayController + + metrics *ingesterMetrics + + chunkFilter chunk.RequestChunkFilterer + extractorWrapper lokilog.SampleExtractorWrapper + pipelineWrapper lokilog.PipelineWrapper + + streamRateCalculator *StreamRateCalculator + + writeLogManager *writefailures.Manager + + customStreamsTracker push.UsageTracker + + // recalculateOwnedStreams periodically checks the ring for changes and recalculates owned streams for each instance. + readRing ring.ReadRing + //recalculateOwnedStreams *recalculateOwnedStreams +} + +// New makes a new Ingester. +func New(cfg Config, clientConfig client.Config, store Store, limits Limits, configs *runtime.TenantConfigs, registerer prometheus.Registerer, writeFailuresCfg writefailures.Cfg, metricsNamespace string, logger log.Logger, customStreamsTracker push.UsageTracker, readRing ring.ReadRing) (*Ingester, error) { + if cfg.ingesterClientFactory == nil { + cfg.ingesterClientFactory = client.New + } + compressionStats.Set(cfg.ChunkEncoding) + targetSizeStats.Set(int64(cfg.TargetChunkSize)) + metrics := newIngesterMetrics(registerer, metricsNamespace) + + segmentWriter, err := wal.NewWalSegmentWriter() + if err != nil { + return nil, err + } + + i := &Ingester{ + cfg: cfg, + logger: logger, + clientConfig: clientConfig, + tenantConfigs: configs, + instances: map[string]*instance{}, + store: store, + periodicConfigs: store.GetSchemaConfigs(), + loopQuit: make(chan struct{}), + flushQueues: make([]*util.PriorityQueue, cfg.ConcurrentFlushes), + tailersQuit: make(chan struct{}), + metrics: metrics, + //flushOnShutdownSwitch: &OnceSwitch{}, + terminateOnShutdown: false, + streamRateCalculator: NewStreamRateCalculator(), + writeLogManager: writefailures.NewManager(logger, registerer, writeFailuresCfg, configs, "ingester_rf1"), + customStreamsTracker: customStreamsTracker, + readRing: readRing, + flushCtx: &flushCtx{ + lock: &sync.RWMutex{}, + flushDone: make(chan struct{}), + newCtxAvailable: make(chan struct{}), + segmentWriter: segmentWriter, + }, + } + //i.replayController = newReplayController(metrics, cfg.WAL, &replayFlusher{i}) + + // TODO: change flush on shutdown + i.lifecycler, err = ring.NewLifecycler(cfg.LifecyclerConfig, i, "ingester-rf1", "ingester-rf1-ring", true, logger, prometheus.WrapRegistererWithPrefix(metricsNamespace+"_", registerer)) + if err != nil { + return nil, err + } + + i.lifecyclerWatcher = services.NewFailureWatcher() + i.lifecyclerWatcher.WatchService(i.lifecycler) + + // Now that the lifecycler has been created, we can create the limiter + // which depends on it. + i.limiter = NewLimiter(limits, metrics, i.lifecycler, cfg.LifecyclerConfig.RingConfig.ReplicationFactor) + + i.Service = services.NewBasicService(i.starting, i.running, i.stopping) + + i.setupAutoForget() + + //if i.cfg.ChunkFilterer != nil { + // i.SetChunkFilterer(i.cfg.ChunkFilterer) + //} + // + //if i.cfg.PipelineWrapper != nil { + // i.SetPipelineWrapper(i.cfg.PipelineWrapper) + //} + // + //if i.cfg.SampleExtractorWrapper != nil { + // i.SetExtractorWrapper(i.cfg.SampleExtractorWrapper) + //} + // + //i.recalculateOwnedStreams = newRecalculateOwnedStreams(i.getInstances, i.lifecycler.ID, i.readRing, cfg.OwnedStreamsCheckInterval, util_log.Logger) + + return i, nil +} + +func (i *Ingester) SetChunkFilterer(chunkFilter chunk.RequestChunkFilterer) { + i.chunkFilter = chunkFilter +} + +func (i *Ingester) SetExtractorWrapper(wrapper lokilog.SampleExtractorWrapper) { + i.extractorWrapper = wrapper +} + +func (i *Ingester) SetPipelineWrapper(wrapper lokilog.PipelineWrapper) { + i.pipelineWrapper = wrapper +} + +// setupAutoForget looks for ring status if `AutoForgetUnhealthy` is enabled +// when enabled, unhealthy ingesters that reach `ring.kvstore.heartbeat_timeout` are removed from the ring every `HeartbeatPeriod` +func (i *Ingester) setupAutoForget() { + if !i.cfg.AutoForgetUnhealthy { + return + } + + go func() { + ctx := context.Background() + err := i.Service.AwaitRunning(ctx) + if err != nil { + level.Error(i.logger).Log("msg", fmt.Sprintf("autoforget received error %s, autoforget is disabled", err.Error())) + return + } + + level.Info(i.logger).Log("msg", fmt.Sprintf("autoforget is enabled and will remove unhealthy instances from the ring after %v with no heartbeat", i.cfg.LifecyclerConfig.RingConfig.HeartbeatTimeout)) + + ticker := time.NewTicker(i.cfg.LifecyclerConfig.HeartbeatPeriod) + defer ticker.Stop() + + var forgetList []string + for range ticker.C { + err := i.lifecycler.KVStore.CAS(ctx, RingKey, func(in interface{}) (out interface{}, retry bool, err error) { + forgetList = forgetList[:0] + if in == nil { + return nil, false, nil + } + + ringDesc, ok := in.(*ring.Desc) + if !ok { + level.Warn(i.logger).Log("msg", fmt.Sprintf("autoforget saw a KV store value that was not `ring.Desc`, got `%T`", in)) + return nil, false, nil + } + + for id, ingester := range ringDesc.Ingesters { + if !ingester.IsHealthy(ring.Reporting, i.cfg.LifecyclerConfig.RingConfig.HeartbeatTimeout, time.Now()) { + if i.lifecycler.ID == id { + level.Warn(i.logger).Log("msg", fmt.Sprintf("autoforget has seen our ID `%s` as unhealthy in the ring, network may be partitioned, skip forgeting ingesters this round", id)) + return nil, false, nil + } + forgetList = append(forgetList, id) + } + } + + if len(forgetList) == len(ringDesc.Ingesters)-1 { + level.Warn(i.logger).Log("msg", fmt.Sprintf("autoforget have seen %d unhealthy ingesters out of %d, network may be partioned, skip forgeting ingesters this round", len(forgetList), len(ringDesc.Ingesters))) + forgetList = forgetList[:0] + return nil, false, nil + } + + if len(forgetList) > 0 { + for _, id := range forgetList { + ringDesc.RemoveIngester(id) + } + return ringDesc, true, nil + } + return nil, false, nil + }) + if err != nil { + level.Warn(i.logger).Log("msg", err) + continue + } + + for _, id := range forgetList { + level.Info(i.logger).Log("msg", fmt.Sprintf("autoforget removed ingester %v from the ring because it was not healthy after %v", id, i.cfg.LifecyclerConfig.RingConfig.HeartbeatTimeout)) + } + i.metrics.autoForgetUnhealthyIngestersTotal.Add(float64(len(forgetList))) + } + }() +} + +// ServeHTTP implements the pattern ring status page. +func (i *Ingester) ServeHTTP(w http.ResponseWriter, r *http.Request) { + i.lifecycler.ServeHTTP(w, r) +} + +func (i *Ingester) starting(ctx context.Context) error { + i.InitFlushQueues() + + // pass new context to lifecycler, so that it doesn't stop automatically when Ingester's service context is done + err := i.lifecycler.StartAsync(context.Background()) + if err != nil { + return err + } + + err = i.lifecycler.AwaitRunning(ctx) + if err != nil { + return err + } + + shutdownMarkerPath := path.Join(i.cfg.ShutdownMarkerPath, shutdownMarkerFilename) + shutdownMarker, err := shutdownMarkerExists(shutdownMarkerPath) + if err != nil { + return errors.Wrap(err, "failed to check ingester shutdown marker") + } + + if shutdownMarker { + level.Info(i.logger).Log("msg", "detected existing shutdown marker, setting unregister and flush on shutdown", "path", shutdownMarkerPath) + i.setPrepareShutdown() + } + + //err = i.recalculateOwnedStreams.StartAsync(ctx) + //if err != nil { + // return fmt.Errorf("can not start recalculate owned streams service: %w", err) + //} + + err = i.lifecycler.AwaitRunning(ctx) + if err != nil { + return fmt.Errorf("can not ensure recalculate owned streams service is running: %w", err) + } + + // start our loop + i.loopDone.Add(1) + go i.loop() + return nil +} + +func (i *Ingester) running(ctx context.Context) error { + var serviceError error + select { + // wait until service is asked to stop + case <-ctx.Done(): + // stop + case err := <-i.lifecyclerWatcher.Chan(): + serviceError = fmt.Errorf("lifecycler failed: %w", err) + } + + // close tailers before stopping our loop + //close(i.tailersQuit) + //for _, instance := range i.getInstances() { + // instance.closeTailers() + //} + + close(i.loopQuit) + i.loopDone.Wait() + return serviceError +} + +// stopping is called when Ingester transitions to Stopping state. +// +// At this point, loop no longer runs, but flushers are still running. +func (i *Ingester) stopping(_ error) error { + i.stopIncomingRequests() + var errs util.MultiError + //errs.Add(i.wal.Stop()) + + //if i.flushOnShutdownSwitch.Get() { + // i.lifecycler.SetFlushOnShutdown(true) + //} + errs.Add(services.StopAndAwaitTerminated(context.Background(), i.lifecycler)) + + for _, flushQueue := range i.flushQueues { + flushQueue.Close() + } + i.flushQueuesDone.Wait() + + //i.streamRateCalculator.Stop() + + // In case the flag to terminate on shutdown is set or this instance is marked to release its resources, + // we need to mark the ingester service as "failed", so Loki will shut down entirely. + // The module manager logs the failure `modules.ErrStopProcess` in a special way. + if i.terminateOnShutdown && errs.Err() == nil { + i.removeShutdownMarkerFile() + return modules.ErrStopProcess + } + return errs.Err() +} + +// stopIncomingRequests is called when ingester is stopping +func (i *Ingester) stopIncomingRequests() { + i.shutdownMtx.Lock() + defer i.shutdownMtx.Unlock() + + i.instancesMtx.Lock() + defer i.instancesMtx.Unlock() + + i.readonly = true +} + +// removeShutdownMarkerFile removes the shutdown marker if it exists. Any errors are logged. +func (i *Ingester) removeShutdownMarkerFile() { + shutdownMarkerPath := path.Join(i.cfg.ShutdownMarkerPath, shutdownMarkerFilename) + exists, err := shutdownMarkerExists(shutdownMarkerPath) + if err != nil { + level.Error(i.logger).Log("msg", "error checking shutdown marker file exists", "err", err) + } + if exists { + err = removeShutdownMarker(shutdownMarkerPath) + if err != nil { + level.Error(i.logger).Log("msg", "error removing shutdown marker file", "err", err) + } + } +} + +func (i *Ingester) loop() { + defer i.loopDone.Done() + + // Delay the first flush operation by up to 0.8x the flush time period. + // This will ensure that multiple ingesters started at the same time do not + // flush at the same time. Flushing at the same time can cause concurrently + // writing the same chunk to object storage, which in AWS S3 leads to being + // rate limited. + jitter := time.Duration(rand.Int63n(int64(float64(i.cfg.FlushCheckPeriod.Nanoseconds()) * 0.8))) + initialDelay := time.NewTimer(jitter) + defer initialDelay.Stop() + + level.Info(i.logger).Log("msg", "sleeping for initial delay before starting periodic flushing", "delay", jitter) + + select { + case <-initialDelay.C: + // do nothing and continue with flush loop + case <-i.loopQuit: + // ingester stopped while waiting for initial delay + return + } + + // Add +/- 20% of flush interval as jitter. + // The default flush check period is 30s so max jitter will be 6s. + j := i.cfg.FlushCheckPeriod / 5 + flushTicker := util.NewTickerWithJitter(i.cfg.FlushCheckPeriod, j) + defer flushTicker.Stop() + + for { + select { + case <-flushTicker.C: + //i.logger.Log("msg", "starting periodic flush") + i.flushCtx.lock.Lock() // Stop new chunks being written while we swap destinations - we'll never unlock as this flushctx can no longer be used. + currentFlushCtx := i.flushCtx + // APIs become unblocked after resetting flushCtx + segmentWriter, err := wal.NewWalSegmentWriter() + if err != nil { + // TODO: handle this properly + panic(err) + } + i.flushCtx = &flushCtx{ + lock: &sync.RWMutex{}, + flushDone: make(chan struct{}), + newCtxAvailable: make(chan struct{}), + segmentWriter: segmentWriter, + } + close(currentFlushCtx.newCtxAvailable) // Broadcast to all waiters that they can now fetch a new flushCtx. Small chance of a race but if they re-fetch the old one, they'll just check again immediately. + // Flush the finished context in the background & then notify watching API requests + // TODO: use multiple flush queues if required + i.flushQueues[0].Enqueue(currentFlushCtx) + + case <-i.loopQuit: + return + } + } +} + +// PrepareShutdown will handle the /ingester/prepare_shutdown endpoint. +// +// Internally, when triggered, this handler will configure the ingester service to release their resources whenever a SIGTERM is received. +// Releasing resources meaning flushing data, deleting tokens, and removing itself from the ring. +// +// It also creates a file on disk which is used to re-apply the configuration if the +// ingester crashes and restarts before being permanently shutdown. +// +// * `GET` shows the status of this configuration +// * `POST` enables this configuration +// * `DELETE` disables this configuration +func (i *Ingester) PrepareShutdown(w http.ResponseWriter, r *http.Request) { + if i.cfg.ShutdownMarkerPath == "" { + w.WriteHeader(http.StatusInternalServerError) + return + } + shutdownMarkerPath := path.Join(i.cfg.ShutdownMarkerPath, shutdownMarkerFilename) + + switch r.Method { + case http.MethodGet: + exists, err := shutdownMarkerExists(shutdownMarkerPath) + if err != nil { + level.Error(i.logger).Log("msg", "unable to check for prepare-shutdown marker file", "path", shutdownMarkerPath, "err", err) + w.WriteHeader(http.StatusInternalServerError) + return + } + + if exists { + util.WriteTextResponse(w, "set") + } else { + util.WriteTextResponse(w, "unset") + } + case http.MethodPost: + if err := createShutdownMarker(shutdownMarkerPath); err != nil { + level.Error(i.logger).Log("msg", "unable to create prepare-shutdown marker file", "path", shutdownMarkerPath, "err", err) + w.WriteHeader(http.StatusInternalServerError) + return + } + + i.setPrepareShutdown() + level.Info(i.logger).Log("msg", "created prepare-shutdown marker file", "path", shutdownMarkerPath) + + w.WriteHeader(http.StatusNoContent) + case http.MethodDelete: + if err := removeShutdownMarker(shutdownMarkerPath); err != nil { + level.Error(i.logger).Log("msg", "unable to remove prepare-shutdown marker file", "path", shutdownMarkerPath, "err", err) + w.WriteHeader(http.StatusInternalServerError) + return + } + + i.unsetPrepareShutdown() + level.Info(i.logger).Log("msg", "removed prepare-shutdown marker file", "path", shutdownMarkerPath) + + w.WriteHeader(http.StatusNoContent) + default: + w.WriteHeader(http.StatusMethodNotAllowed) + } +} + +// setPrepareShutdown toggles ingester lifecycler config to prepare for shutdown +func (i *Ingester) setPrepareShutdown() { + level.Info(i.logger).Log("msg", "preparing full ingester shutdown, resources will be released on SIGTERM") + i.lifecycler.SetFlushOnShutdown(true) + i.lifecycler.SetUnregisterOnShutdown(true) + i.terminateOnShutdown = true + i.metrics.shutdownMarker.Set(1) +} + +func (i *Ingester) unsetPrepareShutdown() { + level.Info(i.logger).Log("msg", "undoing preparation for full ingester shutdown") + i.lifecycler.SetFlushOnShutdown(true) + i.lifecycler.SetUnregisterOnShutdown(i.cfg.LifecyclerConfig.UnregisterOnShutdown) + i.terminateOnShutdown = false + i.metrics.shutdownMarker.Set(0) +} + +// createShutdownMarker writes a marker file to disk to indicate that an ingester is +// going to be scaled down in the future. The presence of this file means that an ingester +// should flush and upload all data when stopping. +func createShutdownMarker(p string) error { + // Write the file, fsync it, then fsync the containing directory in order to guarantee + // it is persisted to disk. From https://man7.org/linux/man-pages/man2/fsync.2.html + // + // > Calling fsync() does not necessarily ensure that the entry in the + // > directory containing the file has also reached disk. For that an + // > explicit fsync() on a file descriptor for the directory is also + // > needed. + file, err := os.Create(p) + if err != nil { + return err + } + + merr := multierror.New() + _, err = file.WriteString(time.Now().UTC().Format(time.RFC3339)) + merr.Add(err) + merr.Add(file.Sync()) + merr.Add(file.Close()) + + if err := merr.Err(); err != nil { + return err + } + + dir, err := os.OpenFile(path.Dir(p), os.O_RDONLY, 0777) + if err != nil { + return err + } + + merr.Add(dir.Sync()) + merr.Add(dir.Close()) + return merr.Err() +} + +// removeShutdownMarker removes the shutdown marker file if it exists. +func removeShutdownMarker(p string) error { + err := os.Remove(p) + if err != nil && !os.IsNotExist(err) { + return err + } + + dir, err := os.OpenFile(path.Dir(p), os.O_RDONLY, 0777) + if err != nil { + return err + } + + merr := multierror.New() + merr.Add(dir.Sync()) + merr.Add(dir.Close()) + return merr.Err() +} + +// shutdownMarkerExists returns true if the shutdown marker file exists, false otherwise +func shutdownMarkerExists(p string) (bool, error) { + s, err := os.Stat(p) + if err != nil && os.IsNotExist(err) { + return false, nil + } + + if err != nil { + return false, err + } + + return s.Mode().IsRegular(), nil +} + +// ShutdownHandler handles a graceful shutdown of the ingester service and +// termination of the Loki process. +func (i *Ingester) ShutdownHandler(w http.ResponseWriter, r *http.Request) { + // Don't allow calling the shutdown handler multiple times + if i.State() != services.Running { + w.WriteHeader(http.StatusServiceUnavailable) + _, _ = w.Write([]byte("Ingester is stopping or already stopped.")) + return + } + params := r.URL.Query() + doFlush := util.FlagFromValues(params, "flush", true) + doDeleteRingTokens := util.FlagFromValues(params, "delete_ring_tokens", false) + doTerminate := util.FlagFromValues(params, "terminate", true) + err := i.handleShutdown(doTerminate, doFlush, doDeleteRingTokens) + + // Stopping the module will return the modules.ErrStopProcess error. This is + // needed so the Loki process is shut down completely. + if err == nil || err == modules.ErrStopProcess { + w.WriteHeader(http.StatusNoContent) + } else { + w.WriteHeader(http.StatusInternalServerError) + _, _ = w.Write([]byte(err.Error())) + } +} + +// handleShutdown triggers the following operations: +// - Change the state of ring to stop accepting writes. +// - optional: Flush all the chunks. +// - optional: Delete ring tokens file +// - Unregister from KV store +// - optional: Terminate process (handled by service manager in loki.go) +func (i *Ingester) handleShutdown(terminate, flush, del bool) error { + i.lifecycler.SetFlushOnShutdown(flush) + i.lifecycler.SetClearTokensOnShutdown(del) + i.lifecycler.SetUnregisterOnShutdown(true) + i.terminateOnShutdown = terminate + return services.StopAndAwaitTerminated(context.Background(), i) +} + +// Push implements logproto.Pusher. +func (i *Ingester) Push(ctx context.Context, req *logproto.PushRequest) (*logproto.PushResponse, error) { + instanceID, err := tenant.TenantID(ctx) + if err != nil { + return nil, err + } else if i.readonly { + return nil, ErrReadOnly + } + + // Set profiling tags + defer pprof.SetGoroutineLabels(ctx) + ctx = pprof.WithLabels(ctx, pprof.Labels("path", "write", "tenant", instanceID)) + pprof.SetGoroutineLabels(ctx) + + instance, err := i.GetOrCreateInstance(instanceID) + if err != nil { + return &logproto.PushResponse{}, err + } + + // Fetch a flush context and try to acquire the RLock + // The only time the Write Lock is held is when this context is no longer usable and a new one is being created. + // In this case, we need to re-read i.flushCtx in order to fetch the new one as soon as it's available. + //The newCtxAvailable chan is closed as soon as the new one is available to avoid a busy loop. + currentFlushCtx := i.flushCtx + for !currentFlushCtx.lock.TryRLock() { + select { + case <-currentFlushCtx.newCtxAvailable: + case <-ctx.Done(): + return &logproto.PushResponse{}, ctx.Err() + } + currentFlushCtx = i.flushCtx + } + err = instance.Push(ctx, req, currentFlushCtx) + currentFlushCtx.lock.RUnlock() + select { + case <-ctx.Done(): + return &logproto.PushResponse{}, ctx.Err() + case <-currentFlushCtx.flushDone: + return &logproto.PushResponse{}, err + } +} + +// GetStreamRates returns a response containing all streams and their current rate +// TODO: It might be nice for this to be human readable, eventually: Sort output and return labels, too? +func (i *Ingester) GetStreamRates(ctx context.Context, _ *logproto.StreamRatesRequest) (*logproto.StreamRatesResponse, error) { + if sp := opentracing.SpanFromContext(ctx); sp != nil { + sp.LogKV("event", "ingester started to handle GetStreamRates") + defer sp.LogKV("event", "ingester finished handling GetStreamRates") + } + + // Set profiling tags + defer pprof.SetGoroutineLabels(ctx) + ctx = pprof.WithLabels(ctx, pprof.Labels("path", "write")) + pprof.SetGoroutineLabels(ctx) + + allRates := i.streamRateCalculator.Rates() + rates := make([]*logproto.StreamRate, len(allRates)) + for idx := range allRates { + rates[idx] = &allRates[idx] + } + return &logproto.StreamRatesResponse{StreamRates: nil}, nil +} + +func (i *Ingester) GetOrCreateInstance(instanceID string) (*instance, error) { //nolint:revive + inst, ok := i.getInstanceByID(instanceID) + if ok { + return inst, nil + } + + i.instancesMtx.Lock() + defer i.instancesMtx.Unlock() + inst, ok = i.instances[instanceID] + if !ok { + var err error + inst, err = newInstance(&i.cfg, i.periodicConfigs, instanceID, i.limiter, i.tenantConfigs, i.metrics, i.chunkFilter, i.pipelineWrapper, i.extractorWrapper, i.streamRateCalculator, i.writeLogManager, i.customStreamsTracker) + if err != nil { + return nil, err + } + i.instances[instanceID] = inst + activeTenantsStats.Set(int64(len(i.instances))) + } + return inst, nil +} + +// asyncStoreMaxLookBack returns a max look back period only if active index type is one of async index stores like `boltdb-shipper` and `tsdb`. +// max look back is limited to from time of async store config. +// It considers previous periodic config's from time if that also has async index type. +// This is to limit the lookback to only async stores where relevant. +func (i *Ingester) asyncStoreMaxLookBack() time.Duration { + activePeriodicConfigIndex := config.ActivePeriodConfig(i.periodicConfigs) + activePeriodicConfig := i.periodicConfigs[activePeriodicConfigIndex] + if activePeriodicConfig.IndexType != types.BoltDBShipperType && activePeriodicConfig.IndexType != types.TSDBType { + return 0 + } + + startTime := activePeriodicConfig.From + if activePeriodicConfigIndex != 0 && (i.periodicConfigs[activePeriodicConfigIndex-1].IndexType == types.BoltDBShipperType || + i.periodicConfigs[activePeriodicConfigIndex-1].IndexType == types.TSDBType) { + startTime = i.periodicConfigs[activePeriodicConfigIndex-1].From + } + + maxLookBack := time.Since(startTime.Time.Time()) + return maxLookBack +} + +// GetChunkIDs is meant to be used only when using an async store like boltdb-shipper or tsdb. +func (i *Ingester) GetChunkIDs(ctx context.Context, req *logproto.GetChunkIDsRequest) (*logproto.GetChunkIDsResponse, error) { + gcr, err := i.getChunkIDs(ctx, req) + err = server_util.ClientGrpcStatusAndError(err) + return gcr, err +} + +// GetChunkIDs is meant to be used only when using an async store like boltdb-shipper or tsdb. +func (i *Ingester) getChunkIDs(ctx context.Context, req *logproto.GetChunkIDsRequest) (*logproto.GetChunkIDsResponse, error) { + orgID, err := tenant.TenantID(ctx) + if err != nil { + return nil, err + } + + // Set profiling tags + defer pprof.SetGoroutineLabels(ctx) + ctx = pprof.WithLabels(ctx, pprof.Labels("path", "read", "type", "chunkIDs", "tenant", orgID)) + pprof.SetGoroutineLabels(ctx) + + asyncStoreMaxLookBack := i.asyncStoreMaxLookBack() + if asyncStoreMaxLookBack == 0 { + return &logproto.GetChunkIDsResponse{}, nil + } + + reqStart := req.Start + reqStart = adjustQueryStartTime(asyncStoreMaxLookBack, reqStart, time.Now()) + + // parse the request + start, end := util.RoundToMilliseconds(reqStart, req.End) + matchers, err := syntax.ParseMatchers(req.Matchers, true) + if err != nil { + return nil, err + } + + // get chunk references + chunksGroups, _, err := i.store.GetChunks(ctx, orgID, start, end, chunk.NewPredicate(matchers, nil), nil) + if err != nil { + return nil, err + } + + // todo (Callum) ingester should maybe store the whole schema config? + s := config.SchemaConfig{ + Configs: i.periodicConfigs, + } + + // build the response + resp := logproto.GetChunkIDsResponse{ChunkIDs: []string{}} + for _, chunks := range chunksGroups { + for _, chk := range chunks { + resp.ChunkIDs = append(resp.ChunkIDs, s.ExternalKey(chk.ChunkRef)) + } + } + + return &resp, nil +} + +// Watch implements grpc_health_v1.HealthCheck. +func (*Ingester) Watch(*grpc_health_v1.HealthCheckRequest, grpc_health_v1.Health_WatchServer) error { + return nil +} + +// ReadinessHandler is used to indicate to k8s when the ingesters are ready for +// the addition removal of another ingester. Returns 204 when the ingester is +// ready, 500 otherwise. +func (i *Ingester) CheckReady(ctx context.Context) error { + if s := i.State(); s != services.Running && s != services.Stopping { + return fmt.Errorf("ingester not ready: %v", s) + } + return i.lifecycler.CheckReady(ctx) +} + +func (i *Ingester) getInstanceByID(id string) (*instance, bool) { + i.instancesMtx.RLock() + defer i.instancesMtx.RUnlock() + + inst, ok := i.instances[id] + return inst, ok +} + +func (i *Ingester) getInstances() []*instance { + i.instancesMtx.RLock() + defer i.instancesMtx.RUnlock() + + instances := make([]*instance, 0, len(i.instances)) + for _, instance := range i.instances { + instances = append(instances, instance) + } + return instances +} + +//// Tail logs matching given query +//func (i *Ingester) Tail(req *logproto.TailRequest, queryServer logproto.Querier_TailServer) error { +// err := i.tail(req, queryServer) +// err = server_util.ClientGrpcStatusAndError(err) +// return err +//} +//func (i *Ingester) tail(req *logproto.TailRequest, queryServer logproto.Querier_TailServer) error { +// select { +// case <-i.tailersQuit: +// return errors.New("Ingester is stopping") +// default: +// } +// +// if req.Plan == nil { +// parsed, err := syntax.ParseLogSelector(req.Query, true) +// if err != nil { +// return err +// } +// req.Plan = &plan.QueryPlan{ +// AST: parsed, +// } +// } +// +// instanceID, err := tenant.TenantID(queryServer.Context()) +// if err != nil { +// return err +// } +// +// instance, err := i.GetOrCreateInstance(instanceID) +// if err != nil { +// return err +// } +// +// expr, ok := req.Plan.AST.(syntax.LogSelectorExpr) +// if !ok { +// return fmt.Errorf("unsupported query expression: want (LogSelectorExpr), got (%T)", req.Plan.AST) +// } +// +// tailer, err := newTailer(instanceID, expr, queryServer, i.cfg.MaxDroppedStreams) +// if err != nil { +// return err +// } +// +// if err := instance.addNewTailer(queryServer.Context(), tailer); err != nil { +// return err +// } +// tailer.loop() +// return nil +//} +// +//// TailersCount returns count of active tail requests from a user +//func (i *Ingester) TailersCount(ctx context.Context, _ *logproto.TailersCountRequest) (*logproto.TailersCountResponse, error) { +// tcr, err := i.tailersCount(ctx) +// err = server_util.ClientGrpcStatusAndError(err) +// return tcr, err +//} +// +//func (i *Ingester) tailersCount(ctx context.Context) (*logproto.TailersCountResponse, error) { +// instanceID, err := tenant.TenantID(ctx) +// if err != nil { +// return nil, err +// } +// +// resp := logproto.TailersCountResponse{} +// +// instance, ok := i.getInstanceByID(instanceID) +// if ok { +// resp.Count = instance.openTailersCount() +// } +// +// return &resp, nil +//} + +func adjustQueryStartTime(maxLookBackPeriod time.Duration, start, now time.Time) time.Time { + if maxLookBackPeriod > 0 { + oldestStartTime := now.Add(-maxLookBackPeriod) + if oldestStartTime.After(start) { + return oldestStartTime + } + } + return start +} + +func (i *Ingester) GetDetectedFields(_ context.Context, r *logproto.DetectedFieldsRequest) (*logproto.DetectedFieldsResponse, error) { + return &logproto.DetectedFieldsResponse{ + Fields: []*logproto.DetectedField{ + { + Label: "foo", + Type: logproto.DetectedFieldString, + Cardinality: 1, + }, + }, + FieldLimit: r.GetFieldLimit(), + }, nil +} diff --git a/pkg/ingester-rf1/instance.go b/pkg/ingester-rf1/instance.go new file mode 100644 index 0000000000000..e5c54549713af --- /dev/null +++ b/pkg/ingester-rf1/instance.go @@ -0,0 +1,299 @@ +package ingesterrf1 + +import ( + "context" + "fmt" + "math" + "net/http" + "sync" + + "github.com/go-kit/log/level" + "github.com/grafana/dskit/httpgrpc" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" + + "github.com/grafana/loki/v3/pkg/analytics" + "github.com/grafana/loki/v3/pkg/chunkenc" + "github.com/grafana/loki/v3/pkg/distributor/writefailures" + "github.com/grafana/loki/v3/pkg/ingester/index" + "github.com/grafana/loki/v3/pkg/loghttp/push" + "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/logql/log" + "github.com/grafana/loki/v3/pkg/logql/syntax" + "github.com/grafana/loki/v3/pkg/runtime" + "github.com/grafana/loki/v3/pkg/storage/chunk" + "github.com/grafana/loki/v3/pkg/storage/config" + "github.com/grafana/loki/v3/pkg/util/constants" + util_log "github.com/grafana/loki/v3/pkg/util/log" + "github.com/grafana/loki/v3/pkg/validation" +) + +var ( + memoryStreams = promauto.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: constants.Loki, + Name: "ingester_rf1_memory_streams", + Help: "The total number of streams in memory per tenant.", + }, []string{"tenant"}) + memoryStreamsLabelsBytes = promauto.NewGauge(prometheus.GaugeOpts{ + Namespace: constants.Loki, + Name: "ingester_rf1_memory_streams_labels_bytes", + Help: "Total bytes of labels of the streams in memory.", + }) + streamsCreatedTotal = promauto.NewCounterVec(prometheus.CounterOpts{ + Namespace: constants.Loki, + Name: "ingester_rf1_streams_created_total", + Help: "The total number of streams created per tenant.", + }, []string{"tenant"}) + streamsRemovedTotal = promauto.NewCounterVec(prometheus.CounterOpts{ + Namespace: constants.Loki, + Name: "ingester_rf1_streams_removed_total", + Help: "The total number of streams removed per tenant.", + }, []string{"tenant"}) + + streamsCountStats = analytics.NewInt("ingester_rf1_streams_count") +) + +type instance struct { + cfg *Config + + buf []byte // buffer used to compute fps. + streams *streamsMap + + index *index.Multi + mapper *FpMapper // using of mapper no longer needs mutex because reading from streams is lock-free + + instanceID string + + streamsCreatedTotal prometheus.Counter + streamsRemovedTotal prometheus.Counter + + //tailers map[uint32]*tailer + tailerMtx sync.RWMutex + + limiter *Limiter + streamCountLimiter *streamCountLimiter + ownedStreamsSvc *ownedStreamService + + configs *runtime.TenantConfigs + + metrics *ingesterMetrics + + chunkFilter chunk.RequestChunkFilterer + pipelineWrapper log.PipelineWrapper + extractorWrapper log.SampleExtractorWrapper + streamRateCalculator *StreamRateCalculator + + writeFailures *writefailures.Manager + + schemaconfig *config.SchemaConfig + + customStreamsTracker push.UsageTracker +} + +func (i *instance) Push(ctx context.Context, req *logproto.PushRequest, flushCtx *flushCtx) error { + rateLimitWholeStream := i.limiter.limits.ShardStreams(i.instanceID).Enabled + + var appendErr error + for _, reqStream := range req.Streams { + s, _, err := i.streams.LoadOrStoreNew(reqStream.Labels, + func() (*stream, error) { + s, err := i.createStream(ctx, reqStream) + return s, err + }, + func(s *stream) error { + return nil + }, + ) + if err != nil { + appendErr = err + continue + } + + _, appendErr = s.Push(ctx, reqStream.Entries, rateLimitWholeStream, i.customStreamsTracker, flushCtx) + } + return appendErr +} + +func newInstance( + cfg *Config, + periodConfigs []config.PeriodConfig, + instanceID string, + limiter *Limiter, + configs *runtime.TenantConfigs, + metrics *ingesterMetrics, + chunkFilter chunk.RequestChunkFilterer, + pipelineWrapper log.PipelineWrapper, + extractorWrapper log.SampleExtractorWrapper, + streamRateCalculator *StreamRateCalculator, + writeFailures *writefailures.Manager, + customStreamsTracker push.UsageTracker, +) (*instance, error) { + fmt.Println("new instance for", instanceID) + invertedIndex, err := index.NewMultiInvertedIndex(periodConfigs, uint32(cfg.IndexShards)) + if err != nil { + return nil, err + } + streams := newStreamsMap() + ownedStreamsSvc := newOwnedStreamService(instanceID, limiter) + c := config.SchemaConfig{Configs: periodConfigs} + i := &instance{ + cfg: cfg, + streams: streams, + buf: make([]byte, 0, 1024), + index: invertedIndex, + instanceID: instanceID, + // + streamsCreatedTotal: streamsCreatedTotal.WithLabelValues(instanceID), + streamsRemovedTotal: streamsRemovedTotal.WithLabelValues(instanceID), + // + //tailers: map[uint32]*tailer{}, + limiter: limiter, + streamCountLimiter: newStreamCountLimiter(instanceID, streams.Len, limiter, ownedStreamsSvc), + ownedStreamsSvc: ownedStreamsSvc, + configs: configs, + metrics: metrics, + chunkFilter: chunkFilter, + pipelineWrapper: pipelineWrapper, + extractorWrapper: extractorWrapper, + + streamRateCalculator: streamRateCalculator, + + writeFailures: writeFailures, + schemaconfig: &c, + + customStreamsTracker: customStreamsTracker, + } + i.mapper = NewFPMapper(i.getLabelsFromFingerprint) + + return i, nil +} + +func (i *instance) createStream(ctx context.Context, pushReqStream logproto.Stream) (*stream, error) { + // record is only nil when replaying WAL. We don't want to drop data when replaying a WAL after + // reducing the stream limits, for instance. + var err error + + labels, err := syntax.ParseLabels(pushReqStream.Labels) + if err != nil { + if i.configs.LogStreamCreation(i.instanceID) { + level.Debug(util_log.Logger).Log( + "msg", "failed to create stream, failed to parse labels", + "org_id", i.instanceID, + "err", err, + "stream", pushReqStream.Labels, + ) + } + return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) + } + + if err != nil { + return i.onStreamCreationError(ctx, pushReqStream, err, labels) + } + + fp := i.getHashForLabels(labels) + + sortedLabels := i.index.Add(logproto.FromLabelsToLabelAdapters(labels), fp) + + chunkfmt, headfmt, err := i.chunkFormatAt(minTs(&pushReqStream)) + if err != nil { + return nil, fmt.Errorf("failed to create stream: %w", err) + } + + s := newStream(chunkfmt, headfmt, i.cfg, i.limiter, i.instanceID, fp, sortedLabels, i.limiter.UnorderedWrites(i.instanceID) /*i.streamRateCalculator,*/, i.metrics, i.writeFailures) + + i.onStreamCreated(s) + + return s, nil +} + +// minTs is a helper to return minimum Unix timestamp (as `model.Time`) +// across all the entries in a given `stream`. +func minTs(stream *logproto.Stream) model.Time { + // NOTE: We choose `min` timestamp because, the chunk is written once then + // added to the index buckets for may be different days. It would better rather to have + // some latest(say v13) indices reference older (say v12) compatible chunks than vice versa. + + streamMinTs := int64(math.MaxInt64) + for _, entry := range stream.Entries { + ts := entry.Timestamp.UnixNano() + if streamMinTs > ts { + streamMinTs = ts + } + } + return model.TimeFromUnixNano(streamMinTs) +} + +// chunkFormatAt returns chunk formats to use at given period of time. +func (i *instance) chunkFormatAt(at model.Time) (byte, chunkenc.HeadBlockFmt, error) { + // NOTE: We choose chunk formats for stream based on it's entries timestamp. + // Rationale being, a single (ingester) instance can be running across multiple schema period + // and choosing correct periodConfig during creation of stream is more accurate rather + // than choosing it during starting of instance itself. + + periodConfig, err := i.schemaconfig.SchemaForTime(at) + if err != nil { + return 0, 0, err + } + + chunkFormat, headblock, err := periodConfig.ChunkFormat() + if err != nil { + return 0, 0, err + } + + return chunkFormat, headblock, nil +} + +func (i *instance) getHashForLabels(ls labels.Labels) model.Fingerprint { + var fp uint64 + fp, i.buf = ls.HashWithoutLabels(i.buf, []string(nil)...) + return i.mapper.MapFP(model.Fingerprint(fp), ls) +} + +// Return labels associated with given fingerprint. Used by fingerprint mapper. +func (i *instance) getLabelsFromFingerprint(fp model.Fingerprint) labels.Labels { + s, ok := i.streams.LoadByFP(fp) + if !ok { + return nil + } + return s.labels +} + +func (i *instance) onStreamCreationError(ctx context.Context, pushReqStream logproto.Stream, err error, labels labels.Labels) (*stream, error) { + if i.configs.LogStreamCreation(i.instanceID) { + level.Debug(util_log.Logger).Log( + "msg", "failed to create stream, exceeded limit", + "org_id", i.instanceID, + "err", err, + "stream", pushReqStream.Labels, + ) + } + + validation.DiscardedSamples.WithLabelValues(validation.StreamLimit, i.instanceID).Add(float64(len(pushReqStream.Entries))) + bytes := 0 + for _, e := range pushReqStream.Entries { + bytes += len(e.Line) + } + validation.DiscardedBytes.WithLabelValues(validation.StreamLimit, i.instanceID).Add(float64(bytes)) + if i.customStreamsTracker != nil { + i.customStreamsTracker.DiscardedBytesAdd(ctx, i.instanceID, validation.StreamLimit, labels, float64(bytes)) + } + return nil, httpgrpc.Errorf(http.StatusTooManyRequests, validation.StreamLimitErrorMsg, labels, i.instanceID) +} + +func (i *instance) onStreamCreated(s *stream) { + memoryStreams.WithLabelValues(i.instanceID).Inc() + memoryStreamsLabelsBytes.Add(float64(len(s.labels.String()))) + i.streamsCreatedTotal.Inc() + //i.addTailersToNewStream(s) + streamsCountStats.Add(1) + i.ownedStreamsSvc.incOwnedStreamCount() + if i.configs.LogStreamCreation(i.instanceID) { + level.Debug(util_log.Logger).Log( + "msg", "successfully created stream", + "org_id", i.instanceID, + "stream", s.labels.String(), + ) + } +} diff --git a/pkg/ingester-rf1/limiter.go b/pkg/ingester-rf1/limiter.go new file mode 100644 index 0000000000000..1957ed54d9145 --- /dev/null +++ b/pkg/ingester-rf1/limiter.go @@ -0,0 +1,226 @@ +package ingesterrf1 + +import ( + "fmt" + "math" + "sync" + "time" + + "golang.org/x/time/rate" + + "github.com/grafana/loki/v3/pkg/distributor/shardstreams" + "github.com/grafana/loki/v3/pkg/validation" +) + +const ( + errMaxStreamsPerUserLimitExceeded = "tenant '%v' per-user streams limit exceeded, streams: %d exceeds calculated limit: %d (local limit: %d, global limit: %d, global/ingesters: %d)" +) + +// RingCount is the interface exposed by a ring implementation which allows +// to count members +type RingCount interface { + HealthyInstancesCount() int +} + +type Limits interface { + UnorderedWrites(userID string) bool + UseOwnedStreamCount(userID string) bool + MaxLocalStreamsPerUser(userID string) int + MaxGlobalStreamsPerUser(userID string) int + PerStreamRateLimit(userID string) validation.RateLimit + ShardStreams(userID string) shardstreams.Config +} + +// Limiter implements primitives to get the maximum number of streams +// an ingester can handle for a specific tenant +type Limiter struct { + limits Limits + ring RingCount + replicationFactor int + metrics *ingesterMetrics + + mtx sync.RWMutex + disabled bool +} + +func (l *Limiter) DisableForWALReplay() { + l.mtx.Lock() + defer l.mtx.Unlock() + l.disabled = true + l.metrics.limiterEnabled.Set(0) +} + +func (l *Limiter) Enable() { + l.mtx.Lock() + defer l.mtx.Unlock() + l.disabled = false + l.metrics.limiterEnabled.Set(1) +} + +// NewLimiter makes a new limiter +func NewLimiter(limits Limits, metrics *ingesterMetrics, ring RingCount, replicationFactor int) *Limiter { + return &Limiter{ + limits: limits, + ring: ring, + replicationFactor: replicationFactor, + metrics: metrics, + } +} + +func (l *Limiter) UnorderedWrites(userID string) bool { + // WAL replay should not discard previously ack'd writes, + // so allow out of order writes while the limiter is disabled. + // This allows replaying unordered WALs into ordered configurations. + if l.disabled { + return true + } + return l.limits.UnorderedWrites(userID) +} + +func (l *Limiter) GetStreamCountLimit(tenantID string) (calculatedLimit, localLimit, globalLimit, adjustedGlobalLimit int) { + // Start by setting the local limit either from override or default + localLimit = l.limits.MaxLocalStreamsPerUser(tenantID) + + // We can assume that streams are evenly distributed across ingesters + // so we do convert the global limit into a local limit + globalLimit = l.limits.MaxGlobalStreamsPerUser(tenantID) + adjustedGlobalLimit = l.convertGlobalToLocalLimit(globalLimit) + + // Set the calculated limit to the lesser of the local limit or the new calculated global limit + calculatedLimit = l.minNonZero(localLimit, adjustedGlobalLimit) + + // If both the local and global limits are disabled, we just + // use the largest int value + if calculatedLimit == 0 { + calculatedLimit = math.MaxInt32 + } + return +} + +func (l *Limiter) minNonZero(first, second int) int { + if first == 0 || (second != 0 && first > second) { + return second + } + + return first +} + +func (l *Limiter) convertGlobalToLocalLimit(globalLimit int) int { + if globalLimit == 0 { + return 0 + } + // todo: change to healthyInstancesInZoneCount() once + // Given we don't need a super accurate count (ie. when the ingesters + // topology changes) and we prefer to always be in favor of the tenant, + // we can use a per-ingester limit equal to: + // (global limit / number of ingesters) * replication factor + numIngesters := l.ring.HealthyInstancesCount() + + // May happen because the number of ingesters is asynchronously updated. + // If happens, we just temporarily ignore the global limit. + if numIngesters > 0 { + return int((float64(globalLimit) / float64(numIngesters)) * float64(l.replicationFactor)) + } + + return 0 +} + +type supplier[T any] func() T + +type streamCountLimiter struct { + tenantID string + limiter *Limiter + defaultStreamCountSupplier supplier[int] + ownedStreamSvc *ownedStreamService +} + +var noopFixedLimitSupplier = func() int { + return 0 +} + +func newStreamCountLimiter(tenantID string, defaultStreamCountSupplier supplier[int], limiter *Limiter, service *ownedStreamService) *streamCountLimiter { + return &streamCountLimiter{ + tenantID: tenantID, + limiter: limiter, + defaultStreamCountSupplier: defaultStreamCountSupplier, + ownedStreamSvc: service, + } +} + +func (l *streamCountLimiter) AssertNewStreamAllowed(tenantID string) error { + streamCountSupplier, fixedLimitSupplier := l.getSuppliers(tenantID) + calculatedLimit, localLimit, globalLimit, adjustedGlobalLimit := l.getCurrentLimit(tenantID, fixedLimitSupplier) + actualStreamsCount := streamCountSupplier() + if actualStreamsCount < calculatedLimit { + return nil + } + + return fmt.Errorf(errMaxStreamsPerUserLimitExceeded, tenantID, actualStreamsCount, calculatedLimit, localLimit, globalLimit, adjustedGlobalLimit) +} + +func (l *streamCountLimiter) getCurrentLimit(tenantID string, fixedLimitSupplier supplier[int]) (calculatedLimit, localLimit, globalLimit, adjustedGlobalLimit int) { + calculatedLimit, localLimit, globalLimit, adjustedGlobalLimit = l.limiter.GetStreamCountLimit(tenantID) + fixedLimit := fixedLimitSupplier() + if fixedLimit > calculatedLimit { + calculatedLimit = fixedLimit + } + return +} + +func (l *streamCountLimiter) getSuppliers(tenant string) (streamCountSupplier, fixedLimitSupplier supplier[int]) { + if l.limiter.limits.UseOwnedStreamCount(tenant) { + return l.ownedStreamSvc.getOwnedStreamCount, l.ownedStreamSvc.getFixedLimit + } + return l.defaultStreamCountSupplier, noopFixedLimitSupplier +} + +type RateLimiterStrategy interface { + RateLimit(tenant string) validation.RateLimit +} + +func (l *Limiter) RateLimit(tenant string) validation.RateLimit { + if l.disabled { + return validation.Unlimited + } + + return l.limits.PerStreamRateLimit(tenant) +} + +type StreamRateLimiter struct { + recheckPeriod time.Duration + recheckAt time.Time + strategy RateLimiterStrategy + tenant string + lim *rate.Limiter +} + +func NewStreamRateLimiter(strategy RateLimiterStrategy, tenant string, recheckPeriod time.Duration) *StreamRateLimiter { + rl := strategy.RateLimit(tenant) + return &StreamRateLimiter{ + recheckPeriod: recheckPeriod, + strategy: strategy, + tenant: tenant, + lim: rate.NewLimiter(rl.Limit, rl.Burst), + } +} + +func (l *StreamRateLimiter) AllowN(at time.Time, n int) bool { + now := time.Now() + if now.After(l.recheckAt) { + l.recheckAt = now.Add(l.recheckPeriod) + + oldLim := l.lim.Limit() + oldBurst := l.lim.Burst() + + next := l.strategy.RateLimit(l.tenant) + + if oldLim != next.Limit || oldBurst != next.Burst { + // Edge case: rate.Inf doesn't advance nicely when reconfigured. + // To simplify, we just create a new limiter after reconfiguration rather + // than alter the existing one. + l.lim = rate.NewLimiter(next.Limit, next.Burst) + } + } + + return l.lim.AllowN(at, n) +} diff --git a/pkg/ingester-rf1/mapper.go b/pkg/ingester-rf1/mapper.go new file mode 100644 index 0000000000000..02d7a4c753e98 --- /dev/null +++ b/pkg/ingester-rf1/mapper.go @@ -0,0 +1,152 @@ +package ingesterrf1 + +import ( + "fmt" + "sort" + "strings" + "sync" + + "github.com/go-kit/log/level" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" + "go.uber.org/atomic" + + util_log "github.com/grafana/loki/v3/pkg/util/log" +) + +const maxMappedFP = 1 << 20 // About 1M fingerprints reserved for mapping. + +var separatorString = string([]byte{model.SeparatorByte}) + +// FpMapper is used to map fingerprints in order to work around fingerprint +// collisions. +type FpMapper struct { + // highestMappedFP has to be aligned for atomic operations. + highestMappedFP atomic.Uint64 + + mtx sync.RWMutex // Protects mappings. + // maps original fingerprints to a map of string representations of + // metrics to the truly unique fingerprint. + mappings map[model.Fingerprint]map[string]model.Fingerprint + + // Returns existing labels for given fingerprint, if any. + // Equality check relies on labels.Labels being sorted. + fpToLabels func(fingerprint model.Fingerprint) labels.Labels +} + +// NewFPMapper returns an fpMapper ready to use. +func NewFPMapper(fpToLabels func(fingerprint model.Fingerprint) labels.Labels) *FpMapper { + if fpToLabels == nil { + panic("nil fpToLabels") + } + + return &FpMapper{ + fpToLabels: fpToLabels, + mappings: map[model.Fingerprint]map[string]model.Fingerprint{}, + } +} + +// MapFP takes a raw fingerprint (as returned by Metrics.FastFingerprint) and +// returns a truly unique fingerprint. The caller must have locked the raw +// fingerprint. +func (m *FpMapper) MapFP(fp model.Fingerprint, metric labels.Labels) model.Fingerprint { + // First check if we are in the reserved FP space, in which case this is + // automatically a collision that has to be mapped. + if fp <= maxMappedFP { + return m.maybeAddMapping(fp, metric) + } + + // Then check the most likely case: This fp belongs to a series that is + // already in memory. + s := m.fpToLabels(fp) + if s != nil { + // FP exists in memory, but is it for the same metric? + if labels.Equal(metric, s) { + // Yupp. We are done. + return fp + } + // Collision detected! + return m.maybeAddMapping(fp, metric) + } + // Metric is not in memory. Before doing the expensive archive lookup, + // check if we have a mapping for this metric in place already. + m.mtx.RLock() + mappedFPs, fpAlreadyMapped := m.mappings[fp] + m.mtx.RUnlock() + if fpAlreadyMapped { + // We indeed have mapped fp historically. + ms := metricToUniqueString(metric) + // fp is locked by the caller, so no further locking of + // 'collisions' required (it is specific to fp). + mappedFP, ok := mappedFPs[ms] + if ok { + // Historical mapping found, return the mapped FP. + return mappedFP + } + } + return fp +} + +// maybeAddMapping is only used internally. It takes a detected collision and +// adds it to the collisions map if not yet there. In any case, it returns the +// truly unique fingerprint for the colliding metric. +func (m *FpMapper) maybeAddMapping(fp model.Fingerprint, collidingMetric labels.Labels) model.Fingerprint { + ms := metricToUniqueString(collidingMetric) + m.mtx.RLock() + mappedFPs, ok := m.mappings[fp] + m.mtx.RUnlock() + if ok { + // fp is locked by the caller, so no further locking required. + mappedFP, ok := mappedFPs[ms] + if ok { + return mappedFP // Existing mapping. + } + // A new mapping has to be created. + mappedFP = m.nextMappedFP() + mappedFPs[ms] = mappedFP + level.Info(util_log.Logger).Log( + "msg", "fingerprint collision detected, mapping to new fingerprint", + "old_fp", fp, + "new_fp", mappedFP, + "metric", ms, + ) + return mappedFP + } + // This is the first collision for fp. + mappedFP := m.nextMappedFP() + mappedFPs = map[string]model.Fingerprint{ms: mappedFP} + m.mtx.Lock() + m.mappings[fp] = mappedFPs + m.mtx.Unlock() + level.Info(util_log.Logger).Log( + "msg", "fingerprint collision detected, mapping to new fingerprint", + "old_fp", fp, + "new_fp", mappedFP, + "metric", collidingMetric, + ) + return mappedFP +} + +func (m *FpMapper) nextMappedFP() model.Fingerprint { + mappedFP := model.Fingerprint(m.highestMappedFP.Inc()) + if mappedFP > maxMappedFP { + panic(fmt.Errorf("more than %v fingerprints mapped in collision detection", maxMappedFP)) + } + return mappedFP +} + +// metricToUniqueString turns a metric into a string in a reproducible and +// unique way, i.e. the same metric will always create the same string, and +// different metrics will always create different strings. In a way, it is the +// "ideal" fingerprint function, only that it is more expensive than the +// FastFingerprint function, and its result is not suitable as a key for maps +// and indexes as it might become really large, causing a lot of hashing effort +// in maps and a lot of storage overhead in indexes. +func metricToUniqueString(m labels.Labels) string { + parts := make([]string, 0, len(m)) + for _, pair := range m { + parts = append(parts, pair.Name+separatorString+pair.Value) + } + sort.Strings(parts) + return strings.Join(parts, separatorString) +} diff --git a/pkg/ingester-rf1/metrics.go b/pkg/ingester-rf1/metrics.go new file mode 100644 index 0000000000000..93291a25ce9ec --- /dev/null +++ b/pkg/ingester-rf1/metrics.go @@ -0,0 +1,297 @@ +package ingesterrf1 + +import ( + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + + "github.com/grafana/loki/v3/pkg/analytics" + "github.com/grafana/loki/v3/pkg/util/constants" + "github.com/grafana/loki/v3/pkg/validation" +) + +type ingesterMetrics struct { + checkpointDeleteFail prometheus.Counter + checkpointDeleteTotal prometheus.Counter + checkpointCreationFail prometheus.Counter + checkpointCreationTotal prometheus.Counter + checkpointDuration prometheus.Summary + checkpointLoggedBytesTotal prometheus.Counter + + walDiskFullFailures prometheus.Counter + walReplayActive prometheus.Gauge + walReplayDuration prometheus.Gauge + walReplaySamplesDropped *prometheus.CounterVec + walReplayBytesDropped *prometheus.CounterVec + walCorruptionsTotal *prometheus.CounterVec + walLoggedBytesTotal prometheus.Counter + walRecordsLogged prometheus.Counter + + recoveredStreamsTotal prometheus.Counter + recoveredChunksTotal prometheus.Counter + recoveredEntriesTotal prometheus.Counter + duplicateEntriesTotal prometheus.Counter + recoveredBytesTotal prometheus.Counter + recoveryBytesInUse prometheus.Gauge + recoveryIsFlushing prometheus.Gauge + + limiterEnabled prometheus.Gauge + + autoForgetUnhealthyIngestersTotal prometheus.Counter + + chunkUtilization prometheus.Histogram + memoryChunks prometheus.Gauge + chunkEntries prometheus.Histogram + chunkSize prometheus.Histogram + chunkCompressionRatio prometheus.Histogram + chunksPerTenant *prometheus.CounterVec + chunkSizePerTenant *prometheus.CounterVec + chunkAge prometheus.Histogram + chunkEncodeTime prometheus.Histogram + chunksFlushFailures prometheus.Counter + chunksFlushedPerReason *prometheus.CounterVec + chunkLifespan prometheus.Histogram + flushedChunksStats *analytics.Counter + flushedChunksBytesStats *analytics.Statistics + flushedChunksLinesStats *analytics.Statistics + flushedChunksAgeStats *analytics.Statistics + flushedChunksLifespanStats *analytics.Statistics + flushedChunksUtilizationStats *analytics.Statistics + + chunksCreatedTotal prometheus.Counter + samplesPerChunk prometheus.Histogram + blocksPerChunk prometheus.Histogram + chunkCreatedStats *analytics.Counter + + // Shutdown marker for ingester scale down + shutdownMarker prometheus.Gauge + + flushQueueLength prometheus.Gauge +} + +// setRecoveryBytesInUse bounds the bytes reports to >= 0. +// TODO(owen-d): we can gain some efficiency by having the flusher never update this after recovery ends. +func (m *ingesterMetrics) setRecoveryBytesInUse(v int64) { + if v < 0 { + v = 0 + } + m.recoveryBytesInUse.Set(float64(v)) +} + +const ( + walTypeCheckpoint = "checkpoint" + walTypeSegment = "segment" + + duplicateReason = "duplicate" +) + +func newIngesterMetrics(r prometheus.Registerer, metricsNamespace string) *ingesterMetrics { + return &ingesterMetrics{ + walDiskFullFailures: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Name: "loki_ingester_rf1_wal_disk_full_failures_total", + Help: "Total number of wal write failures due to full disk.", + }), + walReplayActive: promauto.With(r).NewGauge(prometheus.GaugeOpts{ + Name: "loki_ingester_rf1_wal_replay_active", + Help: "Whether the WAL is replaying", + }), + walReplayDuration: promauto.With(r).NewGauge(prometheus.GaugeOpts{ + Name: "loki_ingester_rf1_wal_replay_duration_seconds", + Help: "Time taken to replay the checkpoint and the WAL.", + }), + walReplaySamplesDropped: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Name: "loki_ingester_rf1_wal_discarded_samples_total", + Help: "WAL segment entries discarded during replay", + }, []string{validation.ReasonLabel}), + walReplayBytesDropped: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Name: "loki_ingester_rf1_wal_discarded_bytes_total", + Help: "WAL segment bytes discarded during replay", + }, []string{validation.ReasonLabel}), + walCorruptionsTotal: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Name: "loki_ingester_rf1_wal_corruptions_total", + Help: "Total number of WAL corruptions encountered.", + }, []string{"type"}), + checkpointDeleteFail: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Name: "loki_ingester_rf1_checkpoint_deletions_failed_total", + Help: "Total number of checkpoint deletions that failed.", + }), + checkpointDeleteTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Name: "loki_ingester_rf1_checkpoint_deletions_total", + Help: "Total number of checkpoint deletions attempted.", + }), + checkpointCreationFail: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Name: "loki_ingester_rf1_checkpoint_creations_failed_total", + Help: "Total number of checkpoint creations that failed.", + }), + checkpointCreationTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Name: "loki_ingester_rf1_checkpoint_creations_total", + Help: "Total number of checkpoint creations attempted.", + }), + checkpointDuration: promauto.With(r).NewSummary(prometheus.SummaryOpts{ + Name: "loki_ingester_rf1_checkpoint_duration_seconds", + Help: "Time taken to create a checkpoint.", + Objectives: map[float64]float64{0.5: 0.05, 0.9: 0.01, 0.99: 0.001}, + }), + walRecordsLogged: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Name: "loki_ingester_rf1_wal_records_logged_total", + Help: "Total number of WAL records logged.", + }), + checkpointLoggedBytesTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Name: "loki_ingester_rf1_checkpoint_logged_bytes_total", + Help: "Total number of bytes written to disk for checkpointing.", + }), + walLoggedBytesTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Name: "loki_ingester_rf1_wal_logged_bytes_total", + Help: "Total number of bytes written to disk for WAL records.", + }), + recoveredStreamsTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Name: "loki_ingester_rf1_wal_recovered_streams_total", + Help: "Total number of streams recovered from the WAL.", + }), + recoveredChunksTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Name: "loki_ingester_rf1_wal_recovered_chunks_total", + Help: "Total number of chunks recovered from the WAL checkpoints.", + }), + recoveredEntriesTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Name: "loki_ingester_rf1_wal_recovered_entries_total", + Help: "Total number of entries recovered from the WAL.", + }), + duplicateEntriesTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Name: "loki_ingester_rf1_wal_duplicate_entries_total", + Help: "Entries discarded during WAL replay due to existing in checkpoints.", + }), + recoveredBytesTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Name: "loki_ingester_rf1_wal_recovered_bytes_total", + Help: "Total number of bytes recovered from the WAL.", + }), + recoveryBytesInUse: promauto.With(r).NewGauge(prometheus.GaugeOpts{ + Name: "loki_ingester_rf1_wal_bytes_in_use", + Help: "Total number of bytes in use by the WAL recovery process.", + }), + recoveryIsFlushing: promauto.With(r).NewGauge(prometheus.GaugeOpts{ + Name: "loki_ingester_rf1_wal_replay_flushing", + Help: "Whether the wal replay is in a flushing phase due to backpressure", + }), + limiterEnabled: promauto.With(r).NewGauge(prometheus.GaugeOpts{ + Name: "loki_ingester_rf1_limiter_enabled", + Help: "Whether the ingester's limiter is enabled", + }), + autoForgetUnhealthyIngestersTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Name: "loki_ingester_rf1_autoforget_unhealthy_ingesters_total", + Help: "Total number of ingesters automatically forgotten", + }), + chunkUtilization: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Name: "ingester_rf1_chunk_utilization", + Help: "Distribution of stored chunk utilization (when stored).", + Buckets: prometheus.LinearBuckets(0, 0.2, 6), + }), + memoryChunks: promauto.With(r).NewGauge(prometheus.GaugeOpts{ + Namespace: constants.Loki, + Name: "ingester_rf1_memory_chunks", + Help: "The total number of chunks in memory.", + }), + chunkEntries: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Name: "ingester_rf1_chunk_entries", + Help: "Distribution of stored lines per chunk (when stored).", + Buckets: prometheus.ExponentialBuckets(200, 2, 9), // biggest bucket is 200*2^(9-1) = 51200 + }), + chunkSize: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Name: "ingester_rf1_chunk_size_bytes", + Help: "Distribution of stored chunk sizes (when stored).", + Buckets: prometheus.ExponentialBuckets(20000, 2, 10), // biggest bucket is 20000*2^(10-1) = 10,240,000 (~10.2MB) + }), + chunkCompressionRatio: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Name: "ingester_rf1_chunk_compression_ratio", + Help: "Compression ratio of chunks (when stored).", + Buckets: prometheus.LinearBuckets(.75, 2, 10), + }), + chunksPerTenant: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Namespace: constants.Loki, + Name: "ingester_rf1_chunks_stored_total", + Help: "Total stored chunks per tenant.", + }, []string{"tenant"}), + chunkSizePerTenant: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Namespace: constants.Loki, + Name: "ingester_rf1_chunk_stored_bytes_total", + Help: "Total bytes stored in chunks per tenant.", + }, []string{"tenant"}), + chunkAge: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Name: "ingester_rf1_chunk_age_seconds", + Help: "Distribution of chunk ages (when stored).", + // with default settings chunks should flush between 5 min and 12 hours + // so buckets at 1min, 5min, 10min, 30min, 1hr, 2hr, 4hr, 10hr, 12hr, 16hr + Buckets: []float64{60, 300, 600, 1800, 3600, 7200, 14400, 36000, 43200, 57600}, + }), + chunkEncodeTime: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Name: "ingester_rf1_chunk_encode_time_seconds", + Help: "Distribution of chunk encode times.", + // 10ms to 10s. + Buckets: prometheus.ExponentialBuckets(0.01, 4, 6), + }), + chunksFlushFailures: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Namespace: constants.Loki, + Name: "ingester_rf1_chunks_flush_failures_total", + Help: "Total number of flush failures.", + }), + chunksFlushedPerReason: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Namespace: constants.Loki, + Name: "ingester_rf1_chunks_flushed_total", + Help: "Total flushed chunks per reason.", + }, []string{"reason"}), + chunkLifespan: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Name: "ingester_rf1_chunk_bounds_hours", + Help: "Distribution of chunk end-start durations.", + // 1h -> 8hr + Buckets: prometheus.LinearBuckets(1, 1, 8), + }), + flushedChunksStats: analytics.NewCounter("ingester_rf1_flushed_chunks"), + flushedChunksBytesStats: analytics.NewStatistics("ingester_rf1_flushed_chunks_bytes"), + flushedChunksLinesStats: analytics.NewStatistics("ingester_rf1_flushed_chunks_lines"), + flushedChunksAgeStats: analytics.NewStatistics("ingester_rf1_flushed_chunks_age_seconds"), + flushedChunksLifespanStats: analytics.NewStatistics("ingester_rf1_flushed_chunks_lifespan_seconds"), + flushedChunksUtilizationStats: analytics.NewStatistics("ingester_rf1_flushed_chunks_utilization"), + chunksCreatedTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Namespace: constants.Loki, + Name: "ingester_rf1_chunks_created_total", + Help: "The total number of chunks created in the ingester.", + }), + samplesPerChunk: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Subsystem: "ingester_rf1", + Name: "samples_per_chunk", + Help: "The number of samples in a chunk.", + + Buckets: prometheus.LinearBuckets(4096, 2048, 6), + }), + blocksPerChunk: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Subsystem: "ingester_rf1", + Name: "blocks_per_chunk", + Help: "The number of blocks in a chunk.", + + Buckets: prometheus.ExponentialBuckets(5, 2, 6), + }), + + chunkCreatedStats: analytics.NewCounter("ingester_chunk_created"), + + shutdownMarker: promauto.With(r).NewGauge(prometheus.GaugeOpts{ + Namespace: constants.Loki, + Subsystem: "ingester_rf1", + Name: "shutdown_marker", + Help: "1 if prepare shutdown has been called, 0 otherwise", + }), + + flushQueueLength: promauto.With(r).NewGauge(prometheus.GaugeOpts{ + Namespace: metricsNamespace, + Subsystem: "ingester_rf1", + Name: "flush_queue_length", + Help: "The total number of series pending in the flush queue.", + }), + } +} diff --git a/pkg/ingester-rf1/owned_streams.go b/pkg/ingester-rf1/owned_streams.go new file mode 100644 index 0000000000000..196d1265d1ba1 --- /dev/null +++ b/pkg/ingester-rf1/owned_streams.go @@ -0,0 +1,74 @@ +package ingesterrf1 + +import ( + "sync" + + "github.com/grafana/dskit/services" + "go.uber.org/atomic" +) + +type ownedStreamService struct { + services.Service + + tenantID string + limiter *Limiter + fixedLimit *atomic.Int32 + ownedStreamCount int + notOwnedStreamCount int + lock sync.RWMutex +} + +func newOwnedStreamService(tenantID string, limiter *Limiter) *ownedStreamService { + svc := &ownedStreamService{ + tenantID: tenantID, + limiter: limiter, + fixedLimit: atomic.NewInt32(0), + } + + svc.updateFixedLimit() + return svc +} + +func (s *ownedStreamService) getOwnedStreamCount() int { + s.lock.RLock() + defer s.lock.RUnlock() + return s.ownedStreamCount +} + +func (s *ownedStreamService) updateFixedLimit() { + limit, _, _, _ := s.limiter.GetStreamCountLimit(s.tenantID) + s.fixedLimit.Store(int32(limit)) +} + +func (s *ownedStreamService) getFixedLimit() int { + return int(s.fixedLimit.Load()) +} + +func (s *ownedStreamService) incOwnedStreamCount() { + s.lock.Lock() + defer s.lock.Unlock() + s.ownedStreamCount++ +} + +func (s *ownedStreamService) incNotOwnedStreamCount() { + s.lock.Lock() + defer s.lock.Unlock() + s.notOwnedStreamCount++ +} + +func (s *ownedStreamService) decOwnedStreamCount() { + s.lock.Lock() + defer s.lock.Unlock() + if s.notOwnedStreamCount > 0 { + s.notOwnedStreamCount-- + return + } + s.ownedStreamCount-- +} + +func (s *ownedStreamService) resetStreamCounts() { + s.lock.Lock() + defer s.lock.Unlock() + s.ownedStreamCount = 0 + s.notOwnedStreamCount = 0 +} diff --git a/pkg/ingester-rf1/ring_client.go b/pkg/ingester-rf1/ring_client.go new file mode 100644 index 0000000000000..534a7468fddf4 --- /dev/null +++ b/pkg/ingester-rf1/ring_client.go @@ -0,0 +1,77 @@ +package ingesterrf1 + +import ( + "context" + "fmt" + + "github.com/go-kit/log" + "github.com/grafana/dskit/ring" + ring_client "github.com/grafana/dskit/ring/client" + "github.com/grafana/dskit/services" + "github.com/prometheus/client_golang/prometheus" + + "github.com/grafana/loki/v3/pkg/ingester-rf1/clientpool" +) + +type RingClient struct { + cfg Config + logger log.Logger + + services.Service + subservices *services.Manager + subservicesWatcher *services.FailureWatcher + ring *ring.Ring + pool *ring_client.Pool +} + +func NewRingClient( + cfg Config, + metricsNamespace string, + registerer prometheus.Registerer, + logger log.Logger, +) (*RingClient, error) { + var err error + registerer = prometheus.WrapRegistererWithPrefix(metricsNamespace+"_", registerer) + ringClient := &RingClient{ + logger: log.With(logger, "component", "ingester-rf1-client"), + cfg: cfg, + } + ringClient.ring, err = ring.New(cfg.LifecyclerConfig.RingConfig, "ingester-rf1", "ingester-rf1-ring", ringClient.logger, registerer) + if err != nil { + return nil, err + } + factory := cfg.factory + if factory == nil { + factory = ring_client.PoolAddrFunc(func(addr string) (ring_client.PoolClient, error) { + return clientpool.NewClient(cfg.ClientConfig, addr) + }) + } + ringClient.pool = clientpool.NewPool("ingester-rf1", cfg.ClientConfig.PoolConfig, ringClient.ring, factory, logger, metricsNamespace) + + ringClient.subservices, err = services.NewManager(ringClient.pool, ringClient.ring) + if err != nil { + return nil, fmt.Errorf("services manager: %w", err) + } + ringClient.subservicesWatcher = services.NewFailureWatcher() + ringClient.subservicesWatcher.WatchManager(ringClient.subservices) + ringClient.Service = services.NewBasicService(ringClient.starting, ringClient.running, ringClient.stopping) + + return ringClient, nil +} + +func (q *RingClient) starting(ctx context.Context) error { + return services.StartManagerAndAwaitHealthy(ctx, q.subservices) +} + +func (q *RingClient) running(ctx context.Context) error { + select { + case <-ctx.Done(): + return nil + case err := <-q.subservicesWatcher.Chan(): + return fmt.Errorf("ingester-rf1 tee subservices failed: %w", err) + } +} + +func (q *RingClient) stopping(_ error) error { + return services.StopManagerAndAwaitStopped(context.Background(), q.subservices) +} diff --git a/pkg/ingester-rf1/stream.go b/pkg/ingester-rf1/stream.go new file mode 100644 index 0000000000000..932c6244bbf20 --- /dev/null +++ b/pkg/ingester-rf1/stream.go @@ -0,0 +1,325 @@ +package ingesterrf1 + +import ( + "bytes" + "context" + "fmt" + "net/http" + "time" + + "github.com/grafana/dskit/httpgrpc" + "github.com/opentracing/opentracing-go" + "github.com/pkg/errors" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" + + "github.com/grafana/loki/v3/pkg/chunkenc" + "github.com/grafana/loki/v3/pkg/distributor/writefailures" + "github.com/grafana/loki/v3/pkg/loghttp/push" + "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/util/flagext" + "github.com/grafana/loki/v3/pkg/validation" +) + +var ErrEntriesExist = errors.New("duplicate push - entries already exist") + +type line struct { + ts time.Time + content string +} + +type stream struct { + limiter *StreamRateLimiter + cfg *Config + tenant string + // Newest chunk at chunks[n-1]. + // Not thread-safe; assume accesses to this are locked by caller. + fp model.Fingerprint // possibly remapped fingerprint, used in the streams map + + labels labels.Labels + labelsString string + labelHash uint64 + labelHashNoShard uint64 + + // most recently pushed line. This is used to prevent duplicate pushes. + // It also determines chunk synchronization when unordered writes are disabled. + lastLine line + + // keeps track of the highest timestamp accepted by the stream. + // This is used when unordered writes are enabled to cap the validity window + // of accepted writes and for chunk synchronization. + highestTs time.Time + + metrics *ingesterMetrics + + //tailers map[uint32]*tailer + //tailerMtx sync.RWMutex + + // entryCt is a counter which is incremented on each accepted entry. + // This allows us to discard WAL entries during replays which were + // already recovered via checkpoints. Historically out of order + // errors were used to detect this, but this counter has been + // introduced to facilitate removing the ordering constraint. + entryCt int64 + + unorderedWrites bool + //streamRateCalculator *StreamRateCalculator + + writeFailures *writefailures.Manager + + chunkFormat byte + chunkHeadBlockFormat chunkenc.HeadBlockFmt +} + +type chunkDesc struct { + chunk *chunkenc.MemChunk + closed bool + synced bool + flushed time.Time + reason string + + lastUpdated time.Time +} + +type entryWithError struct { + entry *logproto.Entry + e error +} + +func newStream( + chunkFormat byte, + headBlockFmt chunkenc.HeadBlockFmt, + cfg *Config, + limits RateLimiterStrategy, + tenant string, + fp model.Fingerprint, + labels labels.Labels, + unorderedWrites bool, + //streamRateCalculator *StreamRateCalculator, + metrics *ingesterMetrics, + writeFailures *writefailures.Manager, +) *stream { + //hashNoShard, _ := labels.HashWithoutLabels(make([]byte, 0, 1024), ShardLbName) + return &stream{ + limiter: NewStreamRateLimiter(limits, tenant, 10*time.Second), + cfg: cfg, + fp: fp, + labels: labels, + labelsString: labels.String(), + labelHash: labels.Hash(), + //labelHashNoShard: hashNoShard, + //tailers: map[uint32]*tailer{}, + metrics: metrics, + tenant: tenant, + //streamRateCalculator: streamRateCalculator, + + unorderedWrites: unorderedWrites, + writeFailures: writeFailures, + chunkFormat: chunkFormat, + chunkHeadBlockFormat: headBlockFmt, + } +} + +// consumeChunk manually adds a chunk to the stream that was received during +// ingester chunk transfer. +// Must hold chunkMtx +// DEPRECATED: chunk transfers are no longer suggested and remain for compatibility. +func (s *stream) consumeChunk(_ context.Context, _ *logproto.Chunk) error { + return nil +} + +func (s *stream) Push( + ctx context.Context, + entries []logproto.Entry, + // Whether nor not to ingest all at once or not. It is a per-tenant configuration. + rateLimitWholeStream bool, + + usageTracker push.UsageTracker, + flushCtx *flushCtx, +) (int, error) { + + toStore, invalid := s.validateEntries(ctx, entries, rateLimitWholeStream, usageTracker) + if rateLimitWholeStream && hasRateLimitErr(invalid) { + return 0, errorForFailedEntries(s, invalid, len(entries)) + } + + bytesAdded, _ := s.storeEntries(ctx, toStore, usageTracker, flushCtx) + + return bytesAdded, errorForFailedEntries(s, invalid, len(entries)) +} + +func errorForFailedEntries(s *stream, failedEntriesWithError []entryWithError, totalEntries int) error { + if len(failedEntriesWithError) == 0 { + return nil + } + + lastEntryWithErr := failedEntriesWithError[len(failedEntriesWithError)-1] + _, ok := lastEntryWithErr.e.(*validation.ErrStreamRateLimit) + outOfOrder := chunkenc.IsOutOfOrderErr(lastEntryWithErr.e) + if !outOfOrder && !ok { + return lastEntryWithErr.e + } + var statusCode int + if outOfOrder { + statusCode = http.StatusBadRequest + } + if ok { + statusCode = http.StatusTooManyRequests + } + // Return a http status 4xx request response with all failed entries. + buf := bytes.Buffer{} + streamName := s.labelsString + + limitedFailedEntries := failedEntriesWithError + if maxIgnore := s.cfg.MaxReturnedErrors; maxIgnore > 0 && len(limitedFailedEntries) > maxIgnore { + limitedFailedEntries = limitedFailedEntries[:maxIgnore] + } + + for _, entryWithError := range limitedFailedEntries { + fmt.Fprintf(&buf, + "entry with timestamp %s ignored, reason: '%s',\n", + entryWithError.entry.Timestamp.String(), entryWithError.e.Error()) + } + + fmt.Fprintf(&buf, "user '%s', total ignored: %d out of %d for stream: %s", s.tenant, len(failedEntriesWithError), totalEntries, streamName) + + return httpgrpc.Errorf(statusCode, buf.String()) +} + +func hasRateLimitErr(errs []entryWithError) bool { + if len(errs) == 0 { + return false + } + + lastErr := errs[len(errs)-1] + _, ok := lastErr.e.(*validation.ErrStreamRateLimit) + return ok +} + +func (s *stream) storeEntries(ctx context.Context, entries []logproto.Entry, usageTracker push.UsageTracker, flushCtx *flushCtx) (int, []*logproto.Entry) { + if sp := opentracing.SpanFromContext(ctx); sp != nil { + sp.LogKV("event", "stream started to store entries", "labels", s.labelsString) + defer sp.LogKV("event", "stream finished to store entries") + } + + var bytesAdded, outOfOrderSamples, outOfOrderBytes int + + storedEntries := make([]*logproto.Entry, 0, len(entries)) + for i := 0; i < len(entries); i++ { + s.entryCt++ + s.lastLine.ts = entries[i].Timestamp + s.lastLine.content = entries[i].Line + if s.highestTs.Before(entries[i].Timestamp) { + s.highestTs = entries[i].Timestamp + } + + bytesAdded += len(entries[i].Line) + storedEntries = append(storedEntries, &entries[i]) + } + flushCtx.segmentWriter.Append(s.tenant, s.labels.String(), s.labels, storedEntries) + s.reportMetrics(ctx, outOfOrderSamples, outOfOrderBytes, 0, 0, usageTracker) + return bytesAdded, storedEntries +} + +func (s *stream) validateEntries(ctx context.Context, entries []logproto.Entry, rateLimitWholeStream bool, usageTracker push.UsageTracker) ([]logproto.Entry, []entryWithError) { + + var ( + outOfOrderSamples, outOfOrderBytes int + rateLimitedSamples, rateLimitedBytes int + validBytes, totalBytes int + failedEntriesWithError []entryWithError + limit = s.limiter.lim.Limit() + lastLine = s.lastLine + highestTs = s.highestTs + toStore = make([]logproto.Entry, 0, len(entries)) + ) + + for i := range entries { + // If this entry matches our last appended line's timestamp and contents, + // ignore it. + // + // This check is done at the stream level so it persists across cut and + // flushed chunks. + // + // NOTE: it's still possible for duplicates to be appended if a stream is + // deleted from inactivity. + if entries[i].Timestamp.Equal(lastLine.ts) && entries[i].Line == lastLine.content { + continue + } + + lineBytes := len(entries[i].Line) + totalBytes += lineBytes + + now := time.Now() + if !rateLimitWholeStream && !s.limiter.AllowN(now, len(entries[i].Line)) { + failedEntriesWithError = append(failedEntriesWithError, entryWithError{&entries[i], &validation.ErrStreamRateLimit{RateLimit: flagext.ByteSize(limit), Labels: s.labelsString, Bytes: flagext.ByteSize(lineBytes)}}) + s.writeFailures.Log(s.tenant, failedEntriesWithError[len(failedEntriesWithError)-1].e) + rateLimitedSamples++ + rateLimitedBytes += lineBytes + continue + } + + // The validity window for unordered writes is the highest timestamp present minus 1/2 * max-chunk-age. + cutoff := highestTs.Add(-s.cfg.MaxChunkAge / 2) + if s.unorderedWrites && !highestTs.IsZero() && cutoff.After(entries[i].Timestamp) { + failedEntriesWithError = append(failedEntriesWithError, entryWithError{&entries[i], chunkenc.ErrTooFarBehind(entries[i].Timestamp, cutoff)}) + s.writeFailures.Log(s.tenant, fmt.Errorf("%w for stream %s", failedEntriesWithError[len(failedEntriesWithError)-1].e, s.labels)) + outOfOrderSamples++ + outOfOrderBytes += lineBytes + continue + } + + validBytes += lineBytes + + lastLine.ts = entries[i].Timestamp + lastLine.content = entries[i].Line + if highestTs.Before(entries[i].Timestamp) { + highestTs = entries[i].Timestamp + } + + toStore = append(toStore, entries[i]) + } + + // Each successful call to 'AllowN' advances the limiter. With all-or-nothing + // ingestion, the limiter should only be advanced when the whole stream can be + // sent + now := time.Now() + if rateLimitWholeStream && !s.limiter.AllowN(now, validBytes) { + // Report that the whole stream was rate limited + rateLimitedSamples = len(toStore) + failedEntriesWithError = make([]entryWithError, 0, len(toStore)) + for i := 0; i < len(toStore); i++ { + failedEntriesWithError = append(failedEntriesWithError, entryWithError{&toStore[i], &validation.ErrStreamRateLimit{RateLimit: flagext.ByteSize(limit), Labels: s.labelsString, Bytes: flagext.ByteSize(len(toStore[i].Line))}}) + rateLimitedBytes += len(toStore[i].Line) + } + } + + //s.streamRateCalculator.Record(s.tenant, s.labelHash, s.labelHashNoShard, totalBytes) + s.reportMetrics(ctx, outOfOrderSamples, outOfOrderBytes, rateLimitedSamples, rateLimitedBytes, usageTracker) + return toStore, failedEntriesWithError +} + +func (s *stream) reportMetrics(ctx context.Context, outOfOrderSamples, outOfOrderBytes, rateLimitedSamples, rateLimitedBytes int, usageTracker push.UsageTracker) { + if outOfOrderSamples > 0 { + name := validation.OutOfOrder + if s.unorderedWrites { + name = validation.TooFarBehind + } + validation.DiscardedSamples.WithLabelValues(name, s.tenant).Add(float64(outOfOrderSamples)) + validation.DiscardedBytes.WithLabelValues(name, s.tenant).Add(float64(outOfOrderBytes)) + if usageTracker != nil { + usageTracker.DiscardedBytesAdd(ctx, s.tenant, name, s.labels, float64(outOfOrderBytes)) + } + } + if rateLimitedSamples > 0 { + validation.DiscardedSamples.WithLabelValues(validation.StreamRateLimit, s.tenant).Add(float64(rateLimitedSamples)) + validation.DiscardedBytes.WithLabelValues(validation.StreamRateLimit, s.tenant).Add(float64(rateLimitedBytes)) + if usageTracker != nil { + usageTracker.DiscardedBytesAdd(ctx, s.tenant, validation.StreamRateLimit, s.labels, float64(rateLimitedBytes)) + } + } +} + +func (s *stream) resetCounter() { + s.entryCt = 0 +} diff --git a/pkg/ingester-rf1/stream_rate_calculator.go b/pkg/ingester-rf1/stream_rate_calculator.go new file mode 100644 index 0000000000000..59272a3fe80a0 --- /dev/null +++ b/pkg/ingester-rf1/stream_rate_calculator.go @@ -0,0 +1,131 @@ +package ingesterrf1 + +import ( + "sync" + "time" + + "github.com/grafana/loki/v3/pkg/logproto" +) + +const ( + // defaultStripeSize is the default number of entries to allocate in the + // stripeSeries list. + defaultStripeSize = 1 << 10 + + // The intent is for a per-second rate so this is hard coded + updateInterval = time.Second +) + +// stripeLock is taken from ruler/storage/wal/series.go +type stripeLock struct { + sync.RWMutex + // Padding to avoid multiple locks being on the same cache line. + _ [40]byte +} + +type StreamRateCalculator struct { + size int + samples []map[string]map[uint64]logproto.StreamRate + locks []stripeLock + stopchan chan struct{} + + rateLock sync.RWMutex + allRates []logproto.StreamRate +} + +func NewStreamRateCalculator() *StreamRateCalculator { + calc := &StreamRateCalculator{ + size: defaultStripeSize, + // Lookup pattern: tenant -> fingerprint -> rate + samples: make([]map[string]map[uint64]logproto.StreamRate, defaultStripeSize), + locks: make([]stripeLock, defaultStripeSize), + stopchan: make(chan struct{}), + } + + for i := 0; i < defaultStripeSize; i++ { + calc.samples[i] = make(map[string]map[uint64]logproto.StreamRate) + } + + go calc.updateLoop() + + return calc +} + +func (c *StreamRateCalculator) updateLoop() { + t := time.NewTicker(updateInterval) + defer t.Stop() + + for { + select { + case <-t.C: + c.updateRates() + case <-c.stopchan: + return + } + } +} + +func (c *StreamRateCalculator) updateRates() { + rates := make([]logproto.StreamRate, 0, c.size) + + for i := 0; i < c.size; i++ { + c.locks[i].Lock() + + tenantRates := c.samples[i] + for _, tenant := range tenantRates { + for _, streamRate := range tenant { + rates = append(rates, logproto.StreamRate{ + Tenant: streamRate.Tenant, + StreamHash: streamRate.StreamHash, + StreamHashNoShard: streamRate.StreamHashNoShard, + Rate: streamRate.Rate, + Pushes: streamRate.Pushes, + }) + } + } + + c.samples[i] = make(map[string]map[uint64]logproto.StreamRate) + c.locks[i].Unlock() + } + + c.rateLock.Lock() + defer c.rateLock.Unlock() + + c.allRates = rates +} + +func (c *StreamRateCalculator) Rates() []logproto.StreamRate { + c.rateLock.RLock() + defer c.rateLock.RUnlock() + + return c.allRates +} + +func (c *StreamRateCalculator) Record(tenant string, streamHash, streamHashNoShard uint64, bytes int) { + i := streamHash & uint64(c.size-1) + + c.locks[i].Lock() + defer c.locks[i].Unlock() + + tenantMap := c.getTenant(i, tenant) + streamRate := tenantMap[streamHash] + streamRate.StreamHash = streamHash + streamRate.StreamHashNoShard = streamHashNoShard + streamRate.Tenant = tenant + streamRate.Rate += int64(bytes) + streamRate.Pushes++ + tenantMap[streamHash] = streamRate + + c.samples[i][tenant] = tenantMap +} + +func (c *StreamRateCalculator) getTenant(idx uint64, tenant string) map[uint64]logproto.StreamRate { + if t, ok := c.samples[idx][tenant]; ok { + return t + } + return make(map[uint64]logproto.StreamRate) +} + +func (c *StreamRateCalculator) Stop() { + close(c.stopchan) +} diff --git a/pkg/ingester-rf1/streams_map.go b/pkg/ingester-rf1/streams_map.go new file mode 100644 index 0000000000000..ccf0f18a40389 --- /dev/null +++ b/pkg/ingester-rf1/streams_map.go @@ -0,0 +1,149 @@ +package ingesterrf1 + +import ( + "sync" + + "github.com/prometheus/common/model" + "go.uber.org/atomic" +) + +type streamsMap struct { + consistencyMtx sync.RWMutex // Keep read/write consistency between other fields + streams *sync.Map // map[string]*stream + streamsByFP *sync.Map // map[model.Fingerprint]*stream + streamsCounter *atomic.Int64 +} + +func newStreamsMap() *streamsMap { + return &streamsMap{ + consistencyMtx: sync.RWMutex{}, + streams: &sync.Map{}, + streamsByFP: &sync.Map{}, + streamsCounter: atomic.NewInt64(0), + } +} + +// Load is lock-free. If usage of the stream is consistency sensitive, must be called inside WithRLock at least +func (m *streamsMap) Load(key string) (*stream, bool) { + return m.load(m.streams, key) +} + +// LoadByFP is lock-free. If usage of the stream is consistency sensitive, must be called inside WithRLock at least +func (m *streamsMap) LoadByFP(fp model.Fingerprint) (*stream, bool) { + return m.load(m.streamsByFP, fp) +} + +// Store must be called inside WithLock +func (m *streamsMap) Store(key string, s *stream) { + m.store(key, s) +} + +// StoreByFP must be called inside WithLock +func (m *streamsMap) StoreByFP(fp model.Fingerprint, s *stream) { + m.store(fp, s) +} + +// Delete must be called inside WithLock +func (m *streamsMap) Delete(s *stream) bool { + _, loaded := m.streams.LoadAndDelete(s.labelsString) + if loaded { + m.streamsByFP.Delete(s.fp) + m.streamsCounter.Dec() + return true + } + return false +} + +// LoadOrStoreNew already has lock inside, do NOT call inside WithLock or WithRLock +func (m *streamsMap) LoadOrStoreNew(key string, newStreamFn func() (*stream, error), postLoadFn func(*stream) error) (*stream, bool, error) { + return m.loadOrStoreNew(m.streams, key, newStreamFn, postLoadFn) +} + +// LoadOrStoreNewByFP already has lock inside, do NOT call inside WithLock or WithRLock +func (m *streamsMap) LoadOrStoreNewByFP(fp model.Fingerprint, newStreamFn func() (*stream, error), postLoadFn func(*stream) error) (*stream, bool, error) { + return m.loadOrStoreNew(m.streamsByFP, fp, newStreamFn, postLoadFn) +} + +// WithLock is a helper function to execute write operations +func (m *streamsMap) WithLock(fn func()) { + m.consistencyMtx.Lock() + defer m.consistencyMtx.Unlock() + fn() +} + +// WithRLock is a helper function to execute consistency sensitive read operations. +// Generally, if a stream loaded from streamsMap will have its chunkMtx locked, chunkMtx.Lock is supposed to be called +// within this function. +func (m *streamsMap) WithRLock(fn func()) { + m.consistencyMtx.RLock() + defer m.consistencyMtx.RUnlock() + fn() +} + +func (m *streamsMap) ForEach(fn func(s *stream) (bool, error)) error { + var c bool + var err error + m.streams.Range(func(_, value interface{}) bool { + c, err = fn(value.(*stream)) + return c + }) + return err +} + +func (m *streamsMap) Len() int { + return int(m.streamsCounter.Load()) +} + +func (m *streamsMap) load(mp *sync.Map, key interface{}) (*stream, bool) { + if v, ok := mp.Load(key); ok { + return v.(*stream), true + } + return nil, false +} + +func (m *streamsMap) store(key interface{}, s *stream) { + if labelsString, ok := key.(string); ok { + m.streams.Store(labelsString, s) + } else { + m.streams.Store(s.labelsString, s) + } + m.streamsByFP.Store(s.fp, s) + m.streamsCounter.Inc() +} + +// newStreamFn: Called if not loaded, with consistencyMtx locked. Must not be nil +// postLoadFn: Called if loaded, with consistencyMtx read-locked at least. Can be nil +func (m *streamsMap) loadOrStoreNew(mp *sync.Map, key interface{}, newStreamFn func() (*stream, error), postLoadFn func(*stream) error) (*stream, bool, error) { + var s *stream + var loaded bool + var err error + m.WithRLock(func() { + if s, loaded = m.load(mp, key); loaded { + if postLoadFn != nil { + err = postLoadFn(s) + } + } + }) + + if loaded { + return s, true, err + } + + m.WithLock(func() { + // Double check + if s, loaded = m.load(mp, key); loaded { + if postLoadFn != nil { + err = postLoadFn(s) + } + return + } + + s, err = newStreamFn() + if err != nil { + return + } + m.store(key, s) + }) + + return s, loaded, err +} diff --git a/pkg/ingester-rf1/tee.go b/pkg/ingester-rf1/tee.go new file mode 100644 index 0000000000000..14d5aa87da3d0 --- /dev/null +++ b/pkg/ingester-rf1/tee.go @@ -0,0 +1,88 @@ +package ingesterrf1 + +import ( + "context" + "errors" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/grafana/dskit/ring" + "github.com/grafana/dskit/user" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + + "github.com/grafana/loki/v3/pkg/distributor" + "github.com/grafana/loki/v3/pkg/logproto" +) + +type Tee struct { + cfg Config + logger log.Logger + ringClient *RingClient + + ingesterAppends *prometheus.CounterVec +} + +func NewTee( + cfg Config, + ringClient *RingClient, + metricsNamespace string, + registerer prometheus.Registerer, + logger log.Logger, +) (*Tee, error) { + registerer = prometheus.WrapRegistererWithPrefix(metricsNamespace+"_", registerer) + + t := &Tee{ + logger: log.With(logger, "component", "ingester-rf1-tee"), + ingesterAppends: promauto.With(registerer).NewCounterVec(prometheus.CounterOpts{ + Name: "ingester_rf1_appends_total", + Help: "The total number of batch appends sent to rf1 ingesters.", + }, []string{"ingester", "status"}), + cfg: cfg, + ringClient: ringClient, + } + + return t, nil +} + +// Duplicate Implements distributor.Tee which is used to tee distributor requests to pattern ingesters. +func (t *Tee) Duplicate(tenant string, streams []distributor.KeyedStream) { + for idx := range streams { + go func(stream distributor.KeyedStream) { + if err := t.sendStream(tenant, stream); err != nil { + level.Error(t.logger).Log("msg", "failed to send stream to ingester-rf1", "err", err) + } + }(streams[idx]) + } +} + +func (t *Tee) sendStream(tenant string, stream distributor.KeyedStream) error { + var descs [1]ring.InstanceDesc + replicationSet, err := t.ringClient.ring.Get(stream.HashKey, ring.WriteNoExtend, descs[:0], nil, nil) + if err != nil { + return err + } + if replicationSet.Instances == nil { + return errors.New("no instances found") + } + addr := replicationSet.Instances[0].Addr + client, err := t.ringClient.pool.GetClientFor(addr) + if err != nil { + return err + } + req := &logproto.PushRequest{ + Streams: []logproto.Stream{ + stream.Stream, + }, + } + + ctx, cancel := context.WithTimeout(user.InjectOrgID(context.Background(), tenant), t.cfg.ClientConfig.RemoteTimeout) + defer cancel() + _, err = client.(logproto.PusherRF1Client).Push(ctx, req) + if err != nil { + t.ingesterAppends.WithLabelValues(addr, "fail").Inc() + return err + } + t.ingesterAppends.WithLabelValues(addr, "success").Inc() + return nil +} diff --git a/pkg/ingester/flush_test.go b/pkg/ingester/flush_test.go index 1287be3d4bfdb..e498faf324ae0 100644 --- a/pkg/ingester/flush_test.go +++ b/pkg/ingester/flush_test.go @@ -37,6 +37,7 @@ import ( "github.com/grafana/loki/v3/pkg/storage/config" "github.com/grafana/loki/v3/pkg/storage/stores/index/stats" "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" + walsegment "github.com/grafana/loki/v3/pkg/storage/wal" "github.com/grafana/loki/v3/pkg/util/constants" "github.com/grafana/loki/v3/pkg/validation" ) @@ -432,6 +433,10 @@ func defaultIngesterTestConfig(t testing.TB) Config { return cfg } +func (s *testStore) PutWal(_ context.Context, _ *walsegment.SegmentWriter) error { + return nil +} + func (s *testStore) Put(ctx context.Context, chunks []chunk.Chunk) error { s.mtx.Lock() defer s.mtx.Unlock() diff --git a/pkg/ingester/ingester_test.go b/pkg/ingester/ingester_test.go index 570452af44eb0..e55824df2a145 100644 --- a/pkg/ingester/ingester_test.go +++ b/pkg/ingester/ingester_test.go @@ -2,7 +2,7 @@ package ingester import ( "fmt" - math "math" + "math" "net" "net/http" "net/http/httptest" @@ -48,6 +48,7 @@ import ( "github.com/grafana/loki/v3/pkg/storage/stores/index/seriesvolume" "github.com/grafana/loki/v3/pkg/storage/stores/index/stats" "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" + "github.com/grafana/loki/v3/pkg/storage/wal" "github.com/grafana/loki/v3/pkg/util/constants" "github.com/grafana/loki/v3/pkg/validation" ) @@ -435,6 +436,10 @@ type mockStore struct { chunks map[string][]chunk.Chunk } +func (s *mockStore) PutWal(_ context.Context, _ *wal.SegmentWriter) error { + return nil +} + func (s *mockStore) Put(ctx context.Context, chunks []chunk.Chunk) error { s.mtx.Lock() defer s.mtx.Unlock() diff --git a/pkg/logproto/ingester-rf1.pb.go b/pkg/logproto/ingester-rf1.pb.go new file mode 100644 index 0000000000000..c9eb2db42f83f --- /dev/null +++ b/pkg/logproto/ingester-rf1.pb.go @@ -0,0 +1,130 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: pkg/logproto/ingester-rf1.proto + +package logproto + +import ( + context "context" + fmt "fmt" + _ "github.com/gogo/protobuf/gogoproto" + proto "github.com/gogo/protobuf/proto" + _ "github.com/gogo/protobuf/types" + push "github.com/grafana/loki/pkg/push" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" + math "math" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package + +func init() { proto.RegisterFile("pkg/logproto/ingester-rf1.proto", fileDescriptor_8ef2a56eb3f3c377) } + +var fileDescriptor_8ef2a56eb3f3c377 = []byte{ + // 250 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x2f, 0xc8, 0x4e, 0xd7, + 0xcf, 0xc9, 0x4f, 0x2f, 0x28, 0xca, 0x2f, 0xc9, 0xd7, 0xcf, 0xcc, 0x4b, 0x4f, 0x2d, 0x2e, 0x49, + 0x2d, 0xd2, 0x2d, 0x4a, 0x33, 0xd4, 0x03, 0x0b, 0x09, 0x71, 0xc0, 0x24, 0xa5, 0x44, 0xd2, 0xf3, + 0xd3, 0xf3, 0x21, 0xea, 0x40, 0x2c, 0x88, 0xbc, 0x94, 0x7c, 0x7a, 0x7e, 0x7e, 0x7a, 0x4e, 0xaa, + 0x3e, 0x98, 0x97, 0x54, 0x9a, 0xa6, 0x5f, 0x92, 0x99, 0x9b, 0x5a, 0x5c, 0x92, 0x98, 0x5b, 0x00, + 0x55, 0x20, 0x8d, 0x62, 0x03, 0x8c, 0x01, 0x95, 0x14, 0x06, 0x49, 0x16, 0x94, 0x16, 0x67, 0x80, + 0x09, 0x88, 0xa0, 0x91, 0x0b, 0x17, 0x67, 0x40, 0x69, 0x71, 0x46, 0x6a, 0x51, 0x90, 0x9b, 0xa1, + 0x90, 0x39, 0x17, 0x0b, 0x88, 0x23, 0x24, 0xaa, 0x07, 0xd7, 0x0a, 0xe2, 0x07, 0xa5, 0x16, 0x96, + 0xa6, 0x16, 0x97, 0x48, 0x89, 0xa1, 0x0b, 0x17, 0x17, 0xe4, 0xe7, 0x15, 0xa7, 0x2a, 0x31, 0x38, + 0xc5, 0x5e, 0x78, 0x28, 0xc7, 0x70, 0xe3, 0xa1, 0x1c, 0xc3, 0x87, 0x87, 0x72, 0x8c, 0x0d, 0x8f, + 0xe4, 0x18, 0x57, 0x3c, 0x92, 0x63, 0x3c, 0xf1, 0x48, 0x8e, 0xf1, 0xc2, 0x23, 0x39, 0xc6, 0x07, + 0x8f, 0xe4, 0x18, 0x5f, 0x3c, 0x92, 0x63, 0xf8, 0xf0, 0x48, 0x8e, 0x71, 0xc2, 0x63, 0x39, 0x86, + 0x0b, 0x8f, 0xe5, 0x18, 0x6e, 0x3c, 0x96, 0x63, 0x88, 0x52, 0x4f, 0xcf, 0x2c, 0xc9, 0x28, 0x4d, + 0xd2, 0x4b, 0xce, 0xcf, 0xd5, 0x4f, 0x2f, 0x4a, 0x4c, 0x4b, 0xcc, 0x4b, 0xd4, 0xcf, 0xc9, 0xcf, + 0xce, 0xd4, 0x2f, 0x33, 0xd6, 0x47, 0xf6, 0x48, 0x12, 0x1b, 0x98, 0x32, 0x06, 0x04, 0x00, 0x00, + 0xff, 0xff, 0x29, 0x6e, 0xb8, 0x46, 0x41, 0x01, 0x00, 0x00, +} + +// Reference imports to suppress errors if they are not otherwise used. +var _ context.Context +var _ grpc.ClientConn + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +const _ = grpc.SupportPackageIsVersion4 + +// PusherRF1Client is the client API for PusherRF1 service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. +type PusherRF1Client interface { + Push(ctx context.Context, in *push.PushRequest, opts ...grpc.CallOption) (*push.PushResponse, error) +} + +type pusherRF1Client struct { + cc *grpc.ClientConn +} + +func NewPusherRF1Client(cc *grpc.ClientConn) PusherRF1Client { + return &pusherRF1Client{cc} +} + +func (c *pusherRF1Client) Push(ctx context.Context, in *push.PushRequest, opts ...grpc.CallOption) (*push.PushResponse, error) { + out := new(push.PushResponse) + err := c.cc.Invoke(ctx, "/logproto.PusherRF1/Push", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// PusherRF1Server is the server API for PusherRF1 service. +type PusherRF1Server interface { + Push(context.Context, *push.PushRequest) (*push.PushResponse, error) +} + +// UnimplementedPusherRF1Server can be embedded to have forward compatible implementations. +type UnimplementedPusherRF1Server struct { +} + +func (*UnimplementedPusherRF1Server) Push(ctx context.Context, req *push.PushRequest) (*push.PushResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method Push not implemented") +} + +func RegisterPusherRF1Server(s *grpc.Server, srv PusherRF1Server) { + s.RegisterService(&_PusherRF1_serviceDesc, srv) +} + +func _PusherRF1_Push_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(push.PushRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(PusherRF1Server).Push(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/logproto.PusherRF1/Push", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(PusherRF1Server).Push(ctx, req.(*push.PushRequest)) + } + return interceptor(ctx, in, info, handler) +} + +var _PusherRF1_serviceDesc = grpc.ServiceDesc{ + ServiceName: "logproto.PusherRF1", + HandlerType: (*PusherRF1Server)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "Push", + Handler: _PusherRF1_Push_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "pkg/logproto/ingester-rf1.proto", +} diff --git a/pkg/logproto/ingester-rf1.proto b/pkg/logproto/ingester-rf1.proto new file mode 100644 index 0000000000000..374d659175bcf --- /dev/null +++ b/pkg/logproto/ingester-rf1.proto @@ -0,0 +1,14 @@ +syntax = "proto3"; + +package logproto; + +import "gogoproto/gogo.proto"; +import "google/protobuf/timestamp.proto"; +import "pkg/logproto/logproto.proto"; +import "pkg/push/push.proto"; + +option go_package = "github.com/grafana/loki/v3/pkg/logproto"; + +service PusherRF1 { + rpc Push(PushRequest) returns (PushResponse) {} +} diff --git a/pkg/loki/config_wrapper.go b/pkg/loki/config_wrapper.go index 4c8da5de23aea..48deb5151bb57 100644 --- a/pkg/loki/config_wrapper.go +++ b/pkg/loki/config_wrapper.go @@ -246,6 +246,21 @@ func applyConfigToRings(r, defaults *ConfigWrapper, rc lokiring.RingConfig, merg r.Ingester.LifecyclerConfig.ObservePeriod = rc.ObservePeriod } + if mergeWithExisting { + r.IngesterRF1.LifecyclerConfig.RingConfig.KVStore = rc.KVStore + r.IngesterRF1.LifecyclerConfig.HeartbeatPeriod = rc.HeartbeatPeriod + r.IngesterRF1.LifecyclerConfig.RingConfig.HeartbeatTimeout = rc.HeartbeatTimeout + r.IngesterRF1.LifecyclerConfig.TokensFilePath = rc.TokensFilePath + r.IngesterRF1.LifecyclerConfig.RingConfig.ZoneAwarenessEnabled = rc.ZoneAwarenessEnabled + r.IngesterRF1.LifecyclerConfig.ID = rc.InstanceID + r.IngesterRF1.LifecyclerConfig.InfNames = rc.InstanceInterfaceNames + r.IngesterRF1.LifecyclerConfig.Port = rc.InstancePort + r.IngesterRF1.LifecyclerConfig.Addr = rc.InstanceAddr + r.IngesterRF1.LifecyclerConfig.Zone = rc.InstanceZone + r.IngesterRF1.LifecyclerConfig.ListenPort = rc.ListenPort + r.IngesterRF1.LifecyclerConfig.ObservePeriod = rc.ObservePeriod + } + if mergeWithExisting { r.Pattern.LifecyclerConfig.RingConfig.KVStore = rc.KVStore r.Pattern.LifecyclerConfig.HeartbeatPeriod = rc.HeartbeatPeriod @@ -669,6 +684,7 @@ func applyIngesterFinalSleep(cfg *ConfigWrapper) { func applyIngesterReplicationFactor(cfg *ConfigWrapper) { cfg.Ingester.LifecyclerConfig.RingConfig.ReplicationFactor = cfg.Common.ReplicationFactor + cfg.IngesterRF1.LifecyclerConfig.RingConfig.ReplicationFactor = cfg.Common.ReplicationFactor } // applyChunkRetain is used to set chunk retain based on having an index query cache configured diff --git a/pkg/loki/loki.go b/pkg/loki/loki.go index 0b2f2a3c91058..e3a856cac83b9 100644 --- a/pkg/loki/loki.go +++ b/pkg/loki/loki.go @@ -11,6 +11,8 @@ import ( rt "runtime" "time" + ingester_rf1 "github.com/grafana/loki/v3/pkg/ingester-rf1" + "go.uber.org/atomic" "github.com/fatih/color" @@ -87,7 +89,9 @@ type Config struct { QueryRange queryrange.Config `yaml:"query_range,omitempty"` Ruler ruler.Config `yaml:"ruler,omitempty"` IngesterClient ingester_client.Config `yaml:"ingester_client,omitempty"` + IngesterRF1Client ingester_client.Config `yaml:"ingester_rf1_client,omitempty"` Ingester ingester.Config `yaml:"ingester,omitempty"` + IngesterRF1 ingester_rf1.Config `yaml:"ingester_rf1,omitempty"` Pattern pattern.Config `yaml:"pattern_ingester,omitempty"` IndexGateway indexgateway.Config `yaml:"index_gateway"` BloomCompactor bloomcompactor.Config `yaml:"bloom_compactor,omitempty" category:"experimental"` @@ -159,7 +163,9 @@ func (c *Config) RegisterFlags(f *flag.FlagSet) { c.CompactorHTTPClient.RegisterFlags(f) c.CompactorGRPCClient.RegisterFlags(f) c.IngesterClient.RegisterFlags(f) + //c.IngesterRF1Client.RegisterFlags(f) c.Ingester.RegisterFlags(f) + c.IngesterRF1.RegisterFlags(f) c.StorageConfig.RegisterFlags(f) c.IndexGateway.RegisterFlags(f) c.BloomGateway.RegisterFlags(f) @@ -332,6 +338,8 @@ type Loki struct { TenantLimits validation.TenantLimits distributor *distributor.Distributor Ingester ingester.Interface + IngesterRF1 ingester_rf1.Interface + IngesterRF1RingClient *ingester_rf1.RingClient PatternIngester *pattern.Ingester PatternRingClient pattern.RingClient Querier querier.Querier @@ -610,6 +618,15 @@ func (t *Loki) readyHandler(sm *services.Manager, shutdownRequested *atomic.Bool } } + // Ingester RF1 has a special check that makes sure that it was able to register into the ring, + // and that all other ring entries are OK too. + if t.IngesterRF1 != nil { + if err := t.IngesterRF1.CheckReady(r.Context()); err != nil { + http.Error(w, "Pattern Ingester not ready: "+err.Error(), http.StatusServiceUnavailable) + return + } + } + // Query Frontend has a special check that makes sure that a querier is attached before it signals // itself as ready if t.frontend != nil { @@ -632,6 +649,10 @@ func (t *Loki) setupModuleManager() error { mm.RegisterModule(InternalServer, t.initInternalServer, modules.UserInvisibleModule) } + /// >---- ./loki -target=ingester + /// Distributor -> gRPC ---> + /// >---- ./loki -target=sexy-ingester + mm.RegisterModule(RuntimeConfig, t.initRuntimeConfig, modules.UserInvisibleModule) mm.RegisterModule(MemberlistKV, t.initMemberlistKV, modules.UserInvisibleModule) mm.RegisterModule(Ring, t.initRing, modules.UserInvisibleModule) @@ -642,6 +663,8 @@ func (t *Loki) setupModuleManager() error { mm.RegisterModule(Store, t.initStore, modules.UserInvisibleModule) mm.RegisterModule(Querier, t.initQuerier) mm.RegisterModule(Ingester, t.initIngester) + mm.RegisterModule(IngesterRF1, t.initIngesterRF1) + mm.RegisterModule(IngesterRF1RingClient, t.initIngesterRF1RingClient, modules.UserInvisibleModule) mm.RegisterModule(IngesterQuerier, t.initIngesterQuerier) mm.RegisterModule(IngesterGRPCInterceptors, t.initIngesterGRPCInterceptors, modules.UserInvisibleModule) mm.RegisterModule(QueryFrontendTripperware, t.initQueryFrontendMiddleware, modules.UserInvisibleModule) @@ -679,8 +702,9 @@ func (t *Loki) setupModuleManager() error { Overrides: {RuntimeConfig}, OverridesExporter: {Overrides, Server}, TenantConfigs: {RuntimeConfig}, - Distributor: {Ring, Server, Overrides, TenantConfigs, PatternRingClient, Analytics}, + Distributor: {Ring, Server, Overrides, TenantConfigs, PatternRingClient, IngesterRF1RingClient, Analytics}, Store: {Overrides, IndexGatewayRing}, + IngesterRF1: {Store, Server, MemberlistKV, TenantConfigs, Analytics}, Ingester: {Store, Server, MemberlistKV, TenantConfigs, Analytics}, Querier: {Store, Ring, Server, IngesterQuerier, PatternRingClient, Overrides, Analytics, CacheGenerationLoader, QuerySchedulerRing}, QueryFrontendTripperware: {Server, Overrides, TenantConfigs}, @@ -697,6 +721,7 @@ func (t *Loki) setupModuleManager() error { BloomBuilder: {Server, BloomStore, Analytics, Store}, PatternIngester: {Server, MemberlistKV, Analytics}, PatternRingClient: {Server, MemberlistKV, Analytics}, + IngesterRF1RingClient: {Server, MemberlistKV, Analytics}, IngesterQuerier: {Ring}, QuerySchedulerRing: {Overrides, MemberlistKV}, IndexGatewayRing: {Overrides, MemberlistKV}, @@ -704,10 +729,10 @@ func (t *Loki) setupModuleManager() error { MemberlistKV: {Server}, Read: {QueryFrontend, Querier}, - Write: {Ingester, Distributor, PatternIngester}, + Write: {Ingester, IngesterRF1, Distributor, PatternIngester}, Backend: {QueryScheduler, Ruler, Compactor, IndexGateway, BloomGateway, BloomCompactor}, - All: {QueryScheduler, QueryFrontend, Querier, Ingester, PatternIngester, Distributor, Ruler, Compactor}, + All: {QueryScheduler, QueryFrontend, Querier, Ingester, IngesterRF1, PatternIngester, Distributor, Ruler, Compactor}, } if t.Cfg.Querier.PerRequestLimitsEnabled { diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 204cecd0ce3ad..bd80062a3cfbd 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -15,6 +15,8 @@ import ( "strings" "time" + ingester_rf1 "github.com/grafana/loki/v3/pkg/ingester-rf1" + "github.com/NYTimes/gziphandler" "github.com/go-kit/log" "github.com/go-kit/log/level" @@ -102,6 +104,8 @@ const ( Querier string = "querier" CacheGenerationLoader string = "cache-generation-loader" Ingester string = "ingester" + IngesterRF1 string = "ingester-rf1" + IngesterRF1RingClient string = "ingester-rf1-ring-client" PatternIngester string = "pattern-ingester" PatternRingClient string = "pattern-ring-client" IngesterQuerier string = "ingester-querier" @@ -328,6 +332,13 @@ func (t *Loki) initDistributor() (services.Service, error) { } t.Tee = distributor.WrapTee(t.Tee, patternTee) } + if t.Cfg.IngesterRF1.Enabled { + rf1Tee, err := ingester_rf1.NewTee(t.Cfg.IngesterRF1, t.IngesterRF1RingClient, t.Cfg.MetricsNamespace, prometheus.DefaultRegisterer, util_log.Logger) + if err != nil { + return nil, err + } + t.Tee = distributor.WrapTee(t.Tee, rf1Tee) + } var err error logger := log.With(util_log.Logger, "component", "distributor") @@ -618,6 +629,63 @@ func (t *Loki) initIngester() (_ services.Service, err error) { return t.Ingester, nil } +func (t *Loki) initIngesterRF1() (_ services.Service, err error) { + logger := log.With(util_log.Logger, "component", "ingester-rf1") + t.Cfg.IngesterRF1.LifecyclerConfig.ListenPort = t.Cfg.Server.GRPCListenPort + + if t.Cfg.IngesterRF1.ShutdownMarkerPath == "" && t.Cfg.Common.PathPrefix != "" { + t.Cfg.IngesterRF1.ShutdownMarkerPath = t.Cfg.Common.PathPrefix + } + if t.Cfg.IngesterRF1.ShutdownMarkerPath == "" { + level.Warn(util_log.Logger).Log("msg", "The config setting shutdown marker path is not set. The /ingester/prepare_shutdown endpoint won't work") + } + + t.IngesterRF1, err = ingester_rf1.New(t.Cfg.IngesterRF1, t.Cfg.IngesterRF1Client, t.Store, t.Overrides, t.tenantConfigs, prometheus.DefaultRegisterer, t.Cfg.Distributor.WriteFailuresLogging, t.Cfg.MetricsNamespace, logger, t.UsageTracker, t.ring) + if err != nil { + fmt.Println("Error initializing ingester rf1", err) + return + } + + if t.Cfg.IngesterRF1.Wrapper != nil { + t.IngesterRF1 = t.Cfg.IngesterRF1.Wrapper.Wrap(t.IngesterRF1) + } + + fmt.Println("registered GRPC") + logproto.RegisterPusherRF1Server(t.Server.GRPC, t.IngesterRF1) + + t.Server.HTTP.Path("/ingester-rf1/ring").Methods("GET", "POST").Handler(t.IngesterRF1) + + if t.Cfg.InternalServer.Enable { + t.InternalServer.HTTP.Path("/ingester-rf1/ring").Methods("GET", "POST").Handler(t.IngesterRF1) + } + + httpMiddleware := middleware.Merge( + serverutil.RecoveryHTTPMiddleware, + ) + t.Server.HTTP.Methods("GET", "POST").Path("/flush").Handler( + httpMiddleware.Wrap(http.HandlerFunc(t.IngesterRF1.FlushHandler)), + ) + t.Server.HTTP.Methods("POST", "GET", "DELETE").Path("/ingester-rf1/prepare_shutdown").Handler( + httpMiddleware.Wrap(http.HandlerFunc(t.IngesterRF1.PrepareShutdown)), + ) + t.Server.HTTP.Methods("POST", "GET").Path("/ingester-rf1/shutdown").Handler( + httpMiddleware.Wrap(http.HandlerFunc(t.IngesterRF1.ShutdownHandler)), + ) + return t.IngesterRF1, nil +} + +func (t *Loki) initIngesterRF1RingClient() (_ services.Service, err error) { + if !t.Cfg.IngesterRF1.Enabled { + return nil, nil + } + ringClient, err := ingester_rf1.NewRingClient(t.Cfg.IngesterRF1, t.Cfg.MetricsNamespace, prometheus.DefaultRegisterer, util_log.Logger) + if err != nil { + return nil, err + } + t.IngesterRF1RingClient = ringClient + return ringClient, nil +} + func (t *Loki) initPatternIngester() (_ services.Service, err error) { if !t.Cfg.Pattern.Enabled { return nil, nil diff --git a/pkg/push/push-rf1.pb.go b/pkg/push/push-rf1.pb.go new file mode 100644 index 0000000000000..7d87f4a1cc718 --- /dev/null +++ b/pkg/push/push-rf1.pb.go @@ -0,0 +1,128 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: pkg/push/push-rf1.proto + +package push + +import ( + context "context" + fmt "fmt" + _ "github.com/gogo/protobuf/gogoproto" + proto "github.com/gogo/protobuf/proto" + _ "github.com/gogo/protobuf/types" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" + math "math" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package + +func init() { proto.RegisterFile("pkg/push/push-rf1.proto", fileDescriptor_4b1742ccc5fd9087) } + +var fileDescriptor_4b1742ccc5fd9087 = []byte{ + // 232 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x12, 0x2f, 0xc8, 0x4e, 0xd7, + 0x2f, 0x28, 0x2d, 0xce, 0x00, 0x13, 0xba, 0x45, 0x69, 0x86, 0x7a, 0x05, 0x45, 0xf9, 0x25, 0xf9, + 0x42, 0x1c, 0x39, 0xf9, 0xe9, 0x60, 0x96, 0x94, 0x48, 0x7a, 0x7e, 0x7a, 0x3e, 0x98, 0xa9, 0x0f, + 0x62, 0x41, 0xe4, 0xa5, 0xe4, 0xd3, 0xf3, 0xf3, 0xd3, 0x73, 0x52, 0xf5, 0xc1, 0xbc, 0xa4, 0xd2, + 0x34, 0xfd, 0x92, 0xcc, 0xdc, 0xd4, 0xe2, 0x92, 0xc4, 0xdc, 0x02, 0xa8, 0x02, 0x61, 0x14, 0x93, + 0x21, 0x82, 0x46, 0x2e, 0x5c, 0x9c, 0x01, 0xa5, 0xc5, 0x19, 0xa9, 0x45, 0x41, 0x6e, 0x86, 0x42, + 0xe6, 0x5c, 0x2c, 0x20, 0x8e, 0x90, 0xa8, 0x1e, 0xcc, 0x2e, 0x3d, 0x10, 0x3f, 0x28, 0xb5, 0xb0, + 0x34, 0xb5, 0xb8, 0x44, 0x4a, 0x0c, 0x5d, 0xb8, 0xb8, 0x20, 0x3f, 0xaf, 0x38, 0x55, 0x89, 0xc1, + 0x29, 0xec, 0xc2, 0x43, 0x39, 0x86, 0x1b, 0x0f, 0xe5, 0x18, 0x3e, 0x3c, 0x94, 0x63, 0x6c, 0x78, + 0x24, 0xc7, 0xb8, 0xe2, 0x91, 0x1c, 0xe3, 0x89, 0x47, 0x72, 0x8c, 0x17, 0x1e, 0xc9, 0x31, 0x3e, + 0x78, 0x24, 0xc7, 0xf8, 0xe2, 0x91, 0x1c, 0xc3, 0x87, 0x47, 0x72, 0x8c, 0x13, 0x1e, 0xcb, 0x31, + 0x5c, 0x78, 0x2c, 0xc7, 0x70, 0xe3, 0xb1, 0x1c, 0x43, 0x94, 0x42, 0x7a, 0x66, 0x49, 0x46, 0x69, + 0x92, 0x5e, 0x72, 0x7e, 0xae, 0x7e, 0x7a, 0x51, 0x62, 0x5a, 0x62, 0x5e, 0xa2, 0x7e, 0x4e, 0x7e, + 0x76, 0xa6, 0x3e, 0xcc, 0xa1, 0x49, 0x6c, 0x60, 0xdb, 0x8c, 0x01, 0x01, 0x00, 0x00, 0xff, 0xff, + 0x48, 0x19, 0x4c, 0x81, 0x15, 0x01, 0x00, 0x00, +} + +// Reference imports to suppress errors if they are not otherwise used. +var _ context.Context +var _ grpc.ClientConn + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +const _ = grpc.SupportPackageIsVersion4 + +// PusherRF1Client is the client API for PusherRF1 service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. +type PusherRF1Client interface { + Push(ctx context.Context, in *PushRequest, opts ...grpc.CallOption) (*PushResponse, error) +} + +type pusherRF1Client struct { + cc *grpc.ClientConn +} + +func NewPusherRF1Client(cc *grpc.ClientConn) PusherRF1Client { + return &pusherRF1Client{cc} +} + +func (c *pusherRF1Client) Push(ctx context.Context, in *PushRequest, opts ...grpc.CallOption) (*PushResponse, error) { + out := new(PushResponse) + err := c.cc.Invoke(ctx, "/logproto.PusherRF1/Push", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// PusherRF1Server is the server API for PusherRF1 service. +type PusherRF1Server interface { + Push(context.Context, *PushRequest) (*PushResponse, error) +} + +// UnimplementedPusherRF1Server can be embedded to have forward compatible implementations. +type UnimplementedPusherRF1Server struct { +} + +func (*UnimplementedPusherRF1Server) Push(ctx context.Context, req *PushRequest) (*PushResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method Push not implemented") +} + +func RegisterPusherRF1Server(s *grpc.Server, srv PusherRF1Server) { + s.RegisterService(&_PusherRF1_serviceDesc, srv) +} + +func _PusherRF1_Push_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(PushRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(PusherRF1Server).Push(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/logproto.PusherRF1/Push", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(PusherRF1Server).Push(ctx, req.(*PushRequest)) + } + return interceptor(ctx, in, info, handler) +} + +var _PusherRF1_serviceDesc = grpc.ServiceDesc{ + ServiceName: "logproto.PusherRF1", + HandlerType: (*PusherRF1Server)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "Push", + Handler: _PusherRF1_Push_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "pkg/push/push-rf1.proto", +} diff --git a/pkg/push/push-rf1.proto b/pkg/push/push-rf1.proto new file mode 100644 index 0000000000000..1c5a3e039341d --- /dev/null +++ b/pkg/push/push-rf1.proto @@ -0,0 +1,13 @@ +syntax = "proto3"; + +package logproto; + +import "gogoproto/gogo.proto"; +import "google/protobuf/timestamp.proto"; +import "pkg/push/push.proto"; + +option go_package = "github.com/grafana/loki/pkg/push"; + +service PusherRF1 { + rpc Push(PushRequest) returns (PushResponse) {} +} diff --git a/pkg/querier/querier_mock_test.go b/pkg/querier/querier_mock_test.go index 20c3b9f1b77c2..60d26fee28d2a 100644 --- a/pkg/querier/querier_mock_test.go +++ b/pkg/querier/querier_mock_test.go @@ -8,6 +8,7 @@ import ( "time" "github.com/grafana/loki/v3/pkg/logql/log" + "github.com/grafana/loki/v3/pkg/storage/wal" "github.com/grafana/loki/v3/pkg/loghttp" @@ -339,6 +340,9 @@ func (s *storeMock) GetChunks(ctx context.Context, userID string, from, through return args.Get(0).([][]chunk.Chunk), args.Get(0).([]*fetcher.Fetcher), args.Error(2) } +func (s *storeMock) PutWal(_ context.Context, _ *wal.SegmentWriter) error { + return errors.New("storeMock.PutWal() has not been mocked") +} func (s *storeMock) Put(_ context.Context, _ []chunk.Chunk) error { return errors.New("storeMock.Put() has not been mocked") } diff --git a/pkg/storage/chunk/client/aws/dynamodb_storage_client.go b/pkg/storage/chunk/client/aws/dynamodb_storage_client.go index 87fd24e127db0..b70c4269ede23 100644 --- a/pkg/storage/chunk/client/aws/dynamodb_storage_client.go +++ b/pkg/storage/chunk/client/aws/dynamodb_storage_client.go @@ -33,6 +33,7 @@ import ( client_util "github.com/grafana/loki/v3/pkg/storage/chunk/client/util" "github.com/grafana/loki/v3/pkg/storage/config" "github.com/grafana/loki/v3/pkg/storage/stores/series/index" + "github.com/grafana/loki/v3/pkg/storage/wal" "github.com/grafana/loki/v3/pkg/util" "github.com/grafana/loki/v3/pkg/util/log" "github.com/grafana/loki/v3/pkg/util/math" @@ -118,6 +119,10 @@ type dynamoDBStorageClient struct { metrics *dynamoDBMetrics } +func (a dynamoDBStorageClient) PutWal(_ context.Context, _ *wal.SegmentWriter) error { + return errors.New("not implemented") +} + // NewDynamoDBIndexClient makes a new DynamoDB-backed IndexClient. func NewDynamoDBIndexClient(cfg DynamoDBConfig, schemaCfg config.SchemaConfig, reg prometheus.Registerer) (index.Client, error) { return newDynamoDBStorageClient(cfg, schemaCfg, reg) diff --git a/pkg/storage/chunk/client/cassandra/storage_client.go b/pkg/storage/chunk/client/cassandra/storage_client.go index d847f9d6b7e2d..732491de2df8a 100644 --- a/pkg/storage/chunk/client/cassandra/storage_client.go +++ b/pkg/storage/chunk/client/cassandra/storage_client.go @@ -23,6 +23,7 @@ import ( "github.com/grafana/loki/v3/pkg/storage/chunk/client/util" "github.com/grafana/loki/v3/pkg/storage/config" "github.com/grafana/loki/v3/pkg/storage/stores/series/index" + "github.com/grafana/loki/v3/pkg/storage/wal" util_log "github.com/grafana/loki/v3/pkg/util/log" ) @@ -567,6 +568,10 @@ func (s *ObjectClient) reconnectReadSession() error { return nil } +func (s *ObjectClient) PutWal(_ context.Context, _ *wal.SegmentWriter) error { + return errors.New("not implemented") +} + // PutChunks implements chunk.ObjectClient. func (s *ObjectClient) PutChunks(ctx context.Context, chunks []chunk.Chunk) error { err := s.putChunks(ctx, chunks) diff --git a/pkg/storage/chunk/client/client.go b/pkg/storage/chunk/client/client.go index 36b65d40b6c2e..800086c6616be 100644 --- a/pkg/storage/chunk/client/client.go +++ b/pkg/storage/chunk/client/client.go @@ -6,6 +6,7 @@ import ( "github.com/grafana/loki/v3/pkg/storage/chunk" "github.com/grafana/loki/v3/pkg/storage/stores/series/index" + "github.com/grafana/loki/v3/pkg/storage/wal" ) var ( @@ -18,6 +19,7 @@ var ( // Client is for storing and retrieving chunks. type Client interface { Stop() + PutWal(ctx context.Context, writer *wal.SegmentWriter) error PutChunks(ctx context.Context, chunks []chunk.Chunk) error GetChunks(ctx context.Context, chunks []chunk.Chunk) ([]chunk.Chunk, error) DeleteChunk(ctx context.Context, userID, chunkID string) error diff --git a/pkg/storage/chunk/client/gcp/bigtable_object_client.go b/pkg/storage/chunk/client/gcp/bigtable_object_client.go index d878bc19bccf0..f7f26ac78e376 100644 --- a/pkg/storage/chunk/client/gcp/bigtable_object_client.go +++ b/pkg/storage/chunk/client/gcp/bigtable_object_client.go @@ -12,6 +12,7 @@ import ( "github.com/grafana/loki/v3/pkg/storage/chunk" "github.com/grafana/loki/v3/pkg/storage/chunk/client" "github.com/grafana/loki/v3/pkg/storage/config" + "github.com/grafana/loki/v3/pkg/storage/wal" "github.com/grafana/loki/v3/pkg/util/math" ) @@ -83,6 +84,10 @@ func (s *bigtableObjectClient) PutChunks(ctx context.Context, chunks []chunk.Chu return nil } +func (s *bigtableObjectClient) PutWal(_ context.Context, _ *wal.SegmentWriter) error { + panic("not implemented") +} + func (s *bigtableObjectClient) GetChunks(ctx context.Context, input []chunk.Chunk) ([]chunk.Chunk, error) { sp, ctx := ot.StartSpanFromContext(ctx, "GetChunks") defer sp.Finish() diff --git a/pkg/storage/chunk/client/grpc/storage_client.go b/pkg/storage/chunk/client/grpc/storage_client.go index 42ee00507e412..8c1284ba1de49 100644 --- a/pkg/storage/chunk/client/grpc/storage_client.go +++ b/pkg/storage/chunk/client/grpc/storage_client.go @@ -9,6 +9,7 @@ import ( "github.com/grafana/loki/v3/pkg/storage/chunk" "github.com/grafana/loki/v3/pkg/storage/config" + "github.com/grafana/loki/v3/pkg/storage/wal" ) type StorageClient struct { @@ -66,6 +67,10 @@ func (s *StorageClient) PutChunks(ctx context.Context, chunks []chunk.Chunk) err return nil } +func (s *StorageClient) PutWal(_ context.Context, _ *wal.SegmentWriter) error { + return errors.New("not implemented") +} + func (s *StorageClient) DeleteChunk(ctx context.Context, _, chunkID string) error { chunkInfo := &ChunkID{ChunkID: chunkID} _, err := s.client.DeleteChunks(ctx, chunkInfo) diff --git a/pkg/storage/chunk/client/metrics.go b/pkg/storage/chunk/client/metrics.go index 76ca20a1bac5f..5e1ba5b41869b 100644 --- a/pkg/storage/chunk/client/metrics.go +++ b/pkg/storage/chunk/client/metrics.go @@ -7,6 +7,7 @@ import ( "github.com/prometheus/client_golang/prometheus/promauto" "github.com/grafana/loki/v3/pkg/storage/chunk" + "github.com/grafana/loki/v3/pkg/storage/wal" "github.com/grafana/loki/v3/pkg/util/constants" ) @@ -60,6 +61,10 @@ func (c MetricsChunkClient) Stop() { c.Client.Stop() } +func (c MetricsChunkClient) PutWal(ctx context.Context, writer *wal.SegmentWriter) error { + return c.Client.PutWal(ctx, writer) +} + func (c MetricsChunkClient) PutChunks(ctx context.Context, chunks []chunk.Chunk) error { if err := c.Client.PutChunks(ctx, chunks); err != nil { return err diff --git a/pkg/storage/chunk/client/object_client.go b/pkg/storage/chunk/client/object_client.go index 7a3b2e40c1663..8a22686fb113a 100644 --- a/pkg/storage/chunk/client/object_client.go +++ b/pkg/storage/chunk/client/object_client.go @@ -13,6 +13,7 @@ import ( "github.com/grafana/loki/v3/pkg/storage/chunk" "github.com/grafana/loki/v3/pkg/storage/chunk/client/util" "github.com/grafana/loki/v3/pkg/storage/config" + "github.com/grafana/loki/v3/pkg/storage/wal" ) // ObjectClient is used to store arbitrary data in Object Store (S3/GCS/Azure/...) @@ -105,6 +106,15 @@ func (o *client) Stop() { o.store.Stop() } +func (o *client) PutWal(ctx context.Context, segment *wal.SegmentWriter) error { + buffer := bytes.NewBuffer(nil) + _, err := segment.WriteTo(buffer) + if err != nil { + return err + } + return o.store.PutObject(ctx, "wal-segment-"+time.Now().UTC().Format(time.RFC3339Nano), bytes.NewReader(buffer.Bytes())) +} + // PutChunks stores the provided chunks in the configured backend. If multiple errors are // returned, the last one sequentially will be propagated up. func (o *client) PutChunks(ctx context.Context, chunks []chunk.Chunk) error { diff --git a/pkg/storage/store.go b/pkg/storage/store.go index 1900803637d7c..3a1e7d131ba12 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -7,6 +7,7 @@ import ( "time" "github.com/grafana/loki/v3/pkg/storage/types" + "github.com/grafana/loki/v3/pkg/storage/wal" "github.com/grafana/loki/v3/pkg/util/httpreq" lokilog "github.com/grafana/loki/v3/pkg/logql/log" @@ -608,3 +609,7 @@ func (f failingChunkWriter) Put(_ context.Context, _ []chunk.Chunk) error { func (f failingChunkWriter) PutOne(_ context.Context, _, _ model.Time, _ chunk.Chunk) error { return errWritingChunkUnsupported } + +func (f failingChunkWriter) PutWal(_ context.Context, _ *wal.SegmentWriter) error { + return errWritingChunkUnsupported +} diff --git a/pkg/storage/stores/composite_store.go b/pkg/storage/stores/composite_store.go index 834d9602727fc..484d8574f3cb3 100644 --- a/pkg/storage/stores/composite_store.go +++ b/pkg/storage/stores/composite_store.go @@ -15,10 +15,16 @@ import ( "github.com/grafana/loki/v3/pkg/storage/stores/index/stats" tsdb_index "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" + "github.com/grafana/loki/v3/pkg/storage/wal" "github.com/grafana/loki/v3/pkg/util" ) +type WalSegmentWriter interface { + PutWal(ctx context.Context, writer *wal.SegmentWriter) error +} + type ChunkWriter interface { + WalSegmentWriter Put(ctx context.Context, chunks []chunk.Chunk) error PutOne(ctx context.Context, from, through model.Time, chunk chunk.Chunk) error } @@ -45,6 +51,7 @@ type Store interface { ChunkWriter ChunkFetcher ChunkFetcherProvider + WalSegmentWriter Stop() } @@ -88,6 +95,12 @@ func (c *CompositeStore) Stores() []Store { return stores } +func (c CompositeStore) PutWal(ctx context.Context, writer *wal.SegmentWriter) error { + // TODO: Understand how to use the forStores method to correctly pick a store for this + err := c.stores[0].PutWal(ctx, writer) + return err +} + func (c CompositeStore) Put(ctx context.Context, chunks []chunk.Chunk) error { for _, chunk := range chunks { err := c.forStores(ctx, chunk.From, chunk.Through, func(innerCtx context.Context, from, through model.Time, store Store) error { diff --git a/pkg/storage/stores/composite_store_test.go b/pkg/storage/stores/composite_store_test.go index 28052c528f7c2..064e19ca8bbf9 100644 --- a/pkg/storage/stores/composite_store_test.go +++ b/pkg/storage/stores/composite_store_test.go @@ -9,6 +9,7 @@ import ( "github.com/pkg/errors" "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/storage/wal" "github.com/grafana/dskit/test" "github.com/prometheus/common/model" @@ -23,6 +24,10 @@ import ( type mockStore int +func (m mockStore) PutWal(_ context.Context, _ *wal.SegmentWriter) error { + return nil +} + func (m mockStore) Put(_ context.Context, _ []chunk.Chunk) error { return nil } diff --git a/pkg/storage/stores/series_store_write.go b/pkg/storage/stores/series_store_write.go index a36ae4510b8e3..2b134472eb2ea 100644 --- a/pkg/storage/stores/series_store_write.go +++ b/pkg/storage/stores/series_store_write.go @@ -13,6 +13,7 @@ import ( "github.com/grafana/loki/v3/pkg/storage/chunk/fetcher" "github.com/grafana/loki/v3/pkg/storage/config" "github.com/grafana/loki/v3/pkg/storage/stores/index" + "github.com/grafana/loki/v3/pkg/storage/wal" "github.com/grafana/loki/v3/pkg/util/constants" "github.com/grafana/loki/v3/pkg/util/spanlogger" ) @@ -65,6 +66,10 @@ func (c *Writer) Put(ctx context.Context, chunks []chunk.Chunk) error { return nil } +func (c *Writer) PutWal(ctx context.Context, segment *wal.SegmentWriter) error { + return c.fetcher.Client().PutWal(ctx, segment) +} + // PutOne implements Store func (c *Writer) PutOne(ctx context.Context, from, through model.Time, chk chunk.Chunk) error { sp, ctx := opentracing.StartSpanFromContext(ctx, "SeriesStore.PutOne") diff --git a/pkg/storage/stores/series_store_write_test.go b/pkg/storage/stores/series_store_write_test.go index cac84a17ebfbf..882fbaa00908b 100644 --- a/pkg/storage/stores/series_store_write_test.go +++ b/pkg/storage/stores/series_store_write_test.go @@ -13,6 +13,7 @@ import ( "github.com/grafana/loki/v3/pkg/storage/chunk" "github.com/grafana/loki/v3/pkg/storage/chunk/fetcher" "github.com/grafana/loki/v3/pkg/storage/config" + "github.com/grafana/loki/v3/pkg/storage/wal" ) type mockCache struct { @@ -55,6 +56,10 @@ type mockChunksClient struct { called int } +func (m *mockChunksClient) PutWal(_ context.Context, _ *wal.SegmentWriter) error { + return nil +} + func (m *mockChunksClient) PutChunks(_ context.Context, _ []chunk.Chunk) error { m.called++ return nil diff --git a/pkg/storage/util_test.go b/pkg/storage/util_test.go index 5ef02e74b1caf..b359132408902 100644 --- a/pkg/storage/util_test.go +++ b/pkg/storage/util_test.go @@ -26,6 +26,7 @@ import ( "github.com/grafana/loki/v3/pkg/storage/stores" index_stats "github.com/grafana/loki/v3/pkg/storage/stores/index/stats" "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" + "github.com/grafana/loki/v3/pkg/storage/wal" loki_util "github.com/grafana/loki/v3/pkg/util" "github.com/grafana/loki/v3/pkg/util/constants" util_log "github.com/grafana/loki/v3/pkg/util/log" @@ -185,7 +186,8 @@ func newMockChunkStore(chunkFormat byte, headfmt chunkenc.HeadBlockFmt, streams return &mockChunkStore{schemas: config.SchemaConfig{}, chunks: chunks, client: &mockChunkStoreClient{chunks: chunks, scfg: config.SchemaConfig{}}} } -func (m *mockChunkStore) Put(_ context.Context, _ []chunk.Chunk) error { return nil } +func (m *mockChunkStore) PutWal(_ context.Context, _ *wal.SegmentWriter) error { return nil } +func (m *mockChunkStore) Put(_ context.Context, _ []chunk.Chunk) error { return nil } func (m *mockChunkStore) PutOne(_ context.Context, _, _ model.Time, _ chunk.Chunk) error { return nil } @@ -292,6 +294,7 @@ func (m mockChunkStoreClient) Stop() { panic("implement me") } +func (m mockChunkStoreClient) PutWal(_ context.Context, _ *wal.SegmentWriter) error { return nil } func (m mockChunkStoreClient) PutChunks(_ context.Context, _ []chunk.Chunk) error { return nil } diff --git a/pkg/storage/wal/segment.go b/pkg/storage/wal/segment.go index 3e4fb0c2fa302..fa8e42cc94a5d 100644 --- a/pkg/storage/wal/segment.go +++ b/pkg/storage/wal/segment.go @@ -10,11 +10,12 @@ import ( "sort" "sync" + "go.uber.org/atomic" + "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" "github.com/grafana/loki/v3/pkg/logproto" - "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb" tsdbindex "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/grafana/loki/v3/pkg/storage/wal/chunks" "github.com/grafana/loki/v3/pkg/storage/wal/index" @@ -29,12 +30,15 @@ var ( streamSegmentPool = sync.Pool{ New: func() interface{} { return &streamSegment{ + lock: &sync.Mutex{}, entries: make([]*logproto.Entry, 0, 4096), } }, } + // 512kb - 20 mb encodedWalSegmentBufferPool = pool.NewBuffer(512*1024, 20*1024*1024, 2) + tenantLabel = "__loki_tenant__" ) func init() { @@ -46,13 +50,15 @@ type streamID struct { } type SegmentWriter struct { - streams map[streamID]*streamSegment - buf1 encoding.Encbuf - inputSize int64 - idxWriter *index.Writer + streams map[streamID]*streamSegment + buf1 encoding.Encbuf + inputSize atomic.Int64 + idxWriter *index.Writer + consistencyMtx *sync.RWMutex } type streamSegment struct { + lock *sync.Mutex lbls labels.Labels entries []*logproto.Entry tenantID string @@ -74,36 +80,52 @@ func NewWalSegmentWriter() (*SegmentWriter, error) { return nil, err } return &SegmentWriter{ - streams: make(map[streamID]*streamSegment, 64), - buf1: encoding.EncWith(make([]byte, 0, 4)), - idxWriter: idxWriter, + streams: make(map[streamID]*streamSegment, 64), + buf1: encoding.EncWith(make([]byte, 0, 4)), + idxWriter: idxWriter, + inputSize: atomic.Int64{}, + consistencyMtx: &sync.RWMutex{}, }, nil } +func (b *SegmentWriter) getOrCreateStream(id streamID, lbls labels.Labels) *streamSegment { + b.consistencyMtx.RLock() + s, ok := b.streams[id] + b.consistencyMtx.RUnlock() + if ok { + return s + } + b.consistencyMtx.Lock() + defer b.consistencyMtx.Unlock() + // Check another thread has not created it + s, ok = b.streams[id] + if ok { + return s + } + if lbls.Get(tenantLabel) == "" { + lbls = labels.NewBuilder(lbls).Set(tenantLabel, id.tenant).Labels() + } + s = streamSegmentPool.Get().(*streamSegment) + s.Reset() + s.lbls = lbls + s.tenantID = id.tenant + b.streams[id] = s + return s +} + // Labels are passed a string `{foo="bar",baz="qux"}` `{foo="foo",baz="foo"}`. labels.Labels => Symbols foo, baz , qux func (b *SegmentWriter) Append(tenantID, labelsString string, lbls labels.Labels, entries []*logproto.Entry) { if len(entries) == 0 { return } for _, e := range entries { - b.inputSize += int64(len(e.Line)) + b.inputSize.Add(int64(len(e.Line))) } id := streamID{labels: labelsString, tenant: tenantID} - s, ok := b.streams[id] - if !ok { - if lbls.Get(tsdb.TenantLabel) == "" { - lbls = labels.NewBuilder(lbls).Set(tsdb.TenantLabel, tenantID).Labels() - } - s = streamSegmentPool.Get().(*streamSegment) - s.Reset() - s.lbls = lbls - s.tenantID = tenantID - s.maxt = entries[len(entries)-1].Timestamp.UnixNano() - s.entries = append(s.entries, entries...) - b.streams[id] = s - return - } + s := b.getOrCreateStream(id, lbls) + s.lock.Lock() + defer s.lock.Unlock() for i, e := range entries { if e.Timestamp.UnixNano() >= s.maxt { s.entries = append(s.entries, entries[i]) @@ -250,12 +272,13 @@ func (b *SegmentWriter) Reset() { streamSegmentPool.Put(s) } b.streams = make(map[streamID]*streamSegment, 64) - b.inputSize = 0 + b.buf1.Reset() + b.inputSize.Store(0) } func (b *SegmentWriter) ToReader() (io.ReadSeekCloser, error) { // snappy compression rate is ~5x , but we can not predict it, so we need to allocate bigger buffer to avoid allocations - buffer := encodedWalSegmentBufferPool.Get(int(b.inputSize / 3)) + buffer := encodedWalSegmentBufferPool.Get(int(b.inputSize.Load() / 3)) _, err := b.WriteTo(buffer) if err != nil { return nil, fmt.Errorf("failed to write segment to create a reader: %w", err) @@ -297,7 +320,7 @@ func (e *EncodedSegmentReader) Close() error { // InputSize returns the total size of the input data written to the writer. // It doesn't account for timestamps and labels. func (b *SegmentWriter) InputSize() int64 { - return b.inputSize + return b.inputSize.Load() } type SegmentReader struct { diff --git a/pkg/storage/wal/segment_test.go b/pkg/storage/wal/segment_test.go index db0e9959ebebe..0e14028bd0531 100644 --- a/pkg/storage/wal/segment_test.go +++ b/pkg/storage/wal/segment_test.go @@ -5,6 +5,7 @@ import ( "context" "fmt" "sort" + "sync" "testing" "time" @@ -14,7 +15,6 @@ import ( "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/logql/syntax" - "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb" "github.com/grafana/loki/v3/pkg/storage/wal/testdata" "github.com/grafana/loki/pkg/push" @@ -121,7 +121,7 @@ func TestWalSegmentWriter_Append(t *testing.T) { require.True(t, ok) lbs, err := syntax.ParseLabels(expected.labels) require.NoError(t, err) - lbs = append(lbs, labels.Label{Name: string(tsdb.TenantLabel), Value: expected.tenant}) + lbs = append(lbs, labels.Label{Name: tenantLabel, Value: expected.tenant}) sort.Sort(lbs) require.Equal(t, lbs, stream.lbls) require.Equal(t, expected.entries, stream.entries) @@ -130,6 +130,163 @@ func TestWalSegmentWriter_Append(t *testing.T) { } } +func BenchmarkConcurrentAppends(t *testing.B) { + type appendArgs struct { + tenant string + labels labels.Labels + entries []*push.Entry + } + + lbls := []labels.Labels{ + labels.FromStrings("container", "foo", "namespace", "dev"), + labels.FromStrings("container", "bar", "namespace", "staging"), + labels.FromStrings("container", "bar", "namespace", "prod"), + } + characters := "abcdefghijklmnopqrstuvwxyz" + tenants := []string{} + // 676 unique tenants (26^2) + for i := 0; i < len(characters); i++ { + for j := 0; j < len(characters); j++ { + tenants = append(tenants, string(characters[i])+string(characters[j])) + } + } + + workChan := make(chan *appendArgs) + var wg sync.WaitGroup + var w *SegmentWriter + for i := 0; i < 100; i++ { + wg.Add(1) + go func(i int) { + for args := range workChan { + w.Append(args.tenant, args.labels.String(), args.labels, args.entries) + } + wg.Done() + }(i) + } + + t.ResetTimer() + for i := 0; i < t.N; i++ { + var err error + w, err = NewWalSegmentWriter() + require.NoError(t, err) + + for _, lbl := range lbls { + for _, r := range tenants { + for i := 0; i < 10; i++ { + workChan <- &appendArgs{ + tenant: r, + labels: lbl, + entries: []*push.Entry{ + {Timestamp: time.Unix(0, int64(i)), Line: fmt.Sprintf("log line %d", i)}, + }, + } + } + } + } + } + close(workChan) + wg.Wait() +} + +func TestConcurrentAppends(t *testing.T) { + type appendArgs struct { + tenant string + labels labels.Labels + entries []*push.Entry + } + dst := bytes.NewBuffer(nil) + + w, err := NewWalSegmentWriter() + require.NoError(t, err) + var wg sync.WaitGroup + workChan := make(chan *appendArgs, 100) + for i := 0; i < 100; i++ { + wg.Add(1) + go func(i int) { + for args := range workChan { + w.Append(args.tenant, args.labels.String(), args.labels, args.entries) + } + wg.Done() + }(i) + } + + lbls := []labels.Labels{ + labels.FromStrings("container", "foo", "namespace", "dev"), + labels.FromStrings("container", "bar", "namespace", "staging"), + labels.FromStrings("container", "bar", "namespace", "prod"), + } + characters := "abcdefghijklmnopqrstuvwxyz" + tenants := []string{} + // 676 unique tenants (26^2) + for i := 0; i < len(characters); i++ { + for j := 0; j < len(characters); j++ { + for k := 0; k < len(characters); k++ { + tenants = append(tenants, string(characters[i])+string(characters[j])+string(characters[k])) + } + } + } + + msgsPerSeries := 10 + msgsGenerated := 0 + for _, r := range tenants { + for _, lbl := range lbls { + for i := 0; i < msgsPerSeries; i++ { + msgsGenerated++ + workChan <- &appendArgs{ + tenant: r, + labels: lbl, + entries: []*push.Entry{ + {Timestamp: time.Unix(0, int64(i)), Line: fmt.Sprintf("log line %d", i)}, + }, + } + } + } + } + close(workChan) + wg.Wait() + + n, err := w.WriteTo(dst) + require.NoError(t, err) + require.True(t, n > 0) + + r, err := NewReader(dst.Bytes()) + require.NoError(t, err) + + iter, err := r.Series(context.Background()) + require.NoError(t, err) + + var expectedSeries, actualSeries []string + + for _, tenant := range tenants { + for _, lbl := range lbls { + expectedSeries = append(expectedSeries, labels.NewBuilder(lbl).Set(tenantLabel, tenant).Labels().String()) + } + } + + msgsRead := 0 + for iter.Next() { + actualSeries = append(actualSeries, iter.At().String()) + chk, err := iter.ChunkReader(nil) + require.NoError(t, err) + // verify all lines + var i int + for chk.Next() { + ts, line := chk.At() + require.Equal(t, int64(i), ts) + require.Equal(t, fmt.Sprintf("log line %d", i), string(line)) + msgsRead++ + i++ + } + require.NoError(t, chk.Err()) + require.NoError(t, chk.Close()) + require.Equal(t, msgsPerSeries, i) + } + require.NoError(t, iter.Err()) + require.ElementsMatch(t, expectedSeries, actualSeries) + require.Equal(t, msgsGenerated, msgsRead) + t.Logf("Generated %d messages between %d tenants", msgsGenerated, len(tenants)) +} + func TestMultiTenantWrite(t *testing.T) { w, err := NewWalSegmentWriter() require.NoError(t, err) @@ -167,7 +324,7 @@ func TestMultiTenantWrite(t *testing.T) { for _, tenant := range tenants { for _, lbl := range lbls { - expectedSeries = append(expectedSeries, labels.NewBuilder(lbl).Set(tsdb.TenantLabel, tenant).Labels().String()) + expectedSeries = append(expectedSeries, labels.NewBuilder(lbl).Set(tenantLabel, tenant).Labels().String()) } } diff --git a/vendor/github.com/grafana/loki/pkg/push/push-rf1.pb.go b/vendor/github.com/grafana/loki/pkg/push/push-rf1.pb.go new file mode 100644 index 0000000000000..7d87f4a1cc718 --- /dev/null +++ b/vendor/github.com/grafana/loki/pkg/push/push-rf1.pb.go @@ -0,0 +1,128 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: pkg/push/push-rf1.proto + +package push + +import ( + context "context" + fmt "fmt" + _ "github.com/gogo/protobuf/gogoproto" + proto "github.com/gogo/protobuf/proto" + _ "github.com/gogo/protobuf/types" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" + math "math" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package + +func init() { proto.RegisterFile("pkg/push/push-rf1.proto", fileDescriptor_4b1742ccc5fd9087) } + +var fileDescriptor_4b1742ccc5fd9087 = []byte{ + // 232 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x12, 0x2f, 0xc8, 0x4e, 0xd7, + 0x2f, 0x28, 0x2d, 0xce, 0x00, 0x13, 0xba, 0x45, 0x69, 0x86, 0x7a, 0x05, 0x45, 0xf9, 0x25, 0xf9, + 0x42, 0x1c, 0x39, 0xf9, 0xe9, 0x60, 0x96, 0x94, 0x48, 0x7a, 0x7e, 0x7a, 0x3e, 0x98, 0xa9, 0x0f, + 0x62, 0x41, 0xe4, 0xa5, 0xe4, 0xd3, 0xf3, 0xf3, 0xd3, 0x73, 0x52, 0xf5, 0xc1, 0xbc, 0xa4, 0xd2, + 0x34, 0xfd, 0x92, 0xcc, 0xdc, 0xd4, 0xe2, 0x92, 0xc4, 0xdc, 0x02, 0xa8, 0x02, 0x61, 0x14, 0x93, + 0x21, 0x82, 0x46, 0x2e, 0x5c, 0x9c, 0x01, 0xa5, 0xc5, 0x19, 0xa9, 0x45, 0x41, 0x6e, 0x86, 0x42, + 0xe6, 0x5c, 0x2c, 0x20, 0x8e, 0x90, 0xa8, 0x1e, 0xcc, 0x2e, 0x3d, 0x10, 0x3f, 0x28, 0xb5, 0xb0, + 0x34, 0xb5, 0xb8, 0x44, 0x4a, 0x0c, 0x5d, 0xb8, 0xb8, 0x20, 0x3f, 0xaf, 0x38, 0x55, 0x89, 0xc1, + 0x29, 0xec, 0xc2, 0x43, 0x39, 0x86, 0x1b, 0x0f, 0xe5, 0x18, 0x3e, 0x3c, 0x94, 0x63, 0x6c, 0x78, + 0x24, 0xc7, 0xb8, 0xe2, 0x91, 0x1c, 0xe3, 0x89, 0x47, 0x72, 0x8c, 0x17, 0x1e, 0xc9, 0x31, 0x3e, + 0x78, 0x24, 0xc7, 0xf8, 0xe2, 0x91, 0x1c, 0xc3, 0x87, 0x47, 0x72, 0x8c, 0x13, 0x1e, 0xcb, 0x31, + 0x5c, 0x78, 0x2c, 0xc7, 0x70, 0xe3, 0xb1, 0x1c, 0x43, 0x94, 0x42, 0x7a, 0x66, 0x49, 0x46, 0x69, + 0x92, 0x5e, 0x72, 0x7e, 0xae, 0x7e, 0x7a, 0x51, 0x62, 0x5a, 0x62, 0x5e, 0xa2, 0x7e, 0x4e, 0x7e, + 0x76, 0xa6, 0x3e, 0xcc, 0xa1, 0x49, 0x6c, 0x60, 0xdb, 0x8c, 0x01, 0x01, 0x00, 0x00, 0xff, 0xff, + 0x48, 0x19, 0x4c, 0x81, 0x15, 0x01, 0x00, 0x00, +} + +// Reference imports to suppress errors if they are not otherwise used. +var _ context.Context +var _ grpc.ClientConn + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +const _ = grpc.SupportPackageIsVersion4 + +// PusherRF1Client is the client API for PusherRF1 service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. +type PusherRF1Client interface { + Push(ctx context.Context, in *PushRequest, opts ...grpc.CallOption) (*PushResponse, error) +} + +type pusherRF1Client struct { + cc *grpc.ClientConn +} + +func NewPusherRF1Client(cc *grpc.ClientConn) PusherRF1Client { + return &pusherRF1Client{cc} +} + +func (c *pusherRF1Client) Push(ctx context.Context, in *PushRequest, opts ...grpc.CallOption) (*PushResponse, error) { + out := new(PushResponse) + err := c.cc.Invoke(ctx, "/logproto.PusherRF1/Push", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// PusherRF1Server is the server API for PusherRF1 service. +type PusherRF1Server interface { + Push(context.Context, *PushRequest) (*PushResponse, error) +} + +// UnimplementedPusherRF1Server can be embedded to have forward compatible implementations. +type UnimplementedPusherRF1Server struct { +} + +func (*UnimplementedPusherRF1Server) Push(ctx context.Context, req *PushRequest) (*PushResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method Push not implemented") +} + +func RegisterPusherRF1Server(s *grpc.Server, srv PusherRF1Server) { + s.RegisterService(&_PusherRF1_serviceDesc, srv) +} + +func _PusherRF1_Push_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(PushRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(PusherRF1Server).Push(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/logproto.PusherRF1/Push", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(PusherRF1Server).Push(ctx, req.(*PushRequest)) + } + return interceptor(ctx, in, info, handler) +} + +var _PusherRF1_serviceDesc = grpc.ServiceDesc{ + ServiceName: "logproto.PusherRF1", + HandlerType: (*PusherRF1Server)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "Push", + Handler: _PusherRF1_Push_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "pkg/push/push-rf1.proto", +} diff --git a/vendor/github.com/grafana/loki/pkg/push/push-rf1.proto b/vendor/github.com/grafana/loki/pkg/push/push-rf1.proto new file mode 100644 index 0000000000000..1c5a3e039341d --- /dev/null +++ b/vendor/github.com/grafana/loki/pkg/push/push-rf1.proto @@ -0,0 +1,13 @@ +syntax = "proto3"; + +package logproto; + +import "gogoproto/gogo.proto"; +import "google/protobuf/timestamp.proto"; +import "pkg/push/push.proto"; + +option go_package = "github.com/grafana/loki/pkg/push"; + +service PusherRF1 { + rpc Push(PushRequest) returns (PushResponse) {} +}