Меня зовут Артем Москальков, я — ведущий инженер данных в Магнит OMNI. В статье я расскажу о том, как мы оптимизировали производительность кластера в ClickHouse.

Частые мелкие вставки данных (INSERT-запросы) в ClickHouse способны серьезно снизить производительность кластера. В нашей системе Kafka ClickHouse долгое время существовала проблема с огромным числом insert-запросов, генерируемых Kafka Sink коннектором, который отправлял отдельный запрос практически на каждое поступающее сообщение. Такая схема приводила к постоянному росту нагрузки: ClickHouse был вынужден обрабатывать тысячи мелких вставок, создавая большое количество партов на диске и расходуя значительные ресурсы на каждую операцию вставки. В результате наблюдалось увеличение времени отклика запросов, учащение фоновых слияний частей и снижение общей пропускной способности системы. Пиковая нагрузка на хосты достигала 3 500 insert-запросов в секунду и больше. Ниже приведен график со статистикой инсертов до проведённой оптимизации кластера.

Количество insert-запросов в секунду (до оптимизации)
Количество insert-запросов в секунду (до оптимизации)

Причина крылась в дефолтной конфигурации коннектора Kafka: данные из топика считывались небольшими порциями и немедленно отправлялись в ClickHouse. Оптимальное же использование ClickHouse предполагает пакетную загрузку данных крупными блоками

Цель оптимизации — настроить Kafka Sink-коннектор так, чтобы он накапливал записи в более крупные батчи перед отправкой в ClickHouse. Мы покажем, как изменение параметров чтения из топика Kafka Consumer позволило в разы сократить частоту insert-запросов при одновременном увеличении среднего размера вставляемого блока данных. Это резко снизило нагрузку на ClickHouse-кластер и улучшило его производительность.

Архитектура и текущий объем нагрузки

Наша инфраструктура — высоконагруженная система, ориентированная на работу с большими объемами данных. В качестве центрального аналитического хранилища мы используем распределенный кластер ClickHouse. Сейчас объем данных в ClickHouse превышает 40 Тб на дисках, и более 200 Тб хранится в S3-хранилище для долговременного хранения и резервного копирования.

Архитектура DWH Magnit Omni

Первичный источник транзакционных данных — PostgreSQL, где размещены порядка 30 баз общим объёмом около 1,5 Тб. Данные из PostgreSQL поступают в Kafka посредством Debezium, который проводит захват изменений в реальном времени (Change Data Capture). PostgreSQL обрабатывает высокую операционную нагрузку, достигающую до 6000 ops по всем базам данных, и до 1500 ops по наиболее нагруженной из них.

Кластер ClickHouse шардирован для балансировки нагрузки и повышения производительности запросов. Для данных Т-0 выделен отдельный шард с двумя репликами, обеспечивающий надежность и отказоустойчивость. На данном шарде располагается 100+ таблиц, которые активно наполняются данными. Средний объем вставленных данных составляет 400+ MiB в минуту. Основной механизм поступления данных — Kafka Sink коннекторы, которые обеспечивают потоковую передачу данных из Kafka непосредственно в ClickHouse.

Kafka Throughput: Входящие и исходящие байты в секунду
Kafka Throughput: Входящие и исходящие байты в секунду

Apache Kafka выступает центральной шиной передачи данных и содержит более 3 Тб активных сообщений. Средний размер пакета сообщений (batch-size-avg) от продюсеров Kafka составляет около 600 байт, а общий объем исходящего трафика от продьюсеров достигает примерно 6 Мб в секунду (Source/Producers Outgoing Bytes per Second). Java Virtual Machine (JVM), выделенная под Kafka, имеет объём памяти 36 Гб. В системе Kafka задействовано более 450 топиков, которые обеспечивают изоляцию потоков данных и их последующую маршрутизацию.

Из-за архитектуры и значительных объемов данных, особенно в ClickHouse и Kafka, мы столкнулись с необходимостью проведения оптимизации для повышения общей производительности системы и улучшения ее пропускной способности.

Оптимизация

Первый этап оптимизации: настройка fetch и poll параметров

Чтобы коннектор формировал крупные батчи, мы скорректировали параметры потребителя Kafka (Kafka Consumer), используемого Sink-коннектором. Ниже в таблице приведены ключевые параметры до и после оптимизации и их роли:

Параметр

Дефолтное значение

Измененное значение

Описание параметра

fetch.min.bytes

1 байт

2 621 440 байт (2,5 Мб)

Минимальный объем данных, который брокер должен собрать перед отдачей потребителю. Брокер Kafka будет ждать накопления этого объема перед отправкой данных коннектору. Это заставляет брокер ждать накопления большого блока сообщений перед отдачей коннектору, что увеличивает размер пачки и снижает частоту обращений к ClickHouse ценой небольшой задержки.

fetch.max.wait.ms

500 мс

72 000 мс (72 с)

Максимальное время, которое брокер будет ждать, чтобы набрать пачку размера fetch.min.bytes. Раньше – 0,5 секунды, теперь – до 60 секунд. Если за это время не накопится 2,5 Мб данных, брокер отдаст все, что успел накопить за интервал. Увеличение этого таймаута позволяет ждать более крупные батчи при низком потоке данных, что также уменьшает частоту вставок.

max.poll.records

500 сообщений

50 000 сообщений

Максимальное число сообщений, возвращаемых за один вызов poll(). По умолчанию 500, мы увеличили до 50 000, чтобы обрабатывать большие фрагменты буфера клиента за меньшее число вызовов. При большом фоновом fetch это позволяет получить большее количество записей вместо сотен мелких порций и снизить накладные расходы на внутренние циклы чтения.

max.poll.interval.ms

300 000 мс (5 мин)

120 000 мс (120 с)

Максимальная задержка между вызовами poll() при работе в составе группы потребителей. По умолчанию 5 минут: если потребитель не вызывает poll дольше, он считается «зависшим» и исключается из группы. Мы намеренно уменьшили этот таймаут, чтобы ускорить обнаружение и реакцию на реальные зависания, сохранив при этом достаточный буфер для нормальной работы коннектора.

Примечание: Значения fetch.min.bytes, fetch.max.wait.ms могут варьироваться в зависимости от целей коннектора, скорости потока и объема сообщений, который приходит в ваш топик. Наша оптимизация Clickhouse кластера подразумевала подбор оптимальных значений по коннекторам для одновременного недопущения роста consumer lag, сохранив при этом скорость вставки батчей не реже чем за полторы – две минуты.

Для коннекторов, где была возможность пожертвовать скоростью вставки за счет увеличения объема батча, мы поднимали значения для параметров конфигурации.

Примерная формула расчета оптимального выбора параметров fetch.min.bytes.:

Образец топика Kafka
Образец топика Kafka
  • объем данных в топике: 528 336 Мб;

  • количество сообщений всего: 2 530 126;

  • вес одного сообщения: 528 336 МБ / 2 530 126 = 0,0002082 Мб = 208,82 байт;

  • трафик: 10 000 сообщений в минуту;

  • объем трафика за минуту = 10 000 * 0,0002082 = 2,082 Мб;

  • Fetch.min.bytes = 2,5 Мб. Повысим порог, если придут более тяжелые сообщения или увеличится трафик;

  • Fetch.max.wait.ms = 72 000 мс. Это время, дольше которого мы не можем ждать, если не накопится достаточный объем.

Проблема таймаутов

После применения новых настроек данные действительно стали накапливаться в большие пачки. Однако поначалу возникли побочные эффекты, связанные с таймаутами, которые мы обнаружили и устранили настройкой дополнительных параметров.

Мы получили разрывы подключения к Kafka: в логах Kafka Client было сообщение о отключении от узла. Брокер разъединялся, потому что коннектор ждал ответ дольше дефолтных 30 с (при новом fetch.max.wait.ms = 72 секунд).

Логи таймаута запроса к брокеру
Логи таймаута запроса к брокеру

Решение: повысить request.timeout.ms на клиенте до 100 с ("request.timeout.ms" = "100000"), чтобы покрыть максимальное время ожидания батча + возможные сетевые задержки. После увеличения этого параметра разрывы соединения прекратились.

Второй этап оптимизации: настройка ClickHouse-коннектора с помощью опции ignorePartitionsWhenBatching

После оптимизации настроек потребителя Kafka на первом этапе (увеличен объём данных, забираемых за один запрос к брокеру), мы столкнулись с ограничением: по умолчанию коннектор Kafka → ClickHouse формирует INSERT-запросы, разбивая полученные сообщения по отдельным батчам согласно партициям Kafka-топика.

Это приводит к тому, что даже при достаточно большом объёме данных, который коннектор получает за один вызов метода poll(), данные всё равно отправляются в ClickHouse множеством мелких транзакций, так как каждый переход к новой партиции топика вынуждает отправить текущий батч и начать формирование нового. В итоге снижается эффективность передачи данных в ClickHouse, так как кластер наиболее эффективен именно при крупных пакетных вставках.

Чтобы устранить это ограничение, мы применили дополнительную оптимизацию на стороне самого коннектора:

Как работает эта настройка?

  1. Сначала Kafka-коннектор делает запросы к брокеру согласно настройкам, выставленным ранее (fetch.min.bytes, fetch.max.wait.ms и другим) и накапливает во внутреннем буфере большие пачки сообщений из различных партиций.

  2. Затем при формировании INSERT-запросов коннектор по умолчанию начинает новый запрос при каждом переходе на новую партицию. С опцией ignorePartitionsWhenBatching=true коннектор игнорирует границы партиций и формирует один общий батч, объединяя сообщения из всех партиций.

Без этой опции даже накопив, к примеру, 5 000 сообщений из 5 партиций, коннектор сделал бы 5 отдельных вставок по ~1 000 строк (по одной на партицию). В документации отмечено, что эффект этой опции зависит от числа задач коннектора (tasks.max): если коннектор запущен с несколькими задачами, каждая получает меньше партиций, и объединение может давать меньший выигрыш.

Таким образом, данная настройка дополняет и усиливает эффект первой оптимизации (настройки fetch):

  • значительно увеличивается размер INSERT-запросов: сообщения разных партиций объединяются в максимально крупные батчи;

  • сокращается общее число INSERT-запросов: вместо множества мелких операций выполняется меньшее число, но более объёмных;

  • уменьшается нагрузка на ClickHouse (меньше парсинга, соединений, транзакций).

Примечание: Настройка (ignorePartitionsWhenBatching) работает только при включённом режиме «at-least-once» (то есть при exactlyOnce=false). Коннектор должен поддерживать чёткие границы по партициям, чтобы гарантировать идемпотентность и точность записей данных. Это означает, что при сбое нужно точно понимать, какие данные уже записаны, а какие нет. Объединение сообщений из разных партиций в один батч нарушает эту возможность, так как теряется привязка сообщений к конкретным партициям и их оффсетам, что делает невозможным точный откат или повтор транзакции при сбоях. В нашем случае отключение exactlyOnce настройки было оправдано, поскольку мы используем ReplacingMergeTree движки в наших таблицах, что позволяет не беспокоится о потенциальных дублях.

Фрагмент настройки коннектора:

"max.poll.interval.ms" = "120000",

"max.poll.records" = "50000",

"fetch.max.wait.ms" = "72000",

"fetch.min.bytes" = "2621440",

"request.timeout.ms" = "100000",

"ignorePartitionsWhenBatching" = "true"

Результаты оптимизации вставок

Эффект первого этапа: включение параметров fetch.min.bytes и fetch.max.wait.ms

На основании проведённых оптимизаций параметров Kafka-коннектора была достигнута значительная оптимизация процесса вставки данных в ClickHouse.

Результат первого этапа оптимизации (пример таблицы CH кластера)
Результат первого этапа оптимизации (пример таблицы CH кластера)

Сравнение метрик до и после оптимизации

  • Среднее количество строк в одной вставке (avg_written_rows):

    • до оптимизации (19:07–19:11) от 4 до 21 строки на один INSERT;

    • после первого этапа (после 19:11) от 1 700 до 2 800 строк на один INSERT.

  • Количество INSERT-запросов в минуту (cnt_inserts) :

    • до оптимизации от 360 до 488 запросов в минуту;

    • после первого этапа всего 8–16 запросов в минуту.

  • Общее количество вставленных строк в минуту (sum_writtem_rows):

    • до оптимизации примерно 1 500–9 300 строк в минуту;

    • после первого этапа выросло до 14 000—45 000 строк в минуту, сохранив стабильную скорость записи, что говорит о минимизации нагрузки на процессы слияния и снижении общего потребления системных ресурсов.

Эффект второго этапа: включение ignorePartitionsWhenBatching=true

После второго этапа оптимизации (включени параметра ignorePartitionsWhenBatching=true), удалось добиться дальнейшего повышения эффективности вставки данных в ClickHouse:

Результат второго этапа оптимизации
Результат второго этапа оптимизации
  • Среднее количество строк в одном INSERT-запросе:

    • после первого этапа (после первой оптимизации): ~1 700–2 800 строк;

    • после второго этапа: увеличилось примерно до 20 000–23 500 строк на один запрос.

  • Количество INSERT-запросов в минуту:

    • после первого этапа: 8–16 запросов;

    • после второго этапа: всего 1 INSERT-запрос в минуту.

  • Суммарное количество строк в минуту:

    • после второго этапа стабилизировалось в районе 19 000–23 500 строк/мин, сохранив необходимую скорость обработки.

В результате последовательного применения двух этапов оптимизации удалось существенно улучшить характеристики процесса вставки данных в ClickHouse.

Метрика

До оптимизаций

После 1 этапа

После 2 этапа

Средний размер одной вставки (строк)

~4–21

~1 700–2 800

~19 000–23 500

Количество INSERT-запросов за минуту

~360–488

~8–16

1

Общий объём вставляемых данных (строк/мин)

~1 500–9 300

~14 000–45 000

19 000–23 500

В качестве наглядного примера эффективности предложенных оптимизаций мы рассмотрели одну из таблиц, регулярно загружаемых из Kafka в ClickHouse. Изначально процесс вставки данных характеризовался большим числом мелких транзакций, что существенно замедляло работу базы. Благодаря двухэтапной оптимизации мы смогли сократить количество запросов более чем в сотню раз, перейдя от нескольких сотен запросов в минуту к единичным, что позволило максимально использовать возможности ClickHouse для пакетной вставки данных.

При этом средний размер одного батча вырос с нескольких десятков строк до нескольких десятков тысяч строк, что полностью соответствует рекомендациям по эффективной работе с ClickHouse. Данные изменения не только снизили нагрузку на СУБД, но и повысили стабильность и предсказуемость работы всей системы. Используя этот подход всего на одной из таблиц, мы на практике убедились, насколько существенно можно повысить производительность и снизить затраты ресурсов при правильном подборе конфигураций коннектора.

Финальные выводы после оптимизации всех проблемных таблиц кластера ClickHouse

По итогам работы над оптимизацией процесса загрузки и обработки данных в кластере ClickHouse мы добились существенного повышения производительности и эффективности системы, а также значительно улучшили показатели стабильности и масштабируемости.

Влияние на кластер:

Метрика

До оптимизации

После оптимизации

Выводы

Влияние на кластер

Среднее количество INSERT-запросов 

~2 500/сек

~150/сек

Снижение нагрузки за счёт уменьшения количества транзакций

Меньше нагрузка на сеть и CPU, оптимальнее тратятся ресурсы на слияние и сортировку

Средний размер батча (INSERT)

~300 строк

~3 000 строк

Повышение эффективности вставки и снижение накладных расходов

Лучшее использование ресурсов, выше стабильность

Пропускная способность вставки (rows/sec)

~1 000–3 000 строк/сек

~5 000–20 000 строк/сек

Увеличение пропускной способности при меньшем числе запросов

Устойчивость под нагрузкой, улучшение масштабируемости, меньше очереди вставки

Среднее время INSERT-запроса

50–120 мс

100–200 мс

Рост времени оправдан объёмом вставок, общая производительность выше

Быстрее общая загрузка данных при том же объёме

Количество compact-партиций

160K–27M

77K–380K

Упростилось управление партициями, снизилась нагрузка на диск

Меньше партиций — быстрее операции с данными

Количество wide-партиций

5–521

40–2091

Оптимизация структуры хранения и уменьшение слияний мелких частей

Более предсказуемая производительность

Коэффициент сжатия

0.24–0.30

0.10–0.38

В среднем эффективность сжатия улучшилась, но появился больший разброс значений

Улучшена производительность хранения и I/O

Среднее время SELECT-запроса

88–1400 мс

50–270 мс

Сокращение задержек, повышение отзывчивости системы

Повышение пользовательского опыта и интерактивности

Пропускная способность SELECT (rows/sec)

500K–3.6M строк/сек

4M–9.6M строк/сек

Существенное улучшение аналитической нагрузки

Обработка больших данных без деградации производительности

Эффективность mark_cache (Hit Rate)

~98,1%

~98,5%

Стабильная работа кэша при увеличенной нагрузке

Минимизация дисковых операций, рост скорости SELECT

Анализируя метрики производительности INSERT-запросов, мы отмечаем, что количество выполняемых запросов сократилось примерно на 94% в сравнении с предыдущей неделей, при этом средний объём одного запроса увеличился в 10 раз. Если ранее кластер регулярно сталкивался с высокой нагрузкой за счёт многочисленных мелких транзакций, то теперь данные записываются крупными пакетами, что позволило существенно снизить нагрузку на систему и повысить её пропускную способность в 5–10 раз. 

Количество insert-запросов в секунду (после оптимизации)
Количество insert-запросов в секунду (после оптимизации)

В результате двух этапов настроек коннектора суммарная нагрузка на ClickHouse резко уменьшилась. До 18 апреля 2025 года система обрабатывала порядка 2 000–3 500 insert-запросов в секунду — на графиках при этом наблюдались частые пики и даже кратковременные сбои из-за перегрузки. После второй оптимизации интенсивность вставок упала до менее чем 500 запросов в секунду. График нагрузки выровнялся и стал ровным, без резких всплесков, что свидетельствует о стабильной работе системы после внедрения этих решений.

При этом, несмотря на рост объёма каждого отдельного INSERT-запроса и увеличение времени выполнения, общая скорость обработки данных возросла многократно, а задержки вставки полностью исчезли (300 отложенных и 53 отклоненных запросов на неделе до оптимизации). Еще одно важное следствие – улучшение работы с партициями: общее количество используемых партиций снизилось, хотя при этом возросла доля wide-партиций, что косвенно подтверждает эффективность применённых стратегий оптимизации и открывает возможности для дальнейших улучшений.

Анализ производительности SELECT-запросов также подтверждает успех оптимизаций. Несмотря на существенное увеличение объёма запросов (рост на 40% по количеству и 30–50% по объёму данных), время выполнения каждого запроса существенно сократилось (среднее время снизилось с 88–1 400 мс до 50–270 мс после оптимизации). Пропускная способность SELECT-запросов возросла в несколько раз, достигая рекордных значений до 9,6 млн строк в секунду (против 3,6 млн ранее). Это позволило системе стабильно справляться с возросшей нагрузкой и продемонстрировало её высокую готовность к дальнейшему масштабированию.

Кроме того, высокий коэффициент использования кэша (mark_cache) остался стабильным, несмотря на рост нагрузки и увеличение количества обрабатываемых партиций, что подтверждает эффективность работы ClickHouse с оптимизированными конфигурациями.

К каким выводам и рекомендациям мы пришли:

  • Рост эффективности вставок. По данным метрик средняя скорость INSERT-запросов увеличилась более чем в 2 раза, а частота их выполнения сократилась в 10 раз, что существенно разгрузило кластер и повысило пропускную способность.

  • Ускорение операций чтения. Время отклика SELECT-запросов снизилось до 50–270 мс, а пропускная способность достигла рекордных 9,6 млн строк/с, что подтвердило возросшую отзывчивость и устойчивость системы.

  • Повышение стабильности и масштабируемости. Новая стратегия позволила избежать задержек и отказов при обработке данных даже в часы пиковой нагрузки.

В целом результаты подтверждают, что проведённые оптимизации дали значительный и стабильный эффект, позволили устранить имевшиеся проблемы с производительностью и открывают широкие перспективы для дальнейшего масштабирования и развития платформы на основе ClickHouse.

Комментарии (0)