Skip to content

Conversation

@aokolnychyi
Copy link
Contributor

This PR enables support for HiveTables in IcebergSource and resolves #8.

@aokolnychyi
Copy link
Contributor Author

@Parth-Brahmbhatt @rdblue

HadoopTables tables = new HadoopTables(conf);
return tables.load(path.get());
} else {
HiveTables tables = new HiveTables(conf);
Copy link
Contributor

Choose a reason for hiding this comment

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

HiveTables creates a connection pool that we will need to close. Maybe we don't need to worry about this for now because Spark 3.0 will use a catalog with a more reasonable life-cycle, but it seems like a bad idea to leak connections. It is also difficult to close this because these are instantiated every time.

What about creating a static cache and getting a HiveTables instance based on the value of hive.metastore.uris?

@rdblue
Copy link
Contributor

rdblue commented Jun 28, 2019

@aokolnychyi, nice work! It is ready other than the connection problem. I think we need to add a static cache to avoid creating too many HMS connections.

import org.junit.Test;

import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS;
import static org.apache.iceberg.types.Types.NestedField.optional;
Copy link
Contributor

@rdsr rdsr Jun 28, 2019

Choose a reason for hiding this comment

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

Baseline may frown at this

Copy link
Contributor Author

Choose a reason for hiding this comment

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

We disable AvoidStaticImport for tests in checkstyle-suppressions.xml. I still try to avoid static imports, but the statement would be too long in this particular case.

Copy link
Contributor

Choose a reason for hiding this comment

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

Makes sense!

@rdblue
Copy link
Contributor

rdblue commented Jun 29, 2019

@aokolnychyi, I merged #240, so could you rebase and update this to use HiveCatalog?

@aokolnychyi
Copy link
Contributor Author

@rdblue @rdsr ready for another review round.

public static HiveCatalog loadCatalog(Configuration conf) {
// metastore URI can be null in local mode
String metastoreUri = conf.get(HiveConf.ConfVars.METASTOREURIS.varname, "");
return CATALOG_CACHE.get(metastoreUri, uri -> new HiveCatalog(conf));
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This might be a bit problematic as we cache the Hadoop conf. Let me think about possible implications.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Okay, one case when this won't work is for hadoop. data source options.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Building a cache Configuration -> HiveCatalog doesn't seem as an option either.

Copy link
Contributor

Choose a reason for hiding this comment

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

We're okay with that for 2.4 support. This will be improved when using a real catalog for Spark 3.0. Also, the hadoop. options are mostly for configuring the read or write, not for controlling the metastore.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@rdblue, you are right. However, hadoop. data source options can be used to set iceberg.compress.metadata if we want to compress metadata only in particular tables.

Ideally, iceberg.compress.metadata should be a table property. It is the only entry in ConfigProperties. The problem is that HadoopTables requires this config upfront to find tables. Maybe, we can circumvent this by trying files with/without gz suffix.

@rdblue rdblue merged commit b898129 into apache:master Jul 2, 2019
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Update iceberg-spark to use Hive tables

3 participants