Fornece uma interface do tipo tabela, somente leitura, para tabelas Apache Iceberg no Amazon S3, Azure, HDFS ou armazenadas localmente.
icebergS3(url [, NOSIGN | access_key_id, secret_access_key, [session_token]] [,format] [,compression_method] [,extra_credentials])
icebergS3(named_collection[, option=value [,..]])
icebergAzure(connection_string|storage_account_url, container_name, blobpath, [,account_name], [,account_key] [,format] [,compression_method])
icebergAzure(named_collection[, option=value [,..]])
icebergHDFS(path_to_table, [,format] [,compression_method])
icebergHDFS(named_collection[, option=value [,..]])
icebergLocal(path_to_table, [,format] [,compression_method])
icebergLocal(named_collection[, option=value [,..]])
A descrição dos argumentos é a mesma que a dos argumentos nas funções de tabela s3, azureBlobStorage, HDFS e file, respectivamente.
format representa o formato dos arquivos de dados na tabela Iceberg.
Para icebergS3, é possível usar um parâmetro opcional extra_credentials para passar um role_arn para controle de acesso baseado em função no ClickHouse Cloud. Consulte Secure S3 para ver as etapas de configuração.
Uma tabela com a estrutura especificada para ler dados da tabela Iceberg especificada.
SELECT * FROM icebergS3('http://test.s3.amazonaws.com/clickhouse-bucket/test_table', 'test', 'test')
Atualmente, o ClickHouse oferece suporte à leitura das versões v1 e v2 do formato Iceberg por meio das funções de tabela icebergS3, icebergAzure, icebergHDFS e icebergLocal e dos motores de tabela IcebergS3, icebergAzure, IcebergHDFS e IcebergLocal.
Definindo uma coleção nomeada
Veja um exemplo de configuração de uma coleção nomeada para armazenar a URL e as credenciais:
<clickhouse>
<named_collections>
<iceberg_conf>
<url>http://test.s3.amazonaws.com/clickhouse-bucket/</url>
<access_key_id>test<access_key_id>
<secret_access_key>test</secret_access_key>
<format>auto</format>
<structure>auto</structure>
</iceberg_conf>
</named_collections>
</clickhouse>
SELECT * FROM icebergS3(iceberg_conf, filename = 'test_table')
DESCRIBE icebergS3(iceberg_conf, filename = 'test_table')
Usando um catálogo de dados
As tabelas Iceberg também podem ser usadas com vários catálogos de dados, como o REST Catalog, o AWS Glue Data Catalog e o Unity Catalog.
Ao usar um catálogo, a maioria dos usuários vai querer usar a database engine DataLakeCatalog, que conecta o ClickHouse ao seu catálogo para descobrir suas tabelas. Você pode usar essa database engine em vez de criar manualmente tabelas individuais com o motor de tabela IcebergS3.
Para usá-los, crie uma tabela com o engine IcebergS3 e forneça as configurações necessárias.
Por exemplo, usando o REST Catalog com armazenamento MinIO:
CREATE TABLE `database_name.table_name`
ENGINE = IcebergS3(
'http://minio:9000/warehouse-rest/table_name/',
'minio_access_key',
'minio_secret_key'
)
Ou, usando o AWS Glue Data Catalog com S3:
CREATE TABLE `my_database.my_table`
ENGINE = IcebergS3(
's3://my-data-bucket/warehouse/my_database/my_table/',
'aws_access_key',
'aws_secret_key'
)
Atualmente, com o CH, é possível ler tabelas Iceberg cujo esquema mudou ao longo do tempo. No momento, oferecemos suporte à leitura de tabelas em que colunas foram adicionadas e removidas, e cuja ordem foi alterada. Também é possível alterar uma coluna em que um valor é obrigatório para outra em que NULL é permitido. Além disso, oferecemos suporte a conversões de tipo permitidas para tipos simples, a saber:
- int -> long
- float -> double
- decimal(P, S) -> decimal(P’, S), em que P’ > P.
Atualmente, não é possível alterar estruturas aninhadas nem os tipos dos elementos dentro de arrays e maps.
O ClickHouse oferece suporte à poda de partições durante consultas SELECT em tabelas Iceberg, o que ajuda a otimizar o desempenho das consultas ao ignorar arquivos de dados irrelevantes. Para ativar a poda de partições, defina use_iceberg_partition_pruning = 1. Para mais informações sobre partition pruning no Iceberg, acesse https://iceberg.apache.org/spec/#partitioning
O ClickHouse oferece suporte ao recurso de viagem no tempo em tabelas Iceberg, permitindo consultar dados históricos com um timestamp específico ou um ID de snapshot.
Atualmente, apenas tabelas Iceberg com exclusões por posição são suportadas.
Os seguintes métodos de exclusão não são suportados:
SELECT * FROM example_table ORDER BY 1
SETTINGS iceberg_timestamp_ms = 1714636800000
SELECT * FROM example_table ORDER BY 1
SETTINGS iceberg_snapshot_id = 3547395809148285433
Observação: não é possível especificar os parâmetros iceberg_timestamp_ms e iceberg_snapshot_id na mesma consulta.
Considerações importantes
-
Snapshots normalmente são criados quando:
-
Novos dados são gravados na tabela
-
Algum tipo de compactação de dados é executado
-
Alterações de esquema normalmente não criam snapshots - Isso leva a comportamentos importantes ao usar viagem no tempo com tabelas que passaram por evolução de esquema.
Todos os cenários usam Spark, porque o CH ainda não oferece suporte à gravação em tabelas Iceberg.
Cenário 1: Alterações no esquema sem novos snapshots
Considere esta sequência de operações:
-- Criar uma tabela com duas colunas
CREATE TABLE IF NOT EXISTS spark_catalog.db.time_travel_example (
order_number bigint,
product_code string
)
USING iceberg
OPTIONS ('format-version'='2')
- - Inserir dados na tabela
INSERT INTO spark_catalog.db.time_travel_example VALUES
(1, 'Mars')
ts1 = now() // Um trecho de pseudocódigo
- - Alterar a tabela para adicionar uma nova coluna
ALTER TABLE spark_catalog.db.time_travel_example ADD COLUMN (price double)
ts2 = now()
- - Inserir dados na tabela
INSERT INTO spark_catalog.db.time_travel_example VALUES (2, 'Venus', 100)
ts3 = now()
- - Consultar a tabela em cada timestamp
SELECT * FROM spark_catalog.db.time_travel_example TIMESTAMP AS OF ts1;
+------------+------------+
|order_number|product_code|
+------------+------------+
| 1| Mars|
+------------+------------+
SELECT * FROM spark_catalog.db.time_travel_example TIMESTAMP AS OF ts2;
+------------+------------+
|order_number|product_code|
+------------+------------+
| 1| Mars|
+------------+------------+
SELECT * FROM spark_catalog.db.time_travel_example TIMESTAMP AS OF ts3;
+------------+------------+-----+
|order_number|product_code|price|
+------------+------------+-----+
| 1| Mars| NULL|
| 2| Venus|100.0|
+------------+------------+-----+
Resultados da consulta em diferentes timestamps:
- Em ts1 & ts2: aparecem apenas as duas colunas originais
- Em ts3: aparecem as três colunas, com NULL no preço da primeira linha
Cenário 2: Diferenças entre o esquema histórico e o atual
Uma consulta de viagem no tempo no momento atual pode mostrar um esquema diferente do da tabela atual:
-- Criar uma tabela
CREATE TABLE IF NOT EXISTS spark_catalog.db.time_travel_example_2 (
order_number bigint,
product_code string
)
USING iceberg
OPTIONS ('format-version'='2')
-- Inserir dados iniciais na tabela
INSERT INTO spark_catalog.db.time_travel_example_2 VALUES (2, 'Venus');
-- Alterar a tabela para adicionar uma nova coluna
ALTER TABLE spark_catalog.db.time_travel_example_2 ADD COLUMN (price double);
ts = now();
-- Consultar a tabela no momento atual usando sintaxe de timestamp
SELECT * FROM spark_catalog.db.time_travel_example_2 TIMESTAMP AS OF ts;
+------------+------------+
|order_number|product_code|
+------------+------------+
| 2| Venus|
+------------+------------+
-- Consultar a tabela no momento atual
SELECT * FROM spark_catalog.db.time_travel_example_2;
+------------+------------+-----+
|order_number|product_code|price|
+------------+------------+-----+
| 2| Venus| NULL|
+------------+------------+-----+
Isso acontece porque ALTER TABLE não cria um novo snapshot; para a tabela atual, o Spark obtém o valor de schema_id do arquivo de metadados mais recente, e não de um snapshot.
Cenário 3: Diferenças entre o esquema histórico e o atual
A segunda é que, ao usar viagem no tempo, você não consegue obter o estado da tabela antes de qualquer dado ter sido gravado nela:
-- Criar uma tabela
CREATE TABLE IF NOT EXISTS spark_catalog.db.time_travel_example_3 (
order_number bigint,
product_code string
)
USING iceberg
OPTIONS ('format-version'='2');
ts = now();
-- Consultar a tabela em um timestamp específico
SELECT * FROM spark_catalog.db.time_travel_example_3 TIMESTAMP AS OF ts; -- Finaliza com erro: Cannot find a snapshot older than ts.
No ClickHouse, o comportamento é consistente com o do Spark. Você pode imaginar as consultas Select do Spark substituídas por consultas Select do ClickHouse, e isso funcionará da mesma forma.
Ao usar a função de tabela iceberg no ClickHouse, o sistema precisa localizar o arquivo metadata.json correto que descreve a estrutura da tabela Iceberg. Veja como esse processo funciona:
Busca de candidatos (em ordem de prioridade)
- Especificação direta do caminho:
*Se você definir
iceberg_metadata_file_path, o sistema usará exatamente esse caminho, combinando-o com o caminho do diretório da tabela Iceberg.
- Quando essa configuração é fornecida, todas as outras configurações de resolução são ignoradas.
-
Correspondência do UUID da tabela:
*Se
iceberg_metadata_table_uuid for especificado, o sistema irá:
*Considerar apenas os arquivos .metadata.json no diretório metadata
*Filtrar os arquivos que contêm um campo table-uuid correspondente ao UUID especificado (sem diferenciar maiúsculas de minúsculas)
-
Busca padrão:
*Se nenhuma das configurações acima for fornecida, todos os arquivos
.metadata.json no diretório metadata passam a ser candidatos
Selecionando o arquivo mais recente
Após identificar os arquivos candidatos usando as regras acima, o sistema determina qual deles é o mais recente:
-
Se
iceberg_recent_metadata_file_by_last_updated_ms_field estiver habilitado:
-
O arquivo com o maior valor de
last-updated-ms é selecionado
-
Caso contrário:
-
O arquivo com o número de versão mais alto é selecionado
-
(A versão aparece como
V em nomes de arquivo no formato V.metadata.json ou V-uuid.metadata.json)
Observação: Todas as configurações mencionadas são configurações de função de tabela (não configurações globais nem configurações no nível da consulta) e devem ser especificadas como mostrado abaixo:
SELECT * FROM iceberg('s3://bucket/path/to/iceberg_table',
SETTINGS iceberg_metadata_table_uuid = 'a90eed4c-f74b-4e5b-b630-096fb9d09021');
Observação: Embora os catálogos Iceberg normalmente cuidem da resolução de metadados, a função de tabela iceberg no ClickHouse interpreta diretamente arquivos armazenados no S3 como tabelas Iceberg, por isso é importante entender essas regras de resolução.
O motor de tabela Iceberg e a função de tabela oferecem suporte a um cache de metadados que armazena informações dos arquivos de manifesto, da lista de manifestos e do JSON de metadados. O cache é armazenado na memória. Esse recurso é controlado pela configuração use_iceberg_metadata_files_cache, que vem habilitada por padrão.
A função de tabela iceberg agora é um alias para icebergS3.
_path — Caminho do arquivo. Tipo: LowCardinality(String).
_file — Nome do arquivo. Tipo: LowCardinality(String).
_size — Tamanho do arquivo em bytes. Tipo: Nullable(UInt64). Se o tamanho do arquivo for desconhecido, o valor será NULL.
_time — Data e hora da última modificação do arquivo. Tipo: Nullable(DateTime). Se esse horário for desconhecido, o valor será NULL.
_etag — O etag do arquivo. Tipo: LowCardinality(String). Se o etag for desconhecido, o valor será NULL.
Gravações em tabelas Iceberg
A partir da versão 25.7, o ClickHouse oferece suporte a modificações nas tabelas Iceberg do usuário.
No momento, este é um recurso experimental, então primeiro você precisa ativá-lo:
SET allow_insert_into_iceberg = 1;
Para criar sua própria tabela Iceberg vazia, use os mesmos comandos da leitura, mas especifique o esquema explicitamente.
As operações de escrita oferecem suporte a todos os formatos de dados da especificação Iceberg, como Parquet, Avro e ORC.
CREATE TABLE iceberg_writes_example
(
x Nullable(String),
y Nullable(Int32)
)
ENGINE = IcebergLocal('/home/scanhex12/iceberg_example/')
Observação: para criar um arquivo indicador de versão, ative a configuração iceberg_use_version_hint.
Se quiser comprimir o arquivo metadata.json, especifique o nome do codec na configuração iceberg_metadata_compression_method.
Após criar uma nova tabela, você pode inserir dados usando a sintaxe padrão do ClickHouse.
INSERT INTO iceberg_writes_example VALUES ('Pavel', 777), ('Ivanov', 993);
SELECT *
FROM iceberg_writes_example
FORMAT VERTICAL;
Row 1:
──────
x: Pavel
y: 777
Row 2:
──────
x: Ivanov
y: 993
A exclusão de linhas excedentes no formato merge-on-read também é compatível com o ClickHouse.
Esta consulta criará um novo snapshot com arquivos de exclusão por posição.
ALTER TABLE iceberg_writes_example DELETE WHERE x != 'Ivanov';
SELECT *
FROM iceberg_writes_example
FORMAT VERTICAL;
Row 1:
──────
x: Ivanov
y: 993
O ClickHouse permite adicionar, remover, modificar ou renomear colunas com tipos simples (que não sejam Tuple, Array nem Map).
ALTER TABLE iceberg_writes_example MODIFY COLUMN y Nullable(Int64);
SHOW CREATE TABLE iceberg_writes_example;
┌─statement─────────────────────────────────────────────────┐
1. │ CREATE TABLE default.iceberg_writes_example ↴│
│↳( ↴│
│↳ `x` Nullable(String), ↴│
│↳ `y` Nullable(Int64) ↴│
│↳) ↴│
│↳ENGINE = IcebergLocal('/home/scanhex12/iceberg_example/') │
└───────────────────────────────────────────────────────────┘
ALTER TABLE iceberg_writes_example ADD COLUMN z Nullable(Int32);
SHOW CREATE TABLE iceberg_writes_example;
┌─statement─────────────────────────────────────────────────┐
1. │ CREATE TABLE default.iceberg_writes_example ↴│
│↳( ↴│
│↳ `x` Nullable(String), ↴│
│↳ `y` Nullable(Int64), ↴│
│↳ `z` Nullable(Int32) ↴│
│↳) ↴│
│↳ENGINE = IcebergLocal('/home/scanhex12/iceberg_example/') │
└───────────────────────────────────────────────────────────┘
SELECT *
FROM iceberg_writes_example
FORMAT VERTICAL;
Row 1:
──────
x: Ivanov
y: 993
z: ᴺᵁᴸᴸ
ALTER TABLE iceberg_writes_example DROP COLUMN z;
SHOW CREATE TABLE iceberg_writes_example;
┌─statement─────────────────────────────────────────────────┐
1. │ CREATE TABLE default.iceberg_writes_example ↴│
│↳( ↴│
│↳ `x` Nullable(String), ↴│
│↳ `y` Nullable(Int64) ↴│
│↳) ↴│
│↳ENGINE = IcebergLocal('/home/scanhex12/iceberg_example/') │
└───────────────────────────────────────────────────────────┘
SELECT *
FROM iceberg_writes_example
FORMAT VERTICAL;
Row 1:
──────
x: Ivanov
y: 993
ALTER TABLE iceberg_writes_example RENAME COLUMN y TO value;
SHOW CREATE TABLE iceberg_writes_example;
┌─statement─────────────────────────────────────────────────┐
1. │ CREATE TABLE default.iceberg_writes_example ↴│
│↳( ↴│
│↳ `x` Nullable(String), ↴│
│↳ `value` Nullable(Int64) ↴│
│↳) ↴│
│↳ENGINE = IcebergLocal('/home/scanhex12/iceberg_example/') │
└───────────────────────────────────────────────────────────┘
SELECT *
FROM iceberg_writes_example
FORMAT VERTICAL;
Row 1:
──────
x: Ivanov
value: 993
O ClickHouse oferece suporte à compactação de tabelas Iceberg. Atualmente, ele pode mesclar arquivos de exclusão por posição aos arquivos de dados enquanto atualiza os metadados. Os IDs de snapshots anteriores e seus timestamps permanecem inalterados, portanto o recurso de viagem no tempo ainda pode ser usado com os mesmos valores.
Como usá-lo:
SET allow_experimental_iceberg_compaction = 1
OPTIMIZE TABLE iceberg_writes_example;
SELECT *
FROM iceberg_writes_example
FORMAT VERTICAL;
Row 1:
──────
x: Ivanov
y: 993
As tabelas Iceberg acumulam snapshots a cada operação INSERT, DELETE ou UPDATE. Com o tempo, isso pode resultar em um grande número de snapshots e arquivos de dados associados. O comando expire_snapshots remove snapshots antigos e limpa os arquivos de dados que não são mais referenciados por nenhum snapshot retido.
Sintaxe:
ALTER TABLE iceberg_table EXECUTE expire_snapshots(
['timestamp']
[, expire_before = 'timestamp']
[, retention_period = '3d']
[, retain_last = 100]
[, snapshot_ids = [1, 2, 3, 4]]
[, dry_run = 1]
);
Por padrão, quais snapshots devem ser mantidos são definidos pela política de retenção (propriedades da tabela min-snapshots-to-keep, max-snapshot-age-ms e substituições por ref). Quando snapshot_ids é especificado, a política de retenção é ignorada e apenas os snapshots listados são considerados para expiração.
Argumentos:
'timestamp' (posicional) ou expire_before = 'timestamp' — uma string de data e hora (por exemplo, '2024-06-01 00:00:00') interpretada no fuso horário do servidor. Funciona como uma trava de segurança: snapshots cujo timestamp-ms seja igual ou posterior a esse valor ficam protegidos contra expiração, mesmo que a política de retenção, de outra forma, os expirasse. Pode ser combinado com snapshot_ids; nesse caso, snapshots listados com timestamp igual ou mais recente que esse valor não expiram.
retention_period = '<duration>' — substitui o history.expire.max-snapshot-age-ms no nível da tabela apenas nesta invocação. Snapshots mais antigos do que essa duração (medida a partir de agora) tornam-se candidatos à expiração. O valor é uma string de duração composta por um ou mais pares {number}{unit} concatenados. Unidades aceitas: y (365 dias), w (7 dias), d (24 horas), h (60 minutos), m (60 segundos), s (1 segundo), ms (1 milissegundo). As unidades podem ser combinadas, por exemplo: '3d', '12h', '1d12h30m', '500ms'.
retain_last = N — substitui o history.expire.min-snapshots-to-keep no nível da tabela apenas nesta invocação. Pelo menos N snapshots são sempre mantidos, independentemente da idade.
snapshot_ids = [id1, id2, ...] — expira exatamente os IDs de snapshot listados (exceto snapshots referenciados pelo snapshot atual, branches ou tags). Esse modo ignora completamente a política de retenção e não pode ser combinado com retention_period nem com retain_last.
dry_run = 1 — calcula o que seria expirado e retorna métricas sem gravar novos metadados nem excluir arquivos.
retention_period e retain_last substituem apenas os padrões de retenção no nível da tabela. Substituições de retenção por ref (branch/tag) configuradas nas propriedades da tabela Iceberg (por exemplo, refs.<branch>.min-snapshots-to-keep) nunca são substituídas — elas sempre entram em vigor conforme especificado nos metadados da tabela.
Exemplo:
SET allow_insert_into_iceberg = 1;
-- Cria alguns snapshots inserindo dados
INSERT INTO iceberg_table VALUES (1);
INSERT INTO iceberg_table VALUES (2);
INSERT INTO iceberg_table VALUES (3);
-- Expira usando apenas a política de retenção
ALTER TABLE iceberg_table EXECUTE expire_snapshots();
-- Expira com fusível de segurança: protege snapshots mais recentes que o timestamp (sintaxe posicional)
ALTER TABLE iceberg_table EXECUTE expire_snapshots('2025-01-01 00:00:00');
-- O mesmo usando a forma de argumento nomeado
ALTER TABLE iceberg_table EXECUTE expire_snapshots(expire_before = '2025-01-01 00:00:00');
-- Substitui os parâmetros de retenção para uma execução
ALTER TABLE iceberg_table EXECUTE expire_snapshots(retention_period = '3d', retain_last = 10);
-- Expira snapshots explícitos
ALTER TABLE iceberg_table EXECUTE expire_snapshots(snapshot_ids = [101, 102, 103]);
-- Pré-visualização em modo dry-run (sem atualizações de metadados, sem exclusão de arquivos)
ALTER TABLE iceberg_table EXECUTE expire_snapshots(retention_period = '1d', dry_run = 1);
Saída:
O comando retorna uma tabela com duas colunas (metric_name String, metric_value Int64), contendo uma linha por métrica. Os nomes das métricas seguem a especificação do Iceberg:
| metric_name | Descrição |
|---|
deleted_data_files_count | Número de arquivos de dados excluídos |
deleted_position_delete_files_count | Número de arquivos de exclusão por posição excluídos |
deleted_equality_delete_files_count | Número de arquivos de exclusão por igualdade excluídos |
deleted_manifest_files_count | Número de arquivos de manifesto excluídos |
deleted_manifest_lists_count | Número de arquivos de lista de manifesto excluídos |
deleted_statistics_files_count | Número de arquivos de estatísticas excluídos (no momento, sempre 0) |
dry_run | 1 para modo de simulação, 0 para execução normal |
O comando executa as seguintes etapas:
- Avalia a política de retenção (veja abaixo) para determinar quais snapshots devem ser preservados
- Se um argumento de timestamp tiver sido fornecido, também protege todos os snapshots nesse timestamp ou posteriores
- Expira os snapshots que não forem retidos pela política nem protegidos pelo limite de timestamp
- Calcula quais arquivos estão associados exclusivamente a snapshots expirados
- No modo normal: gera novos metadados sem os snapshots expirados
- No modo normal: exclui fisicamente listas de manifesto, arquivos de manifesto e arquivos de dados inacessíveis
- No modo
dry_run = 1: ignora as etapas 5 e 6 e retorna apenas as métricas calculadas
Política de retenção de snapshots
O comando expire_snapshots respeita a política de retenção de snapshots do Iceberg. A retenção é configurada por meio de propriedades da tabela Iceberg e de substituições específicas por referência:
| Propriedade | Escopo | Padrão | Descrição |
|---|
history.expire.min-snapshots-to-keep | Tabela | iceberg_expire_default_min_snapshots_to_keep (padrão 1) | Número mínimo de snapshots a manter na cadeia ancestral de cada branch |
history.expire.max-snapshot-age-ms | Tabela | iceberg_expire_default_max_snapshot_age_ms (padrão 432000000, 5 dias) | Idade máxima (em ms) dos snapshots a manter em uma branch |
history.expire.max-ref-age-ms | Tabela | iceberg_expire_default_max_ref_age_ms (padrão ∞) | Idade máxima (em ms) de uma referência de snapshot (branch ou tag) antes que a própria referência seja removida |
Cada referência de snapshot (refs nos metadados do Iceberg) pode substituir esses valores com campos específicos por referência: min-snapshots-to-keep, max-snapshot-age-ms e max-ref-age-ms.
Avaliação da retenção:
- Para cada branch (incluindo
main): a cadeia ancestral é percorrida a partir do head da branch. Os snapshots são mantidos enquanto qualquer uma destas condições for verdadeira:
- O snapshot está entre os primeiros
min-snapshots-to-keep da cadeia
- A idade do snapshot está dentro de
max-snapshot-age-ms (ou seja, now - timestamp-ms <= max-snapshot-age-ms)
- Para tags: o snapshot marcado pela tag é mantido, a menos que a tag tenha excedido seu
max-ref-age-ms, caso em que a referência da tag é removida
- Referências diferentes de
main cuja idade exceda max-ref-age-ms são removidas por completo (a branch main nunca é removida)
- Referências órfãs que apontam para snapshots inexistentes são removidas com um aviso
- O snapshot atual é sempre preservado, independentemente das configurações de retenção
Privilégios necessários:
O privilégio ALTER TABLE EXECUTE é necessário, e ele é um privilégio filho de ALTER TABLE na hierarquia de controle de acesso do ClickHouse. Você pode concedê-lo especificamente ou por meio do privilégio pai:
-- Conceder apenas a permissão EXECUTE
GRANT ALTER TABLE EXECUTE ON my_iceberg_table TO my_user;
-- Ou conceder todas as permissões de ALTER TABLE (inclui ALTER TABLE EXECUTE)
GRANT ALTER TABLE ON my_iceberg_table TO my_user;
- Somente tabelas Iceberg format version 2 são suportadas (snapshots v1 não garantem
manifest-list, que é necessário para identificar com segurança os arquivos a serem removidos)
- O snapshot atual é sempre preservado, mesmo que seja mais antigo que o timestamp especificado
- Exige que a configuração
allow_insert_into_iceberg esteja habilitada
- Exige que a configuração
allow_experimental_expire_snapshots esteja habilitada
- A autorização do próprio catálogo (autenticação do catálogo REST, AWS Glue IAM etc.) é aplicada independentemente quando o ClickHouse atualiza os metadados
Arquivos órfãos são arquivos no armazenamento que não são referenciados por nenhum snapshot nos metadados da tabela Iceberg. Eles se acumulam devido a gravações com falha, limpeza parcial após a compactação e operações interrompidas, causando crescimento descontrolado do armazenamento. O comando remove_orphan_files identifica e remove esses arquivos órfãos.
Sintaxe:
-- Forma posicional: único argumento older_than sem nome
ALTER TABLE iceberg_table EXECUTE remove_orphan_files('timestamp')
-- Forma nomeada
ALTER TABLE iceberg_table EXECUTE remove_orphan_files(
older_than = 'timestamp',
location = 'path',
dry_run = 0|1
)
-- Sem argumentos: usa todos os padrões (older_than = 3 dias atrás)
ALTER TABLE iceberg_table EXECUTE remove_orphan_files()
Parâmetros:
| Parâmetro | Tipo | Padrão | Descrição |
|---|
older_than | String (timestamp) | 3 dias atrás (configurável por iceberg_orphan_files_older_than_seconds) | Considera como candidatos a órfãos apenas os arquivos cuja hora da última modificação seja anterior a este timestamp. É uma proteção para evitar a exclusão de arquivos de gravações em andamento. |
location | String | Localização da tabela | Restringe a varredura a um subdiretório específico dentro da localização da tabela (por exemplo, 'data/' ou 'metadata/'). |
dry_run | UInt64 | 0 | Quando 1, identifica arquivos órfãos e retorna um resumo dos resultados sem excluir nada de fato. |
Exemplos:
-- Remove arquivos órfãos mais antigos que um timestamp específico
ALTER TABLE iceberg_table EXECUTE remove_orphan_files('2026-03-01 00:00:00');
-- Dry run: visualize quais arquivos seriam excluídos
ALTER TABLE iceberg_table EXECUTE remove_orphan_files(dry_run = 1);
-- Escanear apenas o diretório de dados
ALTER TABLE iceberg_table EXECUTE remove_orphan_files(
older_than = '2026-03-01 00:00:00',
location = 'data/'
);
-- Combinar older_than posicional com argumentos nomeados
ALTER TABLE iceberg_table EXECUTE remove_orphan_files(
'2026-03-01 00:00:00',
dry_run = 1
);
Saída:
O comando retorna uma tabela com as colunas metric_name e metric_value, mostrando a contagem de arquivos excluídos (ou que seriam excluídos no modo dry_run) por categoria. As categorias de arquivos são classificadas com base em heurísticas de melhor esforço, seguindo convenções de nomenclatura de arquivos; arquivos que não correspondem a nenhum padrão específico são contabilizados por padrão em deleted_data_files_count:
| metric_name | metric_value |
|---|
| deleted_data_files_count | 5 |
| deleted_position_delete_files_count | 2 |
| deleted_equality_delete_files_count | 0 |
| deleted_manifest_files_count | 3 |
| deleted_manifest_lists_count | 1 |
| deleted_metadata_files_count | 0 |
| deleted_statistics_files_count | 0 |
| skipped_missing_metadata_count | 0 |
| failed_deletions_count | 0 |
Configurações:
| Configuração | Tipo | Padrão | Descrição |
|---|
allow_iceberg_remove_orphan_files | Bool | false | Configuração de controle para habilitar o recurso (experimental). |
iceberg_orphan_files_older_than_seconds | UInt64 | 259200 (3 days) | Limite padrão de older_than, em segundos, quando o argumento é omitido. |
- Requer o Iceberg format version 2 (ou superior). Tabelas da versão 1 são rejeitadas porque não têm ponteiros
manifest-list em snapshots, que são necessários para determinar com segurança o conjunto de arquivos alcançáveis. Executar o comando em uma tabela v1 retorna um erro BAD_ARGUMENTS.
- Requer que as configurações
allow_insert_into_iceberg e allow_iceberg_remove_orphan_files estejam habilitadas
- Recomenda-se executar
expire_snapshots antes de remove_orphan_files, para que os arquivos referenciados exclusivamente por snapshots expirados sejam removidos primeiro
- Use
dry_run = 1 para visualizar os arquivos órfãos antes da exclusão
- O limite
older_than protege contra a exclusão de arquivos de gravações em andamento — o limite padrão de 3 dias oferece uma margem de segurança generosa