Page MenuHomePhabricator

Partition reassignment on kafka-jumbo negatively impacting mw-page-content-change-enrich
Closed, ResolvedPublic

Description

The reassignment described over at https://phabricator.wikimedia.org/T336044#9206891 seemed to have caused issues for mw-page-content-change-enrich (source). The reassignment started on Sept 28th 1:55PM UTC

image.png (175×1 px, 18 KB)

@JAllemandou mentioned

We start seeing a drop in events in/out of our app just before 14:00 UTC - this seems to match

The logs they were seeing were of the form

{"@timestamp":"2023-09-28T15:26:47.761Z","log.level": "INFO","message":"Triggering checkpoint 6 (type=SavepointType{name='Savepoint', postCheckpointAction=NONE, formatType=CANONICAL}) @ 1695914807740 for job 641561a8f514fa3193aa7ddcd82b46f4.", "ecs.version": "1.2.0","process.thread.name":"Checkpoint Timer","log.logger":"org.apache.flink.runtime.checkpoint.CheckpointCoordinator"}

{"@timestamp":"2023-09-28T15:27:05.199Z","log.level": "INFO","message":"[AdminClient clientId=mw-page-content-change-enrich.wikikube-codfw.000-enumerator-admin-client] Node 2004 disconnected.", "ecs.version": "1.2.0","process.thread.name":"kafka-admin-client-thread | mw-page-content-change-enrich.wikikube-codfw.000-enumerator-admin-client","log.logger":"org.apache.kafka.clients.NetworkClient"}

{"@timestamp":"2023-09-28T15:32:05.339Z","log.level": "INFO","message":"[AdminClient clientId=mw-page-content-change-enrich.wikikube-codfw.000-enumerator-admin-client] Node 2003 disconnected.", "ecs.version": "1.2.0","process.thread.name":"kafka-admin-client-thread | mw-page-content-change-enrich.wikikube-codfw.000-enumerator-admin-client","log.logger":"org.apache.kafka.clients.NetworkClient"}

{"@timestamp":"2023-09-28T15:32:54.619Z","log.level": "INFO","message":"[Producer clientId=producer-7] Closing the Kafka producer with timeoutMillis = 3600000 ms.", "ecs.version": "1.2.0","process.thread.name":"process_enrich_with_page_content -> (process_error_enrich_with_page_content, dict_to_row__$null__stream.error -> null__stream.error: Writer -> null__stream.error: Committer, dict_to_row__$kafka__mediawiki.page_content_change.v1 -> kafka__mediawiki.page_content_change.v1: Writer -> kafka__mediawiki.page_content_change.v1: Committer) (1/1)#3","log.logger":"org.apache.kafka.clients.producer.KafkaProducer"}

{"@timestamp":"2023-09-28T15:32:54.621Z","log.level": "INFO","message":"Metrics scheduler closed", "ecs.version": "1.2.0","process.thread.name":"process_enrich_with_page_content -> (process_error_enrich_with_page_content, dict_to_row__$null__stream.error -> null__stream.error: Writer -> null__stream.error: Committer, dict_to_row__$kafka__mediawiki.page_content_change.v1 -> kafka__mediawiki.page_content_change.v1: Writer -> kafka__mediawiki.page_content_change.v1: Committer) (1/1)#3","log.logger":"org.apache.kafka.common.metrics.Metrics"}

{"@timestamp":"2023-09-28T15:32:54.622Z","log.level": "INFO","message":"Closing reporter org.apache.kafka.common.metrics.JmxReporter", "ecs.version": "1.2.0","process.thread.name":"process_enrich_with_page_content -> (process_error_enrich_with_page_content, dict_to_row__$null__stream.error -> null__stream.error: Writer -> null__stream.error: Committer, dict_to_row__$kafka__mediawiki.page_content_change.v1 -> kafka__mediawiki.page_content_change.v1: Writer -> kafka__mediawiki.page_content_change.v1: Committer) (1/1)#3","log.logger":"org.apache.kafka.common.metrics.Metrics"}

{"@timestamp":"2023-09-28T15:32:54.622Z","log.level": "INFO","message":"Metrics reporters closed", "ecs.version": "1.2.0","process.thread.name":"process_enrich_with_page_content -> (process_error_enrich_with_page_content, dict_to_row__$null__stream.error -> null__stream.error: Writer -> null__stream.error: Committer, dict_to_row__$kafka__mediawiki.page_content_change.v1 -> kafka__mediawiki.page_content_change.v1: Writer -> kafka__mediawiki.page_content_change.v1: Committer) (1/1)#3","log.logger":"org.apache.kafka.common.metrics.Metrics"}

{"@timestamp":"2023-09-28T15:32:54.622Z","log.level": "INFO","message":"App info kafka.producer for producer-7 unregistered", "ecs.version": "1.2.0","process.thread.name":"process_enrich_with_page_content -> (process_error_enrich_with_page_content, dict_to_row__$null__stream.error -> null__stream.error: Writer -> null__stream.error: Committer, dict_to_row__$kafka__mediawiki.page_content_change.v1 -> kafka__mediawiki.page_content_change.v1: Writer -> kafka__mediawiki.page_content_change.v1: Committer) (1/1)#3","log.logger":"org.apache.kafka.common.utils.AppInfoParser"}

{"@timestamp":"2023-09-28T15:32:54.622Z","log.level": "INFO","message":"[Producer clientId=producer-8] Closing the Kafka producer with timeoutMillis = 3600000 ms.", "ecs.version": "1.2.0","process.thread.name":"process_enrich_with_page_content -> (process_error_enrich_with_page_content, dict_to_row__$null__stream.error -> null__stream.error: Writer -> null__stream.error: Committer, dict_to_row__$kafka__mediawiki.page_content_change.v1 -> kafka__mediawiki.page_content_change.v1: Writer -> kafka__mediawiki.page_content_change.v1: Committer) (1/1)#3","log.logger":"org.apache.kafka.clients.producer.KafkaProducer"}

{"@timestamp":"2023-09-28T15:32:54.705Z","log.level": "INFO","message":"Metrics scheduler closed", "ecs.version": "1.2.0","process.thread.name":"process_enrich_with_page_content -> (process_error_enrich_with_page_content, dict_to_row__$null__stream.error -> null__stream.error: Writer -> null__stream.error: Committer, dict_to_row__$kafka__mediawiki.page_content_change.v1 -> kafka__mediawiki.page_content_change.v1: Writer -> kafka__mediawiki.page_content_change.v1: Committer) (1/1)#3","log.logger":"org.apache.kafka.common.metrics.Metrics"}

{"@timestamp":"2023-09-28T15:32:54.705Z","log.level": "INFO","message":"Closing reporter org.apache.kafka.common.metrics.JmxReporter", "ecs.version": "1.2.0","process.thread.name":"process_enrich_with_page_content -> (process_error_enrich_with_page_content, dict_to_row__$null__stream.error -> null__stream.error: Writer -> null__stream.error: Committer, dict_to_row__$kafka__mediawiki.page_content_change.v1 -> kafka__mediawiki.page_content_change.v1: Writer -> kafka__mediawiki.page_content_change.v1: Committer) (1/1)#3","log.logger":"org.apache.kafka.common.metrics.Metrics"}

{"@timestamp":"2023-09-28T15:32:54.705Z","log.level": "INFO","message":"Metrics reporters closed", "ecs.version": "1.2.0","process.thread.name":"process_enrich_with_page_content -> (process_error_enrich_with_page_content, dict_to_row__$null__stream.error -> null__stream.error: Writer -> null__stream.error: Committer, dict_to_row__$kafka__mediawiki.page_content_change.v1 -> kafka__mediawiki.page_content_change.v1: Writer -> kafka__mediawiki.page_content_change.v1: Committer) (1/1)#3","log.logger":"org.apache.kafka.common.metrics.Metrics"}

{"@timestamp":"2023-09-28T15:32:54.705Z","log.level": "INFO","message":"App info kafka.producer for producer-8 unregistered", "ecs.version": "1.2.0","process.thread.name":"process_enrich_with_page_content -> (process_error_enrich_with_page_content, dict_to_row__$null__stream.error -> null__stream.error: Writer -> null__stream.error: Committer, dict_to_row__$kafka__mediawiki.page_content_change.v1 -> kafka__mediawiki.page_content_change.v1: Writer -> kafka__mediawiki.page_content_change.v1: Committer) (1/1)#3","log.logger":"org.apache.kafka.common.utils.AppInfoParser"}

These show a failure to produce messages to kafka, topic codfw.mediawiki.page_content_change.v1 : Closing the Kafka producer with timeoutMillis = 3600000 ms..

At the time of the reassignment start, a reassignment throttle was set to 50MB/s. It was then incrementally increased by steps of a couple of 10MB/s each time. However, the commit failures started as soon as the reassignment started, meaning with a throttle of 50MB/s.

Since the reassignment started, the topic has been in the following state (it is a topic with a single partition):

	Topic: codfw.mediawiki.page_content_change.v1	Partition: 0	Leader: 1002	Replicas: 1013,1002,1011,1003,1008,1004	Isr: 1002,1003,1004

The original replicas 1002, 1003, 1004 are still in sync, and streaming data out to 1013,1011,1008, as all 3 original brokers need to be evacuated.

We initially investigated a possible network policy issue, but an helmfile diff didn't show any output, meaning that new brokers should be able to be egress-ed to:

brouberol@deploy2002:/srv/deployment-charts/helmfile.d/services/mw-page-content-change-enrich$ helmfile -e codfw diff
helmfile.yaml: basePath=.
Comparing release=main, chart=wmf-stable/flink-app
helmfile.yaml: basePath=.
brouberol@deploy2002:/srv/deployment-charts/helmfile.d/services/mw-page-content-change-enrich$ helmfile -e eqiad diff
helmfile.yaml: basePath=.
Comparing release=main, chart=wmf-stable/flink-app
helmfile.yaml: basePath=.
brouberol@deploy2002:/srv/deployment-charts/helmfile.d/services/mw-page-content-change-enrich$

After investigation, @JAllemandou found the following log emitted by mw-page-content-change-enrich

{"@timestamp":"2023-09-28T17:53:05.864Z","log.level": "INFO","message":"process_enrich_with_page_content -> (process_error_enrich_with_page_content, dict_to_row__$null__stream.error -> null__stream.error: Writer -> null__stream.error: Committer, dict_to_row__$kafka__mediawiki.page_content_change.v1 -> kafka__mediawiki.page_content_change.v1: Writer -> kafka__mediawiki.page_content_change.v1: Committer) (1/1) (94e62905d349a5968ab3bf1ccb1c9e02_e9b82703c83ea7e9b9e5df8d59eb3c36_0_2) switched from RUNNING to FAILED on flink-app-main-taskmanager-1-1 @ 10.194.164.194 (dataPort=6121).", "ecs.version": "1.2.0","process.thread.name":"flink-akka.actor.default-dispatcher-4","log.logger":"org.apache.flink.runtime.executiongraph.ExecutionGraph","error.type":"org.apache.flink.util.FlinkRuntimeException","error.message":"Failed to send data to Kafka codfw.mediawiki.page_content_change.v1-0@-1 with FlinkKafkaInternalProducer{transactionalId='null', inTransaction=false, closed=false} ","error.stack_trace":"org.apache.flink.util.FlinkRuntimeException: Failed to send data to Kafka codfw.mediawiki.page_content_change.v1-0@-1 with FlinkKafkaInternalProducer{transactionalId='null', inTransaction=false, closed=false} \n\tat org.apache.flink.connector.kafka.sink.KafkaWriter$WriterCallback.throwException(KafkaWriter.java:438)\n\tat org.apache.flink.connector.kafka.sink.KafkaWriter$WriterCallback.lambda$onCompletion$0(KafkaWriter.java:419)\n\tat org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.runThrowing(StreamTaskActionExecutor.java:50)\n\tat org.apache.flink.streaming.runtime.tasks.mailbox.Mail.run(Mail.java:90)\n\tat org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMail(MailboxProcessor.java:398)\n\tat org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMailsNonBlocking(MailboxProcessor.java:383)\n\tat org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMail(MailboxProcessor.java:345)\n\tat org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:229)\n\tat org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:839)\n\tat org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:788)\n\tat org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:952)\n\tat org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:931)\n\tat org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:745)\n\tat org.apache.flink.runtime.taskmanager.Task.run(Task.java:562)\n\tat java.base/java.lang.Thread.run(Thread.java:829)\nCaused by: org.apache.kafka.common.errors.TimeoutException: Expiring 2 record(s) for codfw.mediawiki.page_content_change.v1-0:120000 ms has passed since batch creation\n"}

snipped version:

Failed to send data to Kafka codfw.mediawiki.page_content_change.v1-0 [...]  Expiring 2 record(s) for codfw.mediawiki.page_content_change.v1-0:120000 ms has passed since batch creation

Event Timeline

The only running cluster seems in the k8s eqiad cluster, I just deleted app + taskmanager pods to force their re-creation (helmfile roll_restart doesn't work).

@elukey confirmed that all kafka-jumbo hosts are reachable from the mw-page-content-change-enrich pods (via nsenter), so we can rule out Network Policy issues. The main culprit is backpressure from kafka to the producer caused by the data reassignment.

BTullis triaged this task as High priority.Sep 29 2023, 9:31 AM
BTullis moved this task from Incoming to In Progress on the Data-Platform-SRE board.

I have a doubt, thinking out loud. I am trying to see signs of errors, and so far I checked:

So far I only noticed that the codfw topic stopped to get messages, meanwhile the eqiad one is empty. This makes sense, IIUC, since we have the Flink job only deployed in eqiad (with permissions to pull from main eqiad and push to jumbo eqiad) and not in codfw (IIUC it was undeployed to facilitate the debugging). The page_change source event is only published to codfw at the moment due to the switchover, so unless I am mistaken and I am missing something we are not testing the right app (we should have it deployed in codfw only).

@JAllemandou does it make sense what I wrote above? Should we deploy in codfw and not in eqiad?

Here are the symptoms dislpayed by the flink App at the time of reassignment:

image.png (349×3 px, 92 KB)

We've restarted the app, and its rate of consumed/produced msg/s is low compared to the baseline from yesterday. I'll attempt to reduce the replication throttle to 1MB/s to see if we observe a positivbe impact on the app throughput

brouberol@kafka-jumbo1010:~/topicmappr$ kafka reassign-partitions --reassignment-json-file out-files/reassignment-a-to-d-phase0.json --throttle 100000 --execute
kafka-reassign-partitions --zookeeper conf1007.eqiad.wmnet,conf1008.eqiad.wmnet,conf1009.eqiad.wmnet/kafka/jumbo-eqiad --reassignment-json-file out-files/reassignment-a-to-d-phase0.json --throttle 100000 --execute
There is an existing assignment running.

The errors are back, I see:

Caused by: org.apache.kafka.common.errors.TimeoutException: Expiring 1 record(s) for codfw.mediawiki.page_content_chan
ge.v1-0:120000 ms has passed since batch creation\n

Even with a throttle of 1MB/s, the app is still displaying the same (low) throughput. This does not seem like it's a backpressure issue to me either.

We have investigated the idea of a possible acks=all setting on the producer level that would lead to a timeout, as the new replicas are still catching up, but this setting seem to indicate we have acks=1 (leader only) https://gerrit.wikimedia.org/r/plugins/gitiles/operations/deployment-charts/+/master/helmfile.d/services/mw-page-content-change-enrich/values-main.yaml#40

ACtually, that AT_LEAST_ONCE setting has to do with how Flink batches messages to Kafka according to @JAllemandou, so this does not map to the acks setting.

We checked on the brokers, and found a min.insync.replicas=1 setting, meaning that an acks=all setting on the producer (assuming that's the case) would work with the leader + 1 replica ack-ing the message. So this does not explain what we're seeing here, as even if the 3 new brokers are still catching up, that should'nt cause production issues.

DeliveryGuarantee.AT_LEAST_ONCE: The sink will wait for all outstanding records in the Kafka buffers to be acknowledged by the Kafka producer on a checkpoint. No messages will be lost in case of any issue with the Kafka brokers but messages may be duplicated when Flink restarts because Flink reprocesses old input records.

https://nightlies.apache.org/flink/flink-docs-release-1.17/docs/connectors/datastream/kafka/#fault-tolerance

We are going to re-deploy the flink app in codfw when the kafka partitions re-assignment task (on Jumbo) is done, to verify if it is has anything to do with it. ETA is the weekend (worst case).

bking subscribed.

Hey all, I just saw the alert for mw-page-content-change-enrich and did a helmfile apply to get the service back up. @BTullis messaged me in IRC a few moments later, so I quickly did a helmfile destroy to bring the service back down. Hopefully I didn't mess anything up.

I just silenced mw_page_content_change_enrich in codfw is not running alerts for the next 3 days.

The topic reassignment finished on Sep 30th, 2PM UTC.

image.png (150×699 px, 9 KB)

Once restarted, the app is able to produce to the codfw.mediawiki.page_content_change.v1 topic. The throughput has decreased compared to before the reassignment, but the app is stable nonetheless.

We will swap leadership between brokers 1002 and 1013, and assess any impact on the app performance.

Initial state:

brouberol@kafka-jumbo1010:~/topicmappr/out-files$ cat codfw.mediawiki.page_content_change.v1.phase1.json | jq .
{
  "version": 1,
  "partitions": [
    {
      "topic": "codfw.mediawiki.page_content_change.v1",
      "partition": 0,
      "replicas": [
        1013,
        1011,
        1008,
        1002
      ]
    }
  ]
}
brouberol@kafka-jumbo1010:~/topicmappr/out-files$ kafka reassign-partitions --reassignment-json-file codfw.mediawiki.page_content_change.v1.phase1.json --verify
kafka-reassign-partitions --zookeeper conf1007.eqiad.wmnet,conf1008.eqiad.wmnet,conf1009.eqiad.wmnet/kafka/jumbo-eqiad --reassignment-json-file codfw.mediawiki.page_content_change.v1.phase1.json --verify
Status of partition reassignment: 
ERROR: Assigned replicas (1002,1013,1011,1008) don't match the list of replicas for reassignment (1013,1011,1008,1002) for partition codfw.mediawiki.page_content_change.v1-0
Reassignment of partition codfw.mediawiki.page_content_change.v1-0 failed

The swap between 1002 and 1013 is done:

brouberol@kafka-jumbo1010:~/topicmappr/out-files$ kafka reassign-partitions --reassignment-json-file codfw.mediawiki.page_content_change.v1.phase1.json --execute
kafka-reassign-partitions --zookeeper conf1007.eqiad.wmnet,conf1008.eqiad.wmnet,conf1009.eqiad.wmnet/kafka/jumbo-eqiad --reassignment-json-file codfw.mediawiki.page_content_change.v1.phase1.json --execute
Current partition replica assignment

{"version":1,"partitions":[{"topic":"codfw.mediawiki.page_content_change.v1","partition":0,"replicas":[1002,1013,1011,1008],"log_dirs":["any","any","any","any"]}]}

Save this to use as the --reassignment-json-file option during rollback
Successfully started reassignment of partitions.
brouberol@kafka-jumbo1010:~/topicmappr/out-files$ kafka reassign-partitions --reassignment-json-file codfw.mediawiki.page_content_change.v1.phase1.json --verify
kafka-reassign-partitions --zookeeper conf1007.eqiad.wmnet,conf1008.eqiad.wmnet,conf1009.eqiad.wmnet/kafka/jumbo-eqiad --reassignment-json-file codfw.mediawiki.page_content_change.v1.phase1.json --verify
Status of partition reassignment: 
Reassignment of partition codfw.mediawiki.page_content_change.v1-0 completed successfully
brouberol@kafka-jumbo1010:~/topicmappr/out-files$ kafka topics --describe --topic codfw.mediawiki.page_content_change.v1
kafka-topics --zookeeper conf1007.eqiad.wmnet,conf1008.eqiad.wmnet,conf1009.eqiad.wmnet/kafka/jumbo-eqiad --describe --topic codfw.mediawiki.page_content_change.v1
Topic:codfw.mediawiki.page_content_change.v1	PartitionCount:1	ReplicationFactor:4	Configs:
	Topic: codfw.mediawiki.page_content_change.v1	Partition: 0	Leader: 1002	Replicas: 1013,1011,1008,1002	Isr: 1013,1002,1011,1008

We now need to perform a leader election, or wait for kafka to do it itself

The leadership has been transferred to 1013:

brouberol@kafka-jumbo1010:~/topicmappr/out-files$ kafka topics --describe --topic codfw.mediawiki.page_content_change.v1
kafka-topics --zookeeper conf1007.eqiad.wmnet,conf1008.eqiad.wmnet,conf1009.eqiad.wmnet/kafka/jumbo-eqiad --describe --topic codfw.mediawiki.page_content_change.v1
Topic:codfw.mediawiki.page_content_change.v1	PartitionCount:1	ReplicationFactor:4	Configs:
	Topic: codfw.mediawiki.page_content_change.v1	Partition: 0	Leader: 1013	Replicas: 1013,1011,1008,1002	Isr: 1013,1002,1011,1008

We now remove broker 1002 from the replicas:

brouberol@kafka-jumbo1010:~/topicmappr/out-files$ kafka reassign-partitions --reassignment-json-file codfw.mediawiki.page_content_change.v1.phase1.json --execute
kafka-reassign-partitions --zookeeper conf1007.eqiad.wmnet,conf1008.eqiad.wmnet,conf1009.eqiad.wmnet/kafka/jumbo-eqiad --reassignment-json-file codfw.mediawiki.page_content_change.v1.phase1.json --execute
Current partition replica assignment

{"version":1,"partitions":[{"topic":"codfw.mediawiki.page_content_change.v1","partition":0,"replicas":[1013,1011,1008,1002],"log_dirs":["any","any","any","any"]}]}

Save this to use as the --reassignment-json-file option during rollback
Successfully started reassignment of partitions.
brouberol@kafka-jumbo1010:~/topicmappr/out-files$ kafka reassign-partitions --reassignment-json-file codfw.mediawiki.page_content_change.v1.phase1.json --verify
kafka-reassign-partitions --zookeeper conf1007.eqiad.wmnet,conf1008.eqiad.wmnet,conf1009.eqiad.wmnet/kafka/jumbo-eqiad --reassignment-json-file codfw.mediawiki.page_content_change.v1.phase1.json --verify
Status of partition reassignment: 
Reassignment of partition codfw.mediawiki.page_content_change.v1-0 completed successfully
brouberol@kafka-jumbo1010:~/topicmappr/out-files$ kafka topics --describe --topic codfw.mediawiki.page_content_change.v1
kafka-topics --zookeeper conf1007.eqiad.wmnet,conf1008.eqiad.wmnet,conf1009.eqiad.wmnet/kafka/jumbo-eqiad --describe --topic codfw.mediawiki.page_content_change.v1
Topic:codfw.mediawiki.page_content_change.v1	PartitionCount:1	ReplicationFactor:3	Configs:
	Topic: codfw.mediawiki.page_content_change.v1	Partition: 0	Leader: 1013	Replicas: 1013,1011,1008	Isr: 1013,1011,1008

Earlier today, @JAllemandou and @tchin restarted the streaming app. Messages are now being produced, so the app is working. However, it was not able to catch up, we believe because of T347884. Increasing parallelism by increasing the number of taskmanager replicas allowed the backlog to begin to be processed.

I just manually increased replicas again from 4 to 12 in an attempt to process the backlog faster.

Mentioned in SAL (#wikimedia-operations) [2023-10-02T20:27:32Z] <ottomata> mw-page-content-change-enrich - increase replicas to 12 to process backlog - T347676

Mentioned in SAL (#wikimedia-operations) [2023-10-02T20:56:24Z] <ottomata> mw-page-content-change-enrich - increase replicas to 24 to process backlog - T347676

Mentioned in SAL (#wikimedia-operations) [2023-10-02T20:59:52Z] <ottomata> mw-page-content-change-enrich - CORRECTION - increase replicas to 20 to process backlog - T347676

Change 963051 had a related patch set uploaded (by Ottomata; author: Ottomata):

[operations/deployment-charts@master] mw-page-content-change-enrich - bump to 1.27.0, and set codfw replicas to 12

https://gerrit.wikimedia.org/r/963051

Change 963051 merged by Ottomata:

[operations/deployment-charts@master] mw-page-content-change-enrich - bump to 1.27.0, and set codfw replicas to 12

https://gerrit.wikimedia.org/r/963051

Mentioned in SAL (#wikimedia-operations) [2023-10-03T13:38:16Z] <ottomata> mw-page-content-change-enrich codfw - bump to 1.27.0 and set replicas to 12 while processing backlog - T347676

Change 963080 had a related patch set uploaded (by Ottomata; author: Ottomata):

[operations/deployment-charts@master] mw-page-content-change-enrich - set replicas: 2 for eqiad and codfw

https://gerrit.wikimedia.org/r/963080

Change 963080 merged by jenkins-bot:

[operations/deployment-charts@master] mw-page-content-change-enrich - set replicas: 2 for eqiad and codfw

https://gerrit.wikimedia.org/r/963080