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: refactor IcebergMetadata to support multiple catalogs #6977

Merged
merged 1 commit into from
Aug 16, 2021

Conversation

jackye1995
Copy link
Member

Hi, this is Jack from AWS Athena. As Iceberg recently released 0.11.0, users can now dynamically load Catalog and FileIO implementations. This feature is already supported in Spark and Flink, but Trino's Iceberg connector is strongly coupled with Hive and cannot easily integrate with this feature. We believe that doing this integration can enable a much broader use case for Iceberg through Trino, as Iceberg is now supporting more and more new catalogs like GlueCatalog, NessieCatalog, JdbcCatalog. It will also provide a generic solution for PRs like #6528. There are 2 major blockers we see for integration:

  1. All metadata information are retrieved directly through HiveMetastore rather than Iceberg's Catalog
  2. Reading and writing file has to go through the HdfsEnvironment to leverage existing source and sink in the Hive connector

In this PR, we propose the following approach to update Iceberg connector to complete the integration. The approach is a modified version of our internal Iceberg connector implementation and updated based on the current Trino codebase:

  1. IcebergConfig is treated as the place to load Iceberg catalog properties
  2. IcebergResourceFactory is introduced to load Catalog and FileIO based on session and config information.
  3. IcebergGenericMetadata is introduced to perform all the metadata operations instead of IcebergMetadata.
  4. Please note that we do not want to deprecate the current IcebergMetadata. We think it can be renamed to IcebergHiveMetadata (I did not do in this PR to avoid too many file changes) and extend the generic metadata. The key reason to continue using it would be to leverage Hive's native security features. For example, some APIs such as setSchemaAuthorization are not supported natively in Iceberg.
  5. IcebergMetadataFactory is used to load the correct metadata class based on a flag in Iceberg config.
  6. page source and sink do not need to use Hive's implementations such as HdfsParquetPageSource. In this PR, we only show an example Parquet read code path, which introduces IcebergParquetPageSource that is built upon Iceberg's native SeekableInputStream.

So after all these changes, user can even run this connector without a Hadoop or Hive environment. For example, I am able to start the connector and do a read query of my existing Iceberg warehouse on AWS Glue and S3 with the following configurations:

connector.name=iceberg
iceberg.metadata-impl=io.trino.plugin.iceberg.IcebergGenericMetadata
iceberg.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog
iceberg.io-impl=org.apache.iceberg.aws.s3.S3FileIO
iceberg.warehouse=s3://my-bucket

Please note that this is a draft for discussion, I did not take time to fix unit tests and only tested directly through SQL in CLI. If we can agree on the general approach after discussion, I will start to put this draft into smaller PRs for contribution. I will also post some discussion points as comments in code.

@cla-bot
Copy link

cla-bot bot commented Feb 21, 2021

Thank you for your pull request and welcome to our community. We require contributors to sign our Contributor License Agreement, and we don't seem to have you on file. In order for us to review and merge your code, please submit the signed CLA to cla@trino.io. For more information, see https://github.com/trinodb/cla.

return catalogImpl;
}

@Config("iceberg.catalog-impl")
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe a simpler way is to make this metadataType for users to choose among options like GENERIC, HIVE, etc. There are quite a few places that would require a switch statement and can take advantage of an enum.


@Inject
public IcebergResourceFactory(
StaticMetastoreConfig metastoreConfig,
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Currently I still need to set a dummy value for hive.metastore.uri due to the check in StaticMetastoreConfig, otherwise connector initialization would fail with NPE. Would be great if anyone knows how to avoid initializing that based on the config flag.

}
}

private static Configuration getHadoopConfiguration()
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is for draft only, we need a better way to get the latest Hadoop configuration to initialize Iceberg resource. Currently the connector mostly rely on hdfsEnvironment.getConfiguration(hdfsContext, path), but here we need a way to get configuration without a path. I am not very familiar with this because for Glue we can just pass in null. Currently I just borrowed some code from Hive's ConfigurationUtils to make this work.

}
}

private static Map<String, String> getCatalogProperties(StaticMetastoreConfig metastoreConfig, IcebergConfig icebergConfig)
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In our internal Iceberg connector, we have a well-defined set of catalog properties. However, in open source users should be able to input arbitrary catalog property. This can be done in Spark and Flink easily, but Trino is a bit harder due to its use of a POJO to parse and hold the config map. I can use a string config additional-config and convert it to a map, but that sounds like a hack. Any better suggestions?

private String metadataImpl = IcebergMetadata.class.getName();
private String catalogImpl = HiveCatalog.class.getName();
private String catalogName = "trino";
private String ioImpl = HadoopFileIO.class.getName();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can define enums to define the metastore/file system type like we do for hive connector.

@martint
Copy link
Member

martint commented Feb 26, 2021

@cla-bot check

@cla-bot cla-bot bot added the cla-signed label Feb 26, 2021
@cla-bot
Copy link

cla-bot bot commented Feb 26, 2021

The cla-bot has been summoned, and re-checked this pull request!

@electrum
Copy link
Member

electrum commented Feb 26, 2021

Thanks for sending this draft. Supporting alternate catalogs is definitely a welcome feature and something I've wanted for a while. It might be helpful to have a call about this, but I'll start with some initial comments here:

  • We don't generally support arbitrary, pluggable things in Trino connectors. Everything we ship should be first class, tested, supported, have proper configuration, etc. This is a different philosophy than Hadoop or Spark, which are platforms for people writing programs. We don't use class names in configuration.
  • I don't see any benefit to supporting FileIO. We leverage the Hadoop file system code from the Hive connector that already has configuration, security integration, years of production testing, etc. This is the same reason that we use the metastore client from the Hive connector rather than the raw Hive one that Iceberg uses. While I'd love to get rid of all of the Hadoop related code, that's not feasible, so it's better to use it for everything. If there are things that FileIO supports that we don't support today, we should look at those specific things and figure out the best way to support them.
  • ConnectorMetadata doesn't seem like the right interface to use as the catalog abstraction. It would be better to extract the existing Hive metastore code from IcebergMetadata to be behind an interface like Catalog and SupportsNamespaces.

@rymurr
Copy link

rymurr commented Apr 1, 2021

I am also interested in alternative iceberg catalogs. @electrum and @jackye1995 what do you think the next steps are here and can this be broken down into a few smaller chunks? I would love to help but not sure where I can best do that.

@jackye1995
Copy link
Member Author

jackye1995 commented Apr 2, 2021

@rymurr Hi Ryan, thanks for bringing this up, too many things are going on and I almost forget about this PR.

@electrum thanks for the review, for your comments:

We don't generally support arbitrary, pluggable things in Trino connectors.

I think I am mostly interested in GlueCatalog and @rymurr in NessieCatalog, so we can add them as native types instead of loading them dynamically, that should be fine.

I don't see any benefit to supporting FileIO

Understandable, but as a part of effort to move systems to use AWS SDK v2, we find it difficult to migrate all the Hadoop dependencies, and FileIO interface in Iceberg allows us to bring in that new dependency. I think there is a solution that can satisfy both use cases:

The only difference between the 2 approach is that the current code path runs through the hive connector and is hard-coded to use FSDataInputStream, but after all it's just using the seek and readFully functionality that is exactly the same as Iceberg's SeekableInputStream. So if we can update data sources like HdfsParquetDataSource to accept a more generic input stream, both Hive and FileIO can run through the same read logic in the end. (similar change can also be applied for write)

ConnectorMetadata doesn't seem like the right interface to use as the catalog abstraction.

That would make it basically identical to the existing HiveCatalog in Iceberg. I am certainly happy to go with that approach if we want, but that would require a lot of refactoring of existing code in IcebergMetadata. It would be great to pull more current committers of the Trino Iceberg connector for this.

@electrum
Copy link
Member

as a part of effort to move systems to use AWS SDK v2, we find it difficult to migrate all the Hadoop dependencies

Trino has its own TrinoS3FileSystem and we can update it independently of anything else. The last time I looked at v2, it seemed to be incomplete and missing some things we needed, but perhaps that has changed. I think @pettyjamesm had also looked at this.

We'd need a way to test it at scale with realistic workloads and verify that there aren't any regressions in performance, memory, etc. Given that the S3 support in Trino is stable and has been working well for years, and that v1 seems to be actively maintained, I think there'd need to be significant benefits to upgrading to be worth the potential risk.

We don't want to have a separate S3 implementation for Iceberg, since that would mean duplicating all of the config options that we already have for S3: https://trino.io/docs/current/connector/hive-s3.html

I think we can separate FileIO from the Catalog change, since they seem independent.

That would make it basically identical to the existing HiveCatalog in Iceberg. I am certainly happy to go with that approach if we want, but that would require a lot of refactoring of existing code in IcebergMetadata.

Yes, it will be a large refactoring, but it's what needs to happen to allow multiple implementations. It will make the code cleaner as well, since all the metastore related code will be behind an interface.

It would be great to pull more current committers of the Trino Iceberg connector for this.

That would be me and @phd3.

@jackye1995
Copy link
Member Author

@electrum sounds good, thanks for the explanations. I will make changes in this PR based on the comments so far and set up a meeting with you and @phd3 for more details.

@rymurr
Copy link

rymurr commented Apr 20, 2021

@jackye1995 shout if I can help w/ the catalog refactor.

@jackye1995
Copy link
Member Author

jackye1995 commented Jun 3, 2021

@electrum @rymurr @findepi I have separated out the Glue part of work and make this PR only for refactoring so that the changes are more clear. Glue and other catalog support can be added in different PRs once we agree upon the interface. This also allows us to work on adding new catalogs in parallel.

The approach is mostly described in the Slack room, where the TrinoIcebergCatalog extends the basic Iceberg Catalog interface to offer additional functionalities that Trino has such as permission and view.

I expect other Iceberg native catalogs to be plugged in through something like:

public class TrinoGlueCatalog extends GlueCatalog implements TrinoIcebergCatalog {
    ...
}

For each new session, a new TrinoIcebergCatalog is created, this might seem wasteful for Hive catalog, but it should be fine with just 1 more object for each metadata call, no inner fields are initialized. The reason for that is because for Iceberg native catalogs, session should be used to initialize a new catalog to ensure the access control is set correctly for each session. A caching layer can be added to avoid redundant initialization as the next step.

The current caching logic in IcebergMetadata is refactored into TrinoIcebergCatalogCache to allow all catalogs to use the same caching logic. Instead of caching TableMetadata, it now caches Table, but it is essentially the same amount of information. I changed it because we do not manually construct an Iceberg TableMetadata anymore and can directly create a transaction through TrinoIcebergCatalog.

@rdblue you might also be interested in this, since there are view related APIs in TrinoIcebergCatalog, and you mentioned netflix is planning to add view catalog support to Iceberg. Currently I just add some view APIs based on the need of Trino, so that Hive implementation can use it, and all other implementations automatically throw unsupported exception. Once Iceberg has native view support, we can gradually deprecate the methods here and use native Iceberg methods. Please let me know if the current approach fits the overall roadmap in Iceberg, if not I can adjust it accordingly.

@jackye1995 jackye1995 changed the title Iceberg: dynamic Catalog and FileIO support Iceberg: refactor IcebergMetadata to support multiple catalogs Jun 3, 2021
@phd3 phd3 self-requested a review June 7, 2021 00:01
@findepi findepi requested review from losipiuk and findepi June 8, 2021 12:38
@losipiuk
Copy link
Member

losipiuk commented Jun 9, 2021

@jackye1995 thanks.

I skimmed through PR and I am not convinced that we need the pattern where new TrinoIcebergCatalog instance for each IcebergMetadata method call.
It would be much simpler if we created a single catalog instance for the whole life of IcebergMetadata.

You mentioned that "session should be used to initialize a new catalog to ensure the access control is set correctly for each session".
The other (IMO simpler) approach to achieving that is just to make methods in TrinoIcebergCatalog take Session (or some session-derived information) as a parameter.

That may block us from making TrinoIcebergCatalog extend org.apache.iceberg.catalog.Catalog but I would rather not do that anyway.
If for some implementation (say TrinoGlueCatalog) it makes sense to build on top of GlueCatalog from iceberg lib, it should wrap it, and delegate some calls to it.
It would be cleaner if the exposed interface is solely defined on the Trino side.

The approach I propose simplifies cache management, as then cache becomes sole responsibility of TrinoIcebergCatalog implementation, and is not leaking out to IcebergMetadata.

cc: @findepi, @electrum, @phd3

Copy link
Member

@phd3 phd3 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @jackye1995 for continuing this effort! +1 to @losipiuk's comment. It'd also make it easier for the different implementations to evolve separately in terms of using different Iceberg APIs. SupportNameSpaces and Catalog look like good references to decide the common interface methods (even though not all might be strictly required in Trino atm).

I think we can still see whether it makes sense to keep the cache in IcebergMetadata, and ask the catalog for relevant information only when required. It looks like things we want we can keep are generic iceberg objects. We can keep the cache invisible to catalogs. That way caching logic (and invalidation logic if any in future) doesn't need to be replicated across catalogs.

@jackye1995
Copy link
Member Author

jackye1995 commented Jun 16, 2021

@losipiuk @phd3 thanks for the comments!

The other (IMO simpler) approach to achieving that is just to make methods in TrinoIcebergCatalog take Session (or some session-derived information) as a parameter.

It'd also make it easier for the different implementations to evolve separately in terms of using different Iceberg APIs.

Because Iceberg's catalog is basically designed to be single tenant, for each session a new catalog is required to ensure security. So if we want to let the methods of the catalog take session, I think directly implementing the ConnectorMetadata is the most straight forward appraoch. And this also aligns with what @phd3 says to let them evolve separately.

The original implementation was written with this principle, to have basically 2 ConnectorMetadata implementations, one for Hive and one for all other catalogs. And the comment from @electrum was that we can share more functionalities, which lead to this new implementation that tries to share as much common function calls as possible.

I am fine with either approach, we just need to make a decision here, with:

  1. 2 ConnectorMetadata implementations: I think this is clean, allows 2 approaches to evolve separately, but cannot share much logic (we can have a base class for some caching functionality in the best case)
  2. a new interface that is similar to Iceberg's Catalog but takes session for every single method call: I think this is a bit redundant because it has to manually sync both changes in Iceberg and Trino, unless we have any good reason to do that.
  3. a new interface that extends Iceberg's Catalog: which is the current approach that let these implementations share the most amount of functionalities

Please let me know which way you guys think is better to proceed forward, and I can make change accordingly.

@losipiuk
Copy link
Member

@losipiuk @phd3 thanks for the comments!

The other (IMO simpler) approach to achieving that is just to make methods in TrinoIcebergCatalog take Session (or some session-derived information) as a parameter.

It'd also make it easier for the different implementations to evolve separately in terms of using different Iceberg APIs.

Because Iceberg's catalog is basically designed to be single tenant, for each session a new catalog is required to ensure security. So if we want to let the methods of the catalog take session, I think directly implementing the ConnectorMetadata is the most straight forward appraoch. And this also aligns with what @phd3 says to let them evolve separately.

The original implementation was written with this principle, to have basically 2 ConnectorMetadata implementations, one for Hive and one for all other catalogs. And the comment from @electrum was that we can share more functionalities, which lead to this new implementation that tries to share as much common function calls as possible.

I am fine with either approach, we just need to make a decision here, with:

  1. 2 ConnectorMetadata implementations: I think this is clean, allows 2 approaches to evolve separately, but cannot share much logic (we can have a base class for some caching functionality in the best case)

I think having two separate ConnectorMetadata implementations is a no-go. It implies too much code duplication. After all ConnectorMetadata has much more responsibilities than just serving as a catalog for tables/schemas/views etc.

  1. a new interface that is similar to Iceberg's Catalog but takes session for every single method call: I think this is a bit redundant because it has to manually sync both changes in Iceberg and Trino, unless we have any good reason to do that.
  2. a new interface that extends Iceberg's Catalog: which is the current approach that let these implementations share the most amount of functionalities

From those two I would prefer to go with (2) (even if it is some more boilerplate). I am not a big fan of basing TrinoIcebergCatalog abstraction on the specific interface from the Iceberg library. IMO it is nice to keep interfaces as focused, and as small as possible, and I am pretty sure that in Trino we will not need everything from Catalog / SupportsNamespaces and we do not know how those will evolve.

Please let me know which way you guys think is better to proceed forward, and I can make change accordingly.

As for cache integration, I think what @phd3 proposes (to try to keep cache at ConnectorMetadata layer) is reasonable. If for some reason we see that it is not sufficient, we can pass the cache instance to catalog instance when it is constructed.
Or maybe have a CachingTrinoCatalog wrapper we can use to add caching to any catalog implementation we happen to use. I do not think we need to make a decision regarding this right now.

@electrum please also chime in as you were involved in initial discussions regarding this PR.

@jackye1995
Copy link
Member Author

As for cache integration, I think what @phd3 proposes (to try to keep cache at ConnectorMetadata layer) is reasonable

Yes caching can always be added later for whatever approach we go, I am not super concerned about that.

I think having two separate ConnectorMetadata implementations is a no-go. It implies too much code duplication.

I do not fully get how approach 2 would solve the problem comparing to having 2 connector metadata implementations that just inherit the same base class to avoid duplicated code. Let's take List<String> listSchemaNames(ConnectorSession session) for example:

The Hive version would be:

metastore.getAllDatabases();

with metadata being a singleton. The Iceberg version would need to create a new catalog from the session, and then:

catalog.listNamespaces().stream().map(Namespace::toString).collect(Collectors.toList());

And most of the metadata operations follow a similar pattern. If there is an intermediate interface to adopt both approaches, I think we are just moving the complexity one level down without solving the actual issue of code duplication. And we now have to maintain on more additional interface in the process.

@losipiuk am I misunderstanding the proposal you have or missing anything?

@losipiuk
Copy link
Member

As for cache integration, I think what @phd3 proposes (to try to keep cache at ConnectorMetadata layer) is reasonable

Yes caching can always be added later for whatever approach we go, I am not super concerned about that.

I think having two separate ConnectorMetadata implementations is a no-go. It implies too much code duplication.

I do not fully get how approach 2 would solve the problem comparing to having 2 connector metadata implementations that just inherit the same base class to avoid duplicated code.

Yeah - with the base class you inherit from it would work. I missed that part. But I think it is slightly cleaner to separate the low level catalog access interface with methods like:

  • listSchemas
  • listSchemProperts
  • listTables
  • getTable
  • ...

From higher level methods ConnectorMetadata exposes:

  • beginRefreshMaterializedView
  • beginRefreshMaterializedView
  • applyFilter
  • ...

Technically you can expose the first set of methods as protected abstract methods in the base ConnectorMetadata class you inherit from. But my feeling is that it is cleaner to have those two separated.

@jackye1995
Copy link
Member Author

@losipiuk cool, I think we are actually on the same line then.

I had a discussion with @rdblue about the issue related to authZ difference in Iceberg and Trino, and we came to the conclusion that Iceberg's design did not have much consideration of a dynamic authZ for different calls to the same metastore, and that is something probably worth adding directly in Iceberg core if possible.

I think there can be a middle ground to go with approach 2 by building an authZ layer SessionCatalog<T> on top of the Iceberg catalog implementation, which does what you call "the low level catalog access interface" with a generic session type, and session based methods like listTables(Namespace ns, T session).

Trino will use SessionCatalog<ConnectorSession> where we have TrinoHiveCatalog as Hive implementation and TrinoIcebergCatalog as Iceberg delegated Catalog implementation where Iceberg's no-authZ catalog is cached internally in the implementation, and the actual catalog type is loaded by config.

Then there is a single ConnectorMetadata class that delegates calls to the correct SessionCatalog.

And I would hope to have that SessionCatalog moved to the Iceberg core library one day for Iceberg to have better authZ support. The design on Iceberg side can change, but at least it should not be too hard to migrate once it is available in Iceberg.

Some details around the view interface might need to be discussed in more details as Iceberg is adding ViewCatalog, but apart from that I think this should be a good plan that fits all the requirements discussed. I will start to update the code based on this plan, please let me know if you have any additional suggestions.

@jackye1995
Copy link
Member Author

@losipiuk @phd3 @electrum sorry I was a bit busy with other work in the past few weeks. I have updated based on the plan and rebased master, and it should be good for another review, thanks!

@jackye1995 jackye1995 force-pushed the iceberg-dynamic-catalog branch 2 times, most recently from aae6ebb to 160b136 Compare August 4, 2021 04:31
@jackye1995
Copy link
Member Author

@electrum @losipiuk any updates? I think many people are waiting for this feature, is there any other blocker to merge this PR?

Copy link
Member

@phd3 phd3 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The new interface looks great! Thanks for your work on this PR. added some comments, mostly nits.

Copy link
Member

@electrum electrum left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This looks good to me. Please address the comments from @phd3 and then we can merge this. Thanks for all the work on this. I know that having to keep rebasing a big refactor like this is a pain.

private final String trinoVersion;
private final boolean useUniqueTableLocation;

private final Map<SchemaTableName, TableMetadata> tableMetadataCache = new ConcurrentHashMap<>();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This actually brings up the question of scope of TrinoCatalog. A Trino ConnectorMetadata instance (i.e. IcebergMetadata) is created per SQL transaction via Connector.getMetadata(). Thus, we store the snapshot isolation info in the IcebergMetadata instance.

The snapshot isolation is primarily handled by the snapshotIds map in IcebergMetadata, to ensure we use the same table snapshots for the entire transaction. I believe that tableMetadataCache was added for performance. However, I think since Iceberg (currently?) does not support transactional (historical) metadata, the metadata cache is also required for snapshot isolation.

I would prefer to keep both snapshotIds and tableMetadataCache in IcebergMetadata, since that's the canonical place to store transaction information, and it keeps the catalog implementation simpler.

Going back to the TrinoCatalog scope question, what is expected? Do they need to be thread-safe? Could we turn TrinoCatalogFactory into a Guice Provider and inject TrinoCatalog as a singleton?

@jackye1995
Copy link
Member Author

@phd3 @electrum thanks for the review, I have updated all the codestyle suggestions.

Regarding the scope of TrinoCatalog, form correctness perspective I don't think anything has changed, the TrinoCatalog is created as a part of IcebergMetadata for every transaction. if we change the scope, this will certainly affect coorectness.

Regarding why the metadata cache is in TrinoHiveCatalog instead of IcebergMetadata, an implementation using Iceberg's native catalog will likely not use the same TableMetadata but instead just use the same Table directly to fulfill snapshot isolation, and it internally ensures the same table metadata is used. This is what Spark and Flink does today. So I think it is better to keep the table metadata cache as an internal implementation for now. If we see this becomes a common pattern, we can exteact a CachingTrinoCatalog for that shared feature later.

@jackye1995
Copy link
Member Author

@phd3 sorry some of your comments were hidden, I updated based on those.

@electrum
Copy link
Member

Relevant failure, looks like an exception message needs updating:

 2021-08-17 00:14:01 INFO: FAILURE     /    io.trino.tests.product.iceberg.TestIcebergHiveTablesCompatibility.testIcebergSelectFromHiveTable (Groups: storage_formats, iceberg) took 0.3 seconds
 2021-08-17 00:14:01 SEVERE: Failure cause:
 java.lang.AssertionError: 
 Expecting message:
   <"Query failed (#20210816_182901_01182_ep4qm): Wrong table type: test_iceberg_select_from_hive_1roqcf486yoi$files">
 to match regex:
   <"Query failed \(#\w+\):\Q Not an Iceberg table: default.test_iceberg_select_from_hive_1roqcf486yoi">
 but did not.

@jackye1995
Copy link
Member Author

@phd3 yes what @electrum cited was the reason the UnknownTableTypeException is removed from the system table catch clause. I completely forgot. When it is not an Iceberg table it should throw exception instead of returning no system table.

@electrum electrum merged commit 9d73101 into trinodb:master Aug 16, 2021
@electrum
Copy link
Member

Thanks!

@findepi
Copy link
Member

findepi commented Sep 1, 2021

Does this already allow Iceberg to work with Glue?

@dungdm93
Copy link
Member

dungdm93 commented Sep 5, 2021

Hello @jackye1995
Does trino support iceberg table in Hive define by StorageHandler? Something like:

CREATE TABLE student (
  id bigint,
  name string
) 
STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler';

@ckdarby
Copy link

ckdarby commented Sep 15, 2021

Does this already allow Iceberg to work with Glue?

@jackye1995 I'm wondering the same thing as @findepi, does this contain the work to work for Glue or was it refactoring to allow the ability for that?

@jackye1995
Copy link
Member Author

@findepi @ckdarby @dungdm93 not yet for Glue, good timing to ask, I am writing the PR for that, will publish tomorrow and ping you guys.

For the tables created with Hive storage handler, most likely it would work with the Trino's implementation, but there might be some edge cases I need to check because it's created by a different implementation after all. And Trino only supports a small subset of Iceberg table properties, so if you set some properties it's not likely to be respected by Trino.

@cccs-tom
Copy link
Member

@jackye1995 Do you still also plan to add support for Hadoop catalogs? I know it was never your primary use case, but I am very interested in getting that added to Trino.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Development

Successfully merging this pull request may close these issues.