Skip to content

Commit

Permalink
Support iceberg mtmv query.
Browse files Browse the repository at this point in the history
  • Loading branch information
Jibing-Li committed Dec 19, 2024
1 parent 24328d1 commit 24f9dba
Show file tree
Hide file tree
Showing 12 changed files with 352 additions and 87 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -60,14 +60,9 @@ public boolean isDefaultPartition() {

@Override
public PartitionKeyDesc toPartitionKeyDesc() {
if (partitionKeyRange.hasLowerBound()) {
return PartitionKeyDesc.createFixed(
return PartitionKeyDesc.createFixed(
PartitionInfo.toPartitionValue(partitionKeyRange.lowerEndpoint()),
PartitionInfo.toPartitionValue(partitionKeyRange.upperEndpoint()));
} else {
// For null partition value.
return PartitionKeyDesc.createLessThan(PartitionInfo.toPartitionValue(partitionKeyRange.upperEndpoint()));
}
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -543,7 +543,7 @@ public Optional<SchemaCacheValue> initSchema() {
}

private List<Column> getIcebergSchema() {
return IcebergUtils.getSchema(catalog, dbName, name);
return IcebergUtils.getSchema(catalog, dbName, name, IcebergUtils.LATEST_SNAPSHOT_ID);
}

private List<Column> getHudiSchema() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,9 +27,14 @@
import org.apache.doris.catalog.RangePartitionItem;
import org.apache.doris.common.AnalysisException;
import org.apache.doris.common.DdlException;
import org.apache.doris.datasource.CacheException;
import org.apache.doris.datasource.ExternalSchemaCache;
import org.apache.doris.datasource.ExternalSchemaCache.SchemaCacheKey;
import org.apache.doris.datasource.ExternalTable;
import org.apache.doris.datasource.SchemaCacheValue;
import org.apache.doris.datasource.mvcc.MvccSnapshot;
import org.apache.doris.datasource.mvcc.MvccTable;
import org.apache.doris.datasource.mvcc.MvccUtil;
import org.apache.doris.mtmv.MTMVBaseTableIf;
import org.apache.doris.mtmv.MTMVRefreshContext;
import org.apache.doris.mtmv.MTMVRelatedTableIf;
Expand Down Expand Up @@ -77,7 +82,7 @@
import java.util.Set;
import java.util.stream.Collectors;

public class IcebergExternalTable extends ExternalTable implements MTMVRelatedTableIf, MTMVBaseTableIf {
public class IcebergExternalTable extends ExternalTable implements MTMVRelatedTableIf, MTMVBaseTableIf, MvccTable {

public static final String YEAR = "year";
public static final String MONTH = "month";
Expand All @@ -102,6 +107,8 @@ public String getIcebergCatalogType() {
protected synchronized void makeSureInitialized() {
super.makeSureInitialized();
if (!objectCreated) {
IcebergSchemaCacheKey key = new IcebergSchemaCacheKey(dbName, name, IcebergUtils.LATEST_SNAPSHOT_ID);
initSchema(key);
objectCreated = true;
}
}
Expand All @@ -117,39 +124,23 @@ public void setPartitionColumns(List<Column> partitionColumns) {
}

@Override
public Optional<SchemaCacheValue> initSchema() {
public Optional<SchemaCacheValue> initSchema(SchemaCacheKey key) {
table = IcebergUtils.getIcebergTable(catalog, dbName, name);
List<Column> schema = IcebergUtils.getSchema(catalog, dbName, name);
Snapshot snapshot = table.currentSnapshot();
if (snapshot == null) {
LOG.debug("Table {} is empty", name);
return Optional.of(new IcebergSchemaCacheValue(schema, null, -1, null));
}
long snapshotId = snapshot.snapshotId();
partitionColumns = null;
IcebergPartitionInfo partitionInfo = null;
if (isValidRelatedTable()) {
PartitionSpec spec = table.spec();
partitionColumns = Lists.newArrayList();

// For iceberg table, we only support table with 1 partition column as RelatedTable.
// So we use spec.fields().get(0) to get the partition column.
Types.NestedField col = table.schema().findField(spec.fields().get(0).sourceId());
List<Column> schema = IcebergUtils.getSchema(catalog, dbName, name,
((IcebergSchemaCacheKey) key).getSnapshotId());
List<Column> tmpColumns = Lists.newArrayList();
PartitionSpec spec = table.spec();
for (PartitionField field : spec.fields()) {
Types.NestedField col = table.schema().findField(field.sourceId());
for (Column c : schema) {
if (c.getName().equalsIgnoreCase(col.name())) {
partitionColumns.add(c);
tmpColumns.add(c);
break;
}
}
Preconditions.checkState(partitionColumns.size() == 1,
"Support 1 partition column for iceberg table, but found " + partitionColumns.size());
try {
partitionInfo = loadPartitionInfo();
} catch (AnalysisException e) {
LOG.warn("Failed to load iceberg table {} partition info.", name, e);
}
}
return Optional.of(new IcebergSchemaCacheValue(schema, partitionColumns, snapshotId, partitionInfo));
partitionColumns = tmpColumns;
return Optional.of(new IcebergSchemaCacheValue(schema, partitionColumns));
}

@Override
Expand Down Expand Up @@ -187,6 +178,12 @@ public Table getIcebergTable() {
return IcebergUtils.getIcebergTable(getCatalog(), getDbName(), getName());
}

private IcebergSnapshotCacheValue getIcebergSnapshotCacheValue() {
makeSureInitialized();
return Env.getCurrentEnv().getExtMetaCacheMgr().getIcebergMetadataCache()
.getSnapshotCache(catalog, dbName, name);
}

@Override
public void beforeMTMVRefresh(MTMV mtmv) throws DdlException {
Env.getCurrentEnv().getRefreshManager()
Expand All @@ -195,16 +192,14 @@ public void beforeMTMVRefresh(MTMV mtmv) throws DdlException {

@Override
public Map<String, PartitionItem> getAndCopyPartitionItems(Optional<MvccSnapshot> snapshot) {
return Maps.newHashMap(getPartitionInfoFromCache().getNameToPartitionItem());
makeSureInitialized();
return Maps.newHashMap(getOrFetchSnapshotCacheValue(snapshot).getPartitionInfo().getNameToPartitionItem());
}

private IcebergPartitionInfo getPartitionInfoFromCache() {
@Override
public Map<String, PartitionItem> getNameToPartitionItems(Optional<MvccSnapshot> snapshot) {
makeSureInitialized();
Optional<SchemaCacheValue> schemaCacheValue = getSchemaCacheValue();
if (!schemaCacheValue.isPresent()) {
return new IcebergPartitionInfo();
}
return ((IcebergSchemaCacheValue) schemaCacheValue.get()).getPartitionInfo();
return getOrFetchSnapshotCacheValue(snapshot).getPartitionInfo().getNameToPartitionItem();
}

@Override
Expand All @@ -215,26 +210,26 @@ public PartitionType getPartitionType(Optional<MvccSnapshot> snapshot) {

@Override
public Set<String> getPartitionColumnNames(Optional<MvccSnapshot> snapshot) throws DdlException {
return getPartitionColumnsFromCache().stream().map(Column::getName).collect(Collectors.toSet());
makeSureInitialized();
IcebergSnapshotCacheValue snapshotValue = getOrFetchSnapshotCacheValue(snapshot);
IcebergSchemaCacheValue schemaValue = getIcebergSchemaCacheValue(snapshotValue.getSnapshotId());
return schemaValue.getPartitionColumns().stream().map(Column::getName).collect(Collectors.toSet());
}

@Override
public List<Column> getPartitionColumns(Optional<MvccSnapshot> snapshot) {
return getPartitionColumnsFromCache();
}

private List<Column> getPartitionColumnsFromCache() {
makeSureInitialized();
Optional<SchemaCacheValue> schemaCacheValue = getSchemaCacheValue();
return schemaCacheValue
.map(cacheValue -> ((IcebergSchemaCacheValue) cacheValue).getPartitionColumns())
.orElseGet(Lists::newArrayList);
IcebergSnapshotCacheValue snapshotValue = getOrFetchSnapshotCacheValue(snapshot);
IcebergSchemaCacheValue schemaValue = getIcebergSchemaCacheValue(snapshotValue.getSnapshotId());
return schemaValue.getPartitionColumns();
}

@Override
public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context,
Optional<MvccSnapshot> snapshot) throws AnalysisException {
long latestSnapshotId = getPartitionInfoFromCache().getLatestSnapshotId(partitionName);
makeSureInitialized();
IcebergSnapshotCacheValue snapshotValue = getOrFetchSnapshotCacheValue(snapshot);
long latestSnapshotId = snapshotValue.getPartitionInfo().getLatestSnapshotId(partitionName);
if (latestSnapshotId <= 0) {
throw new AnalysisException("can not find partition: " + partitionName);
}
Expand All @@ -244,16 +239,9 @@ public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshCont
@Override
public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, Optional<MvccSnapshot> snapshot)
throws AnalysisException {
return new MTMVVersionSnapshot(getLatestSnapshotIdFromCache());
}

public long getLatestSnapshotIdFromCache() throws AnalysisException {
makeSureInitialized();
Optional<SchemaCacheValue> schemaCacheValue = getSchemaCacheValue();
if (!schemaCacheValue.isPresent()) {
throw new AnalysisException("Can't find schema cache of table " + name);
}
return ((IcebergSchemaCacheValue) schemaCacheValue.get()).getSnapshotId();
IcebergSnapshotCacheValue snapshotValue = getOrFetchSnapshotCacheValue(snapshot);
return new MTMVVersionSnapshot(snapshotValue.getSnapshotId());
}

@Override
Expand All @@ -268,6 +256,7 @@ public boolean isPartitionColumnAllowNull() {
*/
@Override
public boolean isValidRelatedTable() {
makeSureInitialized();
if (isValidRelatedTableCached) {
return isValidRelatedTable;
}
Expand Down Expand Up @@ -299,8 +288,47 @@ public boolean isValidRelatedTable() {
return isValidRelatedTable;
}

protected IcebergPartitionInfo loadPartitionInfo() throws AnalysisException {
List<IcebergPartition> icebergPartitions = loadIcebergPartition();
@Override
public MvccSnapshot loadSnapshot() {
return new IcebergMvccSnapshot(getIcebergSnapshotCacheValue());
}

public long getLatestSnapshotId() {
if (table == null) {
table = IcebergUtils.getIcebergTable(catalog, dbName, name);
}
Snapshot snapshot = table.currentSnapshot();
return snapshot == null ? IcebergUtils.LATEST_SNAPSHOT_ID : table.currentSnapshot().snapshotId();
}

@Override
public List<Column> getFullSchema() {
Optional<MvccSnapshot> snapshotFromContext = MvccUtil.getSnapshotFromContext(this);
IcebergSnapshotCacheValue cacheValue = getOrFetchSnapshotCacheValue(snapshotFromContext);
return getIcebergSchemaCacheValue(cacheValue.getSnapshotId()).getSchema();
}

@Override
public boolean supportInternalPartitionPruned() {
return true;
}

public IcebergSchemaCacheValue getIcebergSchemaCacheValue(long snapshotId) {
ExternalSchemaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(catalog);
Optional<SchemaCacheValue> schemaCacheValue = cache.getSchemaValue(
new IcebergSchemaCacheKey(dbName, name, snapshotId));
if (!schemaCacheValue.isPresent()) {
throw new CacheException("failed to getSchema for: %s.%s.%s.%s",
null, catalog.getName(), dbName, name, snapshotId);
}
return (IcebergSchemaCacheValue) schemaCacheValue.get();
}

public IcebergPartitionInfo loadPartitionInfo(long snapshotId) throws AnalysisException {
if (!isValidRelatedTable() || snapshotId == IcebergUtils.LATEST_SNAPSHOT_ID) {
return new IcebergPartitionInfo();
}
List<IcebergPartition> icebergPartitions = loadIcebergPartition(snapshotId);
Map<String, IcebergPartition> nameToPartition = Maps.newHashMap();
Map<String, PartitionItem> nameToPartitionItem = Maps.newHashMap();
for (IcebergPartition partition : icebergPartitions) {
Expand All @@ -314,11 +342,11 @@ protected IcebergPartitionInfo loadPartitionInfo() throws AnalysisException {
return new IcebergPartitionInfo(nameToPartitionItem, nameToPartition, partitionNameMap);
}

public List<IcebergPartition> loadIcebergPartition() {
public List<IcebergPartition> loadIcebergPartition(long snapshotId) {
PartitionsTable partitionsTable = (PartitionsTable) MetadataTableUtils
.createMetadataTableInstance(table, MetadataTableType.PARTITIONS);
List<IcebergPartition> partitions = Lists.newArrayList();
try (CloseableIterable<FileScanTask> tasks = partitionsTable.newScan().planFiles()) {
try (CloseableIterable<FileScanTask> tasks = partitionsTable.newScan().useSnapshot(snapshotId).planFiles()) {
for (FileScanTask task : tasks) {
CloseableIterable<StructLike> rows = task.asDataTask().rows();
for (StructLike row : rows) {
Expand Down Expand Up @@ -384,11 +412,12 @@ public IcebergPartition generateIcebergPartition(StructLike row) {
@VisibleForTesting
public Range<PartitionKey> getPartitionRange(String value, String transform)
throws AnalysisException {
// For NULL value, create a lessThan partition for it.
// For NULL value, create a minimum partition for it.
if (value == null) {
PartitionKey nullKey = PartitionKey.createPartitionKey(
Lists.newArrayList(new PartitionValue("0000-01-02")), partitionColumns);
return Range.lessThan(nullKey);
PartitionKey nullLowKey = PartitionKey.createPartitionKey(
Lists.newArrayList(new PartitionValue("0000-01-01")), partitionColumns);
PartitionKey nullUpKey = nullLowKey.successor();
return Range.closedOpen(nullLowKey, nullUpKey);
}
LocalDateTime epoch = Instant.EPOCH.atZone(ZoneId.of("UTC")).toLocalDateTime();
LocalDateTime target;
Expand Down Expand Up @@ -525,4 +554,12 @@ public boolean validRelatedTableCache() {
public void setIsValidRelatedTableCached(boolean isCached) {
this.isValidRelatedTableCached = isCached;
}

private IcebergSnapshotCacheValue getOrFetchSnapshotCacheValue(Optional<MvccSnapshot> snapshot) {
if (snapshot.isPresent()) {
return ((IcebergMvccSnapshot) snapshot.get()).getSnapshotCacheValue();
} else {
return getIcebergSnapshotCacheValue();
}
}
}
Loading

0 comments on commit 24f9dba

Please sign in to comment.