AthenaでIceberg本番運用6ヶ月、タイムトラベルとスキーマ進化で痛い目を見た話
「壊れたデータをロールバックしたい」その一心でIcebergに乗り換えたら、別の沼が待っていた。タイムトラベルやスキーマ進化で実際にハマった失敗談と、6ヶ月で得た知見をまとめます。
Icebergに乗り換えた経緯と最初の後悔
半年前、うちのチームが抱えていたS3データレイクの問題はシンプルだった。Hiveパーティション形式でS3に積み上げたParquetファイルが数TB規模になってきて、「昨日のバッチで壊れたデータをどうやってロールバックするか」という問いに誰も答えられなくなっていた。
最終的に選んだのがAthenaのApache Icebergサポートだった。Delta Lake・Iceberg・Hudi比較の記事でも触れているように、2026年時点ではAthena上のIcebergは相当成熟してきている。ただ、「成熟している」と「すんなり本番運用できる」は別の話で、最初の2週間はかなり痛い目を見た。
教科書的な手順より「これ事前に知りたかった」という話を優先したい。特にタイムトラベルクエリとスキーマ進化周りのハマりどころは、実際に動かして初めてわかることが多かった。
構成全体像:AthenaとIceberg周辺のアーキテクチャ
まず、うちのチームが組んだ全体的な構成を見てほしい。
graph TB
subgraph Sources["データソース"]
APP["アプリケーション DB\nAurora PostgreSQL"]
STREAM["ストリーミング\nKinesis Data Streams"]
BATCH["外部データ\nS3 Transfer"]
end
subgraph Ingestion["取り込み層"]
GLUE_ETL["AWS Glue ETL\nSpark Job"]
FIREHOSE["Kinesis Firehose\nDynamic Partitioning"]
end
subgraph Storage["Iceberg ストレージ層"]
subgraph S3_RAW["S3 Raw Zone"]
RAW["s3://bucket/raw/\nParquet"]
end
subgraph S3_ICE["S3 Iceberg Zone"]
ICE_META["メタデータ\n/metadata/*.json"]
ICE_DATA["データファイル\n/data/*.parquet"]
ICE_MANIFEST["マニフェスト\n/metadata/*.avro"]
end
end
subgraph Catalog["カタログ層"]
GLUE_CAT["AWS Glue\nData Catalog"]
LF["Lake Formation\n権限管理"]
end
subgraph Query["クエリ層"]
ATHENA["Amazon Athena\nEngine v3"]
subgraph Results["クエリ結果"]
S3_RESULT["S3 Result Bucket"]
end
end
subgraph Visualization["可視化・利用"]
QS["Amazon QuickSight"]
NOTEBOOK["SageMaker Studio\nNotebook"]
BI["Metabase\nセルフサービスBI"]
end
subgraph Ops["運用・監視"]
CW["CloudWatch\nメトリクス"]
COMPACTION["Glue Job\n定期Compaction"]
end
APP --> GLUE_ETL
STREAM --> FIREHOSE
BATCH --> GLUE_ETL
GLUE_ETL --> RAW
FIREHOSE --> RAW
RAW --> GLUE_ETL
GLUE_ETL --> ICE_DATA
GLUE_ETL --> ICE_META
ICE_META --> ICE_MANIFEST
ICE_DATA --> GLUE_CAT
ICE_META --> GLUE_CAT
GLUE_CAT --> LF
LF --> ATHENA
ATHENA --> S3_RESULT
S3_RESULT --> QS
S3_RESULT --> NOTEBOOK
S3_RESULT --> BI
ATHENA --> CW
COMPACTION --> ICE_DATA
COMPACTION --> ICE_META
ポイントは2点。まずGlue ETLがIcebergへの書き込みエンジンになっていること(Athena INSERT INTO でも書けるが、大量データはGlue Sparkの方が圧倒的に速い)。次にCompactionを定期Glue Jobとして分離していること。これをサボると後で痛い目を見る——実際にサボって痛い目を見たので後で詳しく書く。
Icebergテーブルの作成と基本操作:最初に理解しておくべきこと
まずテーブル作成から。2026年時点のAthena Engine v3では、CREATE TABLEにtable_type = 'ICEBERG'を指定するだけで使える。
-- Icebergテーブルの作成
CREATE TABLE events_iceberg (
event_id STRING,
user_id STRING,
event_type STRING,
event_time TIMESTAMP,
properties MAP<STRING, STRING>,
created_at TIMESTAMP
)
PARTITIONED BY (year(event_time), month(event_time))
LOCATION 's3://my-datalake-bucket/iceberg/events/'
TBLPROPERTIES (
'table_type' = 'ICEBERG',
'format' = 'parquet',
'write_compression' = 'snappy',
'optimize_rewrite_delete_file_count'= '10',
'vacuum_max_snapshot_age_seconds' = '604800'
);
PARTITIONED BY (year(event_time), month(event_time)) の書き方がHiveと全然違う。Icebergは「時間変換関数」でパーティションを定義する方式で、最初これがわからなくて30分ハマった。Hiveの感覚でPARTITIONED BY (event_time)と書いてエラーを食らうやつ、自分だけじゃないはず。
タイムトラベルクエリ:これが一番使えた機能
Iceberg導入の最大の理由だったタイムトラベルクエリ。実際の使い方はこうなる。
-- 昨日時点のスナップショットを参照
SELECT *
FROM events_iceberg
FOR SYSTEM_TIME AS OF TIMESTAMP '2026-06-03 00:00:00'
WHERE event_type = 'purchase'
LIMIT 100;
-- スナップショットIDで直接参照
SELECT *
FROM events_iceberg
FOR SYSTEM_VERSION AS OF 8754932165847263891;
-- スナップショット一覧の確認
SELECT *
FROM "my_database"."events_iceberg$snapshots"
ORDER BY committed_at DESC
LIMIT 20;
バッチ処理でデータが壊れたとき、「壊れる前の時点のデータ」をサッと取得してリカバリできるのはマジで助かった。以前のHiveパーティション構成だと、バックアップS3バケットからコピーして……という作業が必要で、平均40分かかっていたリカバリが、タイムトラベルで5分以内に収まるようになった。この1点だけでも移行した価値はあったと思っている。
スキーマ進化でハマった話
正直、これが一番痛かった。Icebergはスキーマ進化(Schema Evolution)をサポートしているはずなのに、最初うまく動かなかった。
-- カラム追加(これは問題なく動く)
ALTER TABLE events_iceberg ADD COLUMNS (
session_id STRING COMMENT 'セッションID(2026-05追加)'
);
-- カラムのリネーム(これが曲者)
ALTER TABLE events_iceberg RENAME COLUMN event_type TO event_category;
-- カラムの型変更(INTからBIGINTは可、STRINGへの変換は不可)
ALTER TABLE events_iceberg CHANGE COLUMN user_score user_score BIGINT;
カラムリネーム自体は問題なく通った。問題は「古いデータをタイムトラベルで参照したとき、リネーム前のカラム名でクエリできるか?」という確認を怠ったこと。答えはNoで、タイムトラベルクエリでも新しいカラム名を使わないといけない。これに気付かず、データ品質チェックのクエリがしばらく間違った結果を出し続けていた。データ品質管理の記事で触れているような検証プロセスをIceberg移行直後から入れておくべきだったと、今となっては反省している。
MERGE INTO による Upsert:CDC処理の要
ChangeDataCapture(CDC)処理でのUpsertはIceberg導入の大きなメリットの一つ。実際に使っているパターンがこれ。
-- CDC データのマージ処理
MERGE INTO events_iceberg AS target
USING (
SELECT
event_id,
user_id,
event_type,
event_time,
properties,
created_at,
_op -- CDC操作種別: I/U/D
FROM staging_events
) AS source
ON target.event_id = source.event_id
WHEN MATCHED AND source._op = 'D' THEN
DELETE
WHEN MATCHED AND source._op = 'U' THEN
UPDATE SET
user_id = source.user_id,
event_type = source.event_type,
properties = source.properties
WHEN NOT MATCHED AND source._op != 'D' THEN
INSERT (event_id, user_id, event_type, event_time, properties, created_at)
VALUES (source.event_id, source.user_id, source.event_type,
source.event_time, source.properties, source.created_at);
Hive形式のパーティションテーブルでは絶対できなかった処理で、特に論理削除じゃなくて物理削除が必要なGDPR案件では重宝している。個人的にはこのMERGE INTOが書けるかどうかだけで、Icebergに移行する理由として十分だと思う。
本番で必須になったCompactionとVacuum
Icebergを語る上で避けられないのがファイル管理の話。MERGEやDELETEを繰り返すと、削除ファイル(delete files)が積み上がってクエリが遅くなる。これを放置していたら、2ヶ月後にクエリが3倍遅くなった。週次でCompactionしていたが、全然追いついていなかった。
xychart-beta
title "Compaction前後のクエリレスポンスタイム(秒)"
x-axis ["Week1", "Week2", "Week3", "Week4", "Week5", "Week6", "Week7", "Week8", "Compaction後"]
y-axis "レスポンスタイム(秒)" 0 --> 120
bar [8, 12, 18, 27, 38, 52, 71, 95, 11]
line [8, 12, 18, 27, 38, 52, 71, 95, 11]
Week8の95秒という数字を見たときは流石に焦った。Compaction後に11秒まで戻ったのが救いだったが、これを繰り返したくないので即座に自動化した。
解決策はAthenaのOPTIMIZEコマンドとVACUUMの定期実行。これをGlue Schedulerで自動化している。
-- Compaction(ファイルのマージ)
OPTIMIZE events_iceberg REWRITE DATA
USING BIN_PACK
WHERE event_time >= TIMESTAMP '2026-01-01 00:00:00';
-- 古いスナップショットの削除
VACUUM events_iceberg;
Glue JobでPySparkを使って毎日実行するパターンがこれ。
# Glue ETL Job: iceberg_compaction.py
import boto3
import time
from datetime import datetime, timedelta
def run_athena_query(client, query: str, database: str, output_location: str) -> str:
"""Athenaクエリを実行して完了を待つ"""
response = client.start_query_execution(
QueryString=query,
QueryExecutionContext={'Database': database},
ResultConfiguration={'OutputLocation': output_location}
)
execution_id = response['QueryExecutionId']
# 完了待機(最大30分)
for _ in range(360):
status = client.get_query_execution(
QueryExecutionId=execution_id
)['QueryExecution']['Status']['State']
if status == 'SUCCEEDED':
print(f"Query succeeded: {execution_id}")
return execution_id
elif status in ('FAILED', 'CANCELLED'):
raise RuntimeError(f"Query failed: {execution_id}, status={status}")
time.sleep(5)
raise TimeoutError(f"Query timeout: {execution_id}")
def main():
client = boto3.client('athena', region_name='ap-northeast-1')
database = 'my_datalake'
output_loc = 's3://my-athena-results/compaction/'
tables = ['events_iceberg', 'user_actions_iceberg', 'transactions_iceberg']
for table in tables:
print(f"Starting compaction for {table}")
# 過去30日分をOPTIMIZE
cutoff = (datetime.now() - timedelta(days=30)).strftime('%Y-%m-%d 00:00:00')
optimize_query = f"""
OPTIMIZE {database}.{table} REWRITE DATA
USING BIN_PACK
WHERE event_time >= TIMESTAMP '{cutoff}'
"""
run_athena_query(client, optimize_query, database, output_loc)
# VACUUMで古いスナップショット削除
vacuum_query = f"VACUUM {database}.{table}"
run_athena_query(client, vacuum_query, database, output_loc)
print(f"Compaction completed for {table}")
if __name__ == '__main__':
main()
最初は毎週実行していたが全然追いつかなかった。結局「書き込み頻度の高いテーブルは毎日、ほぼ追記のみのテーブルは週次」という使い分けに落ち着いた。テーブルごとに頻度を変えるのが地味に大事なポイントで、一律で週次にしていた頃は無駄にクエリが遅くなっていた。
コスト・パフォーマンス比較:Hive形式との6ヶ月の実測
移行前のHiveパーティション形式と、Iceberg移行後の実測値を比較するとこうなった。
| 指標 | Hive形式(移行前) | Iceberg(移行後) | 変化 |
|---|---|---|---|
| 平均クエリ時間(全件スキャン) | 42秒 | 18秒 | -57% |
| 平均クエリ時間(フィルタあり) | 28秒 | 7秒 | -75% |
| スキャンデータ量(月平均) | 8.2 TB | 3.1 TB | -62% |
| Athenaコスト(月) | ¥180,000 | ¥68,000 | -62% |
| バッチリカバリ時間(平均) | 40分 | 5分 | -87% |
| スキーマ変更コスト | 手動パーティション再作成 | ALTER TABLE 1行 | 大幅削減 |
| S3ストレージコスト(月) | ¥52,000 | ¥61,000 | +17% |
S3コストだけは微増した。Icebergのメタデータファイル(スナップショット・マニフェスト)が積み上がるから。VACUUMをサボっていた時期は特に顕著だったが、Compactionを自動化してからは安定している。全体のAthenaコストが62%削減できているので、トータルでは十分すぎるくらい元が取れている。
パーティションプルーニングの効果
IcebergのHidden Partitioningは本当に効く。Hive形式ではクエリにWHERE year = 2026 AND month = 6のような条件を明示的に書かないとパーティションプルーニングが効かなかった。IcebergではWHERE event_time BETWEEN '2026-06-01' AND '2026-06-04'と書けばメタデータから自動的に対象ファイルを絞り込んでくれる。
これ、BI担当者からの「なんでクエリが遅いの?」という問い合わせが激減したポイントで、地味に効果が大きかった。MetabaseとSupersetでのBI活用と組み合わせると、エンドユーザーが意識せずにパーティションプルーニングの恩恵を受けられる構成になる。
正直、まだ課題も残っている
全部うまくいったわけじゃないので正直に書く。
Glue Data Catalog連携の微妙な挙動
AthenaでIcebergテーブルを作成した後、Glue Studio上でスキーマが正しく表示されないケースがある。特にMAP型やARRAY型を含むテーブルで発生した。根本原因はGlue CatalogのIceberg対応がまだ部分的なことで、回避策としてaws glue get-tableで直接APIを叩いて確認するようにしている。スマートな解決策とは言えないが、今のところこれで運用できている。
同時書き込みの問題 IcebergはOptimistic Concurrency Controlを使っているが、複数のGlue Jobが同じテーブルに同時書き込みするとコンフリクトが発生することがある。本番では書き込みジョブを直列化して回避しているが、Apache Sparkとの連携でSparkのIcebergコネクタを使うとこの問題が多少緩和される。ここはまだ検証中なので、追って書けることがあれば記事にする。
Row-level DeleteのS3 API呼び出しコスト DELETE文を大量発行するとDelete Fileが爆増してS3のPUT/LIST呼び出しコストが跳ね上がる。これに気付いたのは運用2ヶ月目で、月のS3 APIコストが予想の3倍になっていた。対策はバッチDelete(一度Stagingに出してからMERGE)に切り替えること。設計段階で気付いていれば避けられたはずの失敗だった。
まとめ
6ヶ月Icebergを本番運用して、確実に言えることをまとめる。
- タイムトラベルは実運用でちゃんと使える。バッチ処理の障害復旧時間を40分→5分に短縮できた。ただし、スキーマ変更後のタイムトラベル挙動は事前に必ず検証すること
- Compactionは必須、かつ早めに自動化すべき。放置すると2〜3ヶ月でクエリが数倍遅くなる。うちのチームは最初サボって痛い目を見た
- Athenaコストは60%以上削減できるが、S3メタデータコストの微増は計算に入れておくこと。VACUUMの運用がコスト管理の鍵
- Hidden Partitioningは思った以上に効く。BI利用者がパーティション条件を意識せずに済むようになり、問い合わせが激減した
- 同時書き込みとRow-level Deleteのコストは要注意。設計段階で書き込みパターンを整理しておかないと後で詰む
次にやりたいのがIceberg v2のPosition Delete → Equality Delete移行の検証。Glue 5.0のSpark 3.5と組み合わせると書き込みスループットが改善するらしいが、まだ自分たちの環境では実証できていない。興味ある人は一緒に検証しませんか?
皆さんのチームではデータレイクの障害復旧どうやってます?まだHiveパーティションで頑張ってる人がいたら、Iceberg移行コストは思ったより低いので、小さいテーブルから試してみる価値は十分あると思う。