-
Notifications
You must be signed in to change notification settings - Fork 3k
Flink: Introduce CatalogLoader and TableLoader #1332
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,42 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, | ||
| * software distributed under the License is distributed on an | ||
| * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
| * KIND, either express or implied. See the License for the | ||
| * specific language governing permissions and limitations | ||
| * under the License. | ||
| */ | ||
|
|
||
| package org.apache.iceberg.flink; | ||
|
|
||
| import java.io.Serializable; | ||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.iceberg.catalog.Catalog; | ||
| import org.apache.iceberg.hadoop.HadoopCatalog; | ||
| import org.apache.iceberg.hive.HiveCatalog; | ||
|
|
||
| /** | ||
| * Serializable loader to load an Iceberg {@link Catalog}. | ||
| */ | ||
| public interface CatalogLoader extends Serializable { | ||
|
|
||
| Catalog loadCatalog(Configuration hadoopConf); | ||
|
|
||
| static CatalogLoader hadoop(String name, String warehouseLocation) { | ||
| return conf -> new HadoopCatalog(name, conf, warehouseLocation); | ||
| } | ||
|
|
||
| static CatalogLoader hive(String name, String uri, int clientPoolSize) { | ||
| return conf -> new HiveCatalog(name, uri, clientPoolSize, conf); | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,107 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, | ||
| * software distributed under the License is distributed on an | ||
| * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
| * KIND, either express or implied. See the License for the | ||
| * specific language governing permissions and limitations | ||
| * under the License. | ||
| */ | ||
|
|
||
| package org.apache.iceberg.flink; | ||
|
|
||
| import java.io.Closeable; | ||
| import java.io.IOException; | ||
| import java.io.Serializable; | ||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.iceberg.Table; | ||
| import org.apache.iceberg.catalog.Catalog; | ||
| import org.apache.iceberg.catalog.TableIdentifier; | ||
| import org.apache.iceberg.hadoop.HadoopTables; | ||
|
|
||
| /** | ||
| * Serializable loader to load an Iceberg {@link Table}. | ||
| * Flink needs to get {@link Table} objects in the cluster (for example, to get splits), not just on the client side. | ||
| * So we need an Iceberg table loader to get the {@link Table} object. | ||
| */ | ||
| public interface TableLoader extends Closeable, Serializable { | ||
|
|
||
| void open(Configuration configuration); | ||
|
|
||
| Table loadTable(); | ||
|
|
||
| static TableLoader fromCatalog(CatalogLoader catalogLoader, TableIdentifier identifier) { | ||
| return new CatalogTableLoader(catalogLoader, identifier); | ||
| } | ||
|
|
||
| static TableLoader fromHadoopTable(String location) { | ||
| return new HadoopTableLoader(location); | ||
| } | ||
|
|
||
| class HadoopTableLoader implements TableLoader { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do you really need this? isn't this covered by the
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I saw #1306 , I don't know if I misunderstood something.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Right, I think part of the discussion is that providing a table via this API does not guarantee the atomic commit on table changes, therefore we have I'm not sure which case specifically this specific Flink implementation will use, but as mentioned in #1306 (comment) it seems dangerous to provide tables loaded with
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, There are some comments in In #1306, should be discussing how to support location in the SQL layer. So here is my thoughts:
|
||
|
|
||
| private static final long serialVersionUID = 1L; | ||
|
|
||
| private final String location; | ||
| private transient HadoopTables tables; | ||
|
|
||
| private HadoopTableLoader(String location) { | ||
| this.location = location; | ||
| } | ||
|
|
||
| @Override | ||
| public void open(Configuration configuration) { | ||
| tables = new HadoopTables(configuration); | ||
| } | ||
|
|
||
| @Override | ||
| public Table loadTable() { | ||
| return tables.load(location); | ||
| } | ||
|
|
||
| @Override | ||
| public void close() { | ||
| } | ||
| } | ||
|
|
||
| class CatalogTableLoader implements TableLoader { | ||
|
|
||
| private static final long serialVersionUID = 1L; | ||
|
|
||
| private final CatalogLoader catalogLoader; | ||
| private final String identifier; | ||
|
|
||
| private transient Catalog catalog; | ||
|
|
||
| private CatalogTableLoader(CatalogLoader catalogLoader, TableIdentifier tableIdentifier) { | ||
| this.catalogLoader = catalogLoader; | ||
| this.identifier = tableIdentifier.toString(); | ||
| } | ||
|
|
||
| @Override | ||
| public void open(Configuration configuration) { | ||
| catalog = catalogLoader.loadCatalog(configuration); | ||
| } | ||
|
|
||
| @Override | ||
| public Table loadTable() { | ||
| return catalog.loadTable(TableIdentifier.parse(identifier)); | ||
| } | ||
|
|
||
| @Override | ||
| public void close() throws IOException { | ||
| if (catalog instanceof Closeable) { | ||
| ((Closeable) catalog).close(); | ||
| } | ||
| } | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Those two seems don't have to be private members because I did not see anywhere accessing them except the constructor.