(KO) CUBRID CDC — WAL 위로 흐르는 DML·DDL 스트리밍
목차
학술적 배경
섹션 제목: “학술적 배경”Change Data Capture (CDC) 는 데이터베이스의 내부 쓰기 로그를 다운스트림 이벤트 스트림으로 바꾸는 일이다. 컨슈머가 그 스트림에 반응한다 — Kafka로 흘려 보내거나, 검색 인덱스를 거울처럼 따라 잡거나, denormalised view를 만들거나, compliance 감사용으로 변경 이력을 기록한다. Database Internals (Petrov) 에는 CDC 전용 챕터가 따로 없지만, 이 주제는 5장 (Recovery, WAL) 과 13장 (Distribution, replication) 의 교차점에 위치한다.
이 모델 위에서 모든 CDC 구현은 두 가지 결정을 내려야 한다. 두 결정이 본 문서 골격을 만든다.
- 논리 이벤트는 어디에서 오는가. 두 가지 길이 있다.
(a) 물리 WAL에서 추출.
LOG_*UNDOREDO_DATA와LOG_MVCC_*레코드를 따라 걸으며, 카탈로그와의 cross-reference로 논리 행 이미지를 재구성한다. PostgreSQL의pg_logical이 이 길이며, 옛 CUBRID HA 복제 (log_applier.c::la_apply_*) 도 이 길이다. (b) 엔진이 DML 시점에 발행하는 명시적 논리 레코드. 이 레코드는 처음부터 풍부하게 만들어진다 — 테이블 OID, before 이미지, after 이미지, 트랜잭션 사용자, statement 텍스트. 컨슈머가 카탈로그 lookup 없이 파싱한다. 현대 CUBRID CDC가 이 길을 고른다 — 모든 DML이 정상LOG_*UNDOREDO_DATA와 함께LOG_SUPPLEMENTAL_INFO레코드를 같이 발행한다는 점이다. - Push 인가 pull 인가. push (복제 스타일 — 서버 측 daemon이
로그를 따라가며 컨슈머에게 push) 와 pull (CDC API 스타일 —
컨슈머가 서버에 이 LSA부터 다음 배치를 달라 고 요청). CUBRID
은 두 모델을 모두 지원한다 — HA 복제는 push (
la_apply_log_file가 장기 daemon), CDC API는 pull (cdc_make_loginfo가 request/response).
이 두 답이 보이고 나면, 본 문서의 모든 CUBRID 구조는 그 답 중 하나를 구현하거나 그 답을 더 빠르게 만들기 위해 존재한다는 점이 분명해진다.
DBMS 공통 설계 패턴
섹션 제목: “DBMS 공통 설계 패턴”CDC를 출하하는 모든 엔진은 비슷한 패턴 위에 자기 구현을 얹는다.
위치 cursor를 가진 forward log walking
섹션 제목: “위치 cursor를 가진 forward log walking”컨슈머는 LSA cursor를 들고 다닌다. 서버가 “이 LSA부터 forward로,
N개 레코드 또는 N바이트까지” 를 반환한다. 매 배치마다 컨슈머가
cursor를 다운스트림에 commit한다. 재연결 시 거기서부터 재개된
다. PostgreSQL의 pg_logical slot, Debezium의 offset, MySQL의
binlog position이 모두 같은 아이디어다.
논리 이벤트 타입
섹션 제목: “논리 이벤트 타입”공통 이벤트 어휘 — INSERT, UPDATE, DELETE, BEGIN, COMMIT, ABORT,
그리고 DDL (CREATE / ALTER / DROP). 대부분의 엔진은 이 집합을
작게 (5-7개) 유지한다. 컨슈머가 자라는 파서를 유지하지 않아도
되도록 하기 위해서다. CUBRID의 SUPPLEMENT_REC_TYPE enum
(cubrid-log-manager.md §보조 레코드) 은 11개 타입을 가진다 —
완결성을 위해 trigger-driven INSERT/UPDATE/DELETE 를 별도로
포함한다.
카탈로그 스냅샷 vs 이벤트별 스키마
섹션 제목: “카탈로그 스냅샷 vs 이벤트별 스키마”DDL이 까다로운 케이스다. 컨슈머가 행 이벤트를 읽을 때 테이블 X
의 스키마를 적용해야 한다. 두 길이 있다 — (a) 첫 행 이벤트 전에
카탈로그 스냅샷을 pull하거나, (b) DDL 이벤트를 inline으로 발행
해 컨슈머가 자기 스키마 캐시를 유지하게 한다. CUBRID은 DDL을
inline LOG_SUPPLEMENT_DDL 레코드로 발행한다. 그 레코드가 SQL
텍스트를 들고 다닌다.
Commit 경계에서의 트랜잭션 그룹화
섹션 제목: “Commit 경계에서의 트랜잭션 그룹화”컨슈머는 “트랜잭션 T의 모든 행을 한 배치로, BEGIN과 COMMIT 사이
정렬된 형태로” 보고 싶어한다. producer 가 COMMIT까지 buffer해
놓았다가 한 번에 flush해야 한다는 뜻이다. 비용은 분명하다 —
지연이 트랜잭션 길이에 비례한다. CUBRID의 CDC producer는 trid별
사용자 정보 map (tran_user) 을 두고, trid 별로 이벤트를
정렬한다.
활성-LSA 보존 (keep-alive)
섹션 제목: “활성-LSA 보존 (keep-alive)”archive 제거 daemon은 컨슈머가 아직 필요한 archive를 삭제해서는
안 된다. 각 엔진은 워터마크를 둔다 — PostgreSQL의
replication_slot.confirmed_flush_lsn, MySQL의 binlog 보존
일수, CUBRID의 cdc_min_log_pageid_to_keep
(log_manager.h:235 에 선언).
이론 ↔ CUBRID 명칭 매핑
섹션 제목: “이론 ↔ CUBRID 명칭 매핑”| 이론적 개념 | CUBRID 명칭 |
|---|---|
| 논리 이벤트 레코드 | LOG_SUPPLEMENTAL_INFO 로그 레코드 (cubrid-log-manager.md) |
| 이벤트 type enum | SUPPLEMENT_REC_TYPE (log_record.hpp) — 11개 값 |
| Forward log walker | log_reader 클래스 (log_reader.hpp) |
| Pull-style 컨슈머 진입점 | cdc_make_loginfo (log_manager.c:14835) |
| LSA 검증 | cdc_validate_lsa (log_manager.c:14402) |
| 시간 → LSA lookup | cdc_find_lsa (log_manager.c:14137) |
| DML 이벤트 재구성 | cdc_make_dml_loginfo (log_manager.c:12818) |
| Before-image fetch | cdc_get_undo_record (log_manager.c:11244) |
| Before+after image fetch | cdc_get_recdes (log_manager.c:11330) |
| 이벤트별 메타데이터 | CDC_LOGINFO_ENTRY { next_lsa, length, log_info } (log_impl.h) |
| Producer 측 상태 머신 | CDC_PRODUCER_STATE { WAIT, RUN, DEAD } + CDC_PRODUCER_REQUEST |
| Producer struct | CDC_PRODUCER (log_impl.h) — next_extraction_lsa, 필터, 큐 |
| Consumer 상태 머신 | CDC_CONSUMER_REQUEST (log_impl.h) |
| 공개 C API | src/api/cubrid_log.c — 외부 컨슈머용 DLL 표면 |
| Archive-keep 워터마크 | cdc_min_log_pageid_to_keep (log_manager.h:235) |
| 옛 HA 복제 daemon | la_apply_log_file (log_applier.c:8074) |
| 옛 commit replay | la_log_commit (log_applier.c:6531) |
| 옛 필터 type | REPL_FILTER_TYPE { NONE, INCLUDE_TBL, EXCLUDE_TBL } (log_applier.h:48) |
| Retry 가능 에러 마스크 | LA_RETRY_ON_ERROR 매크로 (log_applier.h:34) |
CUBRID의 구현
섹션 제목: “CUBRID의 구현”CDC 관련 코드는 두 군데에 산다. 현대 CDC API 는 외부 pull-style
컨슈머를 위해 서버가 노출하는 cdc_* 함수들이다 (log_manager.c
안에 약 3000줄). Legacy HA 복제 daemon 은 archive 로그를
slave 에 push하는 la_* 함수들이다 (log_applier.c, 약 233 KB
/ 8000 줄). 둘 다 log_reader 로 로그를 앞으로 걷는다.
현대 API를 먼저 보고, 그 다음 legacy daemon을 본다.
전체 구조
섹션 제목: “전체 구조”flowchart TB
subgraph SRV["Producer 서버"]
DML["DML 트랜잭션"]
LM["log_manager:\nLOG_∗UNDOREDO_DATA +\nLOG_SUPPLEMENTAL_INFO"]
LRD["log_reader\n(forward walker)"]
CDCP["cdc_∗ 함수들\n(producer side)"]
CDCQ["produced queue"]
DML --> LM
LM --> LRD --> CDCP --> CDCQ
end
subgraph CON["컨슈머 (외부)"]
API["cubrid_log API\n(DLL 표면)"]
APP["consumer 앱\n(Kafka publisher,\nDebezium-like)"]
CDCQ --> API --> APP
end
subgraph HA["Legacy HA 복제"]
LASRC["master 의 archive 로그"]
LAS["la_apply_log_file\n(client-mode daemon)"]
LAREPL["la_apply_repl_log"]
LACMT["la_log_commit"]
SLAVE["slave 서버"]
LASRC --> LAS --> LAREPL --> LACMT --> SLAVE
end
LM -.같은 WAL.-> LASRC
이 그림이 보여 주는 두 forward-walking 파이프라인이 같은 WAL을
공유한다. 첫째, 현대 CDC. 서버가 log_manager.c 안에 producer
를 호스팅한다. 컨슈머는 cubrid_log DLL로 연결해 배치를 pull
한다. 둘째, legacy HA. 별도 client-mode 프로세스
(cubrid_replication) 가 master로부터 archive 볼륨을 가져와
연결된 slave 서버에 그것을 재생한다.
LOG_SUPPLEMENTAL_INFO — 현대 이벤트 형식
섹션 제목: “LOG_SUPPLEMENTAL_INFO — 현대 이벤트 형식”현대 CDC는 물리 로그 레코드에서 논리 이벤트를 재구성하지 않는다.
대신 producer 측에서 모든 DML 이 정상 LOG_*UNDOREDO_DATA 와
함께 보조 LOG_SUPPLEMENTAL_INFO 레코드를 발행한다. 보조 레코드
의 페이로드는 자기 기술적이다 — 첫 바이트가 11개 record 종류
중 하나다.
// SUPPLEMENT_REC_TYPE — src/transaction/log_record.hpp:418typedef enum supplement_rec_type{ LOG_SUPPLEMENT_TRAN_USER, /* who: client user name */ LOG_SUPPLEMENT_UNDO_RECORD, /* raw undo image */ LOG_SUPPLEMENT_DDL, /* DDL statement text */
/* DML records: * | LOG_REC_HEADER | TYPE | LENGTH | CLASS OID | UNDO LSA | REDO LSA | */ LOG_SUPPLEMENT_INSERT, LOG_SUPPLEMENT_UPDATE, LOG_SUPPLEMENT_DELETE,
/* Same shape, but emitted from a trigger action: */ LOG_SUPPLEMENT_TRIGGER_INSERT, LOG_SUPPLEMENT_TRIGGER_UPDATE, LOG_SUPPLEMENT_TRIGGER_DELETE,
LOG_SUPPLEMENT_LARGER_REC_TYPE,} SUPPLEMENT_REC_TYPE;
struct log_rec_supplement{ SUPPLEMENT_REC_TYPE rec_type; int length;};DML 레코드는 의도적으로 간접적이다. 행 이미지 자체가 아니라 밑
의 LOG_UNDOREDO_DATA 의 LSA 를 들고 다닌다. CDC producer 가
그 LSA를 따라 가서 데이터 레코드를 디코딩한다. 그 디코딩 함수가
cdc_get_recdes 다. before/after 행 이미지가 그 시점에 비로소
구체화된다. 이렇게 간접적으로 둔 이유는 보조 레코드의 크기를
작게 (~50 바이트) 유지하기 위해서다 — 행 크기와 무관하다는
점이다.
DDL 레코드는 SQL 텍스트를 inline으로 들고 다닌다. 컨슈머가 데이 터 레코드를 파싱하지 않고도 ALTER / DROP을 재생할 수 있다.
CDC producer — pull 구동 log walker
섹션 제목: “CDC producer — pull 구동 log walker”producer의 상태 머신과 설정을 보면 다음과 같다.
// CDC_PRODUCER — src/transaction/log_impl.htypedef enum cdc_producer_state{ CDC_PRODUCER_STATE_WAIT, CDC_PRODUCER_STATE_RUN, CDC_PRODUCER_STATE_DEAD} CDC_PRODUCER_STATE;
typedef struct cdc_producer{ LOG_LSA next_extraction_lsa; /* cursor */
/* Filter configuration */ int all_in_cond; /* match-all flag */ int num_extraction_user; char **extraction_user; /* whitelist of users */ int num_extraction_class; UINT64 *extraction_classoids; /* whitelist of class OIDs */
volatile CDC_PRODUCER_STATE state; volatile CDC_PRODUCER_REQUEST request;
int produced_queue_size;
pthread_mutex_t lock; pthread_cond_t wait_cond;
CDC_TEMP_LOGBUF temp_logbuf[2]; /* double-buffered log pages */
std::unordered_map<TRANID, char *> tran_user; std::unordered_map<TRANID, int> tran_ignore;} CDC_PRODUCER;각 컨슈머의 요청은 다음 흐름으로 처리된다.
- 초기화.
cdc_initialize(log_manager.c:14957) 가 producer instance, 락, condition variable, 그리고 double-buffered 로그 페이지 슬롯을 설정한다. - 설정.
cdc_set_configuration(log_manager.h:239에 선언) 가 필터를 설치한다 — 어떤 사용자를 포함할지, 어떤 class OID 를 추출할지, timeout, 최대 item 수. - LSA 시드. 둘 중 하나다. (a)
cdc_set_extraction_lsa로 명시적 LSA를 주거나, (b)cdc_find_lsa(log_manager.c:14137) 로 “이 wall-clock 시간에 가장 가까운 LSA를 줘” 를 부른다. 두 번째가 컨슈머의 첫 연결 시점에 쓰는 방식이다. - LSA 검증.
cdc_validate_lsa(log_manager.c:14402) 가 LSA가 여전히 archive에 있는 범위인지 확인한다. archive가 그 시점 너머로 제거되었으면 에러를 반환한다. - Pull 루프.
cdc_get_logitem_info(log_manager.h:241에 선언) 가 다음 이벤트 배치를 반환한다. producer thread가 producer mutex 아래에서cdc_make_loginfo(log_manager.c:14835) 를 돌리며,next_extraction_lsa에서 부터 forward로 걸어 가 배치가 가득 찰 때까지 진행한다. - 정리.
cdc_finalize(log_manager.c:15087) 가 연결 해제 시 정리한다.
producer 상태 전이는 다음과 같다.
stateDiagram-v2 [*] --> WAIT: cdc_initialize WAIT --> RUN: 컨슈머 wakeup RUN --> WAIT: 컨슈머 pause / 큐 가득 RUN --> DEAD: cdc_kill_producer WAIT --> DEAD: shutdown DEAD --> [*]: cdc_finalize
이중 enum 인 CDC_PRODUCER_REQUEST 는 컨슈머로부터의 요청
신호다. 컨슈머 측 thread가 이 값을 설정하면 producer 측 thread가
다음 tick에 읽는다. mutex 위에서의 빡빡한 ping-pong 없이도
cdc_pause_producer, cdc_wakeup_producer, cdc_kill_producer
를 구현하기 위한 장치다.
cdc_make_loginfo — producer hot 루프
섹션 제목: “cdc_make_loginfo — producer hot 루프”// cdc_make_loginfo — src/transaction/log_manager.c:14835 (sketch)intcdc_make_loginfo (THREAD_ENTRY *thread_p, LOG_LSA *start_lsa){ /* Walk the log forward starting at start_lsa. */ while (more_to_read && batch_not_full) { record_header = read_log_record_header (start_lsa);
switch (record_header->type) { case LOG_COMMIT: flush_pending_events_for_tran (record_header->trid); /* emit COMMIT event */ break;
case LOG_ABORT: drop_pending_events_for_tran (record_header->trid); break;
case LOG_SUPPLEMENTAL_INFO: supp = read_supplemental_record_payload (start_lsa); switch (supp.rec_type) { case LOG_SUPPLEMENT_INSERT: case LOG_SUPPLEMENT_UPDATE: case LOG_SUPPLEMENT_DELETE: /* indirect: chase to underlying LOG_UNDOREDO_DATA */ cdc_get_recdes (..., supp.undo_lsa, supp.redo_lsa, &undo_recdes, &redo_recdes); if (passes_filter (classoid, user)) cdc_make_dml_loginfo (..., trid, user, dml_type, classoid, &undo_recdes, &redo_recdes, &log_info_entry, /*is_flashback=*/false); break;
case LOG_SUPPLEMENT_DDL: /* DDL statement text is inline. */ if (passes_filter) emit_ddl_event (statement_text); break;
case LOG_SUPPLEMENT_TRAN_USER: tran_user_map[trid] = user_name; break; } break;
case LOG_END_OF_LOG: break_out; }
advance_to_next_record (&start_lsa); }}여기서 짚을 두 속성이 있다. 첫째, 트랜잭션별 buffer. 이벤트
는 trid별로 COMMIT까지 buffer된다. abort가 나면 그 buffer를 버
린다. producer의 std::unordered_map<TRANID, char *> tran_user
가 trid별 메타데이터를 키로 들고 있다. 그 옆에 trid별 이벤트
리스트가 (헤더 발췌에는 보이지 않지만 producer 본문에서 참조
되는 형태로) 함께 산다. 둘째, 필터링은 producer 측이다.
class OID와 사용자 이름이 모두 producer 단계에서 검사된다. 필터
out된 이벤트도 cdc_get_recdes 호출은 통과해야 한다. 비용이
거기 있다는 점이다. 이득은 큐 크기가 raw 이벤트가 아니라 필터
된 이벤트 수에 비례한다는 점이다.
cdc_get_recdes 와 cdc_get_undo_record — backward chase
섹션 제목: “cdc_get_recdes 와 cdc_get_undo_record — backward chase”DML 보조 레코드는 데이터 레코드의 LSA만 들고 다닌다. 행 이미지 를 구체화하려면 producer가 그 LSA를 따라 가야 한다.
// cdc_get_undo_record — src/transaction/log_manager.c:11244 (signature)SCAN_CODE cdc_get_undo_record (THREAD_ENTRY *thread_p, LOG_PAGE *log_page_p, LOG_LSA lsa, RECDES *undo_recdes);
// cdc_get_recdes — src/transaction/log_manager.c:11330 (signature)int cdc_get_recdes (THREAD_ENTRY *thread_p, LOG_LSA *undo_lsa, RECDES *undo_recdes, LOG_LSA *redo_lsa, RECDES *redo_recdes, bool is_flashback);cdc_get_undo_record 가 주어진 LSA에서 LOG_*UNDOREDO_DATA
레코드를 읽고, 호출별 LOG_ZIP 컨텍스트로 압축을 풀어 undo
이미지를 RECDES 로 반환한다. cdc_get_recdes 가 그 wrapper다.
undo와 redo 이미지를 한 번의 호출로 가져 온다. UPDATE 이벤트가
before/after 쌍을 원하는 경우에 쓰인다.
is_flashback 파라미터가 동작을 바꾼다. flashback 모드
(cubrid-flashback.md) 에서는 깨진 사슬과 누락된 페이지에 관용을
보인다. CDC 모드에서는 그 둘이 에러다.
cdc_make_dml_loginfo — DML 이벤트 패킹
섹션 제목: “cdc_make_dml_loginfo — DML 이벤트 패킹”// cdc_make_dml_loginfo — src/transaction/log_manager.c:12818 (signature)int cdc_make_dml_loginfo (THREAD_ENTRY *thread_p, int trid, char *user, CDC_DML_TYPE dml_type, OID classoid, RECDES *undo_recdes, RECDES *redo_recdes, CDC_LOGINFO_ENTRY *dml_entry, bool is_flashback);이 함수가 구체화된 undo/redo RECDES 와 trid/user/dml_type
메타데이터를 받아 CDC_LOGINFO_ENTRY 로 패킹한다. 그 엔트리의
wire 형식이 컨슈머 측 라이브러리가 디코딩하는 형식이다.
// CDC_LOGINFO_ENTRY — src/transaction/log_impl.htypedef struct cdc_loginfo_entry{ LOG_LSA next_lsa; /* LSA after this event — consumer cursor advance */ int length; char *log_info; /* serialised event payload */} CDC_LOGINFO_ENTRY;핸들러가 성공적으로 끝나면 컨슈머가 next_lsa 를 다운스트림에
commit한다. 재연결 시 다음 pull이 그 LSA에서 시작한다.
공개 API 표면
섹션 제목: “공개 API 표면”외부 컨슈머는 log_manager.c 와 직접 link 되지 않는다. src/api/cubrid_log.c
가 그 사이에서, cdc_* 함수들을 안정적인 C ABI로 wrap한다.
DLL 표면은 raw/code-analysis/cubrid/storage/cdc/CUBRID API 문서_v1.6.docx 에 문서화되어 있다. 일반적인 세션은 다음과 같다.
cubrid_log_connect(...)cubrid_log_set_extraction_filter(...) /* tables, users */cubrid_log_set_lsa_by_time(time, &lsa) /* or cubrid_log_set_lsa(lsa) */loop { n = cubrid_log_extract(&entries, &num) for each entry: handle_event(entry) cubrid_log_commit(entries[n-1].next_lsa)}cubrid_log_disconnect()next_lsa 의 다운스트림 내구성은 컨슈머의 책임이다. CUBRID이
컨슈머별 offset을 서버 측에 추적하지 않는다는 점이다.
Archive 보존 — cdc_min_log_pageid_to_keep
섹션 제목: “Archive 보존 — cdc_min_log_pageid_to_keep”로그-archive 제거 daemon (log_wakeup_remove_log_archive_daemon,
cubrid-log-manager.md 에서 다룸) 은 active CDC 컨슈머나 HA
복제 slave 가 여전히 의지하는 가장 작은 LSA에 따라 삭제를 게이팅
한다.
// cdc_min_log_pageid_to_keep — src/transaction/log_manager.h:235extern LOG_PAGEID cdc_min_log_pageid_to_keep ();이 함수가 컨슈머가 붙어 있지 않으면 MAX_LOG_PAGEID 를 반환
한다 (어떤 archive도 제거 가능). 컨슈머가 붙어 있으면 가장 작은
컨슈머 cursor의 pageid를 반환한다.
Legacy HA 복제 — la_* 패밀리
섹션 제목: “Legacy HA 복제 — la_* 패밀리”CDC API 가 등장하기 이전, CUBRID은 client-mode daemon으로 HA 복제를 지원했다. 이 daemon은 slave 호스트에서 동작하면서 master로부터 archive 로그를 가져 온 뒤 로컬 slave 서버에 그것을 재생한다. daemon의 진입점은 다음과 같다.
// la_apply_log_file — src/transaction/log_applier.c:8074 (signature)int la_apply_log_file (const char *database_name, const char *log_path, const int max_mem_size);내부적으로는 다음 루프를 돈다.
- 다음 로그 archive를 가져 온다 (원격 read 또는 공유 filesystem
—
log_path). - forward로 걸으며 레코드별로
la_apply_repl_log(log_applier.c:5739) 를 호출. LOG_COMMIT에서la_log_commit(log_applier.c:6531) 를 호출 — applied-to LSA를 slave의LA_HA_APPLY_INFO행에 기록 해, daemon 재시작 시 같은 자리에서 이어 갈 수 있게 한다.- retry 가능 에러 (deadlock, lock timeout, 페이지 latch abort —
LA_RETRY_ON_ERROR매크로) 가 나면 그 레코드를 다시 시도.
daemon은 테이블 단위 필터링을 REPL_FILTER_TYPE 으로 지원한다.
// REPL_FILTER_TYPE — src/transaction/log_applier.h:48typedef enum{ REPL_FILTER_NONE, REPL_FILTER_INCLUDE_TBL, REPL_FILTER_EXCLUDE_TBL} REPL_FILTER_TYPE;옛 경로는 보조 로그 레코드에 의지하지 않는다. 정상
LOG_*UNDOREDO_DATA 레코드를 따라 걷고, 카탈로그로 행
이미지를 재구성한다. 그래서 이 daemon이 스키마 변경에 더 취약
하다는 점이다 — master와 slave 사이에서 클래스 representation
을 바꾸는 DDL이 replay 로직을 헷갈리게 만들 수 있다.
현대 CDC pull 한 번, 처음부터 끝까지
섹션 제목: “현대 CDC pull 한 번, 처음부터 끝까지”sequenceDiagram
participant App as 컨슈머 앱
participant API as cubrid_log API
participant Q as producer queue
participant Prod as cdc producer thread
participant LR as log_reader
participant LM as log_manager (WAL)
App->>API: "connect + set filter + set LSA"
API->>Prod: "cdc_set_extraction_lsa (X)"
loop each batch
App->>API: "extract(N)"
API->>Q: "pop N entries"
alt 큐가 비어 있음
Q->>Prod: "wake"
Prod->>LR: "walk forward from next_extraction_lsa"
LR->>LM: "fetch_page (LOG_CS read mode)"
LR->>LM: "parse record header"
alt "LOG_SUPPLEMENT_INSERT/UPDATE/DELETE"
Prod->>LM: "cdc_get_recdes (chase undo+redo LSA)"
Prod->>Prod: "cdc_make_dml_loginfo"
Prod->>Q: "push CDC_LOGINFO_ENTRY"
else LOG_SUPPLEMENT_DDL
Prod->>Q: "push DDL entry"
else LOG_COMMIT
Prod->>Q: "trid별 buffer flush + COMMIT push"
end
end
Q-->>API: "N entries"
API-->>App: "entries"
App->>API: "commit(entries[N-1].next_lsa)"
end
소스 코드 가이드
섹션 제목: “소스 코드 가이드”anchor는 심볼명 이다. 라인은 흘러간다.
현대 CDC API
섹션 제목: “현대 CDC API”cdc_initialize(log_manager.c).cdc_finalize(log_manager.c).cdc_set_configuration(log_manager.h) — 필터, timeout.cdc_set_extraction_lsa(log_manager.c) — 명시적 LSA 시드.cdc_find_lsa(log_manager.c) — 시간 → LSA lookup.cdc_validate_lsa(log_manager.c) — archive 범위 검사.cdc_make_loginfo(log_manager.c) — producer hot 루프.cdc_get_logitem_info(log_manager.h) — 컨슈머 배치 fetch.cdc_get_loginfo_metadata(log_manager.h) — 소비 없이 peek.cdc_get_recdes(log_manager.c) — 간접 LSA로부터 undo+redo RECDES 구체화.cdc_get_undo_record(log_manager.c) — 단일 이미지 변종.cdc_make_dml_loginfo(log_manager.c) — DML 이벤트 패킹.cdc_min_log_pageid_to_keep(log_manager.h) — archive 보존 워터마크.cdc_pause_producer/cdc_wakeup_producer/cdc_kill_producer(log_manager.h).cdc_pause_consumer/cdc_wakeup_consumer(log_manager.h).cdc_reinitialize_queue(log_manager.h).cdc_free_extraction_filter/cdc_cleanup/cdc_cleanup_consumer(log_manager.h).cdc_daemons_init/cdc_daemons_destroy(log_manager.h) — producer/consumer thread를 cubthread 매니저에 등록.
Producer 상태와 타입
섹션 제목: “Producer 상태와 타입”CDC_PRODUCER_STATEenum (log_impl.h).CDC_PRODUCER_REQUESTenum (log_impl.h).CDC_CONSUMER_REQUESTenum (log_impl.h).CDC_LOGINFO_ENTRY(log_impl.h).CDC_TEMP_LOGBUF(log_impl.h) — double-buffered 로그 페이지.CDC_PRODUCER(log_impl.h) — 글로벌 producer 상태.
Log walker
섹션 제목: “Log walker”log_reader클래스 (log_reader.hpp) — forward-walking 로그 fetcher. recovery와 flashback과 공유.log_reader::set_lsa_and_fetch_page(log_reader.hpp).LOG_READ_ALIGN,LOG_READ_ADD_ALIGN,LOG_READ_ADVANCE_WHEN_DOESNT_FIT(log_reader.hpp) — 로그 페이지 경계를 처리하는 inline helper.
공개 API
섹션 제목: “공개 API”src/api/cubrid_log.c— DLL 진입 표면 (cubrid_log_*함수).
Legacy HA 복제
섹션 제목: “Legacy HA 복제”la_apply_log_file(log_applier.c) — daemon 진입.la_apply_repl_log(log_applier.c) — record별 디스패치.la_log_commit(log_applier.c) — slave 측 commit 부기.la_init(log_applier.c) — daemon init.la_init_recdes_pool/la_init_cache_pb/la_init_cache_log_buffer/la_init_repl_lists(log_applier.c) — 내부 init.la_init_ha_apply_info(log_applier.c) — slave별 applied-to 부기 행 초기화.la_get_applied_log_info/la_get_copied_log_info(log_applier.h) — 진단.LA_RETRY_ON_ERROR매크로 (log_applier.h) — retry 가능 에러 마스크.REPL_FILTER_TYPE(log_applier.h) — 테이블 단위 필터.log_applier_sql_log.{c,h}— SQL 로그 발행 (감사용 slave 측 텍스트 replay 로그).
이 개정 시점의 위치 힌트 (2026-04-30)
섹션 제목: “이 개정 시점의 위치 힌트 (2026-04-30)”| 심볼 | 파일 | 라인 |
|---|---|---|
SUPPLEMENT_REC_TYPE enum | log_record.hpp | 418 |
log_rec_supplement (struct) | log_record.hpp | 434 |
CDC_LOGINFO_ENTRY (struct) | log_impl.h | 808 |
CDC_TEMP_LOGBUF (struct) | log_impl.h | 815 |
CDC_PRODUCER (struct) | log_impl.h | 821 |
cdc_get_undo_record | log_manager.c | 11244 |
cdc_get_recdes | log_manager.c | 11330 |
cdc_make_dml_loginfo | log_manager.c | 12818 |
cdc_find_lsa | log_manager.c | 14137 |
cdc_validate_lsa | log_manager.c | 14402 |
cdc_set_extraction_lsa | log_manager.c | 14465 |
cdc_make_loginfo | log_manager.c | 14835 |
cdc_initialize | log_manager.c | 14957 |
cdc_finalize | log_manager.c | 15087 |
log_reader (class) | log_reader.hpp | 36 |
la_init | log_applier.c | 6917 |
la_apply_log_file | log_applier.c | 8074 |
la_apply_repl_log | log_applier.c | 5739 |
la_log_commit | log_applier.c | 6531 |
LA_RETRY_ON_ERROR (macro) | log_applier.h | 34 |
REPL_FILTER_TYPE (enum) | log_applier.h | 48 |
소스 검증 (2026-04-30 기준)
섹션 제목: “소스 검증 (2026-04-30 기준)”검증된 사실
섹션 제목: “검증된 사실”-
현대 CDC 와 legacy HA 는 같은 WAL을 공유하는 별도 코드 경로다.
log_applier.c(CS_MODE daemon) 와log_manager.c의cdc_*(서버 측) 둘 다 읽으며 검증. 한쪽에서 다른 쪽으로 의 호출이 없다. 두 파이프라인이 로그를 독립적으로 따라간다는 뜻이다. -
DML 보조 레코드는 간접 이다 — 행 이미지가 아니라 밑의
LOG_UNDOREDO_DATA의 LSA를 들고 다닌다.log_record.hpp:418의SUPPLEMENT_REC_TYPEenum과 423-424 행의 코멘트 (“Contains lsa of logs which contain undo, redo raw record (UPDATE, DELETE, INSERT) | LOG_REC_HEADER | SUPPLEMENT_REC_TYPE | LENGTH | CLASS OID | UNDO LSA (sizeof LOG_LSA) | REDO LSA |”) 로 검증. -
보조 레코드 집합은 11개 항목이다 — TRAN_USER, UNDO_RECORD, DDL, INSERT, UPDATE, DELETE, TRIGGER_INSERT, TRIGGER_UPDATE, TRIGGER_DELETE, 그리고 양 끝의 sentinel.
log_record.hpp:418-432에서 검증.LOG_SUPPLEMENT_LARGER_REC_TYPE가 범위 검사를 위한 상한이며, 새 타입은 그 앞에 추가된다. -
DDL 보조 레코드는 SQL 텍스트를 inline으로 들고 다닌다.
log_record.hpp:421(LOG_SUPPLEMENT_DDL) 와log_impl.h:564의LOG_TDES::ddl_sql_user_text로 검증. 텍스트는 DDL-실행 시점에 잡히고 보조 레코드에 함께 발행된다. 컨슈머가 카탈로그를 round-trip 할 필요가 없게 하기 위함이다. -
CDC producer는 trid 별
tran_usermap을 유지한다.log_impl.h:845에서 검증 (std::unordered_map<TRANID, char *> tran_user). 키는 트랜잭션 id, 값은LOG_SUPPLEMENT_TRAN_USER에서 기록된 클라이언트 사용자 이름이다. -
producer는 3-state 상태 머신을 가진다.
log_impl.h:787-791의CDC_PRODUCER_STATE(WAIT, RUN, DEAD) 와 thread 간 신호 전달용CDC_PRODUCER_REQUEST,CDC_CONSUMER_REQUESTenum으로 검증. -
로그 fetch 는 별도 클래스 (
log_reader) 이며 recovery, CDC, flashback이 모두 재사용한다.log_reader.hpp:36에서 검증. 이 클래스가 LSA → 페이지 fetch + 정렬 + skip 의미론을 encapsulate한다. 헤더 코멘트에 “remaining member after porting features from the LETS structure” 라고 적혀 있다. 클래스가 재사용 가능하도록 처음부터 추출되었다는 점이다. -
log_reader는 thread-safe 가 아니다.log_reader.hpp:31에서 검증 (“NOTE: not thread safe”). 각 producer thread / recovery worker / flashback 세션이 자기log_reader인스턴스 를 만든다. -
HA 복제의 retry-가능 에러는 명시적 whitelist 매크로다.
log_applier.h:34(LA_RETRY_ON_ERROR) 에서 검증. 리스트는 lock timeout, unilateral abort, 페이지 latch timeout, deadlock cycle, TDE cipher 에러를 포함한다. 그 외 에러는 위로 전파 된다. -
archive 보존은 가장 작은 active 컨슈머 cursor에 따라 결정 된다.
log_manager.h:235(cdc_min_log_pageid_to_keep) 에서 검증. archive 제거 daemon (cubrid-log-manager.md) 이 이 워터 마크에 따라 삭제를 게이팅한다. -
producer는 double-buffered 로그 페이지를 쓴다.
log_impl.h:842(CDC_TEMP_LOGBUF temp_logbuf[2]) 에서 검증. 하나는 파싱 중인 페이지, 다른 하나는 다음 read를 위한 prefetch 버퍼다. 순차 walk에서 wait-for-IO 지연이 절반이 된다는 점이다.
미해결 질문
섹션 제목: “미해결 질문”-
trid 별 이벤트 buffer 위치. producer가 commit 되지 않은 이벤트를 buffer해야 한다.
tran_usermap은 헤더에 보이지만, trid 별 이벤트 리스트 구조는log_manager.c내부에 산다. 추적 경로 —cdc_make_loginfo본문 (14835행 부근) 을 읽기. -
LOG_DUMMY_HA_SERVER_STATE레코드에서의 동작. HA 서버 상태 변경이 logging된다. 현대 CDC producer가 이를 이벤트로 surface하는가, 건너뛰는가? 추적 경로 —cdc_make_loginfo안에서LOG_DUMMY_HA_SERVER_STATEswitch arm 을 grep. -
다중-statement DDL 처리. 한
ALTER TABLE이 다수의 카탈 로그 행을 만질 수 있다. producer가 ALTER마다 한 DDL 이벤트를 발행하는가, 카탈로그 mutation마다 발행하는가? 추적 경로 —LOG_SUPPLEMENT_DDL을 발행하는 자리들을 검색. -
컨슈머 reconfigure 의 필터 race. 컨슈머가 stream 도중
cdc_set_configuration을 호출해 클래스를 추가하면, 필터가 적용되기 전에 이미 큐에 있던 이벤트는 어떻게 되는가? 추적 경로 —cdc_set_configuration와 producer 측 필터 검사를 읽기. -
HA 복제 slave failover. slave가 master로 promote되면 LA daemon이 멈춰야 한다. 그 stop 신호는 어디에서 오는가?
la_force_shutdown이 존재하는데, 누가 부르는가? 추적 경로 —src/connection/heartbeat*안에서 호출자를 grep. -
log_applier_sql_log.{c,h}의 용도. slave가 별도로 쓰는 SQL 로그인가? 감사 로그? Replay 검증? 추적 경로 — 이 파일 페어를 읽기.
CUBRID 너머 — 비교 설계와 연구 동향
섹션 제목: “CUBRID 너머 — 비교 설계와 연구 동향”분석이 아닌 포인터(pointers).
-
Debezium (Kafka Connect) — PG, MySQL, MongoDB, Oracle용 플러그형 CDC connector. wire 형식 — Kafka 위 Avro / JSON. CUBRID의
CDC_LOGINFO_ENTRYwire 형식을 Debezium과 비교하면 round-trip 비용이 문서화될 것이다. -
PostgreSQL logical replication — output plugin (
pgoutput,wal2json) 이 물리 WAL walk에서 논리 이벤트를 발행한다. 개념적으로 CUBRID의 legacy HA 에 가깝다. 현대 CDC API의 보조 레코드는 Maxwell의 binlog row event에 더 가깝다. -
MySQL binlog (statement / row / mixed) — 이벤트 발행 모드가 세 개다. row 모드가 CUBRID 현대 CDC 에 가깝다. statement 모드는 legacy HA 복제 쪽에 가깝다.
-
Debezium-style outbox 패턴 — 애플리케이션이 전용 outbox 테이블에 이벤트를 쓰고, CDC가 그것을 publish한다. CUBRID의
LOG_SUPPLEMENT_DDL은 가장 단순한 형태의 같은 아이디어다. 엔진 자체가 outbox writer 역할을 한다. -
Kafka Connect 컨슈머 — CUBRID의 pull-style API는 Kafka Connect source connector에 잘 맞는다. 컨슈머 thread가 connect worker가 된다.
-
Structured Streaming exactly-once 의미론 — 컨슈머가
next_lsa를 commit하는 일은 컨슈머가 2PC를 통합하지 않는 한 at-least-once다. CUBRID의 2PC (cubrid-2pc.md) 가 컨슈머 commit과 결합될 수 있다. 현재 API에서는 그 결합이 노출되지 않는다.
원본 분석 (raw/code-analysis/cubrid/storage/cdc/)
섹션 제목: “원본 분석 (raw/code-analysis/cubrid/storage/cdc/)”CDC 진행상황 공유_v2.pptxCDC 인수인계.pptxALTER, DROP.pptxDML Log sequence.pdfCUBRID API 문서_v1.6.docx
형제 문서
섹션 제목: “형제 문서”knowledge/code-analysis/cubrid/cubrid-log-manager.md—LOG_SUPPLEMENTAL_INFO+SUPPLEMENT_REC_TYPE의 출처.knowledge/code-analysis/cubrid/cubrid-flashback.md— 반대 방향 (역행).log_reader를 공유. 같은 배치에서 진행 중.knowledge/code-analysis/cubrid/cubrid-recovery-manager.md— redo 경로와log_reader클래스를 공유.knowledge/code-analysis/cubrid/cubrid-catalog-manager.md—_db_class등의 갱신이 발행하는LOG_SUPPLEMENT_DDL.knowledge/code-analysis/cubrid/cubrid-2pc.md— exactly-once 소비와 관련된 분산 commit 의미론.
교재 챕터 (knowledge/research/dbms-general/)
섹션 제목: “교재 챕터 (knowledge/research/dbms-general/)”- Database Internals (Petrov), 5장 §Logging, 13장 §Replication.
- Designing Data-Intensive Applications (Kleppmann), 5장 “Replication, 11장 Stream Processing” — CDC framing.
CUBRID 소스 (/data/hgryoo/references/cubrid/)
섹션 제목: “CUBRID 소스 (/data/hgryoo/references/cubrid/)”src/transaction/log_manager.{c,h}—cdc_*현대 API.src/transaction/log_applier.{c,h}— legacy HA daemon.src/transaction/log_applier_sql_log.{c,h}— slave SQL 로그.src/transaction/log_reader.{cpp,hpp}— forward walker.src/api/cubrid_log.c— DLL 진입 표면.