이 글은 CDC 파이프라인 시리즈의 후속편입니다. 이전 글에서 Debezium과 Flink로 CDC 파이프라인을 재설계하고, CDC 기반 Incremental Replication으로 적재 방식을 혁신한 과정을 다뤘습니다. 이번에는 그 이후의 이야기 — “구축한 뒤, 실제로 어떻게 운영하고 있는가“에 대해 공유합니다.
1. 운영의 시작: 구축과 운영은 다른 문제다
CDC 기반 Iceberg 파이프라인을 프로덕션에 올리고 안정적으로 운영하면서, 테이블 수와 운영 기간이 늘어남에 따라 새로운 과제들이 드러나기 시작했습니다.
현재 우리의 Iceberg 테이블 운영 규모는 다음과 같습니다:
10개 데이터베이스, 약 350개의 Iceberg 테이블
시간 단위(hourly) Incremental Merge — Airflow DAG 기반 자동화
Debezium → Kafka → Flink → S3(CDC 로그) → Spark(EMR)를 거쳐 Iceberg에 최종 적재
파이프라인 자체는 안정적으로 동작했지만, 작업 실패 시 어디까지 처리되었는지 추적할 수 없었고, 스냅샷은 끝없이 쌓여갔으며, 파일은 점점 더 잘게 쪼개져 성능을 잠식했습니다.
이 문제들을 이해하고 해결하기 위해서는, 먼저 Iceberg가 내부적으로 데이터를 어떻게 관리하는지를 알아야 합니다.
2. Apache Iceberg란 무엇인가
데이터 레이크의 한계, 그리고 테이블 포맷의 등장
데이터 레이크는 S3 같은 오브젝트 스토리지에 데이터를 저렴하게 보관할 수 있다는 장점이 있지만, 전통적인 Hive 테이블 포맷에는 근본적인 한계가 있었습니다:
트랜잭션 미지원: 쓰기 도중 읽기가 발생하면 불완전한 데이터를 볼 수 있습니다. 동시 쓰기 간의 충돌도 보장되지 않습니다.
비효율적인 데이터 변경: UPDATE나 DELETE를 하려면 전체 파티션을 다시 써야 합니다. CDC처럼 빈번한 변경이 발생하는 워크로드에는 부적합합니다.
디렉토리 기반 파티셔닝의 제약: Hive는 파티션을 디렉토리 경로(
dt=2026-01-01/)로 관리합니다. 파티션 구조를 바꾸려면 테이블 전체를 재생성해야 하고, 쿼리 시 파티션 필터를 명시하지 않으면 전체 스캔이 발생합니다.
Apache Iceberg는 이러한 한계를 해결하기 위해 Netflix에서 설계한 오픈 테이블 포맷(Open Table Format)입니다. 데이터 파일 자체는 기존과 동일한 Parquet나 ORC를 사용하지만, 그 위에 독자적인 메타데이터 계층을 두어 ACID 트랜잭션, 스키마 진화, 파티션 진화, 타임 트래블 같은 기능을 제공합니다.
왜 Iceberg를 선택했는가
우리의 워크로드는 약 350개의 MySQL 테이블에서 CDC 이벤트를 수집하여, 시간 단위로 데이터 레이크에 MERGE하는 것입니다. Iceberg는 MERGE INTO를 통한 원자적 upsert, 스냅샷 기반 ACID 트랜잭션, 스키마 진화, 타임 트래블 등 이 워크로드에 필요한 핵심 기능을 제공합니다. 도입 배경은 이전 글 CDC가 데이터 플랫폼을 바꾸는 방식에서 자세히 다루고 있습니다.
하지만 이 모든 기능은 공짜가 아닙니다. Iceberg의 강력한 기능들은 내부적으로 정교한 메타데이터 구조에 의존하며, 이 구조를 이해하지 않으면 운영 과정에서 예상치 못한 문제에 부딪힙니다.
메타데이터 계층 구조
Iceberg는 데이터를 직접 관리하지 않습니다. 대신, 메타데이터의 계층 구조를 통해 데이터 파일을 추적합니다. 이 구조를 이해하면, 이후에 다룰 스냅샷 폭증, Small File, 메인터넌스의 모든 문제가 자연스럽게 연결됩니다.
Iceberg Catalog (테이블 → current metadata pointer)
└── Metadata File (테이블의 진입점, 스냅샷 목록 포함)
└── Snapshot (테이블의 특정 시점 상태)
└── Manifest List (해당 스냅샷의 모든 매니페스트 목록)
└── Manifest File (데이터 파일들의 메타데이터)
└── Data File (실제 Parquet/ORC 파일)Iceberg Catalog: 테이블 이름(
db.table)을 현재 메타데이터 파일의 위치로 매핑합니다. Glue Catalog, Hive Metastore, REST Catalog 등 다양한 구현이 가능합니다.Metadata File: 테이블의 진입점. 현재 스냅샷이 무엇인지, 스키마는 어떤지, 파티션 스펙은 무엇인지를 기록합니다. 쓰기가 발생하면 새로운 메타데이터 파일이 생성되고, Catalog의 포인터가 갱신됩니다.
Snapshot: 테이블의 특정 시점 상태를 나타내는 포인터입니다. 데이터를 복제하는 것이 아니라, “이 시점에서는 이 매니페스트 리스트를 참조하라“는 참조 정보만 담고 있습니다.
Manifest List: 하나의 스냅샷에 속하는 매니페스트 파일들의 목록입니다. 각 매니페스트가 어떤 파티션 범위를 커버하는지 요약 정보를 포함하여, 쿼리 시 불필요한 매니페스트를 건너뛸 수 있게 합니다.
Manifest File: 실제 데이터 파일의 경로, 파티션 값, 레코드 수, 컬럼 통계(min/max) 등을 기록합니다. 쿼리 엔진은 이 통계를 활용하여 읽을 필요 없는 데이터 파일을 빠르게 스킵합니다.
Data File: 실제 데이터가 저장된 Parquet(또는 ORC) 파일입니다. S3 같은 오브젝트 스토리지에 위치합니다.
스냅샷과 데이터 파일의 관계
Iceberg는 데이터 변경을 처리하는 두 가지 방식을 제공합니다. Copy-on-Write(CoW)는 변경 시 영향받는 파일 전체를 새로 작성하여 읽기 성능이 최적이지만, 상대적으로 쓰기 비용이 높습니다. Merge-on-Read(MoR)는 변경 사항을 별도의 delete file로 기록하여 쓰기를 빠르게 하되, 읽기 시 원본과 병합해야 하므로 읽기 오버헤드가 발생합니다. 우리는 시간 단위 배치로 쓰기 빈도가 낮고, 분석 쿼리의 읽기 성능이 중요하므로 CoW를 선택했습니다.
CoW 방식에서는 데이터가 변경되면 기존 파일을 수정하지 않고, 영향 받는 파티션의 데이터를 새로운 파일로 다시 작성합니다.
Snapshot S1: [파일 A, 파일 B, 파일 C] ↓ 파일 B에 해당하는 파티션에 UPDATE 발생 Snapshot S2: [파일 A, 파일 B', 파일 C] ← 파일 B'는 새로 작성된 파일여기서 중요한 점은, S2가 생성되어도 S1은 여전히 파일 B를 참조하고 있다는 것입니다. 파일 B는 S1이 존재하는 한 삭제할 수 없습니다. 이것이 Iceberg의 타임 트래블(Time Travel) — 과거 스냅샷 ID로 그 시점의 데이터를 그대로 조회할 수 있는 기능 — 의 원리이자, 동시에 스토리지가 누적되는 원인입니다.
Tag와 Branch: 스냅샷의 수명을 제어하는 장치
Iceberg는 스냅샷의 보존 기간을 제어하기 위해 Tag와 Branch를 제공합니다.
Tag: 특정 스냅샷에 이름표를 붙여 보호합니다.
expire_snapshots를 실행해도 태그가 달린 스냅샷은 삭제되지 않습니다. 보존 기간(RETAIN)을 설정하면, 기간이 지난 후 자동으로 태그가 만료되어 다음 expire 대상이 됩니다.Branch: Git의 브랜치와 유사하게, 독립적인 스냅샷 히스토리를 유지합니다. WAP(Write-Audit-Publish) 패턴이나 테스트용 쓰기에 활용됩니다.
우리의 운영에서는 Tag가 핵심적인 역할을 합니다. 뒤에서 자세히 다루겠지만, 하루의 마지막 스냅샷에 EOD 태그를 달아 일간 기준점으로 보존하고, 나머지 hourly 스냅샷은 일정 기간 후 정리하는 방식으로 메인터넌스 정책을 설계했습니다.
메인터넌스가 필요한 이유
이 구조를 이해하면, 왜 Iceberg 테이블에 메인터넌스가 필수적인지 명확해집니다:
스냅샷 누적 → 스토리지 폭증: 매 MERGE마다 새 스냅샷이 생성되고, 이전 스냅샷이 참조하는 데이터 파일은 삭제되지 않습니다. 정리하지 않으면 과거 데이터가 무한히 쌓입니다.
Small File 문제 → 쿼리 성능 저하: 작은 변경이 반복되면서 데이터 파일이 잘게 쪼개지면, 매니페스트가 추적해야 할 파일 수가 급증합니다. 매니페스트 로드 자체가 느려지고, S3 API 호출 수가 폭증합니다.
고아 파일(Orphan Files): 쓰기 도중 실패하거나 메타데이터 커밋이 완료되지 않으면, 데이터 파일은 S3에 생성되었지만 어떤 스냅샷의 매니페스트에도 등록되지 않은 상태로 남습니다.
expire_snapshots는 매니페스트를 기반으로 동작하므로, 애초에 매니페스트에 등록되지 않은 이런 파일은 감지하지 못합니다.
Iceberg는 이를 위해
expire_snapshots,remove_orphan_files,rewrite_data_files같은 메인터넌스 프로시저를 제공하지만, 언제, 어떤 기준으로, 어떤 순서로 실행할지는 운영자의 몫입니다.3. 작업 이력 관리 체계: 운영을 관측 가능하게 만들기
문제: 보이지 않는 파이프라인
350개 테이블을 시간 단위로 머지하는 파이프라인은, 정상적으로 동작하는 한 아무 문제가 없습니다. 하지만 문제는 장애가 발생했을 때 나타납니다.
기존에도 머지 결과를 Slack으로 보내고 있었습니다. 성공/실패 여부는 확인할 수 있었지만, 며칠 전의 특정 테이블 상태를 추적하려면 슬랙 메시지를 뒤져야 했고, 스냅샷 ID 같은 정보는 아예 남기지 않았습니다. 운영에 필요한 이력이 메신저에 흩어져 있으니 사실상 휘발되는 것과 같았습니다.
특정 테이블의 머지가 실패했을 때, 마지막으로 성공한 시점을 슬랙을 뒤져서 찾아야 했습니다
Iceberg 스냅샷을 롤백해야 하는 상황에서, 정확히 어떤 스냅샷으로 돌아가야 하는지 판단할 근거가 없었습니다
"이 테이블이 지금 최신 상태인가?"라는 단순한 질문에도 확신 있게 답할 수 없었습니다
파이프라인의 실행 이력을 시스템적으로 조회하고 활용할 수 있는 장치가 필요했습니다.
해결: 작업 이력 테이블 구축
Iceberg는 자체적으로
table.snapshots메타데이터 테이블을 제공하므로, 스냅샷 이력 자체는 조회할 수 있습니다. 하지만 이 메타데이터에는 비즈니스 맥락이 없습니다 — 어떤 배치(run_dt/run_hour)의 결과인지, CDC를 몇 건 처리했는지, 정합성 검증을 통과했는지 알 수 없습니다. 또한 350개 테이블의 상태를 한눈에 보려면 각 테이블의.snapshots를 개별로 조회해야 합니다.Iceberg가 제공하는 메타데이터 테이블 예시 이 한계를 해결하기 위해, 매 Incremental Merge 실행마다 작업 이력을 기록하는 별도의 히스토리 테이블을 구축했습니다.
이력 테이블은 S3에 Parquet 포맷으로 저장되며, run_dt(실행일)와
db(데이터베이스)로 파티셔닝됩니다. 주요 컬럼은 다음과 같습니다:실행 정보:
table_name,status,started_at,finished_at,auto_retry_count
CDC 통계:
cdc_insert_count,cdc_update_count,cdc_delete_count,cdc_total_count— 해당 배치의 변경 이벤트를 유형별로 기록
정합성 검증:
pk_unique_check_passed— 머지 후 PK 중복 여부 검사cdc_consistency_check_passed— CDC 순변동량cdc_net_expected)과 Iceberg 실제 변동량cdc_net_actual) 비교
Iceberg 스냅샷 메타데이터:
iceberg_snapshot_id— 머지 결과 스냅샷 ID (롤백 시 이 값으로 복원)iceberg_operation— 머지 방식 (overwrite / append)iceberg_total_records,iceberg_total_data_files— 머지 후 테이블 전체 레코드 수와 파일 수iceberg_added_records,iceberg_deleted_records— 해당 배치의 레코드 변화량iceberg_changed_partition_count— 영향받은 파티션 수
에러 정보:
error_type,error_message— 실패 시 에러 유형과 메시지
운영 중인 작업 이력 테이블 예시 핵심은 Iceberg 스냅샷 ID를 실행 시점과 매핑한다는 점입니다. 앞서 설명한 것처럼 Iceberg의 스냅샷은 테이블의 특정 시점 상태를 나타내는 포인터입니다. 이 스냅샷 ID를 작업 이력과 연결함으로써:
장애 추적: 어떤 테이블이 언제 마지막으로 성공했는지 즉시 확인
정확한 롤백: 데이터 이슈 시, history 테이블에서 대상 스냅샷 ID를 조회하여 정확한 시점으로 복원
데이터 정합성 검증: PK 유니크 체크와 CDC 정합성 체크를 매 배치마다 자동 수행하여, 문제가 있으면 즉시 감지
이 히스토리 테이블은 단순한 로그가 아니라, 배치 실행 결과와 Iceberg 스냅샷을 연결해 재시도, 롤백, 정합성 검증을 안전하게 만드는 운영 장치로 활용됩니다.
이력 관리를 통한 재시도의 안전성
히스토리 테이블 도입 이후 가장 크게 달라진 것은 재시도의 안전성입니다.
350개 테이블을 시간 단위로 머지하다 보면, 네트워크 이슈나 일시적인 리소스 부족으로 일부 테이블의 작업이 실패하는 경우가 종종 있습니다. 이때 전체 DAG를 재실행하면, 이미 성공한 테이블까지 다시 머지를 수행하게 됩니다. Iceberg의 CoW 방식에서 불필요한 머지는 새로운 스냅샷과 데이터 파일을 또 생성하므로, 리소스 낭비이자 Small File 문제를 가속시키는 원인이 됩니다.
히스토리 테이블이 있으면 이 문제가 깔끔하게 해결됩니다:
재시도 흐름:
1. DAG 재실행
2. 테이블별로 history 조회 → 해당 run_dt/run_hour에 성공 이력이 있는가?
3. 이미 성공 → 스킵 (불필요한 스냅샷/파일 생성 방지)
4. 실패 또는 이력 없음 → 머지 실행Before: 실패 시 전체 재실행 → 성공한 테이블도 다시 머지 → 불필요한 스냅샷 생성 + 시간 낭비
After: 실패한 테이블만 선별적으로 재처리 → 안전하고 빠른 복구
이 덕분에 운영자가 장애 상황에서 “어디까지 됐지?“를 고민하지 않고 안심하고 재실행 버튼을 누를 수 있게 되었습니다. 히스토리 테이블은 단순한 로그가 아니라, 파이프라인의 멱등성(Idempotency)을 보장하는 장치입니다.
4. 스냅샷 폭증과 메인터넌스: 500TB의 현실
문제: 정책 없이 6개월을 운영하면 벌어지는 일
파이프라인을 처음 프로덕션에 올린 후 약 6개월간은 테이블 커버리지 확대와 파이프라인 안정화에 집중했습니다. 메인터넌스 정책은 안정화 이후의 과제로 계획하고 있었지만, 그 사이에 스냅샷은 빠르게 누적되고 있었습니다.
앞서 설명한 것처럼 Iceberg는 매 쓰기마다 새로운 스냅샷을 생성하고, 이전 스냅샷이 참조하는 데이터 파일은 자동으로 삭제되지 않습니다.
350개 테이블에 시간 단위로 머지가 실행되면, 하루에 약 8,400개(350 × 24)의 스냅샷이 생성됩니다. 별도의 정리 정책 없이 6개월을 운영하자, 상황은 다음과 같았습니다:
항목 | 수치 |
|---|---|
총 스냅샷 | 798,873개 |
테이블당 평균 | 2,420개 |
최대 (단일 테이블) | 4,250개 |
총 스토리지 | ~500TB |
가장 오래된 스냅샷 | 2025-10-13 (~6개월 전) |
각 스냅샷이 참조하는 과거 데이터 파일이 S3에 그대로 보존되어 있었고, 이는 곧 스토리지 비용으로 직결되었습니다. 특히 대형 테이블이 밀집한 데이터베이스의 누적은 심각했습니다:
DB | 스냅샷 수 | 스토리지 |
|---|---|---|
user | 135,716 | 255.8 TB |
loan | 62,124 | 88.3 TB |
lending | 355,026 | 64.4 TB |
payment | 136,691 | 43.8 TB |
expire_snapshots의 동작 원리
단순히 “스냅샷을 삭제한다“고 하면, 실제로 어떤 일이 일어나는지 이해하기 어렵습니다. expire_snapshots는 다음과 같은 과정을 거칩니다:
만료 대상 스냅샷 선별: 지정된 조건(시간, retain 수)에 따라 삭제할 스냅샷을 결정합니다. 단, Tag가 달린 스냅샷은 보호됩니다.
매니페스트 리스트 비교: 만료되는 스냅샷의 매니페스트 리스트와, 살아남는 스냅샷들의 매니페스트 리스트를 비교합니다.
독점 참조 파일 삭제: 만료되는 스냅샷만이 참조하는 데이터 파일 — 즉, 다른 어떤 스냅샷에서도 사용되지 않는 파일 — 을 S3에서 실제로 삭제합니다.
메타데이터 정리: 해당 스냅샷의 매니페스트 리스트와 매니페스트 파일도 정리합니다.
Snapshot S1: [파일 A, 파일 B, 파일 C] ← 만료 대상
Snapshot S2: [파일 A, 파일 B', 파일 C] ← 유지
Snapshot S3: [파일 A, 파일 B', 파일 C'] ← 유지 (현재)
→ S1 만료 시:
- 파일 A: S2, S3에서도 참조 → 삭제하지 않음
- 파일 B: S1만 참조 → S3에서 삭제
- 파일 C: S2에서도 참조 → 삭제하지 않음이 과정에서 모든 살아남는 스냅샷의 매니페스트를 스캔해야 하므로, 스냅샷이 오래 누적될수록 메타데이터 로드에 필요한 메모리와 시간이 늘어납니다. 메인터넌스 정책을 초기에 설정해야 하는 이유이기도 합니다.
해결: EOD 태그 기반 메인터넌스 정책
단순히 “N일 이전 스냅샷을 모두 삭제“하는 방식은 우리 상황에 맞지 않았습니다. Incremental Merge는 시간 단위로 실행되지만, 하루의 마지막 배치(run_hour=23)는 일간 정합성 검증의 기준점이 됩니다. 이 스냅샷은 hourly 스냅샷보다 오래 보존해야 했습니다.
이 문제를 해결하기 위해 앞서 소개한 Iceberg의 Tag 기능을 활용했습니다:
메인터넌스 정책 설계:
EOD 태그 생성: 매일
run_hour=23의 머지가 성공하면, 해당 스냅샷에eod_{run_dt}태그를 부여하고 14일간 보호 (RETAIN 14 DAYS). 14일은 EOD 스냅샷 기반 타임 트래블을 제공하되, 아직 활발히 사용되지 않는 단계이므로 보수적으로 설정한 값입니다. 사용 패턴이 확인되면 조정할 계획입니다.Hourly 스냅샷 정리:
expire_snapshots로 3일 이전의 일반 스냅샷을 삭제 — 태그가 달린 EOD 스냅샷은expire_snapshots의 삭제 대상에서 자동으로 제외. 3일은 데이터 이슈를 인지하고 원인을 파악하여 롤백을 결정하기까지의 대응 시간을 확보하기 위한 기준입니다.고아 파일 정리:
remove_orphan_files로 어떤 스냅샷에도 참조되지 않는 데이터 파일을 삭제합니다. — 쓰기 중 실패로 남은 잔여 파일 정리
테이블당 보존 스냅샷:
├── 최근 3일: 모든 hourly 스냅샷 (~72개)
├── 3일 ~ 14일: EOD 태그된 daily 스냅샷만 (~11개)
└── 14일 이후: 태그 자동 만료 → 다음 expire 시 정리CDC가 없는 시간대(skip-empty)에도 현재 스냅샷에 EOD 태그를 생성하여, 일간 기준점이 빠지지 않도록 했습니다. 안전장치로 retain_last=1을 추가하여, 태그가 없는 테이블이라도 최소 하나의 스냅샷은 반드시 보존합니다.
Airflow 통합과 스케줄링
메인터넌스는 기존 hourly merge DAG에 통합했습니다. 매일 run_hour=23 배치가 완료된 후, 후속 파이프라인과 병렬로 메인터넌스가 실행됩니다.
초기 실행 시 유의점
메인터넌스를 처음 도입하는 시점에는 누적된 스냅샷이 많으므로, expire_snapshots가 스캔해야 할 매니페스트 양이 평소보다 훨씬 많습니다. 우리의 경우 초기 실행에 약 7시간이 소요되었습니다. 이후에는 하루치(테이블당 ~24개)만 정리하므로 빠르게 완료됩니다. 메인터넌스 정책은 가능한 한 초기부터 설정하는 것이 운영 부담을 줄이는 가장 좋은 방법입니다.
스냅샷 메인터넌스 결과
항목 | Before | After | 변화 |
|---|---|---|---|
총 스냅샷 | 798,873 | 16,031 | 98.0% 삭감 |
총 스토리지 | ~500TB | ~50TB | ~450TB 절감 |
월 스토리지 비용 | — | — | ~$10,700/mo 절감 |
DB별 감소율도 97~98%로 일관적이었습니다:
DB | Before | After | 변화 |
|---|---|---|---|
lending | 355,026 | 6,822 | 98.1% |
payment | 136,691 | 2,883 | 97.9% |
user | 135,716 | 2,720 | 98.0% |
loan | 62,124 | 1,148 | 98.2% |
lead | 52,278 | 1,432 | 97.3% |
wallet | 32,467 | 575 | 98.2% |
5. Small File 문제: 98,659개의 파일이 만든 타임아웃
문제: 파일이 너무 많다
스냅샷 메인터넌스로 스토리지를 정리한 뒤에도, 또 다른 성능 문제가 남아 있었습니다. MERGE INTO 작업 중 S3 API 타임아웃이 간헐적으로 발생하고 있었고, 원인을 추적해보니 Small File 문제였습니다.
가장 심각한 테이블인 user_tag의 상태:
항목 | 수치 |
|---|---|
총 파일 수 | 98,659개 |
총 크기 | 38GB |
평균 파일 크기 | 0.4MB |
파티션 수 | 3,111개 |
파티션당 평균 파일 수 | ~32개 |
Iceberg의 이상적인 파일 크기가 128MB~512MB인 것을 감안하면, 0.4MB는 300배 이상 작은 수치입니다. 매니페스트 파일이 98,659개의 데이터 파일을 모두 추적해야 하므로, 쿼리 플래닝 단계에서 이미 상당한 오버헤드가 발생합니다. 여기에 MERGE INTO가 영향받는 파티션의 모든 파일을 읽어야 하므로, S3 GET 요청이 수만 건에 달하며 타임아웃으로 이어졌습니다.
원인: write.distribution-mode와 Spark의 쓰기 동작
이 문제를 이해하려면, Iceberg 테이블에 데이터를 쓸 때 Spark 내부에서 어떤 일이 벌어지는지를 알아야 합니다.
Iceberg의 write.distribution-mode는 쓰기 전에 데이터를 어떻게 분배할 것인가를 결정하는 속성입니다:
none(기본값): 데이터를 셔플하지 않습니다. 각 Spark executor(정확히는 task)가 받은 데이터를 그대로 파일로 기록합니다. 파티션이 여러 executor에 분산되면, 같은 파티션에 여러 개의 작은 파일이 생성됩니다.hash: 파티션 키의 해시값으로 데이터를 셔플합니다. 같은 파티션의 모든 레코드가 하나의 writer로 모이므로, 파티션당 파일 수가 최소화됩니다.range: 정렬 기반으로 데이터를 분배합니다. 정렬 순서가 중요한 경우(예: Z-order) 유용하지만, 셔플 비용이 가장 높습니다.
우리의 상황은 전형적인 none 모드의 부작용이었습니다:
distribution-mode: none
├── Executor 1 → 파일 A (0.3MB) ← 파티션 X의 일부
├── Executor 2 → 파일 B (0.5MB) ← 파티션 X의 일부
├── Executor 3 → 파일 C (0.2MB) ← 파티션 X의 일부
└── ... × 3,111 파티션 × 24시간 × 6개월 = 98,659개
distribution-mode: hash
└── Executor 1 → 파일 A (1.0MB) ← 파티션 X의 전체 → 파일 1개Incremental Merge는 시간 단위로 실행되므로 각 배치의 변경량이 적은데, none 모드에서 여러 executor가 분산 처리하면서 매우 작은 파일이 대량으로 생성된 것입니다.
해결: hash 분배와 타겟 파일 크기
모든 Iceberg 테이블에 두 가지 속성을 적용했습니다:
ALTER TABLE db.table SET TBLPROPERTIES (
'write.distribution-mode' = 'hash',
'write.target-file-size-bytes' = '134217728' -- 128MB
);write.distribution-mode = hash: 파티션 키 기준으로 셔플 후 쓰기. 같은 파티션의 데이터가 하나의 writer로 모이면서 파일 수가 크게 줄어듭니다.write.target-file-size-bytes = 128MB: writer가 이 크기에 도달하면 새 파일을 시작합니다. 대형 파티션에서 과도하게 큰 단일 파일이 생성되는 것을 방지합니다.
셔플 비용이 추가되지만, hourly 배치의 데이터 크기가 작으므로 (테이블당 수 MB~수십 MB) 오버헤드는 무시할 수 있는 수준이었습니다.
적용 범위:
기존 350개 테이블: SSH를 통해
spark-sql로 ALTER TABLE 일괄 실행신규 테이블: 환경별 설정 파일의
table_properties에 추가하여 CREATE TABLE 시 자동 적용
write.distribution-mode는 테이블 속성입니다. ALTER TABLE로 변경하면 이후의 모든 쓰기에 즉시 적용되며, 기존 데이터 파일에는 영향이 없습니다. 과거 파티션의 작은 파일들은 별도의 Compaction(rewrite_data_files)으로 정리해야 합니다.
write.distribution-mode 적용 결과
user_tag 기준 검증 결과:
파티션 시점 | 파일 수 | 평균 크기 |
|---|---|---|
과거 (2020~2021) | 500 ~ 3,558개 | ~0.4MB |
적용 후 (2026-05~) | 1~2개 | ~128MB |
신규 데이터는 의도한 대로 소수의 적정 크기 파일로 기록되고 있습니다. 하지만 과거 파티션의 약 89,000개 파일은 여전히 그대로입니다. write.distribution-mode는 이후의 쓰기에만 영향을 주기 때문입니다. 이미 생성된 파일들을 정리하려면 별도의 Compaction이 필요합니다.
Compaction: 기존 파일 정리
Iceberg는 이를 위해 rewrite_data_files 프로시저를 제공합니다. 이 프로시저는 기존의 작은 파일들을 읽어 타겟 크기에 맞는 새 파일로 다시 작성하고, 매니페스트를 갱신합니다.
CALL system.rewrite_data_files(
table => 'db.user_tag',
options => map(
'target-file-size-bytes', '134217728', -- 128MB
'min-file-size-bytes', '104857600', -- 100MB (이미 적정 크기인 파일은 건너뜀)
'max-file-size-bytes', '180355072' -- 168MB
)
);동작 원리를 좀 더 자세히 들여다보면:
대상 파일 선별: 매니페스트를 스캔하여
min-file-size-bytes보다 작은 파일들을 Compaction 대상으로 선별합니다. 이미 충분히 큰 파일은 건너뜁니다.파일 재작성: 대상 파일들을 읽어
target-file-size-bytes기준으로 새 파일을 작성합니다. 이 과정에서 같은 파티션의 여러 작은 파일이 하나의 큰 파일로 합쳐집니다.메타데이터 갱신: 새로운 스냅샷이 생성되며, 매니페스트가 새 파일을 참조하도록 갱신됩니다. 원본 파일은 이전 스냅샷에서만 참조되므로, 다음
expire_snapshots실행 시 삭제됩니다.
여기서 주목할 점은 Compaction과 Snapshot Maintenance의 연계입니다. rewrite_data_files는 새 스냅샷을 만들 뿐, 원본 파일을 바로 삭제하지는 않습니다. 원본 파일이 실제로 S3에서 삭제되려면, 이전 스냅샷이 expire되어야 합니다. 우리의 메인터넌스 정책(3일 이전 스냅샷 expire)이 이 역할을 자동으로 수행합니다.
Compaction 실행 (Day 0)
├── 새 스냅샷 생성: [기존 작은 파일 100개 → 새 큰 파일 1개]
├── 이전 스냅샷은 여전히 작은 파일 100개 참조 → S3에서 아직 삭제 안 됨
│
expire_snapshots 실행 (Day 3)
├── 이전 스냅샷 만료 → 작은 파일 100개는 독점 참조 상실
└── S3에서 실제 삭제Compaction과 메인터넌스가 별개로 보이지만, 실제로는 하나의 사이클로 동작하는 것입니다.
Compaction 실행과 결과
Compaction을 실행하는 과정에서 예상치 못한 문제를 만났습니다. 대형 테이블에서 rewrite_data_files를 실행하자 S3 타임아웃이 발생했고, Iceberg의 S3 timeout 관련 설정을 조정했지만 효과가 없었습니다. 추적해보니 해당 설정들은 Iceberg가 실제로 property로 노출하지 않아 적용 자체가 되지 않는 것이었습니다.
근본 원인은 EMR 7.13에 포함된 Iceberg 1.10.0의 S3 HTTP connection pool 재사용 버그였습니다. 매 요청마다 새 연결을 생성하면서 연결이 고갈된 것이었고, 1.10.1 jar로 교체하여 해결했습니다.
먼저 가장 심각했던 user_tag 테이블에 Compaction을 실행했습니다:
rewritten_data_files_count: 89,314
added_data_files_count: 1,536
rewritten_bytes_count: 14.6 GB
failed_data_files_count: 089,314개의 파일이 1,536개로 — 98% 감소했습니다. Compaction 이후 MERGE INTO 시 발생하던 S3 API 타임아웃이 해소되었고, Athena 등 분석 쿼리에서도 읽어야 할 파일 수가 대폭 줄어 쿼리 플래닝과 스캔 효율이 개선되었습니다.
이후 전체 350개 테이블에 대해 일괄 Compaction을 실행했습니다. 350개 테이블 전부 성공, 실패 0건. 총 648,789개의 파일이 30,354개로 — 95% 감소했습니다. write.distribution-mode=hash가 이미 적용되어 있으므로 앞으로는 Small File이 쌓이지 않습니다. 이번 Compaction은 과거 데이터를 정리하는 1회성 작업이었습니다.
성능 개선 효과
Compaction과 write.distribution-mode=hash 적용 후, 시간당 Incremental Merge 배치의 DB별 소요시간을 비교했습니다. 같은 시기에 EMR 클러스터 인프라 최적화(전용 클러스터 분리, Dynamic Allocation, Managed Scaling)도 함께 진행했기 때문에, 아래 수치는 Compaction과 인프라 최적화의 복합 효과입니다.
DB | Before (분) | After (분) | 개선 |
|---|---|---|---|
lending (138 tables) | 17.3 | 9.7 | -43% |
user (46 tables) | 15.4 | 9.2 | -40% |
payment (81 tables) | 14.0 | 8.1 | -42% |
loan (25 tables) | 8.8 | 7.0 | -21% |
wallet (11 tables) | 8.8 | 5.2 | -41% |
전체 평균 약 38%의 성능 개선이 이루어졌습니다. 이 중 인프라 최적화가 약 26%p, Compaction과 hash mode가 약 16%p를 기여한 것으로 분석됩니다. 특히 Small File이 심각했던 대형 DB에서 40% 이상의 소요시간이 단축되었고, S3 API 타임아웃으로 간헐 실패하던 테이블들이 안정적으로 처리되기 시작했습니다.
우리의 Small File 전략을 정리하면:
예방: write.distribution-mode=hash로 신규 Small File 생성을 원천 차단
치료:
rewrite_data_files로 기존 Small File을 1회성 Compaction정리: 기존 메인터넌스 정책(
expire_snapshots)이 Compaction 이전 스냅샷과 원본 파일을 자동 삭제
6. 마치며
Iceberg는 ACID 트랜잭션, MERGE INTO, 타임 트래블 같은 강력한 기능을 제공하지만, 운영 관점에서 보면 이 기능들은 모두 메타데이터와 데이터 파일의 누적으로 이어집니다. 매시간 실행되는 Incremental Merge는 새로운 스냅샷을 만들고, 변경된 데이터 파일을 다시 쓰며, 이 과정에서 과거 스냅샷과 Small File이 계속 쌓입니다.
처음 Iceberg를 도입할 때는 이 구조가 잘 보이지 않았습니다. 파이프라인이 정상적으로 동작하는 동안에는 문제가 드러나지 않았지만, 테이블 수가 350개까지 늘어나고 운영 기간이 길어지자 스냅샷 폭증, 스토리지 비용 증가, Small File로 인한 S3 API 타임아웃이 실제 운영 문제로 나타났습니다.
이번 운영을 통해 가장 크게 배운 점은, Iceberg 테이블은 한 번 만들어두면 끝나는 저장소가 아니라는 것입니다. 스냅샷을 얼마나 보존할지, 어떤 시점을 태그로 보호할지, 고아 파일은 언제 정리할지, Small File은 어떻게 예방하고 기존 파일은 언제 Compaction할지까지 함께 설계해야 합니다.
우리는 EOD 태그 기반 스냅샷 정책으로 총 스냅샷을 798,873개에서 16,031개로 줄였고, 스토리지를 약 500TB에서 50TB 수준으로 낮췄습니다. 또한 write.distribution-mode=hash와 rewrite_data_files를 통해 Small File을 정리하고, 인프라 최적화와 함께 배치 성능을 약 38% 개선하면서, 간헐적으로 발생하던 S3 API 타임아웃도 해소했습니다.
작업 이력 테이블은 이 과정에서 중요한 역할을 했습니다. 단순히 로그를 남기는 용도가 아니라, 어떤 배치가 어떤 스냅샷을 만들었는지 추적하고, 실패 시 안전하게 재시도하며, 필요할 때 정확한 시점으로 롤백하기 위한 운영 기준점이 되었습니다. 다만 이력 관리는 목적이라기보다, 메인터넌스와 최적화를 안전하게 수행하기 위한 기반 장치에 가깝습니다.
결국 Iceberg 운영의 핵심은 기능을 사용하는 데서 끝나지 않고, 스냅샷과 데이터 파일의 생명주기를 워크로드에 맞게 지속적으로 관리하는 데 있습니다. 공식 문서가 제공하는 프로시저는 출발점일 뿐이고, 실제 운영에서는 각 조직의 데이터 변경 패턴, 롤백 요구사항, 비용 구조, 배치 주기에 맞는 정책이 필요합니다.
데이터 플랫폼은 한 번 구축하면 끝나는 시스템이 아니라, 운영하면서 계속 조정하고 개선해야 하는 살아있는 시스템입니다. 이 글이 Iceberg를 도입했거나 도입을 검토하는 팀에게, "도입 이후에 무엇을 관리해야 하는가"를 생각해보는 실질적인 참고가 되기를 바랍니다.