Skip to content

Hive read via HiveCatalog documentation#1748

Merged
rdblue merged 4 commits intoapache:masterfrom
ExpediaGroup:docs-hive-read-from-hive-catalog
Nov 20, 2020
Merged

Hive read via HiveCatalog documentation#1748
rdblue merged 4 commits intoapache:masterfrom
ExpediaGroup:docs-hive-read-from-hive-catalog

Conversation

@massdosage
Copy link
Contributor

No description provided.

@github-actions github-actions bot added the docs label Nov 10, 2020

##### Create an Iceberg table
The first step is to create an Iceberg table using the Spark/Java/Python API and `HiveCatalog`. For the purposes of this documentation we will assume that the table is called `table_b` and that the table location is `s3://some_path/table_b`.
TODO: what do we need to set up when we create this table programatically for everything to be registered correctly for read usage in Hive?
Copy link
Contributor Author

@massdosage massdosage Nov 10, 2020

Choose a reason for hiding this comment

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

@pvary What does one need to do in order to get the table set up properly for the Hive read path in this case? What I have tried to do so far is this, first create an Iceberg table using the HiveCatalog like so:

 PartitionSpec spec = PartitionSpec.unpartitioned();
 Schema schema = new Schema(optional(1, "id", Types.LongType.get()), optional(2, "name", Types.StringType.get()));
 SparkSession spark = SparkSession.builder().appName("IcebergTest").getOrCreate();
 Configuration hadoopConfiguration = spark.sparkContext().hadoopConfiguration();
 Catalog catalog = new HiveCatalog(hadoopConfiguration);
 TableIdentifier tableId = TableIdentifier.of("test", "iceberg_table_from_hive_catalog");
 catalog.createTable(tableId, schema, spec);

The table created in Hive by the above has DDL like so:

CREATE EXTERNAL TABLE `iceberg_table_from_hive_catalog`(
  `id` bigint COMMENT '', 
  `name` string COMMENT '')
ROW FORMAT SERDE 
  'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' 
STORED AS INPUTFORMAT 
  'org.apache.hadoop.mapred.FileInputFormat' 
OUTPUTFORMAT 
  'org.apache.hadoop.mapred.FileOutputFormat'
LOCATION
  's3://REDACTED/iceberg_table_from_hive_catalog'
TBLPROPERTIES (
  'metadata_location'='s3://REDACTED/iceberg_table_from_hive_catalog/metadata/00000-7addbbf2-1836-4973-86af-0511ae7577fb.metadata.json', 
  'table_type'='ICEBERG', 
  'transient_lastDdlTime'='1605007216')

Which is obviously incorrect as the StorageHandler hasn't been set etc.. I know you worked on a PR that set this all up properly as long as some config/setup was performed at table creation time. Can you please let me know what I need to do and I'll then document it accordingly once I test it working?

Copy link
Contributor

Choose a reason for hiding this comment

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

I think setting engine.hive.enabled=true should help.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

OK, so I tried adding this:

hadoopConfiguration.set("engine.hive.enabled", "true");

before passing the hadoopConfiguration on to the

HiveCatalog catalog = new HiveCatalog(hadoopConfiguration);

but it doesn't seem to make any difference to the DDL of the table that gets subsequently created.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Oh wait, iceberg.engine.hive.enabled is what it should be, let me try that...

Copy link
Contributor Author

Choose a reason for hiding this comment

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

OK, so, with setting that property the DDL looks better:

CREATE EXTERNAL TABLE `iceberg_table_from_hive_catalog`(
)
ROW FORMAT SERDE 
  'org.apache.iceberg.mr.hive.HiveIcebergSerDe' 
STORED BY 
  'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' 

LOCATION
  's3://REDACTED/iceberg_table_from_hive_catalog'
TBLPROPERTIES (
  'metadata_location'='s3://REDACTED/iceberg_table_from_hive_catalog/metadata/00000-95297465-a383-4f0b-b546-dffe7c4ff778.metadata.json', 
  'table_type'='ICEBERG', 
  'transient_lastDdlTime'='1605024789')

Not sure whether it's an issue that the schema information is missing? I also can't successfully query this table from Hive so I'll look into that next and see what the issue is. Thanks for helping me over the first hurdle though, I'll update this PR shortly with that required config setting.

Copy link
Contributor

Choose a reason for hiding this comment

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

Also, engine.hive.enabled is the table property. When a table has that property, it will be created with the storage handler. I set the property on my table from Spark and it worked.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Also, engine.hive.enabled is the table property. When a table has that property, it will be created with the storage handler. I set the property on my table from Spark and it worked.

Really? From what I can see the code uses the value from here

public static final String ENGINE_HIVE_ENABLED = "iceberg.engine.hive.enabled";
which is prefixed with iceberg as I said above. I tried it both ways and only this way worked but the schema in the Hive table definition is empty so Hive seems to be throwing an error about that when it tries to query it. I haven't tried setting iceberg.mr.catalog=hive in Hive when I do the query, I will try that next and see if it makes any difference and what impact that has on the HadoopTables version of it.

Copy link
Contributor

Choose a reason for hiding this comment

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

The logic for setting up the storage handler is here: https://github.com/apache/iceberg/blob/master/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java#L377-L384

It uses both the environment config and the table property.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, exactly, so in this case when I'm creating a new table it doesn't find the value in the table metadata so it drops through to line 383 and looks in the Configuration using the ConfigProperties key I mentioned above which is iceberg.engine.hive.enabled. I stepped through this code to figure out why it wasn't working with engine.hive.enabled and saw the property it was looking for in the conf is actually iceberg.engine.hive.enabled. So I guess there are two options here:

  1. you explicitly create the table with metadata and set engine.hive.enabled as a table property.

or

  1. you set the value iceberg.engine.hive.enabled explicitly in the conf object (like I'm doing in my example code above) or let this be automatically set by adding it to hive-site.xml.

If we agree on this I can document the above here ^. I

I'll then look into what's necessary in order to query the table and add it.

Copy link
Contributor

Choose a reason for hiding this comment

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

CC @lcspinter as he is working on coming up a generic way to handle properties / configuration values wrt Hive.

We would like to come up with solution where the user could easily understand how the different properties are used:

  • Iceberg table properties
  • HMS table properties
  • HMS serde properties
  • HiveConf
  • Default values

Also we should understand how these properties are used when:

  • Reading Hive / Iceberg tables
  • Writing Hive / Iceberg tables

@pvary
Copy link
Contributor

pvary commented Nov 13, 2020

@massdosage: I try to answer all of the open questions - might miss some, so please feel free to point them out.

  • The property iceberg.engine.hive.enabled/engine.hive.enabled only controls how the HMS metadata values are set (InputFormat/OutputFormat/SerDe/StorageHandler)
  • There are different DDLs for different catalogs, these should work since they are tested with the unit tests as well
-- empty iceberg.mr.catalog means HadoopTables

-- new table Hive + Iceberg
CREATE EXTERNAL TABLE customers 
STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler'
LOCATION '<LOCATION>'
TBLPROPERTIES ('iceberg.mr.table.schema'='<SCHEMA_JSON>', 'iceberg.mr.table.partition.spec'='<PART_SPEC_JSON>');

-- new Hive table above existing Iceberg table
CREATE EXTERNAL TABLE customers 
STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler'
LOCATION '<LOCATION>';

-- HadoopCatalog (same as HadoopTables)
set iceberg.mr.catalog=hadoop;

-- new table Hive + Iceberg
CREATE EXTERNAL TABLE customers 
STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler'
LOCATION '<LOCATION>'
TBLPROPERTIES ('iceberg.mr.table.schema'='<SCHEMA_JSON>', 'iceberg.mr.table.partition.spec'='<PART_SPEC_JSON>');

-- new Hive table above existing Iceberg table
CREATE EXTERNAL TABLE customers 
STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler'
LOCATION '<LOCATION>';

-- HiveCatalog
set iceberg.mr.catalog=hive;

-- new table Hive + Iceberg
CREATE EXTERNAL TABLE customers 
STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler'
TBLPROPERTIES ('iceberg.mr.table.schema'='<SCHEMA_JSON>', 'iceberg.mr.table.partition.spec'='<PART_SPEC_JSON>');

-- new Hive table above existing Iceberg table is an invalid case
  • You might provide the Schema json when creating the table, but the property should not be present on the created HMS table.
  • You said that you have problem selecting data from the tables. What is the exact Exception?

Thanks, and sorry for chiming in late 😢

@massdosage
Copy link
Contributor Author

@pvary thanks for the above. For now my focus is getting a new HiveCatalog table created in such a way that it can be queried. I'll try some variation of what you suggest above with various Hive configuration settings and table properties and see if I can get it working - if so I'll update the steps in this document so you and others can confirm if that's correct, if not I'll add the exceptions I'm seeing here and we can discuss.

@massdosage massdosage marked this pull request as ready for review November 19, 2020 11:53
@rdblue rdblue merged commit 953a7fd into apache:master Nov 20, 2020
@rdblue
Copy link
Contributor

rdblue commented Nov 20, 2020

Thanks @massdosage! Great to have this documented.

@massdosage massdosage deleted the docs-hive-read-from-hive-catalog branch November 20, 2020 09:13
@massdosage
Copy link
Contributor Author

Thanks @massdosage! Great to have this documented.

Sure, I'll hopefully have time in the next week or so to document the HadoopCataog and CustomCatalog paths.

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

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants