数据模型

本文档主要从逻辑层面,描述 Doris 的数据模型,以帮助用户更好的使用 Doris 应对不同的业务场景。

基本概念

在 Doris 中,数据以表(Table)的形式进行逻辑上的描述。 一张表包括行(Row)和列(Column)。Row 即用户的一行数据。Column 用于描述一行数据中不同的字段。

Column 可以分为两大类:Key 和 Value。从业务角度看,Key 和 Value 可以分别对应维度列和指标列。

Doris 的数据模型主要分为3类:

  • Aggregate
  • Unique
  • Duplicate

下面我们分别介绍。

Aggregate 模型

我们以实际的例子来说明什么是聚合模型,以及如何正确的使用聚合模型。

示例1:导入数据聚合

假设业务有如下数据表模式:

ColumnNameTypeAggregationTypeComment
user_idLARGEINT用户id
dateDATE数据灌入日期
cityVARCHAR(20)用户所在城市
ageSMALLINT用户年龄
sexTINYINT用户性别
last_visit_dateDATETIMEREPLACE用户最后一次访问时间
costBIGINTSUM用户总消费
max_dwell_timeINTMAX用户最大停留时间
min_dwell_timeINTMIN用户最小停留时间

如果转换成建表语句则如下(省略建表语句中的 Partition 和 Distribution 信息)

  1. CREATE TABLE IF NOT EXISTS example_db.example_tbl
  2. (
  3. `user_id` LARGEINT NOT NULL COMMENT "用户id",
  4. `date` DATE NOT NULL COMMENT "数据灌入日期时间",
  5. `city` VARCHAR(20) COMMENT "用户所在城市",
  6. `age` SMALLINT COMMENT "用户年龄",
  7. `sex` TINYINT COMMENT "用户性别",
  8. `last_visit_date` DATETIME REPLACE DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间",
  9. `cost` BIGINT SUM DEFAULT "0" COMMENT "用户总消费",
  10. `max_dwell_time` INT MAX DEFAULT "0" COMMENT "用户最大停留时间",
  11. `min_dwell_time` INT MIN DEFAULT "99999" COMMENT "用户最小停留时间"
  12. )
  13. AGGREGATE KEY(`user_id`, `date`, `city`, `age`, `sex`)
  14. DISTRIBUTED BY HASH(`user_id`) BUCKETS 1
  15. PROPERTIES (
  16. "replication_allocation" = "tag.location.default: 1"
  17. );

可以看到,这是一个典型的用户信息和访问行为的事实表。 在一般星型模型中,用户信息和访问行为一般分别存放在维度表和事实表中。这里我们为了更加方便的解释 Doris 的数据模型,将两部分信息统一存放在一张表中。

表中的列按照是否设置了 AggregationType,分为 Key (维度列) 和 Value(指标列)。没有设置 AggregationType 的,如 user_iddateage … 等称为 Key,而设置了 AggregationType 的称为 Value

当我们导入数据时,对于 Key 列相同的行会聚合成一行,而 Value 列会按照设置的 AggregationType 进行聚合。 AggregationType 目前有以下四种聚合方式:

  1. SUM:求和,多行的 Value 进行累加。
  2. REPLACE:替代,下一批数据中的 Value 会替换之前导入过的行中的 Value。
  3. MAX:保留最大值。
  4. MIN:保留最小值。

假设我们有以下导入数据(原始数据):

user_iddatecityagesexlast_visit_datecostmax_dwell_timemin_dwell_time
100002017-10-01北京2002017-10-01 06:00:00201010
100002017-10-01北京2002017-10-01 07:00:001522
100012017-10-01北京3012017-10-01 17:05:4522222
100022017-10-02上海2012017-10-02 12:59:1220055
100032017-10-02广州3202017-10-02 11:20:00301111
100042017-10-01深圳3502017-10-01 10:00:1510033
100042017-10-03深圳3502017-10-03 10:20:221166

我们假设这是一张记录用户访问某商品页面行为的表。我们以第一行数据为例,解释如下:

数据说明
10000用户id,每个用户唯一识别id
2017-10-01数据入库时间,精确到日期
北京用户所在城市
20用户年龄
0性别男(1 代表女性)
2017-10-01 06:00:00用户本次访问该页面的时间,精确到秒
20用户本次访问产生的消费
10用户本次访问,驻留该页面的时间
10用户本次访问,驻留该页面的时间(冗余)

那么当这批数据正确导入到 Doris 中后,Doris 中最终存储如下:

user_iddatecityagesexlast_visit_datecostmax_dwell_timemin_dwell_time
100002017-10-01北京2002017-10-01 07:00:0035102
100012017-10-01北京3012017-10-01 17:05:4522222
100022017-10-02上海2012017-10-02 12:59:1220055
100032017-10-02广州3202017-10-02 11:20:00301111
100042017-10-01深圳3502017-10-01 10:00:1510033
100042017-10-03深圳3502017-10-03 10:20:221166

可以看到,用户 10000 只剩下了一行聚合后的数据。而其余用户的数据和原始数据保持一致。这里先解释下用户 10000 聚合后的数据:

前5列没有变化,从第6列 last_visit_date 开始:

  • 2017-10-01 07:00:00:因为 last_visit_date 列的聚合方式为 REPLACE,所以 2017-10-01 07:00:00 替换了 2017-10-01 06:00:00 保存了下来。

    注:在同一个导入批次中的数据,对于 REPLACE 这种聚合方式,替换顺序不做保证。如在这个例子中,最终保存下来的,也有可能是 2017-10-01 06:00:00。而对于不同导入批次中的数据,可以保证,后一批次的数据会替换前一批次。

  • 35:因为 cost 列的聚合类型为 SUM,所以由 20 + 15 累加获得 35。

  • 10:因为 max_dwell_time 列的聚合类型为 MAX,所以 10 和 2 取最大值,获得 10。

  • 2:因为 min_dwell_time 列的聚合类型为 MIN,所以 10 和 2 取最小值,获得 2。

经过聚合,Doris 中最终只会存储聚合后的数据。换句话说,即明细数据会丢失,用户不能够再查询到聚合前的明细数据了。

示例2:保留明细数据

接示例1,我们将表结构修改如下:

ColumnNameTypeAggregationTypeComment
user_idLARGEINT用户id
dateDATE数据灌入日期
timestampDATETIME数据灌入时间,精确到秒
cityVARCHAR(20)用户所在城市
ageSMALLINT用户年龄
sexTINYINT用户性别
last_visit_dateDATETIMEREPLACE用户最后一次访问时间
costBIGINTSUM用户总消费
max_dwell_timeINTMAX用户最大停留时间
min_dwell_timeINTMIN用户最小停留时间

即增加了一列 timestamp,记录精确到秒的数据灌入时间。 同时,将AGGREGATE KEY设置为AGGREGATE KEY(user_id, date, timestamp, city, age, sex)

导入数据如下:

user_iddatetimestampcityagesexlast_visit_datecostmax_dwell_timemin_dwell_time
100002017-10-012017-10-01 08:00:05北京2002017-10-01 06:00:00201010
100002017-10-012017-10-01 09:00:05北京2002017-10-01 07:00:001522
100012017-10-012017-10-01 18:12:10北京3012017-10-01 17:05:4522222
100022017-10-022017-10-02 13:10:00上海2012017-10-02 12:59:1220055
100032017-10-022017-10-02 13:15:00广州3202017-10-02 11:20:00301111
100042017-10-012017-10-01 12:12:48深圳3502017-10-01 10:00:1510033
100042017-10-032017-10-03 12:38:20深圳3502017-10-03 10:20:221166

那么当这批数据正确导入到 Doris 中后,Doris 中最终存储如下:

user_iddatetimestampcityagesexlast_visit_datecostmax_dwell_timemin_dwell_time
100002017-10-012017-10-01 08:00:05北京2002017-10-01 06:00:00201010
100002017-10-012017-10-01 09:00:05北京2002017-10-01 07:00:001522
100012017-10-012017-10-01 18:12:10北京3012017-10-01 17:05:4522222
100022017-10-022017-10-02 13:10:00上海2012017-10-02 12:59:1220055
100032017-10-022017-10-02 13:15:00广州3202017-10-02 11:20:00301111
100042017-10-012017-10-01 12:12:48深圳3502017-10-01 10:00:1510033
100042017-10-032017-10-03 12:38:20深圳3502017-10-03 10:20:221166

我们可以看到,存储的数据,和导入数据完全一样,没有发生任何聚合。这是因为,这批数据中,因为加入了 timestamp 列,所有行的 Key 都不完全相同。也就是说,只要保证导入的数据中,每一行的 Key 都不完全相同,那么即使在聚合模型下,Doris 也可以保存完整的明细数据。

示例3:导入数据与已有数据聚合

接示例1。假设现在表中已有数据如下:

user_iddatecityagesexlast_visit_datecostmax_dwell_timemin_dwell_time
100002017-10-01北京2002017-10-01 07:00:0035102
100012017-10-01北京3012017-10-01 17:05:4522222
100022017-10-02上海2012017-10-02 12:59:1220055
100032017-10-02广州3202017-10-02 11:20:00301111
100042017-10-01深圳3502017-10-01 10:00:1510033
100042017-10-03深圳3502017-10-03 10:20:221166

我们再导入一批新的数据:

user_iddatecityagesexlast_visit_datecostmax_dwell_timemin_dwell_time
100042017-10-03深圳3502017-10-03 11:22:00441919
100052017-10-03长沙2912017-10-03 18:11:02311

那么当这批数据正确导入到 Doris 中后,Doris 中最终存储如下:

user_iddatecityagesexlast_visit_datecostmax_dwell_timemin_dwell_time
100002017-10-01北京2002017-10-01 07:00:0035102
100012017-10-01北京3012017-10-01 17:05:4522222
100022017-10-02上海2012017-10-02 12:59:1220055
100032017-10-02广州3202017-10-02 11:20:00301111
100042017-10-01深圳3502017-10-01 10:00:1510033
100042017-10-03深圳3502017-10-03 11:22:0055196
100052017-10-03长沙2912017-10-03 18:11:02311

可以看到,用户 10004 的已有数据和新导入的数据发生了聚合。同时新增了 10005 用户的数据。

数据的聚合,在 Doris 中有如下三个阶段发生:

  1. 每一批次数据导入的 ETL 阶段。该阶段会在每一批次导入的数据内部进行聚合。
  2. 底层 BE 进行数据 Compaction 的阶段。该阶段,BE 会对已导入的不同批次的数据进行进一步的聚合。
  3. 数据查询阶段。在数据查询时,对于查询涉及到的数据,会进行对应的聚合。

数据在不同时间,可能聚合的程度不一致。比如一批数据刚导入时,可能还未与之前已存在的数据进行聚合。但是对于用户而言,用户只能查询到聚合后的数据。即不同的聚合程度对于用户查询而言是透明的。用户需始终认为数据以最终的完成的聚合程度存在,而不应假设某些聚合还未发生。(可参阅聚合模型的局限性一节获得更多详情。)

Unique 模型

在某些多维分析场景下,用户更关注的是如何保证 Key 的唯一性,即如何获得 Primary Key 唯一性约束。因此,我们引入了 Unique 数据模型。在1.2版本之前,该模型本质上是聚合模型的一个特例,也是一种简化的表结构表示方式。由于聚合模型的实现方式是读时合并(merge on read),因此在一些聚合查询上性能不佳(参考后续章节聚合模型的局限性的描述),在1.2版本我们引入了Unique模型新的实现方式,写时合并(merge on write),通过在写入时做一些额外的工作,实现了最优的查询性能。写时合并将在未来替换读时合并成为Unique模型的默认实现方式,两者将会短暂的共存一段时间。下面将对两种实现方式分别举例进行说明。

读时合并(与聚合模型相同的实现方式)

ColumnNameTypeIsKeyComment
user_idBIGINTYes用户id
usernameVARCHAR(50)Yes用户昵称
cityVARCHAR(20)No用户所在城市
ageSMALLINTNo用户年龄
sexTINYINTNo用户性别
phoneLARGEINTNo用户电话
addressVARCHAR(500)No用户住址
register_timeDATETIMENo用户注册时间

这是一个典型的用户基础信息表。这类数据没有聚合需求,只需保证主键唯一性。(这里的主键为 user_id + username)。那么我们的建表语句如下:

  1. CREATE TABLE IF NOT EXISTS example_db.example_tbl
  2. (
  3. `user_id` LARGEINT NOT NULL COMMENT "用户id",
  4. `username` VARCHAR(50) NOT NULL COMMENT "用户昵称",
  5. `city` VARCHAR(20) COMMENT "用户所在城市",
  6. `age` SMALLINT COMMENT "用户年龄",
  7. `sex` TINYINT COMMENT "用户性别",
  8. `phone` LARGEINT COMMENT "用户电话",
  9. `address` VARCHAR(500) COMMENT "用户地址",
  10. `register_time` DATETIME COMMENT "用户注册时间"
  11. )
  12. UNIQUE KEY(`user_id`, `username`)
  13. DISTRIBUTED BY HASH(`user_id`) BUCKETS 1
  14. PROPERTIES (
  15. "replication_allocation" = "tag.location.default: 1"
  16. );

而这个表结构,完全同等于以下使用聚合模型描述的表结构:

ColumnNameTypeAggregationTypeComment
user_idBIGINT用户id
usernameVARCHAR(50)用户昵称
cityVARCHAR(20)REPLACE用户所在城市
ageSMALLINTREPLACE用户年龄
sexTINYINTREPLACE用户性别
phoneLARGEINTREPLACE用户电话
addressVARCHAR(500)REPLACE用户住址
register_timeDATETIMEREPLACE用户注册时间

及建表语句:

  1. CREATE TABLE IF NOT EXISTS example_db.example_tbl
  2. (
  3. `user_id` LARGEINT NOT NULL COMMENT "用户id",
  4. `username` VARCHAR(50) NOT NULL COMMENT "用户昵称",
  5. `city` VARCHAR(20) REPLACE COMMENT "用户所在城市",
  6. `age` SMALLINT REPLACE COMMENT "用户年龄",
  7. `sex` TINYINT REPLACE COMMENT "用户性别",
  8. `phone` LARGEINT REPLACE COMMENT "用户电话",
  9. `address` VARCHAR(500) REPLACE COMMENT "用户地址",
  10. `register_time` DATETIME REPLACE COMMENT "用户注册时间"
  11. )
  12. AGGREGATE KEY(`user_id`, `username`)
  13. DISTRIBUTED BY HASH(`user_id`) BUCKETS 1
  14. PROPERTIES (
  15. "replication_allocation" = "tag.location.default: 1"
  16. );

即Unique 模型的读时合并实现完全可以用聚合模型中的 REPLACE 方式替代。其内部的实现方式和数据存储方式也完全一样。这里不再继续举例说明。

SinceVersion 1.2

写时合并

Unqiue模型的写时合并实现,与聚合模型就是完全不同的两种模型了,查询性能更接近于duplicate模型,在有主键约束需求的场景上相比聚合模型有较大的查询性能优势,尤其是在聚合查询以及需要用索引过滤大量数据的查询中。

在 1.2.0 版本中,作为一个新的feature,写时合并默认关闭,用户可以通过添加下面的property来开启

  1. "enable_unique_key_merge_on_write" = "true"

仍然以上面的表为例,建表语句为

  1. CREATE TABLE IF NOT EXISTS example_db.example_tbl
  2. (
  3. `user_id` LARGEINT NOT NULL COMMENT "用户id",
  4. `username` VARCHAR(50) NOT NULL COMMENT "用户昵称",
  5. `city` VARCHAR(20) COMMENT "用户所在城市",
  6. `age` SMALLINT COMMENT "用户年龄",
  7. `sex` TINYINT COMMENT "用户性别",
  8. `phone` LARGEINT COMMENT "用户电话",
  9. `address` VARCHAR(500) COMMENT "用户地址",
  10. `register_time` DATETIME COMMENT "用户注册时间"
  11. )
  12. UNIQUE KEY(`user_id`, `username`)
  13. DISTRIBUTED BY HASH(`user_id`) BUCKETS 1
  14. PROPERTIES (
  15. "replication_allocation" = "tag.location.default: 1",
  16. "enable_unique_key_merge_on_write" = "true"
  17. );

使用这种建表语句建出来的表结构,与聚合模型就完全不同了:

ColumnNameTypeAggregationTypeComment
user_idBIGINT用户id
usernameVARCHAR(50)用户昵称
cityVARCHAR(20)NONE用户所在城市
ageSMALLINTNONE用户年龄
sexTINYINTNONE用户性别
phoneLARGEINTNONE用户电话
addressVARCHAR(500)NONE用户住址
register_timeDATETIMENONE用户注册时间

在开启了写时合并选项的Unique表上,数据在导入阶段就会去将被覆盖和被更新的数据进行标记删除,同时将新的数据写入新的文件。在查询的时候,所有被标记删除的数据都会在文件级别被过滤掉,读取出来的数据就都是最新的数据,消除掉了读时合并中的数据聚合过程,并且能够在很多情况下支持多种谓词的下推。因此在许多场景都能带来比较大的性能提升,尤其是在有聚合查询的情况下。

【注意】

  1. 新的Merge-on-write实现默认关闭,且只能在建表时通过指定property的方式打开。
  2. 旧的Merge-on-read的实现无法无缝升级到新版本的实现(数据组织方式完全不同),如果需要改为使用写时合并的实现版本,需要手动执行insert into unique-mow-table select * from source table.
  3. 在Unique模型上独有的delete sign 和 sequence col,在写时合并的新版实现中仍可以正常使用,用法没有变化。

Duplicate 模型

在某些多维分析场景下,数据既没有主键,也没有聚合需求。因此,我们引入 Duplicate 数据模型来满足这类需求。举例说明。

ColumnNameTypeSortKeyComment
timestampDATETIMEYes日志时间
typeINTYes日志类型
error_codeINTYes错误码
error_msgVARCHAR(1024)No错误详细信息
op_idBIGINTNo负责人id
op_timeDATETIMENo处理时间

建表语句如下:

  1. CREATE TABLE IF NOT EXISTS example_db.example_tbl
  2. (
  3. `timestamp` DATETIME NOT NULL COMMENT "日志时间",
  4. `type` INT NOT NULL COMMENT "日志类型",
  5. `error_code` INT COMMENT "错误码",
  6. `error_msg` VARCHAR(1024) COMMENT "错误详细信息",
  7. `op_id` BIGINT COMMENT "负责人id",
  8. `op_time` DATETIME COMMENT "处理时间"
  9. )
  10. DUPLICATE KEY(`timestamp`, `type`, `error_code`)
  11. DISTRIBUTED BY HASH(`type`) BUCKETS 1
  12. PROPERTIES (
  13. "replication_allocation" = "tag.location.default: 1"
  14. );

这种数据模型区别于 Aggregate 和 Unique 模型。数据完全按照导入文件中的数据进行存储,不会有任何聚合。即使两行数据完全相同,也都会保留。 而在建表语句中指定的 DUPLICATE KEY,只是用来指明底层数据按照那些列进行排序。(更贴切的名称应该为 “Sorted Column”,这里取名 “DUPLICATE KEY” 只是用以明确表示所用的数据模型。关于 “Sorted Column”的更多解释,可以参阅前缀索引)。在 DUPLICATE KEY 的选择上,我们建议适当的选择前 2-4 列就可以。

这种数据模型适用于既没有聚合需求,又没有主键唯一性约束的原始数据的存储。更多使用场景,可参阅聚合模型的局限性小节。

聚合模型的局限性

这里我们针对 Aggregate 模型,来介绍下聚合模型的局限性。

在聚合模型中,模型对外展现的,是最终聚合后的数据。也就是说,任何还未聚合的数据(比如说两个不同导入批次的数据),必须通过某种方式,以保证对外展示的一致性。我们举例说明。

假设表结构如下:

ColumnNameTypeAggregationTypeComment
user_idLARGEINT用户id
dateDATE数据灌入日期
costBIGINTSUM用户总消费

假设存储引擎中有如下两个已经导入完成的批次的数据:

batch 1

user_iddatecost
100012017-11-2050
100022017-11-2139

batch 2

user_iddatecost
100012017-11-201
100012017-11-215
100032017-11-2222

可以看到,用户 10001 分属在两个导入批次中的数据还没有聚合。但是为了保证用户只能查询到如下最终聚合后的数据:

user_iddatecost
100012017-11-2051
100012017-11-215
100022017-11-2139
100032017-11-2222

我们在查询引擎中加入了聚合算子,来保证数据对外的一致性。

另外,在聚合列(Value)上,执行与聚合类型不一致的聚合类查询时,要注意语意。比如我们在如上示例中执行如下查询:

  1. SELECT MIN(cost) FROM table;

得到的结果是 5,而不是 1。

同时,这种一致性保证,在某些查询中,会极大的降低查询效率。

我们以最基本的 count(*) 查询为例:

  1. SELECT COUNT(*) FROM table;

在其他数据库中,这类查询都会很快的返回结果。因为在实现上,我们可以通过如“导入时对行进行计数,保存 count 的统计信息”,或者在查询时“仅扫描某一列数据,获得 count 值”的方式,只需很小的开销,即可获得查询结果。但是在 Doris 的聚合模型中,这种查询的开销非常大

我们以刚才的数据为例:

batch 1

user_iddatecost
100012017-11-2050
100022017-11-2139

batch 2

user_iddatecost
100012017-11-201
100012017-11-215
100032017-11-2222

因为最终的聚合结果为:

user_iddatecost
100012017-11-2051
100012017-11-215
100022017-11-2139
100032017-11-2222

所以,select count(*) from table; 的正确结果应该为 4。但如果我们只扫描 user_id 这一列,如果加上查询时聚合,最终得到的结果是 3(10001, 10002, 10003)。而如果不加查询时聚合,则得到的结果是 5(两批次一共5行数据)。可见这两个结果都是不对的。

为了得到正确的结果,我们必须同时读取 user_iddate 这两列的数据,再加上查询时聚合,才能返回 4 这个正确的结果。也就是说,在 count() 查询中,Doris 必须扫描所有的 AGGREGATE KEY 列(这里就是 user_iddate),并且聚合后,才能得到语意正确的结果。当聚合列非常多时,count() 查询需要扫描大量的数据。

因此,当业务上有频繁的 count(*) 查询时,我们建议用户通过增加一个值恒为 1 的,聚合类型为 SUM 的列来模拟 count(*)。如刚才的例子中的表结构,我们修改如下:

ColumnNameTypeAggregateTypeComment
user_idBIGINT用户id
dateDATE数据灌入日期
costBIGINTSUM用户总消费
countBIGINTSUM用于计算count

增加一个 count 列,并且导入数据中,该列值恒为 1。则 select count(*) from table; 的结果等价于 select sum(count) from table;。而后者的查询效率将远高于前者。不过这种方式也有使用限制,就是用户需要自行保证,不会重复导入 AGGREGATE KEY 列都相同的行。否则,select sum(count) from table; 只能表述原始导入的行数,而不是 select count(*) from table; 的语义。

另一种方式,就是 将如上的 count 列的聚合类型改为 REPLACE,且依然值恒为 1。那么 select sum(count) from table;select count(*) from table; 的结果将是一致的。并且这种方式,没有导入重复行的限制。

Unique模型的写时合并实现

Unique模型的写时合并实现没有聚合模型的局限性,还是以刚才的数据为例,写时合并为每次导入的rowset增加了对应的delete bitmap,来标记哪些数据被覆盖。第一批数据导入后状态如下

batch 1

user_iddatecostdelete bit
100012017-11-2050false
100022017-11-2139false

当第二批数据导入完成后,第一批数据中重复的行就会被标记为已删除,此时两批数据状态如下

batch 1

user_iddatecostdelete bit
100012017-11-2050true
100022017-11-2139false

batch 2

user_iddatecostdelete bit
100012017-11-201false
100012017-11-215false
100032017-11-2222false

在查询时,所有在delete bitmap中被标记删除的数据都不会读出来,因此也无需进行做任何数据聚合,上述数据中有效的行数为4行,查询出的结果也应该是4行,也就可以采取开销最小的方式来获取结果,即前面提到的“仅扫描某一列数据,获得 count 值”的方式。

在测试环境中,count(*) 查询在Unique模型的写时合并实现上的性能,相比聚合模型有10倍以上的提升。

Duplicate 模型

Duplicate 模型没有聚合模型的这个局限性。因为该模型不涉及聚合语意,在做 count(*) 查询时,任意选择一列查询,即可得到语意正确的结果。

key 列

Duplicate、Aggregate、Unique 模型,都会在建表指定 key 列,然而实际上是有所区别的:对于 Duplicate 模型,表的key列,可以认为只是 “排序列”,并非起到唯一标识的作用。而 Aggregate、Unique 模型这种聚合类型的表,key 列是兼顾 “排序列” 和 “唯一标识列”,是真正意义上的“ key 列”。

数据模型的选择建议

因为数据模型在建表时就已经确定,且无法修改。所以,选择一个合适的数据模型非常重要

  1. Aggregate 模型可以通过预聚合,极大地降低聚合查询时所需扫描的数据量和查询的计算量,非常适合有固定模式的报表类查询场景。但是该模型对 count(*) 查询很不友好。同时因为固定了 Value 列上的聚合方式,在进行其他类型的聚合查询时,需要考虑语意正确性。
  2. Unique 模型针对需要唯一主键约束的场景,可以保证主键唯一性约束。但是无法利用 ROLLUP 等预聚合带来的查询优势。
    1. 对于聚合查询有较高性能需求的用户,推荐使用自1.2版本加入的写时合并实现。
    2. Unique 模型仅支持整行更新,如果用户既需要唯一主键约束,又需要更新部分列(例如将多张源表导入到一张 doris 表的情形),则可以考虑使用 Aggregate 模型,同时将非主键列的聚合类型设置为 REPLACE_IF_NOT_NULL。具体的用法可以参考语法手册
  3. Duplicate 适合任意维度的 Ad-hoc 查询。虽然同样无法利用预聚合的特性,但是不受聚合模型的约束,可以发挥列存模型的优势(只读取相关列,而不需要读取所有 Key 列)。