Owen or Carl,

Do you have any thoughts on this approach?  We had previously discussed
this but now that we've looked into it more closely there are a few areas
that are unclear.

HiveMetaHook looks like a good entry point for DDL (though as Adrien
pointed out, it doesn't cover all operations).

However, I'm not clear on where to hook in for DML operations.  Is there a
similar hook for table commits to a table? Seems like hijacking the
MoveTask would be working around the commit problem.

Also, I'm not clear on whether this approach removes the intermediate
copies of task task output for task/job commit, which we probably want
remove.  I assume this could be done with the custom OutputFormat and a
custom OutputCommitter.

-Dan









On Thu, Jul 25, 2019 at 3:37 PM RD <rdsr...@gmail.com> wrote:

> Hi Adrien,
>    We at LinkedIn went through a similar thought process, but given our
> Hive deployment is not that large, we are in the process of considering
> deprecating Hive and asking our users to move to Spark [since Spark
> supports Hive ql].
>
> I'm guessing we'd have to invest in Spark's catalog AFAICT, but we haven't
> investigated this yet.
>
> -Best.
>
>
>
>
>
>
> On Wed, Jul 24, 2019 at 1:53 PM Adrien Guillo
> <adrien.gui...@airbnb.com.invalid> wrote:
>
>> Hi Iceberg folks,
>>
>> In the last few months, we (the data infrastructure team at Airbnb) have
>> been closely following the project. We are currently evaluating potential
>> strategies to migrate our data warehouse to Iceberg. However, we have a
>> very large Hive deployment, which means we can’t really do so without
>> support for Iceberg tables in Hive.
>>
>> We have been thinking about implementation strategies. Here are some
>> thoughts that we would like to share them with you:
>>
>> – Implementing a new `RawStore`
>>
>> This is something that has been mentioned several times on the mailing
>> list and seems to indicate that adding support for Iceberg tables in Hive
>> could be achieved without client-side modifications. Does that mean that
>> the Metastore is the only process manipulating Iceberg metadata (snapshots,
>> manifests)? Does that mean that for instance the `listPartition*` calls to
>> the Metastore return the DataFiles associated with each partition? Per our
>> understanding, it seems that supporting Iceberg tables in Hive with this
>> strategy will most likely require to update the RawStore interface AND will
>> require at least some client-side changes. In addition, with this strategy
>> the Metastore bears new responsibilities, which contradicts one of the
>> Iceberg design goals: offloading more work to jobs and removing the
>> metastore as a bottleneck. In the Iceberg world, not much is needed from
>> the Metastore: it just keeps track of the metadata location and provides a
>> mechanism for atomically updating this location (basically, what is done in
>> the `HiveTableOperations` class). We would like to design a solution that
>> relies  as little as possible on the Metastore so that in future we have
>> the option to replace our fleet of Metastores with a simpler system.
>>
>>
>> – Implementing a new `HiveStorageHandler`
>>
>> We are working on implementing custom `InputFormat` and `OutputFormat`
>> classes for Iceberg (more on that in the next paragraph) and they would fit
>> in nicely with the `HiveStorageHandler` and `HiveStoragePredicateHandler`
>> interfaces. However, the `HiveMetaHook` interface does not seem rich enough
>> to accommodate all the workflows, for instance no hooks run on `ALTER ...`
>>  or `INSERT...` commands.
>>
>>
>>
>> – Proof of concept
>>
>> We set out to write a proof of concept that would allow us to learn and
>> experiment. We based our work on the 2.3 branch. Here’s the state of the
>> project and the paths we explored:
>>
>> DDL commands
>> We support some commands such as `CREABLE TABLE ...`, `DESC ...`, `SHOW
>> PARTITIONS`. They are all implemented in the client and mostly rely on the
>> `HiveCatalog` class to do the work.
>>
>> Read path
>> We are in the process of implementing a custom `FileInputFormat` that
>> receives an Iceberg table identifier and a serialized expression
>> `ExprNodeDesc` as input. This is similar in a lot of ways to what you can
>> find in the `PigParquetReader` class in the `iceberg-pig` package or in
>> `HiveHBaseTableInputFormat` class in Hive.
>>
>>
>> Write path
>> We have made less progress in that front but we see a path forward by
>> implementing a custom `OutputFormat` that would keep track of the files
>> that are being written and gather statistics. Then, each task can dump this
>> information on HDFS. From there, the final Hive `MoveTask` can merge those
>> “pre-manifest” files to create a new snapshot and commit the new version of
>> a table.
>>
>>
>> We hope that our observations will start a healthy conversation about
>> supporting Iceberg tables in Hive :)
>>
>>
>> Cheers,
>> Adrien
>>
>

Reply via email to