Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
42 changes: 42 additions & 0 deletions flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java
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);
}
}
26 changes: 15 additions & 11 deletions flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@
import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
import org.apache.flink.table.expressions.Expression;
import org.apache.flink.util.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.CachingCatalog;
import org.apache.iceberg.Table;
import org.apache.iceberg.catalog.Catalog;
Expand All @@ -71,26 +72,29 @@
*/
public class FlinkCatalog extends AbstractCatalog {

private final Catalog originalCatalog;
private final CatalogLoader catalogLoader;
private final Configuration hadoopConf;
Comment on lines +75 to +76
Copy link
Member

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.

private final Catalog icebergCatalog;
private final String[] baseNamespace;
private final SupportsNamespaces asNamespaceCatalog;
private final Closeable closeable;

public FlinkCatalog(
String catalogName,
String defaultDatabase,
String[] baseNamespace,
Catalog icebergCatalog,
CatalogLoader catalogLoader,
Configuration hadoopConf,
boolean cacheEnabled) {
super(catalogName, defaultDatabase);
this.originalCatalog = icebergCatalog;
this.icebergCatalog = cacheEnabled ? CachingCatalog.wrap(icebergCatalog) : icebergCatalog;
this.hadoopConf = hadoopConf;
this.catalogLoader = catalogLoader;
this.baseNamespace = baseNamespace;
if (icebergCatalog instanceof SupportsNamespaces) {
asNamespaceCatalog = (SupportsNamespaces) icebergCatalog;
} else {
asNamespaceCatalog = null;
}

Catalog originalCatalog = catalogLoader.loadCatalog(hadoopConf);
icebergCatalog = cacheEnabled ? CachingCatalog.wrap(originalCatalog) : originalCatalog;
asNamespaceCatalog = originalCatalog instanceof SupportsNamespaces ? (SupportsNamespaces) originalCatalog : null;
closeable = originalCatalog instanceof Closeable ? (Closeable) originalCatalog : null;
}

@Override
Expand All @@ -99,9 +103,9 @@ public void open() throws CatalogException {

@Override
public void close() throws CatalogException {
if (originalCatalog instanceof Closeable) {
if (closeable != null) {
try {
((Closeable) originalCatalog).close();
closeable.close();
} catch (IOException e) {
throw new CatalogException(e);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,8 +27,6 @@
import org.apache.flink.table.descriptors.CatalogDescriptorValidator;
import org.apache.flink.table.factories.CatalogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.hadoop.HadoopCatalog;
import org.apache.iceberg.hive.HiveCatalog;
import org.apache.iceberg.relocated.com.google.common.base.Splitter;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
Expand All @@ -48,7 +46,7 @@
* </ul>
* <p>
* To use a custom catalog that is not a Hive or Hadoop catalog, extend this class and override
* {@link #buildIcebergCatalog(String, Map)}.
* {@link #createCatalogLoader(String, Map)}.
*/
public class FlinkCatalogFactory implements CatalogFactory {

Expand All @@ -62,37 +60,23 @@ public class FlinkCatalogFactory implements CatalogFactory {
public static final String BASE_NAMESPACE = "base-namespace";

/**
* Build an Iceberg {@link org.apache.iceberg.catalog.Catalog} to be used by this Flink catalog adapter.
* Create an Iceberg {@link org.apache.iceberg.catalog.Catalog} loader to be used by this Flink catalog adapter.
*
* @param name Flink's catalog name
* @param options Flink's catalog options
* @return an Iceberg catalog
* @return an Iceberg catalog loader
*/
protected org.apache.iceberg.catalog.Catalog buildIcebergCatalog(String name, Map<String, String> options) {
Configuration conf = HadoopUtils.getHadoopConfiguration(GlobalConfiguration.loadConfiguration());
return buildIcebergCatalog(name, options, conf);
}

/**
* Build an Iceberg {@link org.apache.iceberg.catalog.Catalog} to be used by this Flink catalog adapter.
*
* @param name Flink's catalog name
* @param options Flink's catalog options
* @param conf Flink's hadoop configuration
* @return an Iceberg catalog
*/
protected org.apache.iceberg.catalog.Catalog buildIcebergCatalog(
String name, Map<String, String> options, Configuration conf) {
protected CatalogLoader createCatalogLoader(String name, Map<String, String> options) {
String catalogType = options.getOrDefault(ICEBERG_CATALOG_TYPE, "hive");
switch (catalogType) {
case "hive":
int clientPoolSize = Integer.parseInt(options.getOrDefault(HIVE_CLIENT_POOL_SIZE, "2"));
String uri = options.get(HIVE_URI);
return new HiveCatalog(name, uri, clientPoolSize, conf);
return CatalogLoader.hive(name, uri, clientPoolSize);

case "hadoop":
String warehouseLocation = options.get(HADOOP_WAREHOUSE_LOCATION);
return new HadoopCatalog(name, conf, warehouseLocation);
return CatalogLoader.hadoop(name, warehouseLocation);

default:
throw new UnsupportedOperationException("Unknown catalog type: " + catalogType);
Expand Down Expand Up @@ -121,12 +105,20 @@ public List<String> supportedProperties() {

@Override
public Catalog createCatalog(String name, Map<String, String> properties) {
org.apache.iceberg.catalog.Catalog catalog = buildIcebergCatalog(name, properties);
return createCatalog(name, properties, clusterHadoopConf());
}

protected Catalog createCatalog(String name, Map<String, String> properties, Configuration hadoopConf) {
CatalogLoader catalogLoader = createCatalogLoader(name, properties);
String defaultDatabase = properties.getOrDefault(DEFAULT_DATABASE, "default");
String[] baseNamespace = properties.containsKey(BASE_NAMESPACE) ?
Splitter.on('.').splitToList(properties.get(BASE_NAMESPACE)).toArray(new String[0]) :
new String[0];
boolean cacheEnabled = Boolean.parseBoolean(properties.getOrDefault("cache-enabled", "true"));
return new FlinkCatalog(name, defaultDatabase, baseNamespace, catalog, cacheEnabled);
return new FlinkCatalog(name, defaultDatabase, baseNamespace, catalogLoader, hadoopConf, cacheEnabled);
}

public static Configuration clusterHadoopConf() {
return HadoopUtils.getHadoopConfiguration(GlobalConfiguration.loadConfiguration());
}
}
107 changes: 107 additions & 0 deletions flink/src/main/java/org/apache/iceberg/flink/TableLoader.java
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 {
Copy link
Contributor

Choose a reason for hiding this comment

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

Do you really need this? isn't this covered by the CatalogTableLoader using hadoop?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I saw #1306 , I don't know if I misunderstood something.
It seems that a simple path can make it easier for users to use.

Copy link
Contributor

@edgarRd edgarRd Aug 20, 2020

Choose a reason for hiding this comment

The 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 HiveCatalog for tables that are backed in systems like s3. While HadoopCatalog for hdfs which does not implement HMS coordination since the filesystem itself has atomic renames.

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 HadoopTables if there's no context of the underlying limitations.

Copy link
Contributor Author

@JingsongLi JingsongLi Aug 20, 2020

Choose a reason for hiding this comment

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

Yes, There are some comments in HadoopCatalog: The HadoopCatalog requires that the underlying file system supports atomic rename. So as HadoopTables.
But I think if users use HadoopTables, HadoopCatalog or TableLoader.fromHadoopTable, users should know, yeah, we need a file system supports atomic rename instead S3 and etc...

In #1306, should be discussing how to support location in the SQL layer.
For Flink, there are two kinds of users: DataStream users and SQL users.

So here is my thoughts:

  • Just like Spark and Mapreduce. For a Flink DataStream programer, he/she can just specify a hadoop location path for reading and writing, or use iceberg Catalog.
  • But for Flink SQL user, he/she must specify an iceberg Catalog to load tables.


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();
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import org.apache.flink.types.Row;
import org.apache.flink.util.ArrayUtils;
import org.apache.flink.util.CloseableIterator;
import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.catalog.Catalog;
import org.apache.iceberg.catalog.Namespace;
import org.apache.iceberg.catalog.SupportsNamespaces;
Expand Down Expand Up @@ -103,10 +104,9 @@ public FlinkCatalogTestBase(String catalogName, String[] baseNamespace) {

FlinkCatalogFactory factory = new FlinkCatalogFactory() {
@Override
protected Catalog buildIcebergCatalog(String name, Map<String, String> options) {
// Flink hadoop configuration depends on system env, it is quiet hard to set from testing. So directly pass
// correct hadoop configuration.
return super.buildIcebergCatalog(name, options, hiveConf);
protected org.apache.flink.table.catalog.Catalog createCatalog(
String name, Map<String, String> properties, Configuration hadoopConf) {
return super.createCatalog(name, properties, hiveConf);
}
};
tEnv.registerCatalog(
Expand Down