Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

iceberg caching catalog refresh table failed #55828

Open
eshishki opened this issue Feb 12, 2025 · 12 comments · May be fixed by #56179
Open

iceberg caching catalog refresh table failed #55828

eshishki opened this issue Feb 12, 2025 · 12 comments · May be fixed by #56179
Assignees
Labels
type/bug Something isn't working

Comments

@eshishki
Copy link
Contributor

The issue seems to be introduced by #53007

on table refresh we get this stacktrace

 [CachingIcebergCatalog.refreshCatalog():385] refresh landing_crm_tj.users metadata cache failed, msg :
com.google.common.util.concurrent.UncheckedExecutionException: java.lang.IllegalArgumentException: Cannot find snapshot with ID 6735346390613841142
        at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2085) ~[spark-dpp-1.0.0.jar:?]
        at com.google.common.cache.LocalCache.get(LocalCache.java:4011) ~[spark-dpp-1.0.0.jar:?]
        at com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:4034) ~[spark-dpp-1.0.0.jar:?]
        at com.google.common.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:5010) ~[spark-dpp-1.0.0.jar:?]
        at com.google.common.cache.LocalCache$LocalLoadingCache.getUnchecked(LocalCache.java:5017) ~[spark-dpp-1.0.0.jar:?]
        at com.starrocks.connector.iceberg.CachingIcebergCatalog.refreshTable(CachingIcebergCatalog.java:287) ~[starrocks-fe.jar:?]
        at com.starrocks.connector.iceberg.CachingIcebergCatalog.refreshTable(CachingIcebergCatalog.java:273) ~[starrocks-fe.jar:?]
        at com.starrocks.connector.iceberg.CachingIcebergCatalog.refreshCatalog(CachingIcebergCatalog.java:327) ~[starrocks-fe.jar:?]
        at com.starrocks.connector.hive.ConnectorTableMetadataProcessor.refreshIcebergCachingCatalog(ConnectorTableMetadataProcessor.java:146) ~[starrocks-fe.jar:?]
        at com.starrocks.connector.hive.ConnectorTableMetadataProcessor.runAfterCatalogReady(ConnectorTableMetadataProcessor.java:96) ~[starrocks-fe.jar:?]
        at com.starrocks.common.util.FrontendDaemon.runOneCycle(FrontendDaemon.java:72) ~[starrocks-fe.jar:?]
        at com.starrocks.common.util.Daemon.run(Daemon.java:98) ~[starrocks-fe.jar:?]
Caused by: java.lang.IllegalArgumentException: Cannot find snapshot with ID 6735346390613841142
        at org.apache.iceberg.relocated.com.google.common.base.Preconditions.checkArgument(Preconditions.java:204) ~[iceberg-bundled-guava-1.7.1.jar:?]
        at org.apache.iceberg.SnapshotScan.useSnapshot(SnapshotScan.java:85) ~[iceberg-core-1.7.1.jar:?]
        at org.apache.iceberg.BaseTableScan.useSnapshot(BaseTableScan.java:25) ~[iceberg-core-1.7.1.jar:?]
        at com.starrocks.connector.iceberg.IcebergCatalog.getPartitions(IcebergCatalog.java:197) ~[starrocks-fe.jar:?]
        at com.starrocks.connector.iceberg.CachingIcebergCatalog.lambda$new$0(CachingIcebergCatalog.java:101) ~[starrocks-fe.jar:?]
        at com.google.common.cache.CacheLoader$FunctionToCacheLoader.load(CacheLoader.java:169) ~[spark-dpp-1.0.0.jar:?]
        at com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3570) ~[spark-dpp-1.0.0.jar:?]
        at com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2312) ~[spark-dpp-1.0.0.jar:?]
        at com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2189) ~[spark-dpp-1.0.0.jar:?]
        at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2079) ~[spark-dpp-1.0.0.jar:?]
        ... 11 more

Cannot find snapshot with ID is a message from iceberg 'SnapshotScan.useSnapshot'
i verified that snapshot do exist, but the error is real too

so the issue seems to be that we try to set this snapshot to metadata table scan, which is not the same as base table
#53007 (comment)

@eshishki eshishki added the type/bug Something isn't working label Feb 12, 2025
@dirtysalt
Copy link
Contributor

@eshishki how do you reproduce this case? or how can I reproduce this case?

@eshishki
Copy link
Contributor Author

@dirtysalt create iceberg connector with caching catalog
set background catalog refresh time to 1 min
create table and insert some data, maybe in spark
select from starrocks, the table is in cache
insert more data in spark
in 1 minute starrocks will try to refresh the table and fail on this

@dirtysalt
Copy link
Contributor

dirtysalt commented Feb 15, 2025

Yeah, I can reproduce that. I almost know what happens.

  • create a partition iceberg table and a mv on that icberg table
  • add a new record to iceberg table, so snaopshot id will change (at spark side)
  • and refresh mv (so to get latest partitions), but use old(cached) snapshot id

And I think the reason is (According to gpt, not sure if it's correct): partition table scan only supports latest snapshot id.

@dirtysalt
Copy link
Contributor

@eshishki I think i have to close this pr. this failure is expected.

the reason why it happens is becuas there could be diff between

  • the latest snapshot id (A) what starrocks think is
  • and the latest snapshot id(B) is actually are

When you update iceberg table out from starrocks(like spark), B will get updated, but A is not(until cache expired). But eventually the snapshot id will be the same.

However here we just want to get partitions of (A) but not (B). So we should pass snapshot_id=A. However becuase of iceberg implementation, if A is not the latest snapshot id, it will fal. But that's ok, we won't get incorrect result.

And if we don't pass snapshot_id=A, this function will return result of partitions in snapshot_id=B. Thart's not what we expect.

@dirtysalt dirtysalt self-assigned this Feb 20, 2025
@eshishki
Copy link
Contributor Author

no, the issue is that if does not support snaphost scan in metascan at all
when you use it in refresh table, you pass valid latest snaphost_id, but you cant do that

@dirtysalt
Copy link
Contributor

yes. another way is to check snapshot id is the latest , to avoid this exception.

but this exception does not have any harm. because we want partitions of snapshot id A, but latest snapshot is B, then we should not get partitions.

@eshishki
Copy link
Contributor Author

no, during refresh catalog, we resolve new table snapshot, we request table partitions of this new snapshot, and fail here, so we dont cache the table and partitions names

@dirtysalt
Copy link
Contributor

let me check with my team mate, I guess maybe it's not correct here.

this snapshot id is still old one. maybe have to use updateOps.current().snapshotid().

            if (!currentLocation.equals(updateLocation)) {
                LOG.info("Refresh iceberg caching catalog table {}.{} from {} to {}",
                        dbName, tableName, currentLocation, updateLocation);
                long baseSnapshotId = currentOps.current().currentSnapshot().snapshotId(); //<---- HERE
                refreshTable(updateTable, baseSnapshotId, dbName, tableName, executorService);
                LOG.info("Finished to refresh iceberg table {}.{}", dbName, tableName);
            }

@eshishki
Copy link
Contributor Author

eshishki commented Feb 22, 2025

no, we call
partitionCache.getUnchecked(updatedIcebergTableName);
it goes to delegate.getPartitions(icebergTable, key.snapshotId, null);
in getPartitions you set
if (snapshotId != -1) {
tableScan = tableScan.useSnapshot(snapshotId);
but the scan is not for updatedIcebergTableName but for MetadataTableType.PARTITIONS

if you wanted just to get the latest partitions and disregard snapshot, ignore that snapshot id

@dirtysalt
Copy link
Contributor

guys, If I say "I want explicity snapshot id = A partitions" explicitly.
but you just ignore snapshot id(don't pass snapshot id) and you give me the partitions which snapshot_id = B.
it's wrong.

here what I want to say is

  • exception is ok,
  • but never give me wrong result.

@eshishki
Copy link
Contributor Author

eshishki commented Feb 22, 2025

i believe you should use the old function https://github.com/StarRocks/starrocks/pull/53007/files#diff-14a0f2a038313d660d400d66ff60bf2546a7c61967a7ba6070c6f9706b8bc414L102
it plans table scan for given snapshot and looks for partition names in the data files

@dirtysalt
Copy link
Contributor

I discard it for perofmance wise. the reason why I need partitions, only when there is materialized view for iceberg partitions mapping. and it can tolerate some delays of refreshing parttions.

so normal scan case, partitions are not needed any more.

I get whay you mean

  • only for latest snapshot, I can use partition meta scan
  • otherwise I have to file scan tasks .

I'll review this design stuff.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
type/bug Something isn't working
Projects
None yet
Development

Successfully merging a pull request may close this issue.

2 participants