You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/10/21 08:26:09 UTC

[GitHub] [hudi] minihippo commented on a diff in pull request #4718: [HUDI-3345][RFC-36] Hudi metastore server

minihippo commented on code in PR #4718:
URL: https://github.com/apache/hudi/pull/4718#discussion_r1001511942


##########
rfc/rfc-36/rfc-36.md:
##########
@@ -0,0 +1,605 @@
+<!--
+  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.
+-->
+# RFC-36: Hudi Metastore Server
+
+## Proposers
+
+- @minihippo
+
+## Approvers
+
+
+## Status
+
+JIRA: [HUDI-3345](https://issues.apache.org/jira/browse/HUDI-3345)
+
+> Please keep the status updated in `rfc/README.md`.
+
+# Hudi Metastore Server
+
+## Abstract
+
+Currently, Hudi is widely used as a table format in the data warehouse. There is a lack of central metastore server to manage the metadata of data lake table. Hive metastore as a commonly used catalog service in the data warehouse on Hadoop cannot store the unique metadata like timeline of the hudi table.
+
+The proposal is to implement an unified metadata management system called hudi metastore server to store the metadata of the hudi table, and be compatible with hive metastore so that other engines can access it without any changes.
+
+## Backgroud
+
+**How Hudi metadata is stored**
+
+The metadata of hudi are table location, configuration and schema, timeline generated by instants, metadata of each commit / instant, which records files created / updated, new records num and so on in this commit. Besides, the information of files in a hudi table is also a part of hudi metadata.
+
+Different from instant or schema recorded by a separate file that is stored under `${tablelocation}/.hoodie` on the HDFS or object storage, files info are managed by the HDFS directly. Hudi gets all files of a table by file listing. File listing is a costly operation and its performance is limited by namenode. In addition, there will be a few invalid files on the file system, which are created by spark speculative tasks(for example) and are not deleted successfully. Getting files by listing will result in inconsistency, so hudi has to store the valid files from each commit metadata, the metadata about files is usually referred to snapshot.
+
+RFC-15 metadata table is a proposal that can solve these problems. However, it only manages the metadata of one table. There is a lack of a unified view.
+
+**The integration of Hive metastore and Hudi metadata lacks a single source of truth.**
+
+Hive metastore server is widely used as a metadata center in the data warehouse on Hadoop. It stores the metadata for hive tables like their schema, location and partitions. Currently, almost all of the storage or computing engines support registering table information to it, discovering and retrieving metadata from it. Meanwhile, cloud service providers like AWS Glue, HUAWEI Cloud, Google Cloud Dataproc, Alibaba Cloud, ByteDance Volcano Engine all provide Apache Hive metastore compatible catalog. It seems that hive metastore has become a standard in the data warehouse.
+
+Different from the traditional table format like hive table, the data lake table not only has schema, partitions and other hive metadata, but also has timeline, snapshot which is unconventional. Hence, the metadata of data lake cannot be managed by HMS directly.
+
+Hudi just syncs the schema and partitions to HMS by now, and other metadata still stores on HDFS or object store. Metadata synchronization between different metadata management systems will result in inconsistency.
+
+## Overview
+
+![architecture](architecture.png)
+
+The hudi metastore server is for metadata management of the data lake table, to support metadata persistency, efficient metadata access and other extensions for data lake. The metadata server managed includes the information of databases and tables, partitions, schemas, instants, instants' meta and files' meta.
+
+The metastore server has two main components: service and storage. The storage is for metadata persistency and the service is to receive the get / put requests from client and return / store the processing result after doing some logical operations on metadata.
+
+The hudi metastore server is / has
+
+- **A metastore server for data lake**
+    -  Different from the traditional table format, the metadata of the data lake has timeline and snapshot concepts, in addition to schema and partitions.
+
+    -  The metastore server is an unified metadata management system for data lake table.
+
+- **Pluggable storage**
+    -  The storage is only responsible for metadata presistency. Therefore, it's doesn't matter what the storage engine is used to store the data, it can be a RDBMS, kv system or file system.
+
+- **Easy to be expanded**
+    -  The service is stateless, so it can be scaled horizontally to support higher QPS. The storage can be split vertically to store more data.
+
+- **Compatible with multiple computing engines**
+    -  The server has an adapter to be compatible with hive metastore server.
+
+## Design
+
+This part has four sections: what the service does, what and how the metadata stores, how the service interacts with the storage when reading and writing a hudi table with an example, and the extension of the metastore server which will be implemented in the next version.
+
+### Service
+
+Service is to receive requests from clients and return results according to the metadata in the storage combined with some processing logic. According to the functional division, the service consists of four parts:
+
+- **table service**
+    -  is for table related requests. To client, it exposes API about database and table CRUD.
+
+- **partition service**
+    -  is for partition related requests. To client, it exposes API about CRUD:
+
+    - support multiple ways of reading, like checking the partition's existence, getting partition info, getting partitions which satisfy a specific condition(partition pruning).
+    - creating or updating API cannot be invoked directly,  only a new commit completion can trigger it.
+    -  dropping a partition not only deletes the partition and files at metadata level, but also triggers a clean action to do the physical clean that deletes the data on the file system.
+
+- **timeline service**
+    -  is for timeline related requests. The basic functions are the same as the `HoodieActiveTimeline`.
+
+    -  To client, it exposes API that
+
+    - the instant CRUD
+    - get the metadata of an instant
+
+- **snapshot service**
+    -  is for file level related request. The old way is that a write client gets all files under the path by file listing, and then builds the file system view by organizing the files as file slices in file groups. After that, get the latest file slice in each file group to read and write. Snapshot service just gives the client the result of the last step.
+
+    -  To client, it exposes API that
+
+    - get the latest snapshot of a partition without multiple file version
+    - get the incremental files after a specified timestamp, for incremental reading
+    - get all files of a partition,for file cleaning
+    - update the snapshot when a new commit completes, it cannot be invoked by client directly.
+
+The specific interface is defined in the `Alpha Version` part.
+
+### Storage
+
+According to the type of the metadata stored, the storage is divided into four parts: table, partition, timeline, snapshot. Table consists of location, schema, table configurations and so on. Partition is just partition metadata. Timeline contains instant and the metadata of each instant. Snapshot has file statuses.
+
+#### Lectotype
+
+**Requirements:** support single row ACID transactions. Almost all write operations require it, like instant creation, state changing and so on.
+
+There are the candidates,
+
+**Hudi table**
+
+pros:
+
+- No external components are introduced and maintained.
+
+crons:
+
+- It is unreasonable that the storage of the hudi metastore server is a hudi table. How does the metadata of the server storage store? There is a paradox.
+
+**RDBMS**
+
+pros:
+
+- database that is suitable for structured data like metadata to store.
+
+- can describe the relation between many kinds of metadata.
+
+crons:
+
+- introduce another system to maintain.
+
+**File system**
+
+pros:
+
+- No external components are introduced and maintained.
+
+crons:
+
+- not suitable for the situation that requires high performance.
+
+- have to do extra work to support the metadata organization.
+
+**Key-value storage**
+
+pros:
+
+- database that is suitable for structured data like metadata to store.
+
+- in-memory data store so that read and write faster.
+
+crons:
+
+- introduce another system to maintain.
+
+- stroage capacity is a limitation.
+
+There are specific requirements of the metastore server in the different scenarios. Through the storage of server is pluggable, considering the general situation of disk storage, good performance of read and write, convenience of development, RDBMS may be a better one to be chosen.
+
+#### Storage Schema
+
+##### Table
+
+- **db**
+
+    -  unique key: name
+
+    - | name         | type   | comment               |
+          | ------------ | ------ | --------------------- |
+      | db_id        | bigint | auto_increment        |
+      | desc         | string | database description  |
+      | location_uri | string | database storage path |
+      | name         | string | database name         |
+      | owner_name   | string | database owner        |
+      | owner_type   | string | database owner type   |
+
+- **db_params**
+
+    -  unique key: param_key, db_id
+
+    - | name        | type   | comment                  |
+          | ----------- | ------ | ------------------------ |
+      | db_id       | bigint | auto_increment           |
+      | param_key   | string | database parameter key   |
+      | param_value | string | database parameter value |
+
+- **tbl**
+
+    -  unique key:  name, db_id
+
+    - | name        | type      | comment                       |
+          | ----------- | --------- | ----------------------------- |
+      | tbl_id      | bigint    | auto_increment                |
+      | db_id       | bigint    | database the table belongs to |
+      | name        | string    | table name                    |
+      | create_time | timestamp | table created time            |
+      | owner       | string    | the table owner               |
+      | location    | string    | table location                |
+
+- **tbl_params**
+
+    -  is used to record table level statistics like file num, total size.
+
+    -  unique key:  param_key, tbl_id
+
+    - | name        | type   | comment               |
+          | ----------- | ------ | --------------------- |
+      | tbl_id      | bigint | auto_increment        |
+      | param_key   | string | table parameter key   |
+      | param_value | string | table parameter value |
+
+- **schema**
+
+    -  unique key: name, type, version, tbl_id
+
+    - | name             | type    | comment                                  |
+          | ---------------- | ------- | ---------------------------------------- |
+      | col_id           | bigint  | auto_increment                           |
+      | tbl_id           | bigint  | table identifier                         |
+      | version          | bigint  | a completed commit identifier            |
+      | name             | string  | partition parameter key                  |
+      | type             | string  | partition parameter value                |
+      | is_partition_col | tinyint | whether the column is a partition column |
+      | comment          | string  | column description                       |
+
+##### Partition
+
+- **partitions**
+
+    -  unique key: part_name, tbl_id
+
+    - | name        | type      | comment                          |
+          | ----------- | --------- | -------------------------------- |
+      | part_id     | bigint    | auto_increment                   |
+      | tbl_id      | bigint    | table the partition belongs to   |
+      | part_name   | string    | partition name                   |
+      | create_time | timestamp | partition created time           |
+      | update_time | timestamp | partition updated time           |
+      | is_deleted  | tinyint   | whether the partition is deleted |
+
+- **partition****_parmas**
+
+    -  unique key: param_key, part_id
+
+    - | name        | type   | comment                        |
+          | ----------- | ------ | ------------------------------ |
+      | part_id     | bigint | partition identifier           |
+      | tbl_id      | bigint | table the partition belongs to |
+      | param_key   | string | partition parameter key        |
+      | param_value | string | partition parameter value      |
+
+- **partition****_key_val**
+
+    -  is used to support partition pruning.
+
+    -  unique key: part_key_name, part_id
+
+    - | name          | type   | comment                        |
+          | ------------- | ------ | ------------------------------ |
+      | part_id       | bigint | partition  identifier          |
+      | tbl_id        | bigint | table the partition belongs to |
+      | part_key_name | string | the partition name             |
+      | part_key_val  | string | the partition value            |
+
+##### Timeline
+
+- **tbl_timestamp**
+
+    -  is used to get a self-increasing timestamp for each table
+
+    -  unique key: part_key_name, part_id
+
+    - | name   | type   | comment               |
+          | ------ | ------ | --------------------- |
+      | tbl_id | bigint | table identifier      |
+      | ts     | string | timestamp for instant |
+
+- **instant**
+
+    -  unique key: ts, action, tbl_id
+
+    - | name       | type      | comment                                      |
+          | ---------- | --------- | -------------------------------------------- |
+      | instant_id | bigint    | auto_increment                               |
+      | tbl_id     | bigint    | table identifier                             |
+      | ts         | string    | instant timestamp                            |
+      | action     | tinyint   | instant action, commit, deltacommit and etc. |
+      | state      | tinyint   | instant state, requested, inflight and etc.  |
+      | duration   | int       | for heartbeat                                |
+      | start_ts   | timestamp | for heartbeat                                |

Review Comment:
   Whether the heartbeat fails, it is judeged by whether duration + start_ts < current_time. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org