Skip to content

Conversation

@pvary
Copy link
Contributor

@pvary pvary commented Sep 23, 2020

This patch enables the following commands from Hive:

  • Table creation (with HadoopTables LOCATION should be provided too):
CREATE EXTERNAL TABLE customers 
STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' 
TBLPROPERTIES ('iceberg.mr.table.schema'='{"type":"struct","fields":[{"id":1,"name":"customer_id","required":true,"type":"long"},{"id":2,"name":"first_name","required":true,"type":"string"}]}');

or

CREATE EXTERNAL TABLE customers 
STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler'
LOCATION 's3://some_bucket/some_path/table_a'
TBLPROPERTIES ('iceberg.mr.table.schema'='{"type":"struct","fields":[{"id":1,"name":"customer_id","required":true,"type":"long"},{"id":2,"name":"first_name","required":true,"type":"string"}]}');
  • Table drop:
DROP TABLE customers;

The backing Iceberg table is created/dropped automatically using the Catalogs new interface methods created by (#1481).
With the help of #1407 this patch will enable CREATE/INSERT/DROP path using Hive queries and backed by Iceberg tables.

The patch consists of the following main changes:

  • The first commit is just the squashed commits of Core: Implement Catalogs.createTable and Catalogs.dropTable #1481 since they are needed here
  • HiveTableOperations / BaseMetastoreCatalog changes to allow initializing already created HMS tables by adding the appropriate Iceberg related data - The exception handling might merit a second look here, opted for the less disrupting change for now
  • Catalogs.canWorkWithoutHive method - not too proud about it but needed to throw an exception if the user wants to keep the backing Iceberg table but trying to drop the Hive table without purging. Any better ideas would be welcome 😄
  • Implementation of the HiveIcebergMetaHook which does the actual job
  • Small change in the HiveIcebergSerDe to handle Schema definition even if the table is not yet created
  • Tests for positive and negative cases

@pvary
Copy link
Contributor Author

pvary commented Sep 23, 2020

If you have time, could you please review and / or try it:
@massdosage, @marton-bod, @rdblue, @omalley
Thanks,
Peter

@rdblue
Copy link
Contributor

rdblue commented Sep 23, 2020

@pvary, why does this set the Iceberg schema using a table property?

@rdblue
Copy link
Contributor

rdblue commented Sep 24, 2020

I think I understand why the schema is used, but I'd like to use types from Hive DDL if possible. I'm not sure whether that would need to change in this PR or in #1481.

@pvary
Copy link
Contributor Author

pvary commented Sep 24, 2020

@pvary, why does this set the Iceberg schema using a table property?

It will be quite convoluted to access the Hive DDL columns and types. It will add another layer of complexity which I would like to address in another PR to keep the changes more reviewer friendly 😄

@massdosage
Copy link
Contributor

I did a little bit of testing with this on a distributed Hive cluster. Results below.

  1. Tried to create a table with no location set, got error java.lang.IllegalArgumentException: Table location not set - that looks correct.
  2. Created a table with location set to an HDFS path - this worked and the location folder was created.
  3. "SELECT *" from this table returned no result - as expected.
  4. I then tried an insert which didn't work, I might have the syntax of the insert command wrong?
    insert into foo.iceberg_customers select named_struct("customer_id","999","first_name","some first name");
    FAILED: SemanticException [Error 10044]: Line 1:12 Cannot insert into target table because column number/types are different 'iceberg_customers': Table insclause-0 has 2 columns, but query has 1 columns.
  5. I then dropped the table - this worked, it was correctly removed from the metastore and the location folder from HDFS.

It might be good to have a HiveRunner test that creates a table, does an insert and then reads the values back to check that that all works end to end?

@pvary
Copy link
Contributor Author

pvary commented Sep 24, 2020

I did a little bit of testing with this on a distributed Hive cluster. Results below.

Really appreciate your help here! Thanks!

  1. I then tried an insert which didn't work, I might have the syntax of the insert command wrong?
    insert into foo.iceberg_customers select named_struct("customer_id","999","first_name","some first name");
    FAILED: SemanticException [Error 10044]: Line 1:12 Cannot insert into target table because column number/types are different 'iceberg_customers': Table insclause-0 has 2 columns, but query has 1 columns.

I would expect this to work (with #1407 also applied)

insert into foo.iceberg_customers select 999,"some first name";

or

insert into foo.iceberg_customers values(999,"some first name");

To be honest, I have never used "named_struct" before 😄

It might be good to have a HiveRunner test that creates a table, does an insert and then reads the values back to check that that all works end to end?

That's the plan, but we need the writer code to get in first 😄

@massdosage
Copy link
Contributor

That's the plan, but we need the writer code to get in first smile

Ah yes, of course, I'm losing track of the different pull requests ;)

The reason I tried select named_struct was I thought that was the only way to match the field names up with their values, I didn't realise one could still do positional inserts using values. Anyway, I tried that and got
FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.tez.TezTask
on the Hive shell, looking in the log I got
2020-09-24T14:19:10,286 ERROR [b4e7cb36-2cbb-42ea-8a22-630b0850c204 main([])]: exec.Task (TezTask.java:execute(230)) - Failed to execute tez graph. java.lang.UnsupportedOperationException: Writing to an Iceberg table with Hive is not supported
which makes sense. Is there any way to get the exception message to appear in the shell, just that one line would be more user friendly than return code 1.

I then set the execution engine to MR and got
java.lang.UnsupportedOperationException: Writing to an Iceberg table with Hive is not supported at org.apache.iceberg.mr.hive.HiveIcebergOutputFormat.checkOutputSpecs(HiveIcebergOutputFormat.java:37)
in the shell. But yes, this is because I don't also have #1407 merged in so that makes sense. If I have some time I'll try again with that. At least the above shows some progress and that the expected code paths are being hit.

@qphien
Copy link
Contributor

qphien commented Sep 28, 2020

I think it is necessary to add a converter to transform Hive DDL schema to iceberg schema instead of specifying iceberg schema in TBLPROPERTIES. Maybe we can use

// Managed Table
CREATE TABLE icebergTable (id int, day string)
STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler'
TBLPROPERTIES('iceberg.mr.table.partition.spec'='day:day')

// External Table
CREATE EXTERNAL TABLE icebergTable (id int, day string)
STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler'
LOCATION 'hdfs://path/to/table'
TBLPROPERTIES('iceberg.mr.table.partition.spec'='id:identity')

to create managed/external table, iceberg.mr.table.partition.spec can be used to specify partition name and transform type.

@rdblue
Copy link
Contributor

rdblue commented Sep 29, 2020

I think it is necessary to add a converter to transform Hive DDL schema to iceberg schema instead of specifying iceberg schema in TBLPROPERTIES

I agree that we want this to be the behavior. @pvary, what is the plan for using the DDL schema instead of a table property?

@pvary
Copy link
Contributor Author

pvary commented Sep 29, 2020

I think it is necessary to add a converter to transform Hive DDL schema to iceberg schema instead of specifying iceberg schema in TBLPROPERTIES

I agree that we want this to be the behavior. @pvary, what is the plan for using the DDL schema instead of a table property?

I agree with you on this. It will be quite convoluted to access the Hive DDL columns and types. It will add another layer of complexity which I would like to address in another PR to keep the changes more reviewer friendly 😄

// is created yet.
// - When we are compiling the Hive query on HiveServer2 side - We only have table information (location/name),
// and we have to read the schema using the table data. This is called multiple times so there is room for
// optimizing here.
Copy link
Contributor

Choose a reason for hiding this comment

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

SchemaParser has a cache, so that should help some.

Peter Vary added 20 commits October 13, 2020 09:09
(cherry picked from commit bc13a60)
- Production code change: For HiveCatalog do not try to load the table in preCreateTable - it should not exist anyway
- Test code:
    - Remove stat related table properties when checking
    - Remove needToCheckSnapshotLocation() use Catalogs.hiveCatalog instead
    - Add new test to check Hive table creation above existing Iceberg table
    - locationForCreateTable for HadoopTable should use the same as createIcebergTable method
@rdblue rdblue merged commit 80ea771 into apache:master Oct 13, 2020
@rdblue
Copy link
Contributor

rdblue commented Oct 13, 2020

Merged! Thanks @pvary for all your work on this one!

@pvary
Copy link
Contributor Author

pvary commented Oct 14, 2020

Merged! Thanks @pvary for all your work on this one!

Thanks for the reviews and the merge @rdblue!

public void commitCreateTable(org.apache.hadoop.hive.metastore.api.Table hmsTable) {
if (icebergTable == null) {
if (Catalogs.hiveCatalog(conf)) {
catalogProperties.put(TableProperties.ENGINE_HIVE_ENABLED, true);

Choose a reason for hiding this comment

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

@pvary @rdblue @massdosage
In the iceberg documentation i could see that
"To enable Hive support globally for an application, set iceberg.engine.hive.enabled=true in its Hadoop configuration."

The value of iceberg.engine.hive.enabled must be true in order to enable hive support. However, I could still use Iceberg's hive support even if I set iceberg.engine.hive.enabled to false. If i understand correctly, the value "iceberg.engine.hive.enabled" is irrelevant.

Because, if it is a hive catalog we are setting engine.hive.enabled as true.
In

if (metadata.properties().get(TableProperties.ENGINE_HIVE_ENABLED) != null) {

We are initially evaluating the 'engine.hive.enabled' value. In cases where it's a Hive catalog, this value is consistently 'True', rendering the 'iceberg.engine.hive.enabled' as unnecessary.

Let me know, if i missed anything in my understanding

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@shivjha30: The code line you have highlighted is for overriding the global settings on table level. So you have 2 levels to enable the hive engine:

  • Catalog level (ConfigProperties.ENGINE_HIVE_ENABLED) - you can set the config in the hadoop configuration of the HiveCatalog
  • Table level (TableProperties.ENGINE_HIVE_ENABLED) - you can set the config in the table properties

Choose a reason for hiding this comment

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

Thanks @pvary for getting back.
I have comprehended the functionality of the two levels concerning the Hive engine activation. If I'm not mistaken, even when we set ConfigProperties.ENGINE_HIVE_ENABLED to FALSE in Hadoop configuration, we are passing the ConfigProperties.ENGINE_HIVE_ENABLED as TRUE in the HiveIcebergMetaHook#commitCreateTable if it's of HiveCatalog type. This action renders the table level configuration irrelevant.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

You are using the metahook if you are using Hive to create the table. We expect that you want to be able to read the table from Hive in this case, so you need the storage handlers to be set.

8n this case, you need the storage handlers on the classpath of the other readers as well.

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

Projects

None yet

Development

Successfully merging this pull request may close these issues.

7 participants