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

Performance Issue with Retrieving Columns from Iceberg Catalog #23468

Open
shohamyamin opened this issue Sep 17, 2024 · 8 comments
Open

Performance Issue with Retrieving Columns from Iceberg Catalog #23468

shohamyamin opened this issue Sep 17, 2024 · 8 comments
Labels
iceberg Iceberg connector performance

Comments

@shohamyamin
Copy link

shohamyamin commented Sep 17, 2024

Hi,

We are experiencing a performance issue when retrieving column metadata from the Iceberg catalog using the following query:

SELECT * FROM iceberg.information_schema.columns;

In our environment this query takes approximately 30 seconds to return results with clean cache, and we have around 500 tables with 3.5k columns in total.

From our analysis of the logs on both the Iceberg catalog and Trino side, it appears that requests to the catalog might be executed sequentially for each table.

Setup Details:

  • Trino version: 453

We tried both catalogs:

link to slack discussion

To investigate the performance issue, I set up a Docker Compose environment with Trino 451 and the latest versions of the REST catalog, and the Nessie catalog (using in-memory configuration), along with MinIO for Iceberg data storage and Jaeger for tracing. This test was conducted on my laptop, which has 32GB of RAM and an Intel Core 9 processor.

I executed the following SQL query for both the REST and Nessie catalogs:

SELECT * FROM iceberg.information_schema.columns;

After each query, I restarted Trino to clear the cache.

Here are the results for 500 tables, each containing 10 columns (a total of 5,000 columns):

  • Nessie catalog: 9.09 seconds
  • REST catalog: 9.03 seconds

I suspect that the reason I observed these better performance results compared to my Kubernetes cluster is due to the local setup. Running MinIO locally allows for a closer proximity to Trino, reducing latency.

Is there any optimization or configuration change we could apply to improve the performance of this query? Additionally, any insights into the sequential request behavior would be helpful.

Thanks in advance for your assistance!

Here are some of the tracing(thanks @mosiac1 for the advice it is very useful ) that i got from open telemetry:

Timelines of both nessie and rest:
nessie_timeline
rest_timeline

You can see in the nessie(for some reson it is more detailed than the rest) timeline that for each table it makes a get from the s3 and it do it one after the other.
nessie_get_column_list_timeline

Here is the tracing as table:
nessie_table
rest_table

Here are the jsons of the tracing if someone wants to load them to jaeger:
rest_jaeger_columns_benchmark.json
nessie_jaeger_columns_benchmark.json.json

@shohamyamin shohamyamin added performance iceberg Iceberg connector labels Sep 17, 2024
@shohamyamin
Copy link
Author

@mosiac1 @ebyhr @findepi I added some tracing information that show the problem and maybe will help to understand why and how to deal with that.

@sopel39
Copy link
Member

sopel39 commented Sep 20, 2024

@shohamyamin perhaps #22739 could help in your case. It requres 454. cc @raunaqmorarka

@alonahmias
Copy link

Do you know the answer to this question?
#22739 (review)
Because the select from information schema is being done by the coordinator

@sopel39
Copy link
Member

sopel39 commented Sep 20, 2024

Yes, its created specifically for coordinator

@shohamyamin
Copy link
Author

The cache is being preformed before the filter columns apply for the specific user that has being logged in?

Because each user has access to different columns
and our problem is that we need that information has the user login to Trino using our system. So the first time should be very fast

Another question this cache is separated from the regular cache?

@mosiac1
Copy link
Contributor

mosiac1 commented Sep 20, 2024

I think its worth understanding why the traces from the Rest catalog do not show S3 operations. I am not very familiar with how Iceberg catalogs work (besides the very basic JDBC one), but from the traces I'm guessing Trino is delegating fetching of metadata to the Rest catalog - in that case there isn't much that can be done on the Trino side.

Another observation is that in the Nessie trace the ConnectorMetadata.streamRelationColumns span is 8.94s, but the self-time (time spent not waiting on children) is 5752.43ms, which implies Trino is spending some time on processing as well, this isn't bound just by IO.

In any case, its probably a good idea to do the S3 reads in parallel.

For your test, was Trino is a coordinator-only setup? Wondering is this would generate multiple splits, but I doubt that.

IMHO parallel reads of metadata files in Trino would be a band-aid solution. If you want this to fast (sub 500ms) you likely want to delegate the a Rest Iceberg catalog that reads from a Redis cache or Postgres DB without ever going to S3.

@shohamyamin
Copy link
Author

shohamyamin commented Sep 20, 2024

Yes, the Trino setup for this test is indeed coordinator-only.
And I agreed that making thous s3 requests in parallel will improve significant time

The s3 GET requests are not preformed by Trino?

About that self-time that can't be affected also by that trino need to wait for the responses?

@shohamyamin
Copy link
Author

shohamyamin commented Sep 23, 2024

@mosiac1 , it looks like this is the for loop performing the S3 get request synchronously:

link to code

Edit:

The relevant code is in the iceberg plugin:
link to code - iceberg metadata

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
iceberg Iceberg connector performance
Development

No branches or pull requests

4 participants