From 01787cc36a3156ff88439669a9648e214283a6bb Mon Sep 17 00:00:00 2001 From: yuanlihan Date: Thu, 1 Aug 2019 00:58:26 +0800 Subject: [PATCH 01/26] Enable partition discovery when loading data from parquet file --- be/src/exec/parquet_reader.cpp | 34 +++++++++---- be/src/exec/parquet_reader.h | 3 +- be/src/exec/parquet_scanner.cpp | 2 +- be/test/exec/parquet_scanner_test.cpp | 11 ++-- .../Contents/Data Manipulation/broker_load.md | 15 ++++++ .../apache/doris/common/util/BrokerUtil.java | 50 +++++++++++-------- .../apache/doris/planner/BrokerScanNode.java | 38 ++++++++++++++ gensrc/thrift/PlanNodes.thrift | 2 + 8 files changed, 119 insertions(+), 36 deletions(-) diff --git a/be/src/exec/parquet_reader.cpp b/be/src/exec/parquet_reader.cpp index a60295b0fab111..de1acd12f1d6ed 100644 --- a/be/src/exec/parquet_reader.cpp +++ b/be/src/exec/parquet_reader.cpp @@ -34,8 +34,8 @@ namespace doris { // Broker -ParquetReaderWrap::ParquetReaderWrap(FileReader *file_reader) : - _total_groups(0), _current_group(0), _rows_of_group(0), _current_line_of_group(0) { +ParquetReaderWrap::ParquetReaderWrap(FileReader *file_reader, const std::map& partition_columns) : + _partition_columns(partition_columns), _total_groups(0), _current_group(0), _rows_of_group(0), _current_line_of_group(0) { _parquet = std::shared_ptr(new ParquetFile(file_reader)); _properties = parquet::ReaderProperties(); _properties.enable_buffered_stream(); @@ -122,13 +122,17 @@ Status ParquetReaderWrap::column_indices(const std::vector& tup for (auto slot_desc : tuple_slot_descs) { // Get the Column Reader for the boolean column auto iter = _map_column.find(slot_desc->col_name()); - if (iter == _map_column.end()) { - std::stringstream str_error; - str_error << "Invalid Column Name:" << slot_desc->col_name(); - LOG(WARNING) << str_error.str(); - return Status::InvalidArgument(str_error.str()); + if (iter != _map_column.end()) { + _parquet_column_ids.emplace_back(iter->second); + } else { + auto iter_1 = _partition_columns.find(slot_desc->col_name()); + if (iter_1 == _partition_columns.end()) { + std::stringstream str_error; + str_error << "Invalid Column Name:" << slot_desc->col_name(); + LOG(WARNING) << str_error.str(); + return Status::InvalidArgument(str_error.str()); + } } - _parquet_column_ids.emplace_back(iter->second); } return Status::OK(); } @@ -201,13 +205,23 @@ Status ParquetReaderWrap::read(Tuple* tuple, const std::vector& uint8_t tmp_buf[128] = {0}; int32_t wbtyes = 0; const uint8_t *value = nullptr; + int index = 0; int column_index = 0; try { size_t slots = tuple_slot_descs.size(); for (size_t i = 0; i < slots; ++i) { auto slot_desc = tuple_slot_descs[i]; - column_index = i;// column index in batch record - switch (_parquet_column_type[i]) { + auto iter = _partition_columns.find(slot_desc->col_name()); + if (iter != _partition_columns.end()) { + std::string partitioned_field = iter->second; + value = reinterpret_cast(partitioned_field.c_str()); + wbtyes = partitioned_field.size(); + fill_slot(tuple, slot_desc, mem_pool, value, wbtyes); + continue; + } else { + column_index = index++; // column index in batch record + } + switch (_parquet_column_type[column_index]) { case arrow::Type::type::STRING: { auto str_array = std::dynamic_pointer_cast(_batch->column(column_index)); if (str_array->IsNull(_current_line_of_group)) { diff --git a/be/src/exec/parquet_reader.h b/be/src/exec/parquet_reader.h index defe3d9ebf09fd..107588d79d9aa8 100644 --- a/be/src/exec/parquet_reader.h +++ b/be/src/exec/parquet_reader.h @@ -68,7 +68,7 @@ class ParquetFile : public arrow::io::RandomAccessFile { // Reader of broker parquet file class ParquetReaderWrap { public: - ParquetReaderWrap(FileReader *file_reader); + ParquetReaderWrap(FileReader *file_reader, const std::map& partition_columns); virtual ~ParquetReaderWrap(); // Read @@ -85,6 +85,7 @@ class ParquetReaderWrap { Status handle_timestamp(const std::shared_ptr& ts_array, uint8_t *buf, int32_t *wbtyes); private: + const std::map& _partition_columns; parquet::ReaderProperties _properties; std::shared_ptr _parquet; diff --git a/be/src/exec/parquet_scanner.cpp b/be/src/exec/parquet_scanner.cpp index c69d19940d1e75..b5ab0d7793ab78 100644 --- a/be/src/exec/parquet_scanner.cpp +++ b/be/src/exec/parquet_scanner.cpp @@ -141,7 +141,7 @@ Status ParquetScanner::open_next_reader() { file_reader->close(); continue; } - _cur_file_reader = new ParquetReaderWrap(file_reader.release()); + _cur_file_reader = new ParquetReaderWrap(file_reader.release(), range.partition_columns); return _cur_file_reader->init_parquet_reader(_src_slot_descs); } } diff --git a/be/test/exec/parquet_scanner_test.cpp b/be/test/exec/parquet_scanner_test.cpp index 4bea0130c33672..e95472e8b947f8 100644 --- a/be/test/exec/parquet_scanner_test.cpp +++ b/be/test/exec/parquet_scanner_test.cpp @@ -68,14 +68,14 @@ class ParquetSannerTest : public testing::Test { #define TUPLE_ID_DST 0 #define TUPLE_ID_SRC 1 -#define CLOMN_NUMBERS 19 +#define CLOMN_NUMBERS 20 #define DST_TUPLE_SLOT_ID_START 1 -#define SRC_TUPLE_SLOT_ID_START 20 +#define SRC_TUPLE_SLOT_ID_START 21 int ParquetSannerTest::create_src_tuple(TDescriptorTable& t_desc_table, int next_slot_id) { const char *clomnNames[] = {"log_version", "log_time", "log_time_stamp", "js_version", "vst_cookie", "vst_ip", "vst_user_id", "vst_user_agent", "device_resolution", "page_url", "page_refer_url", "page_yyid", "page_type", "pos_type", "content_id", "media_id", - "spm_cnt", "spm_pre", "scm_cnt"}; + "spm_cnt", "spm_pre", "scm_cnt", "partition_column"}; for (int i = 0; i < CLOMN_NUMBERS; i++) { TSlotDescriptor slot_desc; @@ -201,7 +201,7 @@ int ParquetSannerTest::create_dst_tuple(TDescriptorTable& t_desc_table, int next const char *clomnNames[] = {"log_version", "log_time", "log_time_stamp", "js_version", "vst_cookie", "vst_ip", "vst_user_id", "vst_user_agent", "device_resolution", "page_url", "page_refer_url", "page_yyid", "page_type", "pos_type", "content_id", "media_id", - "spm_cnt", "spm_pre", "scm_cnt"}; + "spm_cnt", "spm_pre", "scm_cnt", "partition_column"}; for (int i = 3; i < CLOMN_NUMBERS; i++, byteOffset+=16) { TSlotDescriptor slot_desc; @@ -435,6 +435,9 @@ TEST_F(ParquetSannerTest, normal) { range.size = -1; range.format_type = TFileFormatType::FORMAT_PARQUET; range.splittable = true; + + std::map partition_columns = {{"partition_column", "value"}}; + range.__set_partition_columns(partition_columns); #if 1 range.path = "./be/test/exec/test_data/parquet_scanner/localfile.parquet"; range.file_type = TFileType::FILE_LOCAL; diff --git a/docs/help/Contents/Data Manipulation/broker_load.md b/docs/help/Contents/Data Manipulation/broker_load.md index 3f83280c16da4d..0f0e70c3e3ff46 100644 --- a/docs/help/Contents/Data Manipulation/broker_load.md +++ b/docs/help/Contents/Data Manipulation/broker_load.md @@ -353,6 +353,9 @@ WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); 8. 导入Parquet文件中数据 指定FORMAT 为parquet, 默认是通过文件后缀判断 + 如果导入路径为目录(base_path),则递归地列出该目录下的所有parquet文件 + 如果需要,则会根据表中定义的字段类型解析base_path目录下的partitioned fields,实现类似Spark中读parquet文件 + LOAD LABEL example_db.label9 ( DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/file") @@ -361,6 +364,18 @@ (k1, k2, k3) ) WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); + + LOAD LABEL example_db.label10 + ( + DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir") + INTO TABLE `my_table` + FORMAT AS "parquet" + (k1, k2, k3) + ) + WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); + + hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir目录下包括如下文件:[hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/k1=key1/xxx.parquet, hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/k1=key2/xxx.parquet, ...] + 如果my_table中定义了列k1,则会从文件path中提取k1对应的partitioned field的值,并完成数据导入 ## keyword BROKER LOAD diff --git a/fe/src/main/java/org/apache/doris/common/util/BrokerUtil.java b/fe/src/main/java/org/apache/doris/common/util/BrokerUtil.java index a6cb66832c5369..725d8160a2444c 100644 --- a/fe/src/main/java/org/apache/doris/common/util/BrokerUtil.java +++ b/fe/src/main/java/org/apache/doris/common/util/BrokerUtil.java @@ -61,28 +61,10 @@ public static void parseBrokerFile(String path, BrokerDesc brokerDesc, List parsePartitionedFields(String filePath) { + for (BrokerFileGroup fileGroup : fileGroups) { + String fileFormat = fileGroup.getFileFormat(); + if ((fileFormat == null || !fileFormat.toLowerCase().equals("parquet")) || !filePath.endsWith(".parquet")) { + continue; + } + for (String base : fileGroup.getFilePaths()) { + if (base.endsWith("/*")) { + base = base.substring(0, base.indexOf("/*")); + } + if (!filePath.startsWith(base)) { + continue; + } + String subPath = filePath.substring(base.length()); + String[] strings = subPath.split("/"); + Map partitionedFields = new HashMap<>(); + for (String str : strings) { + if (str == null || str.isEmpty() || !str.contains("=")) { + continue; + } + String[] pair = str.split("="); + if (pair.length != 2) { + continue; + } + Column column = targetTable.getColumn(pair[0]); + if (column == null) { + continue; + } + partitionedFields.put(pair[0], pair[1]); + } + return partitionedFields; + } + } + return Collections.emptyMap(); + } + // If fileFormat is not null, we use fileFormat instead of check file's suffix private void processFileGroup( String fileFormat, @@ -650,6 +687,7 @@ private TBrokerRangeDesc createBrokerRangeDesc(long curFileOffset, TBrokerFileSt rangeDesc.setStart_offset(curFileOffset); rangeDesc.setSize(rangeBytes); rangeDesc.setFile_size(fileStatus.size); + rangeDesc.setPartition_columns(parsePartitionedFields(fileStatus.path)); return rangeDesc; } diff --git a/gensrc/thrift/PlanNodes.thrift b/gensrc/thrift/PlanNodes.thrift index d587e34b58b8c9..fd3529fd3a3d9c 100644 --- a/gensrc/thrift/PlanNodes.thrift +++ b/gensrc/thrift/PlanNodes.thrift @@ -114,6 +114,8 @@ struct TBrokerRangeDesc { 7: optional Types.TUniqueId load_id // total size of the file 8: optional i64 file_size + // partition columns of this file(parquet only) + 9: optional map partition_columns } struct TBrokerScanRangeParams { From cefe1794d4b9feb9e5e8377750994b53c411cab9 Mon Sep 17 00:00:00 2001 From: Mingyu Chen Date: Thu, 1 Aug 2019 10:26:06 +0800 Subject: [PATCH 02/26] Fix bug that replicas of a tablet may be located on same host (#1517) Doris support deploy multi BE on one host. So when allocating BE for replicas of a tablet, we should select different host. But there is a bug in tablet scheduler that same host may be selected for one tablet. This patch will fix this problem. There are some places related to this problem: 1. Create Table There is no bug in Create Table process. 2. Tablet Scheduler Fixed when selecting BE for REPLICA_MISSING and REPLICA_RELOCATING. Fixed when balance the tablet. 3. Colocate Table Balancer Fixed when selecting BE for repairing colocate backend sequence. Not fix in colocate group balance. Leave it to colocate repairing. 4. Tablet report Tablet report may add replica to catalog. But I did not check the host here, Tablet Scheduler will fix it. --- .../operation/tablet-repair-and-balance.md | 9 +++ .../doris/catalog/ColocateTableIndex.java | 11 ---- .../java/org/apache/doris/catalog/Tablet.java | 13 +++-- .../doris/clone/ColocateTableBalancer.java | 55 +++++++++++++++++-- .../org/apache/doris/clone/LoadBalancer.java | 21 ++++++- .../apache/doris/clone/TabletSchedCtx.java | 16 +++++- .../apache/doris/clone/TabletScheduler.java | 49 +++++++++++++++-- .../doris/system/SystemInfoService.java | 1 + 8 files changed, 149 insertions(+), 26 deletions(-) diff --git a/docs/documentation/cn/administrator-guide/operation/tablet-repair-and-balance.md b/docs/documentation/cn/administrator-guide/operation/tablet-repair-and-balance.md index 020a74ff0e47fa..0051d8af9a17d3 100644 --- a/docs/documentation/cn/administrator-guide/operation/tablet-repair-and-balance.md +++ b/docs/documentation/cn/administrator-guide/operation/tablet-repair-and-balance.md @@ -134,6 +134,8 @@ TabletChecker 作为常驻的后台进程,会定期检查所有分片的状态 删除一个非 Colocation Group 中指定的副本分布 BE 节点上的副本。 +Doris 在选择副本节点时,不会将同一个 Tablet 的副本部署在同一个 host 的不同 BE 上。保证了即使同一个 host 上的所有 BE 都挂掉,也不会造成全部副本丢失。 + ### 调度优先级 TabletScheduler 里等待被调度的分片会根据状态不同,赋予不同的优先级。优先级高的分片将会被优先调度。目前有以下几种优先级。 @@ -189,6 +191,8 @@ TabletScheduler 里等待被调度的分片会根据状态不同,赋予不同 Doris 会自动进行集群内的副本均衡。均衡的主要思想,是对某些分片,先在低负载的节点上创建一个副本,然后再删除这些分片在高负载节点上的副本。同时,因为不同存储介质的存在,在同一个集群内的不同 BE 节点上,可能存在或不存在一种或两种存储介质。我们要求存储介质为 A 的分片在均衡后,尽量依然存储在存储介质 A 中。所以我们根据存储介质,对集群的 BE 节点进行划分。然后针对不同的存储介质的 BE 节点集合,进行负载均衡调度。 +同样,副本均衡会保证不会将同一个 Tablet 的副本部署在同一个 host 的 BE 上。 + ### BE 节点负载 我们用 ClusterLoadStatistics(CLS)表示一个 cluster 中各个 Backend 的负载均衡情况。TabletScheduler 根据这个统计值,来触发集群均衡。我们当前通过 **磁盘使用率** 和 **副本数量** 两个指标,为每个BE计算一个 loadScore,作为 BE 的负载分数。分数越高,表示该 BE 的负载越重。 @@ -635,4 +639,9 @@ TabletScheduler 在每轮调度时,都会通过 LoadBalancer 来选择一定 * 在某些情况下,默认的副本修复和均衡策略可能会导致网络被打满(多发生在千兆网卡,且每台 BE 的磁盘数量较多的情况下)。此时需要调整一些参数来减少同时进行的均衡和修复任务数。 +* 目前针对 Colocate Table 的副本的均衡策略无法保证同一个 Tablet 的副本不会分布在同一个 host 的 BE 上。但 Colocate Table 的副本的修复策略会检测到这种分布错误并校正。但可能会出现,校正后,均衡策略再次认为副本不均衡而重新均衡。从而导致在两种状态间不停交替,无法使 Colocate Group 达成稳定。针对这种情况,我们建议在使用 Colocate 属性时,尽量保证集群是同构的,以减小副本分布在同一个 host 上的概率。 + + + + diff --git a/fe/src/main/java/org/apache/doris/catalog/ColocateTableIndex.java b/fe/src/main/java/org/apache/doris/catalog/ColocateTableIndex.java index 4316cd9a5a1eb8..c8a685f1bab090 100644 --- a/fe/src/main/java/org/apache/doris/catalog/ColocateTableIndex.java +++ b/fe/src/main/java/org/apache/doris/catalog/ColocateTableIndex.java @@ -173,17 +173,6 @@ public GroupId addTableToGroup(long dbId, OlapTable tbl, String groupName, Group } } - // This is for manual recovery. - public void addTableToGroup(GroupId groupId, long tableId) { - writeLock(); - try { - group2Tables.put(groupId, tableId); - table2Group.put(tableId, groupId); - } finally { - writeUnlock(); - } - } - public void addBackendsPerBucketSeq(GroupId groupId, List> backendsPerBucketSeq) { writeLock(); try { diff --git a/fe/src/main/java/org/apache/doris/catalog/Tablet.java b/fe/src/main/java/org/apache/doris/catalog/Tablet.java index c4cc461ae1fdf5..3e15ac1ddc1670 100644 --- a/fe/src/main/java/org/apache/doris/catalog/Tablet.java +++ b/fe/src/main/java/org/apache/doris/catalog/Tablet.java @@ -383,12 +383,15 @@ public Pair getHealthStatusWithPriority( int stable = 0; int availableInCluster = 0; + Set hosts = Sets.newHashSet(); for (Replica replica : replicas) { - long backendId = replica.getBackendId(); - Backend backend = systemInfoService.getBackend(backendId); + Backend backend = systemInfoService.getBackend(replica.getBackendId()); if (backend == null || !backend.isAlive() || replica.getState() == ReplicaState.CLONE - || replica.isBad()) { - // this replica is not alive + || replica.isBad() || !hosts.add(backend.getHost())) { + // this replica is not alive, + // or if this replica is on same host with another replica, we also treat it as 'dead', + // so that Tablet Scheduler will create a new replica on different host. + // ATTN: Replicas on same host is a bug of previous Doris version, so we fix it by this way. continue; } alive++; @@ -422,7 +425,7 @@ public Pair getHealthStatusWithPriority( // condition explain: // 1. alive < replicationNum: replica is missing or bad // 2. replicas.size() >= availableBackendsNum: the existing replicas occupies all available backends - // 3. availableBackendsNum >= replicationNum: make sure after deleting, there will be a backend for new replica. + // 3. availableBackendsNum >= replicationNum: make sure after deleting, there will be at least one backend for new replica. // 4. replicationNum > 1: if replication num is set to 1, do not delete any replica, for safety reason return Pair.create(TabletStatus.FORCE_REDUNDANT, TabletSchedCtx.Priority.VERY_HIGH); } else if (alive < (replicationNum / 2) + 1) { diff --git a/fe/src/main/java/org/apache/doris/clone/ColocateTableBalancer.java b/fe/src/main/java/org/apache/doris/clone/ColocateTableBalancer.java index 91b1a6b9c09af3..19707dc884dbfb 100644 --- a/fe/src/main/java/org/apache/doris/clone/ColocateTableBalancer.java +++ b/fe/src/main/java/org/apache/doris/clone/ColocateTableBalancer.java @@ -38,6 +38,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -104,7 +105,7 @@ protected void runOneCycle() { * and backend 3 is dead, so we will find an available backend(eg. backend 4) to replace backend 3. * [[1, 2, 4], [4, 1, 2], [3, 4, 1], [2, 3, 4], [1, 2, 3]] * - * NOTICE that we only replace the backends in first bucket. that is, we only replace + * NOTICE that in this example, we only replace the #3 backend in first bucket. That is, we only replace * one bucket in one group at each round. because we need to use newly-updated cluster load statistic to * find next available backend. and cluster load statistic is updated every 20 seconds. */ @@ -143,9 +144,33 @@ private void relocateGroup() { } if (unavailableBeId == -1) { - // all backends in this group are available, check next group - continue; + // all backends in this group are available. + // But in previous version we had a bug that replicas of a tablet may be located on same host. + // we have to check it. + List> backendsPerBucketsSeq = colocateIndex.getBackendsPerBucketSeq(groupId); + Set hosts = Sets.newHashSet(); + OUT: for (List backendIds : backendsPerBucketsSeq) { + for (Long beId : backendIds) { + Backend be = infoService.getBackend(beId); + if (be == null) { + // backend can be dropped any time, just skip this bucket + break; + } + if (!hosts.add(be.getHost())) { + // find replicas on same host. simply mark this backend as unavailable, + // so that following step will find another backend + unavailableBeId = beId; + break OUT; + } + } + } + + if (unavailableBeId == -1) { + // if everything is ok, continue + continue; + } } + Preconditions.checkState(unavailableBeId != -1); // find the first bucket which contains the unavailable backend LOG.info("backend {} is unavailable in colocate group {}", unavailableBeId, groupId); @@ -229,7 +254,7 @@ private long selectSubstituteBackend(int tabletOrderIdx, GroupId groupId, long u * There is an unsolved problem of finding a new backend for data migration: * Different table(partition) in this group may in different storage medium(SSD or HDD). If one backend * is down, the best solution is to find a backend which has both SSD and HDD, and replicas can be - * relocated in corresponding medium. + * relocated in corresponding storage medium. * But in fact, backends can be heterogeneous, which may only has SSD or HDD. If we choose to strictly * find backends with expecting storage medium, this may lead to a consequence that most of replicas * are gathered in a small portion of backends. @@ -242,10 +267,32 @@ private long selectSubstituteBackend(int tabletOrderIdx, GroupId groupId, long u LOG.warn("failed to relocate backend for colocate group: {}, no backends found", groupId); return -1; } + + // the selected backend should not be on same host of other backends of this bucket. + // here we generate a host set for further checking. + SystemInfoService infoService = Catalog.getCurrentSystemInfo(); + Set excludeHosts = Sets.newHashSet(); + for (Long excludeBeId : excludeBeIds) { + Backend be = infoService.getBackend(excludeBeId); + if (be == null) { + LOG.info("Backend {} has been dropped when finding backend for colocate group {}", excludeBeId, groupId); + return -1; + } + excludeHosts.add(be.getHost()); + } + Preconditions.checkState(excludeBeIds.size() >= excludeHosts.size()); + // beStats is ordered by load score, ascend. so finding the available from first to last BackendLoadStatistic choosenBe = null; for (BackendLoadStatistic beStat : beStats) { if (beStat.isAvailable() && beStat.getBeId() != unavailableBeId && !excludeBeIds.contains(beStat.getBeId())) { + Backend be = infoService.getBackend(beStat.getBeId()); + if (be == null) { + continue; + } + if (excludeHosts.contains(be.getHost())) { + break; + } choosenBe = beStat; break; } diff --git a/fe/src/main/java/org/apache/doris/clone/LoadBalancer.java b/fe/src/main/java/org/apache/doris/clone/LoadBalancer.java index b1b8a1aa0dc0c0..77d3ac9a2814e2 100644 --- a/fe/src/main/java/org/apache/doris/clone/LoadBalancer.java +++ b/fe/src/main/java/org/apache/doris/clone/LoadBalancer.java @@ -25,6 +25,8 @@ import org.apache.doris.clone.SchedException.Status; import org.apache.doris.clone.TabletSchedCtx.Priority; import org.apache.doris.clone.TabletScheduler.PathSlot; +import org.apache.doris.system.Backend; +import org.apache.doris.system.SystemInfoService; import org.apache.doris.task.AgentBatchTask; import org.apache.doris.thrift.TStorageMedium; @@ -50,10 +52,12 @@ public class LoadBalancer { private Map statisticMap; private TabletInvertedIndex invertedIndex; + private SystemInfoService infoService; public LoadBalancer(Map statisticMap) { this.statisticMap = statisticMap; this.invertedIndex = Catalog.getCurrentInvertedIndex(); + this.infoService = Catalog.getCurrentSystemInfo(); } public List selectAlternativeTablets() { @@ -224,12 +228,18 @@ public void createBalanceTask(TabletSchedCtx tabletCtx, Map back List replicas = tabletCtx.getReplicas(); // Check if this tablet has replica on high load backend. + // Also create a set to save hosts of this tablet. + Set hosts = Sets.newHashSet(); boolean hasHighReplica = false; for (Replica replica : replicas) { if (highBe.stream().anyMatch(b -> b.getBeId() == replica.getBackendId())) { hasHighReplica = true; - break; } + Backend be = infoService.getBackend(replica.getBackendId()); + if (be == null) { + throw new SchedException(Status.UNRECOVERABLE, "backend is dropped: " + replica.getBackendId()); + } + hosts.add(be.getHost()); } if (!hasHighReplica) { throw new SchedException(Status.UNRECOVERABLE, "no replica on high load backend"); @@ -259,6 +269,15 @@ public void createBalanceTask(TabletSchedCtx tabletCtx, Map back boolean setDest = false; for (BackendLoadStatistic beStat : lowBe) { if (beStat.isAvailable() && !replicas.stream().anyMatch(r -> r.getBackendId() == beStat.getBeId())) { + // check if on same host. + Backend lowBackend = infoService.getBackend(beStat.getBeId()); + if (lowBackend == null) { + continue; + } + if (hosts.contains(lowBackend.getHost())) { + continue; + } + // no replica on this low load backend // 1. check if this clone task can make the cluster more balance. List availPaths = Lists.newArrayList(); diff --git a/fe/src/main/java/org/apache/doris/clone/TabletSchedCtx.java b/fe/src/main/java/org/apache/doris/clone/TabletSchedCtx.java index e0adbe149721b3..76822b698bd610 100644 --- a/fe/src/main/java/org/apache/doris/clone/TabletSchedCtx.java +++ b/fe/src/main/java/org/apache/doris/clone/TabletSchedCtx.java @@ -411,9 +411,23 @@ public long getTabletSize() { return max; } - // database lock should be held. + /* + * check if existing replicas are on same BE. + * database lock should be held. + */ public boolean containsBE(long beId) { + String host = infoService.getBackend(beId).getHost(); for (Replica replica : tablet.getReplicas()) { + Backend be = infoService.getBackend(replica.getBackendId()); + if (be == null) { + // BE has been dropped, just return true, so that the caller will not choose this BE. + return true; + } + if (host.equals(be.getHost())) { + return true; + } + // actually there is no need to check BE id anymore, because if hosts are not same, BE ids are + // not same either. But for psychological comfort, leave this check here. if (replica.getBackendId() == beId) { return true; } diff --git a/fe/src/main/java/org/apache/doris/clone/TabletScheduler.java b/fe/src/main/java/org/apache/doris/clone/TabletScheduler.java index 9f2186480be149..4074ab35a734ef 100644 --- a/fe/src/main/java/org/apache/doris/clone/TabletScheduler.java +++ b/fe/src/main/java/org/apache/doris/clone/TabletScheduler.java @@ -591,7 +591,7 @@ private void handleTabletByTypeAndStatus(TabletStatus status, TabletSchedCtx tab * So we need to find a destination backend to clone a new replica as possible as we can. * 1. find an available path in a backend as destination: * 1. backend need to be alive. - * 2. backend of existing replicas should be excluded. + * 2. backend of existing replicas should be excluded. (should not be on same host either) * 3. backend has available slot for clone. * 4. replica can fit in the path (consider the threshold of disk capacity and usage percent). * 5. try to find a path with lowest load score. @@ -670,6 +670,7 @@ private void handleRedundantReplica(TabletSchedCtx tabletCtx, boolean force) thr || deleteCloneReplica(tabletCtx, force) || deleteReplicaWithFailedVersion(tabletCtx, force) || deleteReplicaWithLowerVersion(tabletCtx, force) + || deleteReplicaOnSameHost(tabletCtx, force) || deleteReplicaNotInCluster(tabletCtx, force) || deleteReplicaOnHighLoadBackend(tabletCtx, force)) { // if we delete at least one redundant replica, we still throw a SchedException with status FINISHED @@ -745,12 +746,47 @@ private boolean deleteReplicaWithLowerVersion(TabletSchedCtx tabletCtx, boolean return false; } + private boolean deleteReplicaOnSameHost(TabletSchedCtx tabletCtx, boolean force) { + ClusterLoadStatistic statistic = statisticMap.get(tabletCtx.getCluster()); + if (statistic == null) { + return false; + } + + // collect replicas of this tablet. + // host -> (replicas on same host) + Map> hostToReplicas = Maps.newHashMap(); + for (Replica replica : tabletCtx.getReplicas()) { + Backend be = infoService.getBackend(replica.getBackendId()); + if (be == null) { + // this case should be handled in deleteBackendDropped() + return false; + } + List replicas = hostToReplicas.get(be.getHost()); + if (replicas == null) { + replicas = Lists.newArrayList(); + hostToReplicas.put(be.getHost(), replicas); + } + replicas.add(replica); + } + + // find if there are replicas on same host, if yes, delete one. + for (List replicas : hostToReplicas.values()) { + if (replicas.size() > 1) { + // delete one replica from replicas on same host. + // better to choose high load backend + return deleteFromHighLoadBackend(tabletCtx, replicas, force, statistic); + } + } + + return false; + } + private boolean deleteReplicaNotInCluster(TabletSchedCtx tabletCtx, boolean force) { for (Replica replica : tabletCtx.getReplicas()) { Backend be = infoService.getBackend(replica.getBackendId()); if (be == null) { // this case should be handled in deleteBackendDropped() - continue; + return false; } if (!be.getOwnerClusterName().equals(tabletCtx.getCluster())) { deleteReplicaInternal(tabletCtx, replica, "not in cluster", force); @@ -766,9 +802,14 @@ private boolean deleteReplicaOnHighLoadBackend(TabletSchedCtx tabletCtx, boolean return false; } + return deleteFromHighLoadBackend(tabletCtx, tabletCtx.getReplicas(), force, statistic); + } + + private boolean deleteFromHighLoadBackend(TabletSchedCtx tabletCtx, List replicas, + boolean force, ClusterLoadStatistic statistic) { Replica chosenReplica = null; double maxScore = 0; - for (Replica replica : tabletCtx.getReplicas()) { + for (Replica replica : replicas) { BackendLoadStatistic beStatistic = statistic.getBackendLoadStatistic(replica.getBackendId()); if (beStatistic == null) { continue; @@ -944,7 +985,7 @@ private RootPathLoadStatistic chooseAvailableDestPath(TabletSchedCtx tabletCtx, if (!bes.isAvailable()) { continue; } - // exclude BE which already has replica of this tablet + // exclude host which already has replica of this tablet if (tabletCtx.containsBE(bes.getBeId())) { continue; } diff --git a/fe/src/main/java/org/apache/doris/system/SystemInfoService.java b/fe/src/main/java/org/apache/doris/system/SystemInfoService.java index 8f64f5d10c8420..81688bc86a50f4 100644 --- a/fe/src/main/java/org/apache/doris/system/SystemInfoService.java +++ b/fe/src/main/java/org/apache/doris/system/SystemInfoService.java @@ -766,6 +766,7 @@ public synchronized List seqChooseBackendIds(int backendNum, boolean needA // put backend with same host in same list final List srcBackends = getClusterBackends(clusterName); + // host -> BE list Map> backendMaps = Maps.newHashMap(); for (Backend backend : srcBackends) { if (backendMaps.containsKey(backend.getHost())){ From 6c21a5a484bf897bba92ae906f8bbd6800c5b037 Mon Sep 17 00:00:00 2001 From: ZHAO Chun Date: Sat, 3 Aug 2019 22:49:35 +0800 Subject: [PATCH 03/26] Switch MAKE_TEST off in build.sh (#1579) --- build.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sh b/build.sh index a27f6fdf23b4b3..3d67d021b6a9c8 100755 --- a/build.sh +++ b/build.sh @@ -160,7 +160,7 @@ if [ ${BUILD_BE} -eq 1 ] ; then fi mkdir -p ${DORIS_HOME}/be/build/ cd ${DORIS_HOME}/be/build/ - cmake -DWITH_MYSQL=${WITH_MYSQL} -DWITH_LZO=${WITH_LZO} ../ + cmake -DMAKE_TEST=OFF -DWITH_MYSQL=${WITH_MYSQL} -DWITH_LZO=${WITH_LZO} ../ make -j${PARALLEL} make install cd ${DORIS_HOME} From 938c6d4cdff2d84592430f98918f9a460c392bb4 Mon Sep 17 00:00:00 2001 From: EmmyMiao87 <522274284@qq.com> Date: Sun, 4 Aug 2019 15:54:03 +0800 Subject: [PATCH 04/26] Thrown TabletQuorumFailedException in commitTxn (#1575) The TabletQuorumFailedException will be thrown in commitTxn while the success replica num of tablet is less then quorom replica num. The Hadoop load does not handle this exception because the push task will retry it later. The streaming broker, insert, stream and mini load will catch this exception and abort the txn after that. --- .../org/apache/doris/load/LoadChecker.java | 3 ++ .../doris/load/loadv2/BrokerLoadJob.java | 6 ++- .../transaction/GlobalTransactionMgr.java | 10 +++-- .../TabletQuorumFailedException.java | 44 +++++++++++++++++++ .../transaction/GlobalTransactionMgrTest.java | 29 +++++++----- 5 files changed, 75 insertions(+), 17 deletions(-) create mode 100644 fe/src/main/java/org/apache/doris/transaction/TabletQuorumFailedException.java diff --git a/fe/src/main/java/org/apache/doris/load/LoadChecker.java b/fe/src/main/java/org/apache/doris/load/LoadChecker.java index d4479f8fe13439..83bb0abe3ca7f3 100644 --- a/fe/src/main/java/org/apache/doris/load/LoadChecker.java +++ b/fe/src/main/java/org/apache/doris/load/LoadChecker.java @@ -51,6 +51,7 @@ import org.apache.doris.thrift.TTaskType; import org.apache.doris.transaction.GlobalTransactionMgr; import org.apache.doris.transaction.TabletCommitInfo; +import org.apache.doris.transaction.TabletQuorumFailedException; import org.apache.doris.transaction.TransactionState; import org.apache.doris.transaction.TransactionStatus; @@ -333,6 +334,8 @@ private void tryCommitJob(LoadJob job, Database db) { tabletCommitInfos.add(new TabletCommitInfo(tabletId, replica.getBackendId())); } globalTransactionMgr.commitTransaction(job.getDbId(), job.getTransactionId(), tabletCommitInfos); + } catch (TabletQuorumFailedException e) { + // wait the upper application retry } catch (UserException e) { LOG.warn("errors while commit transaction [{}], cancel the job {}, reason is {}", transactionState.getTransactionId(), job, e); diff --git a/fe/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java b/fe/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java index 649c594dc87353..ca0e8c4b3c58b3 100644 --- a/fe/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java +++ b/fe/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java @@ -374,6 +374,10 @@ private void onLoadingTaskFinished(BrokerLoadingTaskAttachment attachment) { } db.writeLock(); try { + LOG.info(new LogBuilder(LogKey.LOAD_JOB, id) + .add("txn_id", transactionId) + .add("msg", "Load job try to commit txn") + .build()); Catalog.getCurrentGlobalTransactionMgr().commitTransaction( dbId, transactionId, commitInfos, new LoadJobFinalOperation(id, loadingStatus, progress, loadStartTimestamp, @@ -381,7 +385,7 @@ private void onLoadingTaskFinished(BrokerLoadingTaskAttachment attachment) { } catch (UserException e) { LOG.warn(new LogBuilder(LogKey.LOAD_JOB, id) .add("database_id", dbId) - .add("error_msg", "Failed to commit txn.") + .add("error_msg", "Failed to commit txn with error:" + e.getMessage()) .build(), e); cancelJobWithoutCheck(new FailMsg(FailMsg.CancelType.LOAD_RUN_FAIL, e.getMessage()),true); return; diff --git a/fe/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java b/fe/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java index ed46f4e063d33e..2800ee15e9144b 100644 --- a/fe/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java +++ b/fe/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java @@ -396,12 +396,14 @@ public void commitTransaction(long dbId, long transactionId, List errorBackendIdsForTablet = Sets.newHashSet(); + + public TabletQuorumFailedException(long transactionId, long tabletId, + int successReplicaNum, int quorumReplicaNum, + Set errorBackendIdsForTablet) { + super(String.format(TABLET_QUORUM_FAILED_MSG, transactionId, tabletId, + successReplicaNum, quorumReplicaNum, + Joiner.on(",").join(errorBackendIdsForTablet)), + transactionId); + this.tabletId = tabletId; + this.errorBackendIdsForTablet = errorBackendIdsForTablet; + } +} diff --git a/fe/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java b/fe/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java index 5abc132e8ab415..fb5375933dd938 100644 --- a/fe/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java +++ b/fe/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java @@ -17,10 +17,7 @@ package org.apache.doris.transaction; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.*; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.CatalogTestUtil; @@ -226,10 +223,14 @@ public void testCommitTransactionWithOneFailed() throws UserException { transTablets = Lists.newArrayList(); transTablets.add(tabletCommitInfo1); transTablets.add(tabletCommitInfo3); - masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId2, transTablets); - transactionState = masterTransMgr.getTransactionState(transactionId2); - // check status is prepare, because the commit failed - assertEquals(TransactionStatus.PREPARE, transactionState.getTransactionStatus()); + try { + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId2, transTablets); + Assert.fail(); + } catch (TabletQuorumFailedException e) { + transactionState = masterTransMgr.getTransactionState(transactionId2); + // check status is prepare, because the commit failed + assertEquals(TransactionStatus.PREPARE, transactionState.getTransactionStatus()); + } // check replica version Partition testPartition = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1) .getPartition(CatalogTestUtil.testPartition1); @@ -533,10 +534,14 @@ public void testFinishTransactionWithOneFailed() throws UserException { transTablets = Lists.newArrayList(); transTablets.add(tabletCommitInfo1); transTablets.add(tabletCommitInfo3); - masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId2, transTablets); - transactionState = masterTransMgr.getTransactionState(transactionId2); - // check status is prepare, because the commit failed - assertEquals(TransactionStatus.PREPARE, transactionState.getTransactionStatus()); + try { + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId2, transTablets); + Assert.fail(); + } catch (TabletQuorumFailedException e) { + transactionState = masterTransMgr.getTransactionState(transactionId2); + // check status is prepare, because the commit failed + assertEquals(TransactionStatus.PREPARE, transactionState.getTransactionStatus()); + } // commit the second transaction with 1,2,3 success tabletCommitInfo1 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId1); From 81cbffd2e853032f694d943440f3ab5767f2addd Mon Sep 17 00:00:00 2001 From: yuanlihan Date: Sun, 4 Aug 2019 23:28:24 +0800 Subject: [PATCH 05/26] Read date type column from parquet --- be/src/exec/parquet_reader.cpp | 29 +++++++++++++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/be/src/exec/parquet_reader.cpp b/be/src/exec/parquet_reader.cpp index de1acd12f1d6ed..0c7242edd116bf 100644 --- a/be/src/exec/parquet_reader.cpp +++ b/be/src/exec/parquet_reader.cpp @@ -397,6 +397,35 @@ Status ParquetReaderWrap::read(Tuple* tuple, const std::vector& } break; } + case arrow::Type::type::DATE32: { + auto ts_array = std::dynamic_pointer_cast(_batch->column(column_index)); + if (ts_array->IsNull(_current_line_of_group)) { + RETURN_IF_ERROR(set_field_null(tuple, slot_desc)); + } else { + time_t timestamp = (time_t)((int64_t)ts_array->Value(_current_line_of_group) * 24 * 60 * 60); + tm* local; + local = localtime(×tamp); + char* to = reinterpret_cast(&tmp_buf); + wbtyes = (uint32_t)strftime(to, 64, "%Y-%m-%d", local); + fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbtyes); + } + break; + } + case arrow::Type::type::DATE64: { + auto ts_array = std::dynamic_pointer_cast(_batch->column(column_index)); + if (ts_array->IsNull(_current_line_of_group)) { + RETURN_IF_ERROR(set_field_null(tuple, slot_desc)); + } else { + // convert milliseconds to seconds + time_t timestamp = (time_t)((int64_t)ts_array->Value(_current_line_of_group) / 1000); + tm* local; + local = localtime(×tamp); + char* to = reinterpret_cast(&tmp_buf); + wbtyes = (uint32_t)strftime(to, 64, "%Y-%m-%d %H:%M:%S", local); + fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbtyes); + } + break; + } default: { // other type not support. std::stringstream str_error; From 0ab22e4c3b09b32a44a310f0e12a4a485678b16a Mon Sep 17 00:00:00 2001 From: yuanlihan Date: Sun, 4 Aug 2019 23:31:27 +0800 Subject: [PATCH 06/26] Enable partition discovery for broker loading --- be/src/exec/broker_scanner.cpp | 39 +++++++--- be/src/exec/broker_scanner.h | 2 + be/test/exec/broker_scan_node_test.cpp | 71 ++++++++++++++++-- .../Contents/Data Manipulation/broker_load.md | 52 +++++++++---- .../Data Manipulation/manipulation_stmt.md | 36 +++++++++ fe/src/main/cup/sql_parser.cup | 16 +++- .../doris/analysis/DataDescription.java | 19 +++++ .../org/apache/doris/common/FeConstants.java | 2 +- .../apache/doris/common/FeMetaVersion.java | 2 + .../apache/doris/load/BrokerFileGroup.java | 20 +++++ .../apache/doris/planner/BrokerScanNode.java | 74 ++++++++++--------- fe/src/main/jflex/sql_scanner.flex | 1 + 12 files changed, 261 insertions(+), 73 deletions(-) diff --git a/be/src/exec/broker_scanner.cpp b/be/src/exec/broker_scanner.cpp index 19cd829f6826e6..062a476ab537ea 100644 --- a/be/src/exec/broker_scanner.cpp +++ b/be/src/exec/broker_scanner.cpp @@ -54,6 +54,7 @@ BrokerScanner::BrokerScanner(RuntimeState* state, _cur_decompressor(nullptr), _next_range(0), _cur_line_reader_eof(false), + _partition_columns(), _scanner_eof(false), _skip_next_line(false) { } @@ -236,6 +237,9 @@ Status BrokerScanner::open_line_reader() { // _decompressor may be NULL if this is not a compressed file RETURN_IF_ERROR(create_decompressor(range.format_type)); + // set partitioned columns + _partition_columns = range.partition_columns; + // open line reader switch (range.format_type) { case TFileFormatType::FORMAT_CSV_PLAIN: @@ -452,6 +456,17 @@ bool BrokerScanner::convert_one_row( return fill_dest_tuple(line, tuple, tuple_pool); } +inline void BrokerScanner::fill_slot(SlotDescriptor* slot_desc, const Slice& value) { + if (slot_desc->is_nullable() && is_null(value)) { + _src_tuple->set_null(slot_desc->null_indicator_offset()); + } + _src_tuple->set_not_null(slot_desc->null_indicator_offset()); + void* slot = _src_tuple->get_slot(slot_desc->tuple_offset()); + StringValue* str_slot = reinterpret_cast(slot); + str_slot->ptr = value.data; + str_slot->len = value.size; +} + // Convert one row to this tuple bool BrokerScanner::line_to_src_tuple(const Slice& line) { @@ -469,7 +484,7 @@ bool BrokerScanner::line_to_src_tuple(const Slice& line) { split_line(line, &values); } - if (values.size() < _src_slot_descs.size()) { + if (values.size() + _partition_columns.size() < _src_slot_descs.size()) { std::stringstream error_msg; error_msg << "actual column number is less than schema column number. " << "actual number: " << values.size() << " sep: " << _value_separator << ", " @@ -478,7 +493,7 @@ bool BrokerScanner::line_to_src_tuple(const Slice& line) { error_msg.str()); _counter->num_rows_filtered++; return false; - } else if (values.size() > _src_slot_descs.size()) { + } else if (values.size() + _partition_columns.size() > _src_slot_descs.size()) { std::stringstream error_msg; error_msg << "actual column number is more than schema column number. " << "actual number: " << values.size() << " sep: " << _value_separator << ", " @@ -489,18 +504,18 @@ bool BrokerScanner::line_to_src_tuple(const Slice& line) { return false; } - for (int i = 0; i < values.size(); ++i) { + int file_column_index = 0; + for (int i = 0; i < _src_slot_descs.size(); ++i) { auto slot_desc = _src_slot_descs[i]; - const Slice& value = values[i]; - if (slot_desc->is_nullable() && is_null(value)) { - _src_tuple->set_null(slot_desc->null_indicator_offset()); - continue; + auto iter = _partition_columns.find(slot_desc->col_name()); + if (iter != _partition_columns.end()) { + std::string partitioned_field = iter->second; + const Slice value = Slice(partitioned_field.c_str(), partitioned_field.size()); + fill_slot(slot_desc, value); + } else { + const Slice& value = values[file_column_index++]; + fill_slot(slot_desc, value); } - _src_tuple->set_not_null(slot_desc->null_indicator_offset()); - void* slot = _src_tuple->get_slot(slot_desc->tuple_offset()); - StringValue* str_slot = reinterpret_cast(slot); - str_slot->ptr = value.data; - str_slot->len = value.size; } return true; diff --git a/be/src/exec/broker_scanner.h b/be/src/exec/broker_scanner.h index b9a798694dc8d9..ea464dfc48f6d3 100644 --- a/be/src/exec/broker_scanner.h +++ b/be/src/exec/broker_scanner.h @@ -104,6 +104,7 @@ class BrokerScanner : public BaseScanner { //Status init_expr_ctxes(); Status line_to_src_tuple(); + void fill_slot(SlotDescriptor* slot_desc, const Slice& value); bool line_to_src_tuple(const Slice& line); private:; const std::vector& _ranges; @@ -120,6 +121,7 @@ private:; Decompressor* _cur_decompressor; int _next_range; bool _cur_line_reader_eof; + std::map _partition_columns; bool _scanner_eof; diff --git a/be/test/exec/broker_scan_node_test.cpp b/be/test/exec/broker_scan_node_test.cpp index 066f468dfa99cb..bc4cc533d2455d 100644 --- a/be/test/exec/broker_scan_node_test.cpp +++ b/be/test/exec/broker_scan_node_test.cpp @@ -153,7 +153,33 @@ void BrokerScanNodeTest::init_desc_table() { slot_desc.nullIndicatorByte = 0; slot_desc.nullIndicatorBit = -1; slot_desc.colName = "k3"; - slot_desc.slotIdx = 2; + slot_desc.slotIdx = 3; + slot_desc.isMaterialized = true; + + t_desc_table.slotDescriptors.push_back(slot_desc); + } + // k4(partitioned column) + { + TSlotDescriptor slot_desc; + + slot_desc.id = next_slot_id++; + slot_desc.parent = 0; + TTypeDesc type; + { + TTypeNode node; + node.__set_type(TTypeNodeType::SCALAR); + TScalarType scalar_type; + scalar_type.__set_type(TPrimitiveType::INT); + node.__set_scalar_type(scalar_type); + type.types.push_back(node); + } + slot_desc.slotType = type; + slot_desc.columnPos = 1; + slot_desc.byteOffset = 12; + slot_desc.nullIndicatorByte = 0; + slot_desc.nullIndicatorBit = -1; + slot_desc.colName = "k4"; + slot_desc.slotIdx = 4; slot_desc.isMaterialized = true; t_desc_table.slotDescriptors.push_back(slot_desc); @@ -164,7 +190,7 @@ void BrokerScanNodeTest::init_desc_table() { // TTupleDescriptor dest TTupleDescriptor t_tuple_desc; t_tuple_desc.id = 0; - t_tuple_desc.byteSize = 12; + t_tuple_desc.byteSize = 16; t_tuple_desc.numNullBytes = 0; t_tuple_desc.tableId = 0; t_tuple_desc.__isset.tableId = true; @@ -251,7 +277,34 @@ void BrokerScanNodeTest::init_desc_table() { slot_desc.nullIndicatorByte = 0; slot_desc.nullIndicatorBit = -1; slot_desc.colName = "k3"; - slot_desc.slotIdx = 2; + slot_desc.slotIdx = 3; + slot_desc.isMaterialized = true; + + t_desc_table.slotDescriptors.push_back(slot_desc); + } + // k4(partitioned column) + { + TSlotDescriptor slot_desc; + + slot_desc.id = next_slot_id++; + slot_desc.parent = 1; + TTypeDesc type; + { + TTypeNode node; + node.__set_type(TTypeNodeType::SCALAR); + TScalarType scalar_type; + scalar_type.__set_type(TPrimitiveType::VARCHAR); + scalar_type.__set_len(65535); + node.__set_scalar_type(scalar_type); + type.types.push_back(node); + } + slot_desc.slotType = type; + slot_desc.columnPos = 1; + slot_desc.byteOffset = 48; + slot_desc.nullIndicatorByte = 0; + slot_desc.nullIndicatorBit = -1; + slot_desc.colName = "k4"; + slot_desc.slotIdx = 4; slot_desc.isMaterialized = true; t_desc_table.slotDescriptors.push_back(slot_desc); @@ -261,7 +314,7 @@ void BrokerScanNodeTest::init_desc_table() { // TTupleDescriptor source TTupleDescriptor t_tuple_desc; t_tuple_desc.id = 1; - t_tuple_desc.byteSize = 48; + t_tuple_desc.byteSize = 60; t_tuple_desc.numNullBytes = 0; t_tuple_desc.tableId = 0; t_tuple_desc.__isset.tableId = true; @@ -297,7 +350,7 @@ void BrokerScanNodeTest::init() { varchar_type.types.push_back(node); } - for (int i = 0; i < 3; ++i) { + for (int i = 0; i < 4; ++i) { TExprNode cast_expr; cast_expr.node_type = TExprNodeType::CAST_EXPR; cast_expr.type = int_type; @@ -319,7 +372,7 @@ void BrokerScanNodeTest::init() { slot_ref.type = varchar_type; slot_ref.num_children = 0; slot_ref.__isset.slot_ref = true; - slot_ref.slot_ref.slot_id = 4 + i; + slot_ref.slot_ref.slot_id = 5 + i; slot_ref.slot_ref.tuple_id = 1; TExpr expr; @@ -327,7 +380,7 @@ void BrokerScanNodeTest::init() { expr.nodes.push_back(slot_ref); _params.expr_of_dest_slot.emplace(i + 1, expr); - _params.src_slot_ids.push_back(4 + i); + _params.src_slot_ids.push_back(5 + i); } // _params.__isset.expr_of_dest_slot = true; _params.__set_dest_tuple_id(0); @@ -367,6 +420,8 @@ TEST_F(BrokerScanNodeTest, normal) { range.file_type = TFileType::FILE_LOCAL; range.format_type = TFileFormatType::FORMAT_CSV_PLAIN; range.splittable = true; + std::map partition_columns = {{"k4", "1"}}; + range.__set_partition_columns(partition_columns); broker_scan_range.ranges.push_back(range); scan_range_params.scan_range.__set_broker_scan_range(broker_scan_range); @@ -386,6 +441,8 @@ TEST_F(BrokerScanNodeTest, normal) { range.file_type = TFileType::FILE_LOCAL; range.format_type = TFileFormatType::FORMAT_CSV_PLAIN; range.splittable = true; + std::map partition_columns = {{"k4", "2"}}; + range.__set_partition_columns(partition_columns); broker_scan_range.ranges.push_back(range); scan_range_params.scan_range.__set_broker_scan_range(broker_scan_range); diff --git a/docs/help/Contents/Data Manipulation/broker_load.md b/docs/help/Contents/Data Manipulation/broker_load.md index 0f0e70c3e3ff46..9666cc1209dc38 100644 --- a/docs/help/Contents/Data Manipulation/broker_load.md +++ b/docs/help/Contents/Data Manipulation/broker_load.md @@ -39,6 +39,7 @@ [PARTITION (p1, p2)] [COLUMNS TERMINATED BY "column_separator"] [FORMAT AS "file_type"] + [BASE_PATH AS "base_path"] [(column_list)] [SET (k1 = func(k2))] @@ -65,6 +66,10 @@ file_type: 用于指定导入文件的类型,例如:parquet、csv。默认值通过文件后缀名判断。 + + base_path: + + 用于指定作为Partition Discovery的基础路径。 column_list: @@ -352,10 +357,7 @@ ) WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); - 8. 导入Parquet文件中数据 指定FORMAT 为parquet, 默认是通过文件后缀判断 - 如果导入路径为目录(base_path),则递归地列出该目录下的所有parquet文件 - 如果需要,则会根据表中定义的字段类型解析base_path目录下的partitioned fields,实现类似Spark中读parquet文件 - + 8. 导入Parquet文件中数据 指定FORMAT 为parquet, 默认是通过文件后缀判断 LOAD LABEL example_db.label9 ( DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/file") @@ -363,19 +365,39 @@ FORMAT AS "parquet" (k1, k2, k3) ) - WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); + WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); - LOAD LABEL example_db.label10 - ( - DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir") - INTO TABLE `my_table` - FORMAT AS "parquet" - (k1, k2, k3) - ) - WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); + 9. 通过Partition Discovery提取文件路径中的压缩字段 + 如果导入路径为目录,则递归地列出该目录下的所有parquet文件 + 如果需要,则会根据表中定义的字段类型解析文件路径中的partitioned fields,实现类似Spark中读parquet文件 + 1. 不指定Partition Discovery的基础路径(BASE_PATH) + LOAD LABEL example_db.label10 + ( + DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir") + INTO TABLE `my_table` + FORMAT AS "parquet" + (k1, k2, k3) + ) + WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); + + hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir目录下包括如下文件:[hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/k1=key1/xxx.parquet, hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/k1=key2/xxx.parquet, ...] + 则会从文件path中提取k1对应的partitioned field的值,并完成数据导入 + + 2. 指定Partition Discovery的基础路径(BASE_PATH) + LOAD LABEL example_db.label11 + ( + DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/city=beijing/utc_date=2019-06-26") + INTO TABLE `my_table` + FORMAT AS "csv" + BASE_PATH AS "hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/" + (k1, k2, k3, utc_date,city) + SET (uniq_id = md5sum(k1, city)) + ) + WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); + + hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/city=beijing/utc_date=2019-06-26目录下包括如下文件:[hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/city=beijing/utc_date=2019-06-26/0000.csv, hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/city=beijing/utc_date=2019-06-26/0001.csv, ...] + 假设CSV文件中仅包括3列(k1, k2和k3),则会根据base_path(hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/)提取文件路径的中的city和utc_date字段 - hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir目录下包括如下文件:[hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/k1=key1/xxx.parquet, hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/k1=key2/xxx.parquet, ...] - 如果my_table中定义了列k1,则会从文件path中提取k1对应的partitioned field的值,并完成数据导入 ## keyword BROKER LOAD diff --git a/docs/help/Contents/Data Manipulation/manipulation_stmt.md b/docs/help/Contents/Data Manipulation/manipulation_stmt.md index cc55e5f25602a9..01087ac64e8723 100644 --- a/docs/help/Contents/Data Manipulation/manipulation_stmt.md +++ b/docs/help/Contents/Data Manipulation/manipulation_stmt.md @@ -48,6 +48,7 @@ [PARTITION (p1, p2)] [COLUMNS TERMINATED BY "column_separator"] [FORMAT AS "file_type"] + [BASE_PATH AS "base_path"] [(column_list)] [SET (k1 = func(k2))] @@ -74,6 +75,10 @@ file_type: 用于指定导入文件的类型,例如:parquet、csv。默认值通过文件后缀名判断。 + + base_path: + + 用于指定作为partition discovery的基础路径。 column_list: @@ -279,6 +284,37 @@ ) WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); + 9. 通过Partition Discovery提取文件路径中的压缩字段 + 如果导入路径为目录,则递归地列出该目录下的所有parquet文件 + 如果需要,则会根据表中定义的字段类型解析文件路径中的partitioned fields,实现类似Spark中读parquet文件 + 1. 不指定Partition Discovery的基础路径(BASE_PATH) + LOAD LABEL example_db.label10 + ( + DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir") + INTO TABLE `my_table` + FORMAT AS "parquet" + (k1, k2, k3) + ) + WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); + + hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir目录下包括如下文件:[hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/k1=key1/xxx.parquet, hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/k1=key2/xxx.parquet, ...] + 则会从文件path中提取k1对应的partitioned field的值,并完成数据导入 + + 2. 指定Partition Discovery的基础路径(BASE_PATH) + LOAD LABEL example_db.label11 + ( + DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/city=beijing/utc_date=2019-06-26") + INTO TABLE `my_table` + FORMAT AS "csv" + BASE_PATH AS "hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/" + (k1, k2, k3, utc_date,city) + SET (uniq_id = md5sum(k1, city)) + ) + WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); + + hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/city=beijing/utc_date=2019-06-26目录下包括如下文件:[hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/city=beijing/utc_date=2019-06-26/0000.csv, hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/city=beijing/utc_date=2019-06-26/0001.csv, ...] + 假设CSV文件中仅包括3列(k1, k2和k3),则会根据base_path(hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/)提取文件路径的中的city和utc_date字段 + ## keyword LOAD diff --git a/fe/src/main/cup/sql_parser.cup b/fe/src/main/cup/sql_parser.cup index 5dbcfe99d66977..bc4ee491d6c303 100644 --- a/fe/src/main/cup/sql_parser.cup +++ b/fe/src/main/cup/sql_parser.cup @@ -191,7 +191,7 @@ parser code {: // Total keywords of doris terminal String KW_ADD, KW_ADMIN, KW_AFTER, KW_AGGREGATE, KW_ALL, KW_ALTER, KW_AND, KW_ANTI, KW_AS, KW_ASC, KW_AUTHORS, - KW_BACKEND, KW_BACKUP, KW_BETWEEN, KW_BEGIN, KW_BIGINT, KW_BOOLEAN, KW_BOTH, KW_BROKER, KW_BACKENDS, KW_BY, + KW_BACKEND, KW_BACKUP, KW_BASE_PATH, KW_BETWEEN, KW_BEGIN, KW_BIGINT, KW_BOOLEAN, KW_BOTH, KW_BROKER, KW_BACKENDS, KW_BY, KW_CANCEL, KW_CASE, KW_CAST, KW_CHAIN, KW_CHAR, KW_CHARSET, KW_CLUSTER, KW_CLUSTERS, KW_COLLATE, KW_COLLATION, KW_COLUMN, KW_COLUMNS, KW_COMMENT, KW_COMMIT, KW_COMMITTED, KW_CONFIG, KW_CONNECTION, KW_CONNECTION_ID, KW_CONSISTENT, KW_COUNT, KW_CREATE, KW_CROSS, KW_CURRENT, KW_CURRENT_USER, @@ -403,7 +403,7 @@ nonterminal List alter_table_clause_list; // nonterminal String keyword, ident, ident_or_text, variable_name, text_or_password, charset_name_or_default, old_or_new_charset_name_or_default, opt_collate, - collation_name_or_default, type_func_name_keyword, type_function_name, opt_file_format; + collation_name_or_default, type_func_name_keyword, type_function_name, opt_file_format, opt_base_path; nonterminal String opt_db, opt_partition_name, procedure_or_function, opt_comment, opt_engine; nonterminal ColumnDef.DefaultValue opt_default_value; @@ -1050,10 +1050,11 @@ data_desc ::= opt_partitions:partitionNames opt_field_term:colSep opt_file_format:fileFormat + opt_base_path:basePath opt_col_list:colList opt_col_mapping_list:colMappingList {: - RESULT = new DataDescription(tableName, partitionNames, files, colList, colSep, fileFormat, isNeg, colMappingList); + RESULT = new DataDescription(tableName, partitionNames, files, colList, colSep, fileFormat, basePath, isNeg, colMappingList); :} ; @@ -1110,6 +1111,13 @@ opt_file_format ::= {: RESULT = format; :} ; +opt_base_path ::= + /* Empty */ + {: RESULT = null; :} + | KW_BASE_PATH KW_AS ident_or_text:basePath + {: RESULT = basePath; :} + ; + opt_col_list ::= {: RESULT = null; @@ -3924,6 +3932,8 @@ keyword ::= {: RESULT = id; :} | KW_FORMAT:id {: RESULT = id; :} + | KW_BASE_PATH:id + {: RESULT = id; :} | KW_FUNCTION:id {: RESULT = id; :} | KW_END:id diff --git a/fe/src/main/java/org/apache/doris/analysis/DataDescription.java b/fe/src/main/java/org/apache/doris/analysis/DataDescription.java index 981ba82567e785..2e200fde2b1df4 100644 --- a/fe/src/main/java/org/apache/doris/analysis/DataDescription.java +++ b/fe/src/main/java/org/apache/doris/analysis/DataDescription.java @@ -60,6 +60,7 @@ public class DataDescription { private final List columnNames; private final ColumnSeparator columnSeparator; private final String fileFormat; + private final String basePath; private final boolean isNegative; private final List columnMappingList; @@ -85,6 +86,7 @@ public DataDescription(String tableName, this.columnNames = columnNames; this.columnSeparator = columnSeparator; this.fileFormat = null; + this.basePath = null; this.isNegative = isNegative; this.columnMappingList = columnMappingList; } @@ -95,6 +97,7 @@ public DataDescription(String tableName, List columnNames, ColumnSeparator columnSeparator, String fileFormat, + String basePath, boolean isNegative, List columnMappingList) { this.tableName = tableName; @@ -103,10 +106,22 @@ public DataDescription(String tableName, this.columnNames = columnNames; this.columnSeparator = columnSeparator; this.fileFormat = fileFormat; + this.basePath = basePath; this.isNegative = isNegative; this.columnMappingList = columnMappingList; } + public DataDescription(String tableName, + List partitionNames, + List filePaths, + List columnNames, + ColumnSeparator columnSeparator, + String fileFormat, + boolean isNegative, + List columnMappingList) { + this(tableName, partitionNames, filePaths, columnNames, columnSeparator, fileFormat, null, isNegative, columnMappingList); + } + public String getTableName() { return tableName; } @@ -127,6 +142,10 @@ public String getFileFormat() { return fileFormat; } + public String getBasePath() { + return basePath; + } + public String getColumnSeparator() { if (columnSeparator == null) { return null; diff --git a/fe/src/main/java/org/apache/doris/common/FeConstants.java b/fe/src/main/java/org/apache/doris/common/FeConstants.java index 084a5b642cc9ab..95683e20ac3693 100644 --- a/fe/src/main/java/org/apache/doris/common/FeConstants.java +++ b/fe/src/main/java/org/apache/doris/common/FeConstants.java @@ -35,5 +35,5 @@ public class FeConstants { // general model // Current meta data version. Use this version to write journals and image - public static int meta_version = FeMetaVersion.VERSION_57; + public static int meta_version = FeMetaVersion.VERSION_58; } diff --git a/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java b/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java index 62acd73006cdd5..05721da5a989b4 100644 --- a/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java +++ b/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java @@ -124,4 +124,6 @@ public final class FeMetaVersion { public static final int VERSION_56 = 56; // for base index using different id public static final int VERSION_57 = 57; + // basePath for partition discovery when loading data using broker scan + public static final int VERSION_58 = 58; } diff --git a/fe/src/main/java/org/apache/doris/load/BrokerFileGroup.java b/fe/src/main/java/org/apache/doris/load/BrokerFileGroup.java index 075134e3cd1f77..3af4a38615434f 100644 --- a/fe/src/main/java/org/apache/doris/load/BrokerFileGroup.java +++ b/fe/src/main/java/org/apache/doris/load/BrokerFileGroup.java @@ -58,6 +58,7 @@ public class BrokerFileGroup implements Writable { private String lineDelimiter; // fileFormat may be null, which means format will be decided by file's suffix private String fileFormat; + private String basePath; private boolean isNegative; private List partitionIds; private List fileFieldNames; @@ -82,6 +83,7 @@ public BrokerFileGroup(BrokerTable table) throws AnalysisException { public BrokerFileGroup(DataDescription dataDescription) { this.dataDescription = dataDescription; + this.basePath = dataDescription.getBasePath(); exprColumnMap = dataDescription.getParsedExprMap(); } @@ -159,6 +161,10 @@ public String getFileFormat() { return fileFormat; } + public String getBasePath() { + return basePath; + } + public boolean isNegative() { return isNegative; } @@ -208,6 +214,7 @@ public String toString() { sb.append(",valueSeparator=").append(valueSeparator) .append(",lineDelimiter=").append(lineDelimiter) .append(",fileFormat=").append(fileFormat) + .append(",basePath=").append(basePath) .append(",isNegative=").append(isNegative); sb.append(",fileInfos=["); int idx = 0; @@ -275,6 +282,13 @@ public void write(DataOutput out) throws IOException { out.writeBoolean(true); Text.writeString(out, fileFormat); } + // basePath + if (basePath == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + Text.writeString(out, basePath); + } } @Override @@ -328,6 +342,12 @@ public void readFields(DataInput in) throws IOException { fileFormat = Text.readString(in); } } + // basePath + if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_58) { + if (in.readBoolean()) { + basePath = Text.readString(in); + } + } } public static BrokerFileGroup read(DataInput in) throws IOException { diff --git a/fe/src/main/java/org/apache/doris/planner/BrokerScanNode.java b/fe/src/main/java/org/apache/doris/planner/BrokerScanNode.java index 1a103dfaf7243c..84d8e02357e244 100644 --- a/fe/src/main/java/org/apache/doris/planner/BrokerScanNode.java +++ b/fe/src/main/java/org/apache/doris/planner/BrokerScanNode.java @@ -589,51 +589,55 @@ private TFileFormatType formatType(String fileFormat, String path) { } } - private Map parsePartitionedFields(String filePath) { - for (BrokerFileGroup fileGroup : fileGroups) { - String fileFormat = fileGroup.getFileFormat(); - if ((fileFormat == null || !fileFormat.toLowerCase().equals("parquet")) || !filePath.endsWith(".parquet")) { - continue; - } + private Map parsePartitionedFields(String filePath, BrokerFileGroup fileGroup) { + String basePath = fileGroup.getBasePath(); + if (basePath == null) { + // find the default base path for (String base : fileGroup.getFilePaths()) { if (base.endsWith("/*")) { base = base.substring(0, base.indexOf("/*")); } - if (!filePath.startsWith(base)) { - continue; + if (filePath.startsWith(base)) { + basePath = base; + break; } - String subPath = filePath.substring(base.length()); - String[] strings = subPath.split("/"); - Map partitionedFields = new HashMap<>(); - for (String str : strings) { - if (str == null || str.isEmpty() || !str.contains("=")) { - continue; - } - String[] pair = str.split("="); - if (pair.length != 2) { - continue; - } - Column column = targetTable.getColumn(pair[0]); - if (column == null) { - continue; - } - partitionedFields.put(pair[0], pair[1]); - } - return partitionedFields; } } - return Collections.emptyMap(); + + if (basePath == null || !filePath.startsWith(basePath)) { + return Collections.emptyMap(); + } + List fileFieldNames = fileGroup.getFileFieldNames(); + String subPath = filePath.substring(basePath.length()); + String[] strings = subPath.split("/"); + Map partitionedFields = new HashMap<>(); + for (String str : strings) { + if (str == null || str.isEmpty() || !str.contains("=")) { + continue; + } + String[] pair = str.split("="); + if (pair.length != 2) { + continue; + } + if (!fileFieldNames.contains(pair[0])) { + continue; + } + partitionedFields.put(pair[0], pair[1]); + } + return partitionedFields; } // If fileFormat is not null, we use fileFormat instead of check file's suffix private void processFileGroup( - String fileFormat, - TBrokerScanRangeParams params, + ParamCreateContext context, List fileStatuses) throws UserException { if (fileStatuses == null || fileStatuses.isEmpty()) { return; } + String fileFormat = context.fileGroup.getFileFormat(); + TBrokerScanRangeParams params = context.params; + BrokerFileGroup fileGroup = context.fileGroup; TScanRangeLocations curLocations = newLocations(params, brokerDesc.getName()); long curInstanceBytes = 0; @@ -647,11 +651,11 @@ private void processFileGroup( // Now only support split plain text if (formatType == TFileFormatType.FORMAT_CSV_PLAIN && fileStatus.isSplitable) { long rangeBytes = bytesPerInstance - curInstanceBytes; - TBrokerRangeDesc rangeDesc = createBrokerRangeDesc(curFileOffset, fileStatus, formatType, rangeBytes); + TBrokerRangeDesc rangeDesc = createBrokerRangeDesc(curFileOffset, fileStatus, formatType, rangeBytes, fileGroup); brokerScanRange(curLocations).addToRanges(rangeDesc); curFileOffset += rangeBytes; } else { - TBrokerRangeDesc rangeDesc = createBrokerRangeDesc(curFileOffset, fileStatus, formatType, leftBytes); + TBrokerRangeDesc rangeDesc = createBrokerRangeDesc(curFileOffset, fileStatus, formatType, leftBytes, fileGroup); brokerScanRange(curLocations).addToRanges(rangeDesc); curFileOffset = 0; i++; @@ -663,7 +667,7 @@ private void processFileGroup( curInstanceBytes = 0; } else { - TBrokerRangeDesc rangeDesc = createBrokerRangeDesc(curFileOffset, fileStatus, formatType, leftBytes); + TBrokerRangeDesc rangeDesc = createBrokerRangeDesc(curFileOffset, fileStatus, formatType, leftBytes, fileGroup); brokerScanRange(curLocations).addToRanges(rangeDesc); curFileOffset = 0; curInstanceBytes += leftBytes; @@ -678,7 +682,7 @@ private void processFileGroup( } private TBrokerRangeDesc createBrokerRangeDesc(long curFileOffset, TBrokerFileStatus fileStatus, - TFileFormatType formatType, long rangeBytes) { + TFileFormatType formatType, long rangeBytes, BrokerFileGroup fileGroup) { TBrokerRangeDesc rangeDesc = new TBrokerRangeDesc(); rangeDesc.setFile_type(TFileType.FILE_BROKER); rangeDesc.setFormat_type(formatType); @@ -687,7 +691,7 @@ private TBrokerRangeDesc createBrokerRangeDesc(long curFileOffset, TBrokerFileSt rangeDesc.setStart_offset(curFileOffset); rangeDesc.setSize(rangeBytes); rangeDesc.setFile_size(fileStatus.size); - rangeDesc.setPartition_columns(parsePartitionedFields(fileStatus.path)); + rangeDesc.setPartition_columns(parsePartitionedFields(fileStatus.path, fileGroup)); return rangeDesc; } @@ -706,7 +710,7 @@ public void finalize(Analyzer analyzer) throws UserException { } catch (AnalysisException e) { throw new UserException(e.getMessage()); } - processFileGroup(context.fileGroup.getFileFormat(), context.params, fileStatuses); + processFileGroup(context, fileStatuses); } if (LOG.isDebugEnabled()) { for (TScanRangeLocations locations : locationsList) { diff --git a/fe/src/main/jflex/sql_scanner.flex b/fe/src/main/jflex/sql_scanner.flex index 07d1fb61999444..feaeb12e0091ac 100644 --- a/fe/src/main/jflex/sql_scanner.flex +++ b/fe/src/main/jflex/sql_scanner.flex @@ -155,6 +155,7 @@ import org.apache.doris.common.util.SqlUtils; keywordMap.put("following", new Integer(SqlParserSymbols.KW_FOLLOWING)); keywordMap.put("for", new Integer(SqlParserSymbols.KW_FOR)); keywordMap.put("format", new Integer(SqlParserSymbols.KW_FORMAT)); + keywordMap.put("base_path", new Integer(SqlParserSymbols.KW_BASE_PATH)); keywordMap.put("from", new Integer(SqlParserSymbols.KW_FROM)); keywordMap.put("frontend", new Integer(SqlParserSymbols.KW_FRONTEND)); keywordMap.put("frontends", new Integer(SqlParserSymbols.KW_FRONTENDS)); From bf8e0811d1d6b160ad66375cf810783c2c529fd9 Mon Sep 17 00:00:00 2001 From: yuanlihan Date: Mon, 5 Aug 2019 00:14:27 +0800 Subject: [PATCH 07/26] Update comment --- gensrc/thrift/PlanNodes.thrift | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gensrc/thrift/PlanNodes.thrift b/gensrc/thrift/PlanNodes.thrift index fd3529fd3a3d9c..bbbfe2a7d9325b 100644 --- a/gensrc/thrift/PlanNodes.thrift +++ b/gensrc/thrift/PlanNodes.thrift @@ -114,7 +114,7 @@ struct TBrokerRangeDesc { 7: optional Types.TUniqueId load_id // total size of the file 8: optional i64 file_size - // partition columns of this file(parquet only) + // partition columns of this file 9: optional map partition_columns } From 93a3577baa27c8ce08d4eb9c4a2d2ee0bb930a8e Mon Sep 17 00:00:00 2001 From: Mingyu Chen Date: Mon, 5 Aug 2019 16:16:43 +0800 Subject: [PATCH 08/26] Support multi partition column when creating table (#1574) When creating table with OLAP engine, use can specify multi parition columns. eg: PARTITION BY RANGE(`date`, `id`) ( PARTITION `p201701_1000` VALUES LESS THAN ("2017-02-01", "1000"), PARTITION `p201702_2000` VALUES LESS THAN ("2017-03-01", "2000"), PARTITION `p201703_all` VALUES LESS THAN ("2017-04-01") ) Notice that load by hadoop cluster does not support multi parition column table. --- be/src/exec/tablet_info.cpp | 47 +++++++++++++++---- be/src/exec/tablet_info.h | 34 +++++++++----- .../cn/getting-started/data-partition.md | 40 +++++++++++++++- .../help/Contents/Data Definition/ddl_stmt.md | 7 +-- .../doris/analysis/CreateTableStmt.java | 4 -- .../doris/analysis/RangePartitionDesc.java | 6 +-- .../apache/doris/catalog/PartitionInfo.java | 6 +++ .../apache/doris/catalog/PartitionKey.java | 18 ++++--- .../doris/catalog/RangePartitionInfo.java | 11 +++-- .../main/java/org/apache/doris/load/Load.java | 27 +++++++---- .../apache/doris/load/PullLoadSourceInfo.java | 1 + .../doris/load/loadv2/BrokerLoadJob.java | 2 +- .../org/apache/doris/load/loadv2/LoadJob.java | 5 +- .../apache/doris/planner/DataSplitSink.java | 4 +- .../apache/doris/planner/OlapTableSink.java | 20 +++++--- .../doris/planner/RangePartitionPruner.java | 3 ++ .../java/org/apache/doris/qe/DdlExecutor.java | 2 +- .../apache/doris/task/PullLoadEtlTask.java | 4 +- .../org/apache/doris/task/PullLoadJob.java | 2 + .../org/apache/doris/task/PullLoadJobMgr.java | 1 + .../doris/task/PullLoadPendingTask.java | 1 + .../org/apache/doris/task/PullLoadTask.java | 1 + .../doris/task/PullLoadTaskPlanner.java | 3 +- .../doris/load/loadv2/BrokerLoadJobTest.java | 2 +- gensrc/thrift/Descriptors.thrift | 7 +++ 25 files changed, 186 insertions(+), 72 deletions(-) diff --git a/be/src/exec/tablet_info.cpp b/be/src/exec/tablet_info.cpp index c4fc5b3b5cc996..2ad37491f705aa 100644 --- a/be/src/exec/tablet_info.cpp +++ b/be/src/exec/tablet_info.cpp @@ -150,7 +150,6 @@ OlapTablePartitionParam::OlapTablePartitionParam( std::shared_ptr schema, const TOlapTablePartitionParam& t_param) : _schema(schema), _t_param(t_param), - _partition_slot_desc(nullptr), _mem_tracker(new MemTracker()), _mem_pool(new MemPool(_mem_tracker.get())) { } @@ -170,11 +169,23 @@ Status OlapTablePartitionParam::init() { ss << "partition column not found, column=" << _t_param.partition_column; return Status::InternalError(ss.str()); } - _partition_slot_desc = it->second; + _partition_slot_descs.push_back(it->second); + } else { + DCHECK(_t_param.__isset.partition_columns); + for (auto& part_col : _t_param.partition_columns) { + auto it = slots_map.find(part_col); + if (it == std::end(slots_map)) { + std::stringstream ss; + ss << "partition column not found, column=" << part_col; + return Status::InternalError(ss.str()); + } + _partition_slot_descs.push_back(it->second); + } } + _partitions_map.reset( new std::map( - OlapTablePartKeyComparator(_partition_slot_desc))); + OlapTablePartKeyComparator(_partition_slot_descs))); if (_t_param.__isset.distributed_columns) { for (auto& col : _t_param.distributed_columns) { auto it = slots_map.find(col); @@ -191,12 +202,22 @@ Status OlapTablePartitionParam::init() { const TOlapTablePartition& t_part = _t_param.partitions[i]; OlapTablePartition* part = _obj_pool.add(new OlapTablePartition()); part->id = t_part.id; + if (t_part.__isset.start_key) { - RETURN_IF_ERROR(_create_partition_key(t_part.start_key, &part->start_key)); + // deprecated, use start_keys instead + std::vector exprs = { t_part.start_key }; + RETURN_IF_ERROR(_create_partition_keys(exprs, &part->start_key)); + } else if (t_part.__isset.start_keys) { + RETURN_IF_ERROR(_create_partition_keys(t_part.start_keys, &part->start_key)); } if (t_part.__isset.end_key) { - RETURN_IF_ERROR(_create_partition_key(t_part.end_key, &part->end_key)); + // deprecated, use end_keys instead + std::vector exprs = { t_part.end_key }; + RETURN_IF_ERROR(_create_partition_keys(exprs, &part->end_key)); + } else if (t_part.__isset.end_keys) { + RETURN_IF_ERROR(_create_partition_keys(t_part.end_keys, &part->end_key)); } + part->num_buckets = t_part.num_buckets; auto num_indexes = _schema->indexes().size(); if (t_part.indexes.size() != num_indexes) { @@ -242,10 +263,19 @@ bool OlapTablePartitionParam::find_tablet(Tuple* tuple, return false; } -Status OlapTablePartitionParam::_create_partition_key(const TExprNode& t_expr, Tuple** part_key) { +Status OlapTablePartitionParam::_create_partition_keys(const std::vector& t_exprs, Tuple** part_key) { Tuple* tuple = (Tuple*)_mem_pool->allocate(_schema->tuple_desc()->byte_size()); - void* slot = tuple->get_slot(_partition_slot_desc->tuple_offset()); - tuple->set_not_null(_partition_slot_desc->null_indicator_offset()); + for (int i = 0; i < t_exprs.size(); i++) { + const TExprNode& t_expr = t_exprs[i]; + RETURN_IF_ERROR(_create_partition_key(t_expr, tuple, _partition_slot_descs[i])); + } + *part_key = tuple; + return Status::OK(); +} + +Status OlapTablePartitionParam::_create_partition_key(const TExprNode& t_expr, Tuple* tuple, SlotDescriptor* slot_desc) { + void* slot = tuple->get_slot(slot_desc->tuple_offset()); + tuple->set_not_null(slot_desc->null_indicator_offset()); switch (t_expr.node_type) { case TExprNodeType::DATE_LITERAL: { if (!reinterpret_cast(slot)->from_date_str( @@ -293,7 +323,6 @@ Status OlapTablePartitionParam::_create_partition_key(const TExprNode& t_expr, T return Status::InternalError(ss.str()); } } - *part_key = tuple; return Status::OK(); } diff --git a/be/src/exec/tablet_info.h b/be/src/exec/tablet_info.h index 6ba2b99e2ec00e..ffef971f8b9dc0 100644 --- a/be/src/exec/tablet_info.h +++ b/be/src/exec/tablet_info.h @@ -104,7 +104,8 @@ struct OlapTablePartition { class OlapTablePartKeyComparator { public: - OlapTablePartKeyComparator(SlotDescriptor* slot_desc) : _slot_desc(slot_desc) { } + OlapTablePartKeyComparator(const std::vector& slot_descs) : + _slot_descs(slot_descs) { } // return true if lhs < rhs // 'nullptr' is max value, but 'null' is min value bool operator()(const Tuple* lhs, const Tuple* rhs) const { @@ -114,16 +115,23 @@ class OlapTablePartKeyComparator { return true; } - bool lhs_null = lhs->is_null(_slot_desc->null_indicator_offset()); - bool rhs_null = rhs->is_null(_slot_desc->null_indicator_offset()); - if (lhs_null || rhs_null) { return !rhs_null; } - - auto lhs_value = lhs->get_slot(_slot_desc->tuple_offset()); - auto rhs_value = rhs->get_slot(_slot_desc->tuple_offset()); - return RawValue::lt(lhs_value, rhs_value, _slot_desc->type()); + for (auto slot_desc : _slot_descs) { + bool lhs_null = lhs->is_null(slot_desc->null_indicator_offset()); + bool rhs_null = rhs->is_null(slot_desc->null_indicator_offset()); + if (lhs_null && rhs_null) { continue; } + if (lhs_null || rhs_null) { return !rhs_null; } + + auto lhs_value = lhs->get_slot(slot_desc->tuple_offset()); + auto rhs_value = rhs->get_slot(slot_desc->tuple_offset()); + + int res = RawValue::compare(lhs_value, rhs_value, slot_desc->type()); + if (res != 0) { return res < 0; } + } + // equal, return false + return false; } private: - SlotDescriptor* _slot_desc; + std::vector _slot_descs; }; // store an olap table's tablet information @@ -150,7 +158,9 @@ class OlapTablePartitionParam { } std::string debug_string() const; private: - Status _create_partition_key(const TExprNode& t_expr, Tuple** part_key); + Status _create_partition_keys(const std::vector& t_exprs, Tuple** part_key); + + Status _create_partition_key(const TExprNode& t_expr, Tuple* tuple, SlotDescriptor* slot_desc); uint32_t _compute_dist_hash(Tuple* key) const; @@ -160,7 +170,7 @@ class OlapTablePartitionParam { // start_key is nullptr means the lower bound is boundless return true; } - OlapTablePartKeyComparator comparator(_partition_slot_desc); + OlapTablePartKeyComparator comparator(_partition_slot_descs); return !comparator(key, part->start_key); } private: @@ -168,7 +178,7 @@ class OlapTablePartitionParam { std::shared_ptr _schema; TOlapTablePartitionParam _t_param; - SlotDescriptor* _partition_slot_desc; + std::vector _partition_slot_descs; std::vector _distributed_slot_descs; ObjectPool _obj_pool; diff --git a/docs/documentation/cn/getting-started/data-partition.md b/docs/documentation/cn/getting-started/data-partition.md index bb4f103cda9891..0224af16257505 100644 --- a/docs/documentation/cn/getting-started/data-partition.md +++ b/docs/documentation/cn/getting-started/data-partition.md @@ -86,7 +86,7 @@ Doris 支持两层的数据划分。第一层是 Partition,仅支持 Range 的 1. Partition - * Partition 列只能指定一列。且必须为 KEY 列。 + * Partition 列可以指定一列或多列。分区类必须为 KEY 列。多列分区的使用方式在后面 **多列分区** 小结介绍。 * Partition 的区间界限是左闭右开。比如如上示例,如果想在 p201702 存储所有2月份的数据,则分区值需要输入 "2017-03-01",即范围为:[2017-02-01, 2017-03-01)。 * 不论分区列是什么类型,在写分区值时,都需要加双引号。 * 分区列通常为时间列,以方便的管理新旧数据。 @@ -156,7 +156,7 @@ Doris 支持两层的数据划分。第一层是 Partition,仅支持 Range 的 * 如果使用了 Partition,则 `DISTRIBUTED ...` 语句描述的是数据在**各个分区内**的划分规则。如果不使用 Partition,则描述的是对整个表的数据的划分规则。 * 分桶列可以是多列,但必须为 Key 列。分桶列可以和 Partition 列相同或不同。 - * 分桶列的选择,是在 **查询吞吐** 和 **查询延迟** 之间的一种权衡: + * 分桶列的选择,是在 **查询吞吐** 和 **查询并发** 之间的一种权衡: 1. 如果选择多个分桶列,则数据分布更均匀。但如果查询条件不包含所有分桶列的等值条件的话,一个查询会扫描所有分桶。这样查询的吞吐会增加,但是单个查询的延迟也会增加。这个方式适合大吞吐低并发的查询场景。 2. 如果仅选择一个或少数分桶列,则点查询可以仅查询一个分桶。这种方式适合高并发的点查询场景。 @@ -174,6 +174,42 @@ Doris 支持两层的数据划分。第一层是 Partition,仅支持 Range 的 * 举一些例子:假设在有10台BE,每台BE一块磁盘的情况下。如果一个表总大小为 500MB,则可以考虑4-8个分片。5GB:8-16个。50GB:32个。500GB:建议分区,每个分区大小在 50GB 左右,每个分区16-32个分片。5TB:建议分区,每个分区大小在 50GB 左右,每个分区16-32个分片。 > 注:表的数据量可以通过 `show data` 命令查看,结果除以副本数,即表的数据量。 + +#### 多列分区 + +Doris 支持指定多列作为分区列,示例如下: + +``` +PARTITION BY RANGE(`date`, `id`) +( + PARTITION `p201701_1000` VALUES LESS THAN ("2017-02-01", "1000"), + PARTITION `p201702_2000` VALUES LESS THAN ("2017-03-01", "2000"), + PARTITION `p201703_all` VALUES LESS THAN ("2017-04-01") +) +``` + +在以上示例中,我们指定 `date`(DATE 类型) 和 `id`(INT 类型) 作为分区列。以上示例最终得到的分区如下: + +``` +* p201701_1000: [(MIN_VALUE, MIN_VALUE), ("2017-02-01", "1000") ) +* p201702_2000: [("2017-02-01", "1000"), ("2017-03-01", "2000") ) +* p201703_all: [("2017-03-01", "2000"), ("2017-04-01", MIN_VALUE)) +``` + +注意,最后一个分区用户缺省只指定了 `date` 列的分区值,所以 `id` 列的分区值会默认填充 `MIN_VALUE`。当用户插入数据时,分区列值会按照顺序依次比较,最终得到对应的分区。举例如下: + +``` +* 数据 --> 分区 +* 2017-01-01, 200 --> p201701_1000 +* 2017-01-01, 2000 --> p201701_1000 +* 2017-02-01, 100 --> p201701_1000 +* 2017-02-01, 2000 --> p201702_2000 +* 2017-02-15, 5000 --> p201702_2000 +* 2017-03-01, 2000 --> p201703_all +* 2017-03-10, 1 --> p201703_all +* 2017-04-01, 1000 --> 无法导入 +* 2017-05-01, 1000 --> 无法导入 +``` ### PROPERTIES diff --git a/docs/help/Contents/Data Definition/ddl_stmt.md b/docs/help/Contents/Data Definition/ddl_stmt.md index e723d24348cc1f..147511f1e329f1 100644 --- a/docs/help/Contents/Data Definition/ddl_stmt.md +++ b/docs/help/Contents/Data Definition/ddl_stmt.md @@ -139,10 +139,10 @@ 4. partition_desc 1) Range 分区 语法: - PARTITION BY RANGE (k1) + PARTITION BY RANGE (k1, k2, ...) ( - PARTITION partition_name VALUES LESS THAN MAXVALUE|("value1") - PARTITION partition_name VALUES LESS THAN MAXVALUE|("value2") + PARTITION partition_name VALUES LESS THAN MAXVALUE|("value1", "value2", ...) + PARTITION partition_name VALUES LESS THAN MAXVALUE|("value1", "value2", ...) ... ) 说明: @@ -152,6 +152,7 @@ TINYINT, SMALLINT, INT, BIGINT, LARGEINT, DATE, DATETIME 3) 分区为左闭右开区间,首个分区的左边界为做最小值 4) NULL 值只会存放在包含最小值的分区中。当包含最小值的分区被删除后,NULL 值将无法导入。 + 5) 可以指定一列或多列作为分区列。如果分区值缺省,则会默认填充最小值。 注意: 1) 分区一般用于时间维度的数据管理 diff --git a/fe/src/main/java/org/apache/doris/analysis/CreateTableStmt.java b/fe/src/main/java/org/apache/doris/analysis/CreateTableStmt.java index 041f33df9a523a..bb43b9b4823748 100644 --- a/fe/src/main/java/org/apache/doris/analysis/CreateTableStmt.java +++ b/fe/src/main/java/org/apache/doris/analysis/CreateTableStmt.java @@ -287,10 +287,6 @@ public void analyze(Analyzer analyzer) throws AnalysisException, UserException { } RangePartitionDesc rangePartitionDesc = (RangePartitionDesc) partitionDesc; - if (rangePartitionDesc.getPartitionColNames().size() != 1) { - throw new AnalysisException("Only allow partitioned by one column"); - } - rangePartitionDesc.analyze(columnDefs, properties); } diff --git a/fe/src/main/java/org/apache/doris/analysis/RangePartitionDesc.java b/fe/src/main/java/org/apache/doris/analysis/RangePartitionDesc.java index 5360f9a12f9a8a..3e539e4da9b970 100644 --- a/fe/src/main/java/org/apache/doris/analysis/RangePartitionDesc.java +++ b/fe/src/main/java/org/apache/doris/analysis/RangePartitionDesc.java @@ -173,9 +173,9 @@ public PartitionInfo toPartitionInfo(List schema, Map part * VALUE LESS THEN (80) * * key range is: - * ( {MIN, MIN, MIN}, {10, 100, 1000} ) - * [ {10, 100, 500}, {50, 500, ? } ) - * [ {50, 500, ? }, {80, ?, ? } ) + * ( {MIN, MIN, MIN}, {10, 100, 1000} ) + * [ {10, 100, 1000}, {50, 500, MIN } ) + * [ {50, 500, MIN }, {80, MIN, MIN } ) */ RangePartitionInfo rangePartitionInfo = new RangePartitionInfo(partitionColumns); for (SingleRangePartitionDesc desc : singleRangePartitionDescs) { diff --git a/fe/src/main/java/org/apache/doris/catalog/PartitionInfo.java b/fe/src/main/java/org/apache/doris/catalog/PartitionInfo.java index 3028ce8ade77e0..b0d98b7e10b420 100644 --- a/fe/src/main/java/org/apache/doris/catalog/PartitionInfo.java +++ b/fe/src/main/java/org/apache/doris/catalog/PartitionInfo.java @@ -38,6 +38,8 @@ public class PartitionInfo implements Writable { protected Map idToDataProperty; // partition id -> replication num protected Map idToReplicationNum; + // true if the partition has multi partition columns + protected boolean isMultiColumnPartition = false; public PartitionInfo() { // for persist @@ -87,6 +89,10 @@ public static PartitionInfo read(DataInput in) throws IOException { return partitionInfo; } + public boolean isMultiColumnPartition() { + return isMultiColumnPartition; + } + public String toSql(OlapTable table, List partitionId) { return ""; } diff --git a/fe/src/main/java/org/apache/doris/catalog/PartitionKey.java b/fe/src/main/java/org/apache/doris/catalog/PartitionKey.java index 0feb5554114aca..f34633ec1cb77c 100644 --- a/fe/src/main/java/org/apache/doris/catalog/PartitionKey.java +++ b/fe/src/main/java/org/apache/doris/catalog/PartitionKey.java @@ -26,6 +26,7 @@ import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; +import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; @@ -75,8 +76,7 @@ public static PartitionKey createPartitionKey(List keys, List co // fill the vacancy with MIN for (; i < columns.size(); ++i) { Type type = Type.fromPrimitiveType(columns.get(i).getDataType()); - partitionKey.keys.add( - LiteralExpr.createInfinity(type, false)); + partitionKey.keys.add(LiteralExpr.createInfinity(type, false)); partitionKey.types.add(columns.get(i).getDataType()); } @@ -127,8 +127,8 @@ public boolean isMaxValue() { return true; } - @Override // compare with other PartitionKey. used for partition prune + @Override public int compareTo(PartitionKey other) { int this_key_len = this.keys.size(); int other_key_len = other.keys.size(); @@ -184,10 +184,10 @@ public String toSql() { value = dateLiteral.toSql(); } } - if (keys.size() - 1 == i) { - strBuilder.append("(").append(value).append(")"); - } else { - strBuilder.append("(").append(value).append("), "); + strBuilder.append(value); + + if (keys.size() - 1 != i) { + strBuilder.append(", "); } i++; } @@ -198,9 +198,7 @@ public String toSql() { public String toString() { StringBuilder builder = new StringBuilder(); builder.append("types: ["); - for (PrimitiveType type : types) { - builder.append(type.toString()); - } + builder.append(Joiner.on(", ").join(types)); builder.append("]; "); builder.append("keys: ["); diff --git a/fe/src/main/java/org/apache/doris/catalog/RangePartitionInfo.java b/fe/src/main/java/org/apache/doris/catalog/RangePartitionInfo.java index 118cb22ea6ef5b..404ed8c84e93dc 100644 --- a/fe/src/main/java/org/apache/doris/catalog/RangePartitionInfo.java +++ b/fe/src/main/java/org/apache/doris/catalog/RangePartitionInfo.java @@ -73,6 +73,7 @@ public RangePartitionInfo(List partitionColumns) { super(PartitionType.RANGE); this.partitionColumns = partitionColumns; this.idToRange = new HashMap>(); + this.isMultiColumnPartition = partitionColumns.size() > 1; } public List getPartitionColumns() { @@ -367,6 +368,8 @@ public void readFields(DataInput in) throws IOException { partitionColumns.add(column); } + this.isMultiColumnPartition = partitionColumns.size() > 1; + counter = in.readInt(); for (int i = 0; i < counter; i++) { long partitionId = in.readLong(); @@ -405,20 +408,20 @@ public String toSql(OlapTable table, List partitionId) { // first partition if (!range.lowerEndpoint().isMinValue()) { sb.append("PARTITION ").append(FeNameFormat.FORBIDDEN_PARTITION_NAME).append(idx) - .append(" VALUES LESS THAN ").append(range.lowerEndpoint().toSql()); + .append(" VALUES LESS THAN (").append(range.lowerEndpoint().toSql()).append(")"); sb.append(",\n"); } } else { Preconditions.checkNotNull(lastRange); if (!lastRange.upperEndpoint().equals(range.lowerEndpoint())) { sb.append("PARTITION ").append(FeNameFormat.FORBIDDEN_PARTITION_NAME).append(idx) - .append(" VALUES LESS THAN ").append(range.lowerEndpoint().toSql()); + .append(" VALUES LESS THAN (").append(range.lowerEndpoint().toSql()).append(")"); sb.append(",\n"); } } - sb.append("PARTITION ").append(partitionName).append(" VALUES LESS THAN "); - sb.append(range.upperEndpoint().toSql()); + sb.append("PARTITION ").append(partitionName).append(" VALUES LESS THAN ("); + sb.append(range.upperEndpoint().toSql()).append(")"); if (partitionId != null) { partitionId.add(entry.getKey()); diff --git a/fe/src/main/java/org/apache/doris/load/Load.java b/fe/src/main/java/org/apache/doris/load/Load.java index 4a290c117b73cd..536e8dd14a4974 100644 --- a/fe/src/main/java/org/apache/doris/load/Load.java +++ b/fe/src/main/java/org/apache/doris/load/Load.java @@ -17,14 +17,6 @@ package org.apache.doris.load; -import com.google.common.base.Joiner; -import com.google.common.base.Preconditions; -import com.google.common.base.Strings; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; -import com.google.gson.Gson; -import org.apache.commons.lang.StringUtils; import org.apache.doris.analysis.BinaryPredicate; import org.apache.doris.analysis.CancelLoadStmt; import org.apache.doris.analysis.ColumnSeparator; @@ -98,6 +90,16 @@ import org.apache.doris.transaction.TransactionState; import org.apache.doris.transaction.TransactionState.LoadJobSourceType; import org.apache.doris.transaction.TransactionStatus; + +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import com.google.gson.Gson; + +import org.apache.commons.lang.StringUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -519,7 +521,7 @@ private LoadJob createLoadJob(LoadStmt stmt, EtlJobType etlJobType, Map>> tableToPartitionSources = Maps.newHashMap(); for (DataDescription dataDescription : dataDescriptions) { // create source - checkAndCreateSource(db, dataDescription, tableToPartitionSources, job.getDeleteFlag()); + checkAndCreateSource(db, dataDescription, tableToPartitionSources, job.getDeleteFlag(), etlJobType); job.addTableName(dataDescription.getTableName()); } for (Entry>> tableEntry : tableToPartitionSources.entrySet()) { @@ -648,7 +650,7 @@ private LoadJob createLoadJob(LoadStmt stmt, EtlJobType etlJobType, public static void checkAndCreateSource(Database db, DataDescription dataDescription, Map>> tableToPartitionSources, - boolean deleteFlag) + boolean deleteFlag, EtlJobType jobType) throws DdlException { Source source = new Source(dataDescription.getFilePaths()); long tableId = -1; @@ -668,6 +670,11 @@ public static void checkAndCreateSource(Database db, DataDescription dataDescrip throw new DdlException("Table [" + tableName + "] is not olap table"); } + if (((OlapTable) table).getPartitionInfo().isMultiColumnPartition() && jobType == EtlJobType.HADOOP) { + throw new DdlException("Load by hadoop cluster does not support table with multi partition columns." + + " Table: " + table.getName() + ". Try using broker load. See 'help broker load;'"); + } + // check partition if (dataDescription.getPartitionNames() != null && !dataDescription.getPartitionNames().isEmpty() && diff --git a/fe/src/main/java/org/apache/doris/load/PullLoadSourceInfo.java b/fe/src/main/java/org/apache/doris/load/PullLoadSourceInfo.java index a211185502ae19..b611467fb0f198 100644 --- a/fe/src/main/java/org/apache/doris/load/PullLoadSourceInfo.java +++ b/fe/src/main/java/org/apache/doris/load/PullLoadSourceInfo.java @@ -34,6 +34,7 @@ /** * PullLoadSourceInfo */ +@Deprecated public class PullLoadSourceInfo implements Writable { private static final Logger LOG = LogManager.getLogger(PullLoadSourceInfo.class); diff --git a/fe/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java b/fe/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java index ca0e8c4b3c58b3..3f7d7044ff90bc 100644 --- a/fe/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java +++ b/fe/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java @@ -101,7 +101,7 @@ public static BrokerLoadJob fromLoadStmt(LoadStmt stmt) throws DdlException { throw new DdlException("Database[" + dbName + "] does not exist"); } // check data source info - LoadJob.checkDataSourceInfo(db, stmt.getDataDescriptions()); + LoadJob.checkDataSourceInfo(db, stmt.getDataDescriptions(), EtlJobType.BROKER); // create job try { diff --git a/fe/src/main/java/org/apache/doris/load/loadv2/LoadJob.java b/fe/src/main/java/org/apache/doris/load/loadv2/LoadJob.java index 0792682d6120a1..41f7f35ab07ef4 100644 --- a/fe/src/main/java/org/apache/doris/load/loadv2/LoadJob.java +++ b/fe/src/main/java/org/apache/doris/load/loadv2/LoadJob.java @@ -259,13 +259,14 @@ protected void setJobProperties(Map properties) throws DdlExcept } } - protected static void checkDataSourceInfo(Database db, List dataDescriptions) throws DdlException { + protected static void checkDataSourceInfo(Database db, List dataDescriptions, + EtlJobType jobType) throws DdlException { for (DataDescription dataDescription : dataDescriptions) { // loadInfo is a temporary param for the method of checkAndCreateSource. // >> Map>> loadInfo = Maps.newHashMap(); // only support broker load now - Load.checkAndCreateSource(db, dataDescription, loadInfo, false); + Load.checkAndCreateSource(db, dataDescription, loadInfo, false, jobType); } } diff --git a/fe/src/main/java/org/apache/doris/planner/DataSplitSink.java b/fe/src/main/java/org/apache/doris/planner/DataSplitSink.java index e817a28e7d3368..8e8f25ba8f0def 100644 --- a/fe/src/main/java/org/apache/doris/planner/DataSplitSink.java +++ b/fe/src/main/java/org/apache/doris/planner/DataSplitSink.java @@ -31,8 +31,8 @@ import org.apache.doris.catalog.PartitionKey; import org.apache.doris.catalog.PartitionType; import org.apache.doris.catalog.PrimitiveType; -import org.apache.doris.catalog.Type; import org.apache.doris.catalog.RangePartitionInfo; +import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.ErrorCode; import org.apache.doris.thrift.TAggregationType; @@ -51,6 +51,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Range; + import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -59,6 +60,7 @@ import java.util.Set; // This class used to split data read from file to batch +@Deprecated public class DataSplitSink extends DataSink { private static final Logger LOG = LogManager.getLogger(Planner.class); diff --git a/fe/src/main/java/org/apache/doris/planner/OlapTableSink.java b/fe/src/main/java/org/apache/doris/planner/OlapTableSink.java index aa42f3652507a1..f69754afe6c9e5 100644 --- a/fe/src/main/java/org/apache/doris/planner/OlapTableSink.java +++ b/fe/src/main/java/org/apache/doris/planner/OlapTableSink.java @@ -217,12 +217,11 @@ private TOlapTablePartitionParam createPartition(long dbId, OlapTable table) thr switch (partType) { case RANGE: { RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) table.getPartitionInfo(); - // range partition's only has one column - Preconditions.checkArgument(rangePartitionInfo.getPartitionColumns().size() == 1, - "number columns of range partition is not 1, number_columns=" - + rangePartitionInfo.getPartitionColumns().size()); - partitionParam.setPartition_column(rangePartitionInfo.getPartitionColumns().get(0).getName()); + for (Column partCol : rangePartitionInfo.getPartitionColumns()) { + partitionParam.addToPartition_columns(partCol.getName()); + } + int partColNum = rangePartitionInfo.getPartitionColumns().size(); DistributionInfo selectedDistInfo = null; for (Partition partition : table.getPartitions()) { if (partitionSet != null && !partitionSet.contains(partition.getName())) { @@ -231,12 +230,19 @@ private TOlapTablePartitionParam createPartition(long dbId, OlapTable table) thr TOlapTablePartition tPartition = new TOlapTablePartition(); tPartition.setId(partition.getId()); Range range = rangePartitionInfo.getRange(partition.getId()); + // set start keys if (range.hasLowerBound() && !range.lowerEndpoint().isMinValue()) { - tPartition.setStart_key(range.lowerEndpoint().getKeys().get(0).treeToThrift().getNodes().get(0)); + for (int i = 0; i < partColNum; i++) { + tPartition.addToStart_keys(range.lowerEndpoint().getKeys().get(i).treeToThrift().getNodes().get(0)); + } } + // set end keys if (range.hasUpperBound() && !range.upperEndpoint().isMaxValue()) { - tPartition.setEnd_key(range.upperEndpoint().getKeys().get(0).treeToThrift().getNodes().get(0)); + for (int i = 0; i < partColNum; i++) { + tPartition.addToEnd_keys(range.upperEndpoint().getKeys().get(i).treeToThrift().getNodes().get(0)); + } } + for (MaterializedIndex index : partition.getMaterializedIndices()) { tPartition.addToIndexes(new TOlapTableIndexTablets(index.getId(), Lists.newArrayList( index.getTablets().stream().map(Tablet::getId).collect(Collectors.toList())))); diff --git a/fe/src/main/java/org/apache/doris/planner/RangePartitionPruner.java b/fe/src/main/java/org/apache/doris/planner/RangePartitionPruner.java index 245f1e6f0e75ad..d91ae0886860d6 100644 --- a/fe/src/main/java/org/apache/doris/planner/RangePartitionPruner.java +++ b/fe/src/main/java/org/apache/doris/planner/RangePartitionPruner.java @@ -94,6 +94,8 @@ private Collection prune(RangeMap rangeMap, if (filter.lowerBoundInclusive && filter.upperBoundInclusive && filter.lowerBound != null && filter.upperBound != null && 0 == filter.lowerBound.compareLiteral(filter.upperBound)) { + + // eg: [10, 10], [null, null] if (filter.lowerBound instanceof NullLiteral && filter.upperBound instanceof NullLiteral) { // replace Null with min value LiteralExpr minKeyValue = LiteralExpr.createInfinity( @@ -109,6 +111,7 @@ private Collection prune(RangeMap rangeMap, maxKey.popColumn(); return result; } + // no in predicate BoundType lowerType = filter.lowerBoundInclusive ? BoundType.CLOSED : BoundType.OPEN; BoundType upperType = filter.upperBoundInclusive ? BoundType.CLOSED : BoundType.OPEN; diff --git a/fe/src/main/java/org/apache/doris/qe/DdlExecutor.java b/fe/src/main/java/org/apache/doris/qe/DdlExecutor.java index e1faaf089e20ed..9ad683fcf933a0 100644 --- a/fe/src/main/java/org/apache/doris/qe/DdlExecutor.java +++ b/fe/src/main/java/org/apache/doris/qe/DdlExecutor.java @@ -111,7 +111,7 @@ public static void execute(Catalog catalog, DdlStmt ddlStmt, String origStmt) th } else { if (Config.disable_hadoop_load) { throw new DdlException("Load job by hadoop cluster is disabled." - + " Try use broker load. See 'help broker load;'"); + + " Try using broker load. See 'help broker load;'"); } jobType = EtlJobType.HADOOP; } diff --git a/fe/src/main/java/org/apache/doris/task/PullLoadEtlTask.java b/fe/src/main/java/org/apache/doris/task/PullLoadEtlTask.java index 76d817bbb0560b..24648c61f7bd18 100644 --- a/fe/src/main/java/org/apache/doris/task/PullLoadEtlTask.java +++ b/fe/src/main/java/org/apache/doris/task/PullLoadEtlTask.java @@ -26,11 +26,13 @@ import com.google.common.collect.Maps; -import java.util.Map; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.util.Map; + // Used to process pull load etl task +@Deprecated public class PullLoadEtlTask extends LoadEtlTask { private static final Logger LOG = LogManager.getLogger(PullLoadEtlTask.class); private PullLoadJobMgr mgr; diff --git a/fe/src/main/java/org/apache/doris/task/PullLoadJob.java b/fe/src/main/java/org/apache/doris/task/PullLoadJob.java index e77a77706dec32..fa2e520a9e4cc8 100644 --- a/fe/src/main/java/org/apache/doris/task/PullLoadJob.java +++ b/fe/src/main/java/org/apache/doris/task/PullLoadJob.java @@ -20,6 +20,7 @@ import org.apache.doris.load.LoadJob; import com.google.common.collect.Sets; + import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -27,6 +28,7 @@ import java.util.Set; // One pull load job +@Deprecated public class PullLoadJob { private static final Logger LOG = LogManager.getLogger(PullLoadTask.class); diff --git a/fe/src/main/java/org/apache/doris/task/PullLoadJobMgr.java b/fe/src/main/java/org/apache/doris/task/PullLoadJobMgr.java index f3e4865ef95b3a..385007b2e9a60a 100644 --- a/fe/src/main/java/org/apache/doris/task/PullLoadJobMgr.java +++ b/fe/src/main/java/org/apache/doris/task/PullLoadJobMgr.java @@ -33,6 +33,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.locks.ReentrantLock; +@Deprecated public class PullLoadJobMgr { private static final Logger LOG = LogManager.getLogger(PullLoadJobMgr.class); diff --git a/fe/src/main/java/org/apache/doris/task/PullLoadPendingTask.java b/fe/src/main/java/org/apache/doris/task/PullLoadPendingTask.java index f820169d46d5c0..66532127f4e50b 100644 --- a/fe/src/main/java/org/apache/doris/task/PullLoadPendingTask.java +++ b/fe/src/main/java/org/apache/doris/task/PullLoadPendingTask.java @@ -40,6 +40,7 @@ import java.util.Map; // Making a pull load job to some tasks +@Deprecated public class PullLoadPendingTask extends LoadPendingTask { private static final Logger LOG = LogManager.getLogger(PullLoadPendingTask.class); diff --git a/fe/src/main/java/org/apache/doris/task/PullLoadTask.java b/fe/src/main/java/org/apache/doris/task/PullLoadTask.java index e312e5af0fda4c..3ac0ebf96dd3aa 100644 --- a/fe/src/main/java/org/apache/doris/task/PullLoadTask.java +++ b/fe/src/main/java/org/apache/doris/task/PullLoadTask.java @@ -42,6 +42,7 @@ import java.util.UUID; // A pull load task is used to process one table of this pull load job. +@Deprecated public class PullLoadTask { private static final Logger LOG = LogManager.getLogger(PullLoadTask.class); // Input parameter diff --git a/fe/src/main/java/org/apache/doris/task/PullLoadTaskPlanner.java b/fe/src/main/java/org/apache/doris/task/PullLoadTaskPlanner.java index d07b6a197c7b3f..c9b65736815b4b 100644 --- a/fe/src/main/java/org/apache/doris/task/PullLoadTaskPlanner.java +++ b/fe/src/main/java/org/apache/doris/task/PullLoadTaskPlanner.java @@ -27,8 +27,8 @@ import org.apache.doris.catalog.Column; import org.apache.doris.catalog.OlapTable; import org.apache.doris.common.AnalysisException; -import org.apache.doris.common.UserException; import org.apache.doris.common.NotImplementedException; +import org.apache.doris.common.UserException; import org.apache.doris.planner.BrokerScanNode; import org.apache.doris.planner.DataPartition; import org.apache.doris.planner.DataSplitSink; @@ -49,6 +49,7 @@ import java.util.List; // Planner used to generate a plan for pull load ETL work +@Deprecated public class PullLoadTaskPlanner { private static final Logger LOG = LogManager.getLogger(PullLoadTaskPlanner.class); diff --git a/fe/src/test/java/org/apache/doris/load/loadv2/BrokerLoadJobTest.java b/fe/src/test/java/org/apache/doris/load/loadv2/BrokerLoadJobTest.java index d99cd8685035a2..e114aef9fb506e 100644 --- a/fe/src/test/java/org/apache/doris/load/loadv2/BrokerLoadJobTest.java +++ b/fe/src/test/java/org/apache/doris/load/loadv2/BrokerLoadJobTest.java @@ -141,7 +141,7 @@ public void testFromLoadStmt(@Injectable LoadStmt loadStmt, @Mock public void checkAndCreateSource(Database db, DataDescription dataDescription, Map>> tableToPartitionSources, - boolean deleteFlag) { + boolean deleteFlag, EtlJobType jobType) { } }; diff --git a/gensrc/thrift/Descriptors.thrift b/gensrc/thrift/Descriptors.thrift index b8ffcb035bd91a..2d10dbd8775657 100644 --- a/gensrc/thrift/Descriptors.thrift +++ b/gensrc/thrift/Descriptors.thrift @@ -118,6 +118,7 @@ struct TOlapTableIndexTablets { // its a closed-open range struct TOlapTablePartition { 1: required i64 id + // deprecated, use 'start_keys' and 'end_keys' instead 2: optional Exprs.TExprNode start_key 3: optional Exprs.TExprNode end_key @@ -125,6 +126,9 @@ struct TOlapTablePartition { 4: required i32 num_buckets 5: required list indexes + + 6: optional list start_keys + 7: optional list end_keys } struct TOlapTablePartitionParam { @@ -133,6 +137,7 @@ struct TOlapTablePartitionParam { 3: required i64 version // used to split a logical table to multiple paritions + // deprecated, use 'partition_columns' instead 4: optional string partition_column // used to split a partition to multiple tablets @@ -140,6 +145,8 @@ struct TOlapTablePartitionParam { // partitions 6: required list partitions + + 7: optional list partition_columns } struct TOlapTableIndexSchema { From eda55a7394fcec2f7b6c0aefd1628f9d63911815 Mon Sep 17 00:00:00 2001 From: Mingyu Chen Date: Mon, 5 Aug 2019 16:19:05 +0800 Subject: [PATCH 09/26] Fix bug that unable to delete replica if version is missing (#1585) If there is a redundant replica on BE which version is missing, the tablet report logic can not drop it correctly. --- fe/src/main/java/org/apache/doris/master/ReportHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/src/main/java/org/apache/doris/master/ReportHandler.java b/fe/src/main/java/org/apache/doris/master/ReportHandler.java index 4a70d9fc45762a..b2e85ceacf20be 100644 --- a/fe/src/main/java/org/apache/doris/master/ReportHandler.java +++ b/fe/src/main/java/org/apache/doris/master/ReportHandler.java @@ -631,7 +631,7 @@ private static void deleteFromBackend(Map backendTablets, for (Long tabletId : backendTablets.keySet()) { TTablet backendTablet = backendTablets.get(tabletId); for (TTabletInfo backendTabletInfo : backendTablet.getTablet_infos()) { - boolean needDelete = false; + boolean needDelete = true; if (!foundTabletsWithValidSchema.contains(tabletId) && isBackendReplicaHealthy(backendTabletInfo)) { // if this tablet is not in meta. try adding it. From d938f9a6ead36d966275a067c292fc77dd27e053 Mon Sep 17 00:00:00 2001 From: Dayue Gao Date: Tue, 6 Aug 2019 10:40:16 +0800 Subject: [PATCH 10/26] Implement the initial version of BetaRowset (#1568) --- be/src/olap/olap_define.h | 2 +- be/src/olap/rowset/CMakeLists.txt | 2 + be/src/olap/rowset/alpha_rowset.cpp | 59 ++-------- be/src/olap/rowset/alpha_rowset.h | 28 ++--- be/src/olap/rowset/alpha_rowset_writer.cpp | 5 +- be/src/olap/rowset/beta_rowset.cpp | 125 +++++++++++++++++++++ be/src/olap/rowset/beta_rowset.h | 69 ++++++++++++ be/src/olap/rowset/rowset.cpp | 54 +++++++++ be/src/olap/rowset/rowset.h | 69 ++++++++---- be/src/olap/rowset/rowset_meta.h | 50 +++++---- be/src/olap/storage_engine.cpp | 2 +- be/src/olap/txn_manager.cpp | 2 +- gensrc/proto/olap_file.proto | 43 ++++--- 13 files changed, 378 insertions(+), 132 deletions(-) create mode 100644 be/src/olap/rowset/beta_rowset.cpp create mode 100644 be/src/olap/rowset/beta_rowset.h create mode 100644 be/src/olap/rowset/rowset.cpp diff --git a/be/src/olap/olap_define.h b/be/src/olap/olap_define.h index c8b8c62ef7f829..225ff1d20bd54c 100644 --- a/be/src/olap/olap_define.h +++ b/be/src/olap/olap_define.h @@ -351,7 +351,7 @@ enum OLAPStatus { OLAP_ERR_ROWSET_WRITER_INIT = -3100, OLAP_ERR_ROWSET_SAVE_FAILED = -3101, OLAP_ERR_ROWSET_GENERATE_ID_FAILED = -3102, - OLAP_ERR_ROWSET_DELETE_SEGMENT_GROUP_FILE_FAILED = -3103, + OLAP_ERR_ROWSET_DELETE_FILE_FAILED = -3103, OLAP_ERR_ROWSET_BUILDER_INIT = -3104, OLAP_ERR_ROWSET_TYPE_NOT_FOUND = -3105, OLAP_ERR_ROWSET_ALREADY_EXIST = -3106, diff --git a/be/src/olap/rowset/CMakeLists.txt b/be/src/olap/rowset/CMakeLists.txt index 494ad09118eedb..ca2acfa0df5f36 100644 --- a/be/src/olap/rowset/CMakeLists.txt +++ b/be/src/olap/rowset/CMakeLists.txt @@ -35,11 +35,13 @@ add_library(Rowset STATIC run_length_integer_writer.cpp segment_reader.cpp segment_writer.cpp + rowset.cpp rowset_id_generator.cpp rowset_meta_manager.cpp alpha_rowset.cpp alpha_rowset_reader.cpp alpha_rowset_writer.cpp alpha_rowset_meta.cpp + beta_rowset.cpp rowset_id_generator.cpp ) diff --git a/be/src/olap/rowset/alpha_rowset.cpp b/be/src/olap/rowset/alpha_rowset.cpp index 35e4a10564c529..bee85c5cf99120 100644 --- a/be/src/olap/rowset/alpha_rowset.cpp +++ b/be/src/olap/rowset/alpha_rowset.cpp @@ -24,26 +24,10 @@ namespace doris { AlphaRowset::AlphaRowset(const TabletSchema* schema, - const std::string rowset_path, + std::string rowset_path, DataDir* data_dir, RowsetMetaSharedPtr rowset_meta) - : _schema(schema), - _rowset_path(rowset_path), - _data_dir(data_dir), - _rowset_meta(rowset_meta), - _is_cumulative_rowset(false), - _is_pending_rowset(false) { - if (!_rowset_meta->has_version()) { - _is_pending_rowset = true; - } - if (!_is_pending_rowset) { - Version version = _rowset_meta->version(); - if (version.first == version.second) { - _is_cumulative_rowset = false; - } else { - _is_cumulative_rowset = true; - } - } + : Rowset(schema, std::move(rowset_path), data_dir, std::move(rowset_meta)) { } OLAPStatus AlphaRowset::init() { @@ -103,40 +87,26 @@ std::shared_ptr AlphaRowset::create_reader() { } OLAPStatus AlphaRowset::remove() { - LOG(INFO) << "begin to remove rowset: " << rowset_id(); + LOG(INFO) << "begin to remove files in rowset " << unique_id(); for (auto segment_group : _segment_groups) { bool ret = segment_group->delete_all_files(); if (!ret) { LOG(WARNING) << "delete segment group files failed." << " tablet id:" << segment_group->get_tablet_id() << ", rowset path:" << segment_group->rowset_path_prefix(); - return OLAP_ERR_ROWSET_DELETE_SEGMENT_GROUP_FILE_FAILED; + return OLAP_ERR_ROWSET_DELETE_FILE_FAILED; } } return OLAP_SUCCESS; } -RowsetMetaSharedPtr AlphaRowset::rowset_meta() const { - return _rowset_meta; -} - -void AlphaRowset::set_version_and_version_hash(Version version, VersionHash version_hash) { - _rowset_meta->set_version(version); - _rowset_meta->set_version_hash(version_hash); - // set the rowset state to VISIBLE - _rowset_meta->set_rowset_state(VISIBLE); - - if (rowset_meta()->has_delete_predicate()) { - rowset_meta()->mutable_delete_predicate()->set_version(version.first); - return; - } - +void AlphaRowset::make_visible_extra(Version version, VersionHash version_hash) { AlphaRowsetMetaSharedPtr alpha_rowset_meta = std::dynamic_pointer_cast(_rowset_meta); vector published_segment_groups; alpha_rowset_meta->get_segment_groups(&published_segment_groups); int32_t segment_group_idx = 0; - for (auto segment_group : _segment_groups) { + for (auto& segment_group : _segment_groups) { segment_group->set_version(version); segment_group->set_version_hash(version_hash); segment_group->set_pending_finished(); @@ -147,8 +117,6 @@ void AlphaRowset::set_version_and_version_hash(Version version, VersionHash ver for (auto& segment_group_meta : published_segment_groups) { alpha_rowset_meta->add_segment_group(segment_group_meta); } - - _is_pending_rowset = false; } OLAPStatus AlphaRowset::make_snapshot(const std::string& snapshot_path, @@ -216,10 +184,6 @@ OLAPStatus AlphaRowset::remove_old_files(std::vector* files_to_remo return OLAP_SUCCESS; } -bool AlphaRowset::is_pending() const { - return _is_pending_rowset; -} - OLAPStatus AlphaRowset::split_range( const RowCursor& start_key, const RowCursor& end_key, @@ -320,7 +284,7 @@ OLAPStatus AlphaRowset::split_range( bool AlphaRowset::check_path(const std::string& path) { std::set valid_paths; - for (auto segment_group : _segment_groups) { + for (auto& segment_group : _segment_groups) { for (int i = 0; i < segment_group->num_segments(); ++i) { std::string data_path = segment_group->construct_data_file_path(i); std::string index_path = segment_group->construct_index_file_path(i); @@ -337,7 +301,7 @@ OLAPStatus AlphaRowset::_init_segment_groups() { _alpha_rowset_meta->get_segment_groups(&segment_group_metas); for (auto& segment_group_meta : segment_group_metas) { std::shared_ptr segment_group; - if (_is_pending_rowset) { + if (_is_pending) { segment_group.reset(new SegmentGroup(_rowset_meta->tablet_id(), _rowset_meta->rowset_id(), _schema, _rowset_path, false, segment_group_meta.segment_group_id(), segment_group_meta.num_segments(), true, @@ -385,7 +349,7 @@ OLAPStatus AlphaRowset::_init_segment_groups() { } _segment_groups.push_back(segment_group); } - if (_is_cumulative_rowset && _segment_groups.size() > 1) { + if (_is_cumulative && _segment_groups.size() > 1) { LOG(WARNING) << "invalid segment group meta for cumulative rowset. segment group size:" << _segment_groups.size(); return OLAP_ERR_ENGINE_LOAD_INDEX_TABLE_ERROR; @@ -435,9 +399,4 @@ OLAPStatus AlphaRowset::reset_sizeinfo() { return OLAP_SUCCESS; } -std::string AlphaRowset::unique_id() { - // rowset path + rowset_id is unique for a rowset - return _rowset_path + "/" + std::to_string(rowset_id()); -} - } // namespace doris diff --git a/be/src/olap/rowset/alpha_rowset.h b/be/src/olap/rowset/alpha_rowset.h index beaa8e8b15cf49..36d6d894d345c8 100644 --- a/be/src/olap/rowset/alpha_rowset.h +++ b/be/src/olap/rowset/alpha_rowset.h @@ -36,8 +36,11 @@ using AlphaRowsetSharedPtr = std::shared_ptr; class AlphaRowset : public Rowset { public: - AlphaRowset(const TabletSchema* schema, const std::string rowset_path, - DataDir* data_dir, RowsetMetaSharedPtr rowset_meta); + AlphaRowset(const TabletSchema* schema, + std::string rowset_path, + DataDir* data_dir, + RowsetMetaSharedPtr rowset_meta); + virtual ~AlphaRowset() {} static bool is_valid_rowset_path(std::string path); @@ -52,10 +55,6 @@ class AlphaRowset : public Rowset { OLAPStatus remove() override; - RowsetMetaSharedPtr rowset_meta() const override; - - void set_version_and_version_hash(Version version, VersionHash version_hash) override; - OLAPStatus make_snapshot(const std::string& snapshot_path, std::vector* success_links) override; OLAPStatus copy_files_to_path(const std::string& dest_path, @@ -69,8 +68,6 @@ class AlphaRowset : public Rowset { OLAPStatus remove_old_files(std::vector* files_to_remove) override; - bool is_pending() const override; - OLAPStatus split_range( const RowCursor& start_key, const RowCursor& end_key, @@ -83,11 +80,9 @@ class AlphaRowset : public Rowset { // info by using segment's info OLAPStatus reset_sizeinfo(); - std::string unique_id() override; - - const std::string& rowset_path() const { - return _rowset_path; - } +protected: + // add custom logic when rowset is published + void make_visible_extra(Version version, VersionHash version_hash) override; private: OLAPStatus _init_segment_groups(); @@ -97,13 +92,8 @@ class AlphaRowset : public Rowset { private: friend class AlphaRowsetWriter; friend class AlphaRowsetReader; - const TabletSchema* _schema; - std::string _rowset_path; - DataDir* _data_dir; - RowsetMetaSharedPtr _rowset_meta; + std::vector> _segment_groups; - bool _is_cumulative_rowset; - bool _is_pending_rowset; }; } // namespace doris diff --git a/be/src/olap/rowset/alpha_rowset_writer.cpp b/be/src/olap/rowset/alpha_rowset_writer.cpp index 393498fc19c817..102b9d9bdecb42 100644 --- a/be/src/olap/rowset/alpha_rowset_writer.cpp +++ b/be/src/olap/rowset/alpha_rowset_writer.cpp @@ -152,6 +152,7 @@ RowsetSharedPtr AlphaRowsetWriter::build() { LOG(WARNING) << "invalid writer state before build, state:" << _writer_state; return nullptr; } + int total_num_segments = 0; for (auto& segment_group : _segment_groups) { if (segment_group->load() != OLAP_SUCCESS) { return nullptr; @@ -166,6 +167,7 @@ RowsetSharedPtr AlphaRowsetWriter::build() { SegmentGroupPB segment_group_pb; segment_group_pb.set_segment_group_id(segment_group->segment_group_id()); segment_group_pb.set_num_segments(segment_group->num_segments()); + total_num_segments += segment_group->num_segments(); segment_group_pb.set_index_size(segment_group->index_size()); segment_group_pb.set_data_size(segment_group->data_size()); segment_group_pb.set_num_rows(segment_group->num_rows()); @@ -188,6 +190,7 @@ RowsetSharedPtr AlphaRowsetWriter::build() { = std::dynamic_pointer_cast(_current_rowset_meta); alpha_rowset_meta->add_segment_group(segment_group_pb); } + _current_rowset_meta->set_num_segments(total_num_segments); if (_is_pending_rowset) { _current_rowset_meta->set_rowset_state(COMMITTED); } else { @@ -242,7 +245,7 @@ OLAPStatus AlphaRowsetWriter::garbage_collection() { LOG(WARNING) << "delete segment group files failed." << " tablet id:" << segment_group->get_tablet_id() << ", rowset path:" << segment_group->rowset_path_prefix(); - return OLAP_ERR_ROWSET_DELETE_SEGMENT_GROUP_FILE_FAILED; + return OLAP_ERR_ROWSET_DELETE_FILE_FAILED; } } return OLAP_SUCCESS; diff --git a/be/src/olap/rowset/beta_rowset.cpp b/be/src/olap/rowset/beta_rowset.cpp new file mode 100644 index 00000000000000..b7b793f59a5417 --- /dev/null +++ b/be/src/olap/rowset/beta_rowset.cpp @@ -0,0 +1,125 @@ +// 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. + +#include "olap/rowset/beta_rowset.h" + +#include +#include // for remove() +#include // for link() +#include "gutil/strings/substitute.h" +#include "olap/utils.h" + +namespace doris { + +BetaRowset::BetaRowset(const TabletSchema* schema, + string rowset_path, + DataDir* data_dir, + RowsetMetaSharedPtr rowset_meta) + : Rowset(schema, std::move(rowset_path), data_dir, std::move(rowset_meta)) { +} + +OLAPStatus BetaRowset::init() { + if (is_inited()) { + return OLAP_SUCCESS; + } + // TODO init segment readers + set_inited(true); + return OLAP_SUCCESS; +} + +OLAPStatus BetaRowset::load(bool use_cache) { + // TODO load segment footers + return OLAP_SUCCESS; +} + +std::shared_ptr BetaRowset::create_reader() { + // TODO return BetaRowsetReader or RowwiseIterator? + return nullptr; +} + +OLAPStatus BetaRowset::remove() { + // TODO should we close and remove all segment reader first? + // TODO should we rename the method to remove_files() to be more specific? + LOG(INFO) << "begin to remove files in rowset " << unique_id(); + bool success = true; + for (int i = 0; i < num_segments(); ++i) { + std::string path = _segment_file_path(_rowset_path, i); + LOG(INFO) << "deleting " << path; + if (::remove(path.c_str()) != 0) { + char errmsg[64]; + LOG(WARNING) << "failed to delete file. err=" << strerror_r(errno, errmsg, 64) + << ", path=" << path; + success = false; + } + } + if (!success) { + LOG(WARNING) << "failed to remove files in rowset " << unique_id(); + return OLAP_ERR_ROWSET_DELETE_FILE_FAILED; + } + return OLAP_SUCCESS; +} + +OLAPStatus BetaRowset::make_snapshot(const std::string& snapshot_path, std::vector* success_links) { + // TODO should we rename this method to `hard_link_files_to` to be more general? + for (int i = 0; i < num_segments(); ++i) { + std::string dst_link_path = _segment_file_path(snapshot_path, i); + if (check_dir_existed(dst_link_path)) { + LOG(WARNING) << "failed to make snapshot, file already exist: " << dst_link_path; + return OLAP_ERR_FILE_ALREADY_EXIST; + } + std::string src_file_path = _segment_file_path(_rowset_path, i); + if (link(src_file_path.c_str(), dst_link_path.c_str()) != 0) { + LOG(WARNING) << "fail to create hard link. from=" << src_file_path << ", " + << "to=" << dst_link_path << ", " << "errno=" << Errno::no(); + return OLAP_ERR_OS_ERROR; + } + success_links->push_back(dst_link_path); + } + return OLAP_SUCCESS; +} + +OLAPStatus BetaRowset::copy_files_to_path(const std::string& dest_path, std::vector* success_files) { + for (int i = 0; i < num_segments(); ++i) { + std::string dst_path = _segment_file_path(dest_path, i); + if (check_dir_existed(dst_path)) { + LOG(WARNING) << "file already exist: " << dst_path; + return OLAP_ERR_FILE_ALREADY_EXIST; + } + std::string src_path = _segment_file_path(_rowset_path, i); + if (copy_file(src_path, dst_path) != OLAP_SUCCESS) { + LOG(WARNING) << "fail to copy file. from=" << src_path << ", to=" << dst_path + << ", errno=" << Errno::no(); + return OLAP_ERR_OS_ERROR; + } + success_files->push_back(dst_path); + } + return OLAP_SUCCESS; +} + +bool BetaRowset::check_path(const std::string& path) { + std::set valid_paths; + for (int i = 0; i < num_segments(); ++i) { + valid_paths.insert(_segment_file_path(_rowset_path, i)); + } + return valid_paths.find(path) != valid_paths.end(); +} + +std::string BetaRowset::_segment_file_path(const std::string& dir, int segment_id) { + return strings::Substitute("$0/$1_$2.dat", dir, rowset_id(), segment_id); +} + +} // namespace doris \ No newline at end of file diff --git a/be/src/olap/rowset/beta_rowset.h b/be/src/olap/rowset/beta_rowset.h new file mode 100644 index 00000000000000..6acfaeb130a340 --- /dev/null +++ b/be/src/olap/rowset/beta_rowset.h @@ -0,0 +1,69 @@ +// 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. + +#ifndef DORIS_SRC_OLAP_ROWSET_BETA_ROWSET_H_ +#define DORIS_SRC_OLAP_ROWSET_BETA_ROWSET_H_ + +#include "olap/olap_define.h" +#include "olap/rowset/rowset.h" +#include "olap/rowset/rowset_meta.h" +#include "olap/data_dir.h" + +namespace doris { + +class BetaRowset; +using BetaRowsetSharedPtr = std::shared_ptr; + +class BetaRowset : public Rowset { +public: + BetaRowset(const TabletSchema* schema, + std::string rowset_path, + DataDir* data_dir, + RowsetMetaSharedPtr rowset_meta); + + virtual ~BetaRowset() {} + + OLAPStatus init() override; + + OLAPStatus load(bool use_cache = true) override; + + std::shared_ptr create_reader() override; + + OLAPStatus remove() override; + + OLAPStatus make_snapshot(const std::string& snapshot_path, + std::vector* success_links) override; + + OLAPStatus copy_files_to_path(const std::string& dest_path, + std::vector* success_files) override; + + // only applicable to alpha rowset, no op here + OLAPStatus remove_old_files(std::vector* files_to_remove) override { + return OLAP_SUCCESS; + }; + + bool check_path(const std::string& path) override; + +private: + std::string _segment_file_path(const std::string& segment_dir, int segment_id); + + // TODO segment readers member +}; + +} // namespace doris + +#endif //DORIS_SRC_OLAP_ROWSET_BETA_ROWSET_H_ diff --git a/be/src/olap/rowset/rowset.cpp b/be/src/olap/rowset/rowset.cpp new file mode 100644 index 00000000000000..f9d3faa73a53f8 --- /dev/null +++ b/be/src/olap/rowset/rowset.cpp @@ -0,0 +1,54 @@ +// 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. + +#include "olap/rowset/rowset.h" + +namespace doris { + +Rowset::Rowset(const TabletSchema *schema, + std::string rowset_path, + DataDir *data_dir, + RowsetMetaSharedPtr rowset_meta) + : _schema(schema), + _rowset_path(std::move(rowset_path)), + _data_dir(data_dir), + _rowset_meta(std::move(rowset_meta)) { + + _is_pending = !_rowset_meta->has_version(); + if (_is_pending) { + _is_cumulative = false; + } else { + Version version = _rowset_meta->version(); + _is_cumulative = version.first != version.second; + } +} + +void Rowset::make_visible(Version version, VersionHash version_hash) { + _is_pending = false; + _rowset_meta->set_version(version); + _rowset_meta->set_version_hash(version_hash); + _rowset_meta->set_rowset_state(VISIBLE); + + if (_rowset_meta->has_delete_predicate()) { + _rowset_meta->mutable_delete_predicate()->set_version(version.first); + return; + } + make_visible_extra(version, version_hash); +} + +} // namespace doris + diff --git a/be/src/olap/rowset/rowset.h b/be/src/olap/rowset/rowset.h index 07eac354ef2ff8..6f58ae6d4121ac 100644 --- a/be/src/olap/rowset/rowset.h +++ b/be/src/olap/rowset/rowset.h @@ -26,31 +26,33 @@ namespace doris { +class DataDir; class Rowset; using RowsetSharedPtr = std::shared_ptr; - -class RowsetWriter; class RowsetReader; +class TabletSchema; class Rowset : public std::enable_shared_from_this { public: - Rowset() : _is_inited(false), _is_loaded(false), _need_delete_file(false) { - } + Rowset(const TabletSchema* schema, + std::string rowset_path, + DataDir* data_dir, + RowsetMetaSharedPtr rowset_meta); virtual ~Rowset() { } // this api is for init related objects in memory virtual OLAPStatus init() = 0; - virtual bool is_inited() { + bool is_inited() const { return _is_inited; } - virtual void set_inited(bool inited) { + void set_inited(bool inited) { _is_inited = inited; } - virtual bool is_loaded() { + bool is_loaded() const { return _is_loaded; } @@ -58,6 +60,17 @@ class Rowset : public std::enable_shared_from_this { _is_loaded= loaded; } + RowsetMetaSharedPtr rowset_meta() const { + return _rowset_meta; + } + + bool is_pending() const { + return _is_pending; + } + + // publish rowset to make it visible to read + void make_visible(Version version, VersionHash version_hash); + // helper class to access RowsetMeta int64_t start_version() const { return rowset_meta()->version().first; } int64_t end_version() const { return rowset_meta()->version().second; } @@ -75,6 +88,7 @@ class Rowset : public std::enable_shared_from_this { int64_t partition_id() const { return rowset_meta()->partition_id(); } // flag for push delete rowset bool delete_flag() const { return rowset_meta()->delete_flag(); } + int64_t num_segments() const { return rowset_meta()->num_segments(); } void to_rowset_pb(RowsetMetaPB* rs_meta) { return rowset_meta()->to_rowset_pb(rs_meta); } // this api is for lazy loading data @@ -83,12 +97,9 @@ class Rowset : public std::enable_shared_from_this { virtual std::shared_ptr create_reader() = 0; + // remove all files in this rowset virtual OLAPStatus remove() = 0; - virtual RowsetMetaSharedPtr rowset_meta() const = 0; - - virtual void set_version_and_version_hash(Version version, VersionHash version_hash) = 0; - virtual OLAPStatus make_snapshot(const std::string& snapshot_path, std::vector* success_links) = 0; virtual OLAPStatus copy_files_to_path(const std::string& dest_path, @@ -96,27 +107,37 @@ class Rowset : public std::enable_shared_from_this { virtual OLAPStatus remove_old_files(std::vector* files_to_remove) = 0; - virtual bool is_pending() const = 0; - + // return whether `path` is one of the files in this rowset virtual bool check_path(const std::string& path) = 0; - virtual std::string unique_id() = 0; + // return an unique identifier string for this rowset + std::string unique_id() const { + return _rowset_path + "/" + std::to_string(rowset_id()); + } - bool need_delete_file() { + bool need_delete_file() const { return _need_delete_file; } - void set_need_delete_file(bool need_delete_file) { - if (_need_delete_file == true) { - return; - } - _need_delete_file = need_delete_file; + void set_need_delete_file() { + _need_delete_file = true; } -private: - bool _is_inited; - bool _is_loaded; - bool _need_delete_file; +protected: + // allow subclass to add custom logic when rowset is being published + virtual void make_visible_extra(Version version, VersionHash version_hash) {} + + const TabletSchema* _schema; + std::string _rowset_path; + DataDir* _data_dir; + RowsetMetaSharedPtr _rowset_meta; + // init in constructor + bool _is_pending; // rowset is pending iff it's not in visible state + bool _is_cumulative; // rowset is cumulative iff it's visible and start version < end version + + bool _is_inited = false; + bool _is_loaded = false; + bool _need_delete_file = false; }; } // namespace doris diff --git a/be/src/olap/rowset/rowset_meta.h b/be/src/olap/rowset/rowset_meta.h index e10a2476fb2d91..d8a4b84227d3f7 100644 --- a/be/src/olap/rowset/rowset_meta.h +++ b/be/src/olap/rowset/rowset_meta.h @@ -71,7 +71,7 @@ class RowsetMeta { return ret; } - int64_t rowset_id() { + int64_t rowset_id() const { return _rowset_meta_pb.rowset_id(); } @@ -79,7 +79,7 @@ class RowsetMeta { _rowset_meta_pb.set_rowset_id(rowset_id); } - int64_t tablet_id() { + int64_t tablet_id() const { return _rowset_meta_pb.tablet_id(); } @@ -87,7 +87,7 @@ class RowsetMeta { _rowset_meta_pb.set_tablet_id(tablet_id); } - TabletUid tablet_uid() { + TabletUid tablet_uid() const { return _rowset_meta_pb.tablet_uid(); } @@ -95,7 +95,7 @@ class RowsetMeta { *(_rowset_meta_pb.mutable_tablet_uid()) = tablet_uid.to_proto(); } - int64_t txn_id() { + int64_t txn_id() const { return _rowset_meta_pb.txn_id(); } @@ -103,7 +103,7 @@ class RowsetMeta { _rowset_meta_pb.set_txn_id(txn_id); } - int32_t tablet_schema_hash() { + int32_t tablet_schema_hash() const { return _rowset_meta_pb.tablet_schema_hash(); } @@ -111,7 +111,7 @@ class RowsetMeta { _rowset_meta_pb.set_tablet_schema_hash(tablet_schema_hash); } - RowsetTypePB rowset_type() { + RowsetTypePB rowset_type() const { return _rowset_meta_pb.rowset_type(); } @@ -119,7 +119,7 @@ class RowsetMeta { _rowset_meta_pb.set_rowset_type(rowset_type); } - RowsetStatePB rowset_state() { + RowsetStatePB rowset_state() const { return _rowset_meta_pb.rowset_state(); } @@ -127,7 +127,7 @@ class RowsetMeta { _rowset_meta_pb.set_rowset_state(rowset_state); } - Version version() { + Version version() const { return { _rowset_meta_pb.start_version(), _rowset_meta_pb.end_version() }; } @@ -137,7 +137,7 @@ class RowsetMeta { _rowset_meta_pb.set_end_version(version.second); } - bool has_version() { + bool has_version() const { return _rowset_meta_pb.has_start_version() && _rowset_meta_pb.has_end_version(); } @@ -158,7 +158,7 @@ class RowsetMeta { _rowset_meta_pb.set_end_version(end_version); } - VersionHash version_hash() { + VersionHash version_hash() const { return _rowset_meta_pb.version_hash(); } @@ -166,7 +166,7 @@ class RowsetMeta { _rowset_meta_pb.set_version_hash(version_hash); } - int64_t num_rows() { + int64_t num_rows() const { return _rowset_meta_pb.num_rows(); } @@ -174,7 +174,7 @@ class RowsetMeta { _rowset_meta_pb.set_num_rows(num_rows); } - size_t total_disk_size() { + size_t total_disk_size() const { return _rowset_meta_pb.total_disk_size(); } @@ -182,7 +182,7 @@ class RowsetMeta { _rowset_meta_pb.set_total_disk_size(total_disk_size); } - size_t data_disk_size() { + size_t data_disk_size() const { return _rowset_meta_pb.data_disk_size(); } @@ -190,7 +190,7 @@ class RowsetMeta { _rowset_meta_pb.set_data_disk_size(data_disk_size); } - size_t index_disk_size() { + size_t index_disk_size() const { return _rowset_meta_pb.index_disk_size(); } @@ -216,11 +216,11 @@ class RowsetMeta { *new_zone_map = zone_map; } - bool has_delete_predicate() { + bool has_delete_predicate() const { return _rowset_meta_pb.has_delete_predicate(); } - const DeletePredicatePB& delete_predicate() { + const DeletePredicatePB& delete_predicate() const { return _rowset_meta_pb.delete_predicate(); } @@ -233,7 +233,7 @@ class RowsetMeta { *new_delete_condition = delete_predicate; } - bool empty() { + bool empty() const { return _rowset_meta_pb.empty(); } @@ -241,7 +241,7 @@ class RowsetMeta { _rowset_meta_pb.set_empty(empty); } - PUniqueId load_id() { + PUniqueId load_id() const { return _rowset_meta_pb.load_id(); } @@ -251,7 +251,7 @@ class RowsetMeta { new_load_id->set_lo(load_id.lo()); } - bool delete_flag() { + bool delete_flag() const { return _rowset_meta_pb.delete_flag(); } @@ -267,7 +267,7 @@ class RowsetMeta { return _rowset_meta_pb.set_creation_time(creation_time); } - int64_t partition_id() { + int64_t partition_id() const { return _rowset_meta_pb.partition_id(); } @@ -275,7 +275,15 @@ class RowsetMeta { return _rowset_meta_pb.set_partition_id(partition_id); } - void to_rowset_pb(RowsetMetaPB* rs_meta_pb) { + int64_t num_segments() const { + return _rowset_meta_pb.num_segments(); + } + + void set_num_segments(int64_t num_segments) { + _rowset_meta_pb.set_num_segments(num_segments); + } + + void to_rowset_pb(RowsetMetaPB* rs_meta_pb) const { *rs_meta_pb = _rowset_meta_pb; } diff --git a/be/src/olap/storage_engine.cpp b/be/src/olap/storage_engine.cpp index c0e74945743619..8877299346ac57 100644 --- a/be/src/olap/storage_engine.cpp +++ b/be/src/olap/storage_engine.cpp @@ -767,7 +767,7 @@ void StorageEngine::add_unused_rowset(RowsetSharedPtr rowset) { << ", unique id:" << rowset->unique_id(); auto it = _unused_rowsets.find(rowset->unique_id()); if (it == _unused_rowsets.end()) { - rowset->set_need_delete_file(true); + rowset->set_need_delete_file(); _unused_rowsets[rowset->unique_id()] = rowset; } _gc_mutex.unlock(); diff --git a/be/src/olap/txn_manager.cpp b/be/src/olap/txn_manager.cpp index 80dc877749c995..a856510c797a43 100755 --- a/be/src/olap/txn_manager.cpp +++ b/be/src/olap/txn_manager.cpp @@ -232,7 +232,7 @@ OLAPStatus TxnManager::publish_txn(OlapMeta* meta, TPartitionId partition_id, TT if (rowset_ptr != nullptr) { // TODO(ygl): rowset is already set version here, memory is changed, if save failed // it maybe a fatal error - rowset_ptr->set_version_and_version_hash(version, version_hash); + rowset_ptr->make_visible(version, version_hash); OLAPStatus save_status = RowsetMetaManager::save(meta, tablet_uid, rowset_ptr->rowset_id(), rowset_ptr->rowset_meta().get()); diff --git a/gensrc/proto/olap_file.proto b/gensrc/proto/olap_file.proto index 96002b13980ba4..7dfc27ffd5b21c 100644 --- a/gensrc/proto/olap_file.proto +++ b/gensrc/proto/olap_file.proto @@ -76,25 +76,40 @@ message RowsetMetaPB { required int64 rowset_id = 1; optional int64 partition_id = 2; optional int64 tablet_id = 3; + // only for pending rowset optional int64 txn_id = 4; - optional int32 tablet_schema_hash = 5; // tablet id and schema hash could find tablet - optional RowsetTypePB rowset_type = 6; // alpha + // tablet id and schema hash could find tablet + optional int32 tablet_schema_hash = 5; + optional RowsetTypePB rowset_type = 6; optional RowsetStatePB rowset_state = 7; - optional int64 start_version = 8; // PDelta.start_version - optional int64 end_version = 9; // PDelta.end_version - optional int64 version_hash = 10; // PDelta.version_hash - optional int64 num_rows = 11; // calculated from segment group - optional int64 total_disk_size = 12; // calculated by index + data - optional int64 data_disk_size = 13; // calculated sum(segmentgroup.data_size) - optional int64 index_disk_size = 14; // calculated sum(segmentgroup.index_size) - // column min/max/null flag statistic info - repeated ZoneMap zone_maps = 15; // not set, not used currently + // only for visible rowset. For alpha rowset, it's the same as PDelta.start_version + optional int64 start_version = 8; + // only for visible rowset. For alpha rowset, it's the same as PDelta.end_version + optional int64 end_version = 9; + // only for visible rowset. For alpha rowset, it's the same as PDelta.version_hash + optional int64 version_hash = 10; + // total number of rows. For alpha rowset, it's calculated from segment group + optional int64 num_rows = 11; + // calculated by index + data + optional int64 total_disk_size = 12; + // calculated sum(segmentgroup.data_size) + optional int64 data_disk_size = 13; + // calculated sum(segmentgroup.index_size) + optional int64 index_disk_size = 14; + // rowset level column min/max/null statistics, not not used currently + repeated ZoneMap zone_maps = 15; optional DeletePredicatePB delete_predicate = 16; - optional bool empty = 17; // calculated from segment group + // calculated from segment group + optional bool empty = 17; + // only for pending rowset optional PUniqueId load_id = 18; - optional bool delete_flag = 19; // not set, get from olap index header msg? - optional int64 creation_time = 20; // PDelta.creation_time + // not set, get from olap index header msg? + optional bool delete_flag = 19; + // For alpha rowset, it's the same as PDelta.creation_time + optional int64 creation_time = 20; optional PUniqueId tablet_uid = 21; + // total number of segments + optional int64 num_segments = 22; // spare field id for future use optional AlphaRowsetExtraMetaPB alpha_rowset_extra_meta_pb = 50; } From ec7b9e421f0a2b57d945b956c655c4050d34cbfc Mon Sep 17 00:00:00 2001 From: yiguolei Date: Tue, 6 Aug 2019 17:14:39 +0800 Subject: [PATCH 11/26] Acquire tablet map write lock during tablet gc (#1588) --- be/src/olap/tablet_manager.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/be/src/olap/tablet_manager.cpp b/be/src/olap/tablet_manager.cpp index 01f1738df4f260..833541556659e1 100755 --- a/be/src/olap/tablet_manager.cpp +++ b/be/src/olap/tablet_manager.cpp @@ -1012,7 +1012,9 @@ OLAPStatus TabletManager::start_trash_sweep() { do { sleep(1); clean_num = 0; - ReadLock rlock(&_tablet_map_lock); + // should get write lock here, because it will remove tablet from shut_down_tablets + // and get tablet will access shut_down_tablets + WriteLock wlock(&_tablet_map_lock); auto it = _shutdown_tablets.begin(); for (; it != _shutdown_tablets.end();) { // check if the meta has the tablet info and its state is shutdown From b2e678dfc13ddc080903a0369aaaa32da89605f6 Mon Sep 17 00:00:00 2001 From: ZHAO Chun Date: Tue, 6 Aug 2019 17:15:11 +0800 Subject: [PATCH 12/26] Support Segment for BetaRowset (#1577) We create a new segment format for BetaRowset. New format merge data file and index file into one file. And we create a new format for short key index. In origin code index is stored in format like RowCusor which is not efficient to compare. Now we encode multiple column into binary, and we assure that this binary is sorted same with the key columns. --- be/src/gutil/endian.h | 153 ++++++++++ be/src/olap/CMakeLists.txt | 5 + be/src/olap/column_mapping.h | 4 +- be/src/olap/delete_handler.cpp | 1 + be/src/olap/delete_handler.h | 6 +- be/src/olap/delta_writer.cpp | 1 + be/src/olap/delta_writer.h | 3 +- be/src/olap/field.h | 14 + be/src/olap/iterators.h | 19 ++ be/src/olap/key_coder.cpp | 80 +++++ be/src/olap/key_coder.h | 224 ++++++++++++++ be/src/olap/rowset/rowset_reader.h | 1 + be/src/olap/rowset/rowset_reader_context.h | 10 +- be/src/olap/rowset/rowset_writer.h | 4 +- be/src/olap/rowset/segment_v2/column_writer.h | 12 + be/src/olap/rowset/segment_v2/segment.cpp | 183 +++++++++++ be/src/olap/rowset/segment_v2/segment.h | 120 ++++++++ .../rowset/segment_v2/segment_iterator.cpp | 258 ++++++++++++++++ .../olap/rowset/segment_v2/segment_iterator.h | 88 ++++++ .../olap/rowset/segment_v2/segment_writer.cpp | 172 +++++++++++ .../olap/rowset/segment_v2/segment_writer.h | 88 ++++++ be/src/olap/schema_change.cpp | 45 +++ be/src/olap/schema_change.h | 46 --- be/src/olap/short_key_index.cpp | 117 +++++++ be/src/olap/short_key_index.h | 274 +++++++++++++++++ be/src/olap/storage_engine.h | 2 - be/src/olap/tablet.h | 1 - be/src/olap/types.h | 10 + be/src/olap/uint24.h | 13 +- be/src/util/CMakeLists.txt | 1 + be/src/util/slice.cpp | 28 ++ be/src/util/slice.h | 4 + be/test/olap/CMakeLists.txt | 3 + be/test/olap/key_coder_test.cpp | 287 ++++++++++++++++++ .../olap/rowset/segment_v2/segment_test.cpp | 259 ++++++++++++++++ be/test/olap/short_key_index_test.cpp | 161 ++++++++++ be/test/olap/tablet_schema_helper.h | 54 ++++ gensrc/proto/segment_v2.proto | 49 ++- run-ut.sh | 3 + 39 files changed, 2732 insertions(+), 71 deletions(-) create mode 100644 be/src/olap/key_coder.cpp create mode 100644 be/src/olap/key_coder.h create mode 100644 be/src/olap/rowset/segment_v2/segment.cpp create mode 100644 be/src/olap/rowset/segment_v2/segment.h create mode 100644 be/src/olap/rowset/segment_v2/segment_iterator.cpp create mode 100644 be/src/olap/rowset/segment_v2/segment_iterator.h create mode 100644 be/src/olap/rowset/segment_v2/segment_writer.cpp create mode 100644 be/src/olap/rowset/segment_v2/segment_writer.h create mode 100644 be/src/olap/short_key_index.cpp create mode 100644 be/src/olap/short_key_index.h create mode 100644 be/src/util/slice.cpp create mode 100644 be/test/olap/key_coder_test.cpp create mode 100644 be/test/olap/rowset/segment_v2/segment_test.cpp create mode 100644 be/test/olap/short_key_index_test.cpp create mode 100644 be/test/olap/tablet_schema_helper.h diff --git a/be/src/gutil/endian.h b/be/src/gutil/endian.h index f6b2485b6d8bab..6b8a0bc772d414 100644 --- a/be/src/gutil/endian.h +++ b/be/src/gutil/endian.h @@ -32,6 +32,18 @@ inline uint64 gbswap_64(uint64 host_int) { #endif // bswap_64 } +inline unsigned __int128 gbswap_128(unsigned __int128 host_int) { + return static_cast(bswap_64(static_cast(host_int >> 64))) | + (static_cast(bswap_64(static_cast(host_int))) << 64); +} + +// Swap bytes of a 24-bit value. +inline uint32_t bswap_24(uint32_t x) { + return ((x & 0x0000ffULL) << 16) | + ((x & 0x00ff00ULL)) | + ((x & 0xff0000ULL) >> 16); +} + #ifdef IS_LITTLE_ENDIAN // Definitions for ntohl etc. that don't require us to include @@ -188,4 +200,145 @@ class LittleEndian { #define gntohll(x) ghtonll(x) #define ntohll(x) htonll(x) +// Utilities to convert numbers between the current hosts's native byte +// order and big-endian byte order (same as network byte order) +// +// Load/Store methods are alignment safe +class BigEndian { +public: +#ifdef IS_LITTLE_ENDIAN + + static uint16 FromHost16(uint16 x) { return bswap_16(x); } + static uint16 ToHost16(uint16 x) { return bswap_16(x); } + + static uint32 FromHost24(uint32 x) { return bswap_24(x); } + static uint32 ToHost24(uint32 x) { return bswap_24(x); } + + static uint32 FromHost32(uint32 x) { return bswap_32(x); } + static uint32 ToHost32(uint32 x) { return bswap_32(x); } + + static uint64 FromHost64(uint64 x) { return gbswap_64(x); } + static uint64 ToHost64(uint64 x) { return gbswap_64(x); } + + static unsigned __int128 FromHost128(unsigned __int128 x) { return gbswap_128(x); } + static unsigned __int128 ToHost128(unsigned __int128 x) { return gbswap_128(x); } + + static bool IsLittleEndian() { return true; } + +#elif defined IS_BIG_ENDIAN + + static uint16 FromHost16(uint16 x) { return x; } + static uint16 ToHost16(uint16 x) { return x; } + + static uint32 FromHost24(uint32 x) { return x; } + static uint32 ToHost24(uint32 x) { return x; } + + static uint32 FromHost32(uint32 x) { return x; } + static uint32 ToHost32(uint32 x) { return x; } + + static uint64 FromHost64(uint64 x) { return x; } + static uint64 ToHost64(uint64 x) { return x; } + + static uint128 FromHost128(uint128 x) { return x; } + static uint128 ToHost128(uint128 x) { return x; } + + static bool IsLittleEndian() { return false; } + +#endif /* ENDIAN */ + // Functions to do unaligned loads and stores in little-endian order. + static uint16 Load16(const void *p) { + return ToHost16(UNALIGNED_LOAD16(p)); + } + + static void Store16(void *p, uint16 v) { + UNALIGNED_STORE16(p, FromHost16(v)); + } + + static uint32 Load32(const void *p) { + return ToHost32(UNALIGNED_LOAD32(p)); + } + + static void Store32(void *p, uint32 v) { + UNALIGNED_STORE32(p, FromHost32(v)); + } + + static uint64 Load64(const void *p) { + return ToHost64(UNALIGNED_LOAD64(p)); + } + + // Build a uint64 from 1-8 bytes. + // 8 * len least significant bits are loaded from the memory with + // BigEndian order. The 64 - 8 * len most significant bits are + // set all to 0. + // In latex-friendly words, this function returns: + // $\sum_{i=0}^{len-1} p[i] 256^{i}$, where p[i] is unsigned. + // + // This function is equivalent with: + // uint64 val = 0; + // memcpy(&val, p, len); + // return ToHost64(val); + // TODO(user): write a small benchmark and benchmark the speed + // of a memcpy based approach. + // + // For speed reasons this function does not work for len == 0. + // The caller needs to guarantee that 1 <= len <= 8. + static uint64 Load64VariableLength(const void * const p, int len) { + assert(len >= 1 && len <= 8); + uint64 val = Load64(p); + uint64 mask = 0; + --len; + do { + mask = (mask << 8) | 0xff; + // (--len >= 0) is about 10 % faster than (len--) in some benchmarks. + } while (--len >= 0); + return val & mask; + } + + static void Store64(void *p, uint64 v) { + UNALIGNED_STORE64(p, FromHost64(v)); + } + + static uint128 Load128(const void *p) { + return uint128( + ToHost64(UNALIGNED_LOAD64(p)), + ToHost64(UNALIGNED_LOAD64(reinterpret_cast(p) + 1))); + } + + static void Store128(void *p, const uint128 v) { + UNALIGNED_STORE64(p, FromHost64(Uint128High64(v))); + UNALIGNED_STORE64(reinterpret_cast(p) + 1, + FromHost64(Uint128Low64(v))); + } + + // Build a uint128 from 1-16 bytes. + // 8 * len least significant bits are loaded from the memory with + // BigEndian order. The 128 - 8 * len most significant bits are + // set all to 0. + static uint128 Load128VariableLength(const void *p, int len) { + if (len <= 8) { + return uint128(Load64VariableLength(static_cast(p)+8, + len)); + } else { + return uint128( + Load64VariableLength(p, len-8), + Load64(static_cast(p)+8)); + } + } + + // Load & Store in machine's word size. + static uword_t LoadUnsignedWord(const void *p) { + if (sizeof(uword_t) == 8) + return Load64(p); + else + return Load32(p); + } + + static void StoreUnsignedWord(void *p, uword_t v) { + if (sizeof(uword_t) == 8) + Store64(p, v); + else + Store32(p, v); + } +}; // BigEndian + #endif // UTIL_ENDIAN_ENDIAN_H_ diff --git a/be/src/olap/CMakeLists.txt b/be/src/olap/CMakeLists.txt index c494c8d0f6c6a6..1790f9effa7f98 100644 --- a/be/src/olap/CMakeLists.txt +++ b/be/src/olap/CMakeLists.txt @@ -41,6 +41,7 @@ add_library(Olap STATIC hll.cpp in_list_predicate.cpp in_stream.cpp + key_coder.cpp lru_cache.cpp memtable.cpp merger.cpp @@ -63,6 +64,7 @@ add_library(Olap STATIC serialize.cpp storage_engine.cpp data_dir.cpp + short_key_index.cpp snapshot_manager.cpp stream_index_common.cpp stream_index_reader.cpp @@ -84,6 +86,9 @@ add_library(Olap STATIC rowset/segment_v2/encoding_info.cpp rowset/segment_v2/ordinal_page_index.cpp rowset/segment_v2/binary_dict_page.cpp + rowset/segment_v2/segment.cpp + rowset/segment_v2/segment_iterator.cpp + rowset/segment_v2/segment_writer.cpp rowset_factory.cpp task/engine_batch_load_task.cpp task/engine_checksum_task.cpp diff --git a/be/src/olap/column_mapping.h b/be/src/olap/column_mapping.h index 1cebe4eab3e544..7b19d9c5f111ec 100644 --- a/be/src/olap/column_mapping.h +++ b/be/src/olap/column_mapping.h @@ -18,10 +18,10 @@ #ifndef DORIS_BE_SRC_OLAP_COLUMN_MAPPING_H #define DORIS_BE_SRC_OLAP_COLUMN_MAPPING_H -#include "olap/wrapper_field.h" - namespace doris { +class WrapperField; + struct ColumnMapping { ColumnMapping() : ref_column(-1), default_value(nullptr) {} virtual ~ColumnMapping() {} diff --git a/be/src/olap/delete_handler.cpp b/be/src/olap/delete_handler.cpp index 70e7146247ecff..b6c36d64f6227e 100644 --- a/be/src/olap/delete_handler.cpp +++ b/be/src/olap/delete_handler.cpp @@ -29,6 +29,7 @@ #include "gen_cpp/olap_file.pb.h" #include "olap/olap_common.h" #include "olap/utils.h" +#include "olap/olap_cond.h" using apache::thrift::ThriftDebugString; using std::numeric_limits; diff --git a/be/src/olap/delete_handler.h b/be/src/olap/delete_handler.h index 9e0f7689a34736..0c23a005ab91f6 100644 --- a/be/src/olap/delete_handler.h +++ b/be/src/olap/delete_handler.h @@ -23,14 +23,14 @@ #include "gen_cpp/AgentService_types.h" #include "gen_cpp/olap_file.pb.h" -#include "olap/field.h" -#include "olap/olap_cond.h" #include "olap/olap_define.h" -#include "olap/row_cursor.h" +#include "olap/tablet_schema.h" namespace doris { typedef google::protobuf::RepeatedPtrField DelPredicateArray; +class Conditions; +class RowCursor; class DeleteConditionHandler { public: diff --git a/be/src/olap/delta_writer.cpp b/be/src/olap/delta_writer.cpp index d593aef5921a54..d3f63478c4e967 100644 --- a/be/src/olap/delta_writer.cpp +++ b/be/src/olap/delta_writer.cpp @@ -18,6 +18,7 @@ #include "olap/delta_writer.h" #include "olap/schema.h" +#include "olap/memtable.h" #include "olap/data_dir.h" #include "olap/rowset/alpha_rowset_writer.h" #include "olap/rowset/rowset_meta_manager.h" diff --git a/be/src/olap/delta_writer.h b/be/src/olap/delta_writer.h index 16db33d4aab312..c909147b075100 100644 --- a/be/src/olap/delta_writer.h +++ b/be/src/olap/delta_writer.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_SRC_DELTA_WRITER_H #define DORIS_BE_SRC_DELTA_WRITER_H -#include "olap/memtable.h" #include "olap/storage_engine.h" #include "olap/tablet.h" #include "olap/schema_change.h" @@ -30,6 +29,8 @@ namespace doris { class SegmentGroup; +class MemTable; +class Schema; enum WriteType { LOAD = 1, diff --git a/be/src/olap/field.h b/be/src/olap/field.h index d6eef2a7753075..460c7bd936277e 100644 --- a/be/src/olap/field.h +++ b/be/src/olap/field.h @@ -26,6 +26,7 @@ #include "olap/olap_define.h" #include "olap/tablet_schema.h" #include "olap/types.h" +#include "olap/key_coder.h" #include "olap/utils.h" #include "olap/row_cursor_cell.h" #include "runtime/mem_pool.h" @@ -57,12 +58,14 @@ class Field { Field(const TabletColumn& column) : _type_info(get_type_info(column.type())), _agg_info(get_aggregate_info(column.aggregation(), column.type())), + _key_coder(get_key_coder(column.type())), _index_size(column.index_length()), _is_nullable(column.is_nullable()) { } Field(FieldType type) : _type_info(get_type_info(type)), _agg_info(get_aggregate_info(OLAP_FIELD_AGGREGATION_NONE, type)), + _key_coder(get_key_coder(type)), _index_size(_type_info->size()), _is_nullable(true) { } @@ -70,6 +73,7 @@ class Field { Field(const FieldAggregationMethod& agg, const FieldType& type, bool is_nullable) : _type_info(get_type_info(type)), _agg_info(get_aggregate_info(agg, type)), + _key_coder(get_key_coder(type)), _index_size(-1), _is_nullable(is_nullable) { } @@ -77,6 +81,7 @@ class Field { Field(const FieldAggregationMethod& agg, const FieldType& type, size_t index_size, bool is_nullable) : _type_info(get_type_info(type)), _agg_info(get_aggregate_info(agg, type)), + _key_coder(get_key_coder(type)), _index_size(index_size), _is_nullable(is_nullable) { } @@ -233,10 +238,19 @@ class Field { FieldType type() const { return _type_info->type(); } const TypeInfo* type_info() const { return _type_info; } bool is_nullable() const { return _is_nullable; } + + void encode_ascending(const void* value, std::string* buf) const { + _key_coder->encode_ascending(value, _index_size, buf); + } + + Status decode_ascending(Slice* encoded_key, uint8_t* cell_ptr, Arena* arena) const { + return _key_coder->decode_ascending(encoded_key, _index_size, cell_ptr, arena); + } private: // Field的最大长度,单位为字节,通常等于length, 变长字符串不同 const TypeInfo* _type_info; const AggregateInfo* _agg_info; + const KeyCoder* _key_coder; uint16_t _index_size; bool _is_nullable; }; diff --git a/be/src/olap/iterators.h b/be/src/olap/iterators.h index d54cef13ff6840..bbb2ce62906b2e 100644 --- a/be/src/olap/iterators.h +++ b/be/src/olap/iterators.h @@ -17,14 +17,33 @@ #pragma once +#include + #include "common/status.h" namespace doris { +class RowCursor; class RowBlockV2; class Schema; struct StorageReadOptions { + // lower_bound defines the smallest key at which iterator will + // return data. + // If lower_bound is null, won't return + std::shared_ptr lower_bound; + + // If include_lower_bound is true, data equal with lower_bound will + // be read + bool include_lower_bound; + + // upper_bound defines the extend upto which the iterator can return + // data. + std::shared_ptr upper_bound; + + // If include_upper_bound is true, data equal with upper_bound will + // be read + bool include_upper_bound; }; // Used to read data in RowBlockV2 one by one diff --git a/be/src/olap/key_coder.cpp b/be/src/olap/key_coder.cpp new file mode 100644 index 00000000000000..68f2ace8961bcb --- /dev/null +++ b/be/src/olap/key_coder.cpp @@ -0,0 +1,80 @@ +// 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. + +#include "olap/key_coder.h" + +#include + +namespace doris { + +template +KeyCoder::KeyCoder(TraitsType traits) + : _encode_ascending(traits.encode_ascending), + _decode_ascending(traits.decode_ascending) { +} + +// Helper class used to get KeyCoder +class KeyCoderResolver { +public: + ~KeyCoderResolver() { + for (auto& iter : _coder_map) { + delete iter.second; + } + } + + static KeyCoderResolver* instance() { + static KeyCoderResolver s_instance; + return &s_instance; + } + + KeyCoder* get_coder(FieldType field_type) const { + auto it = _coder_map.find(field_type); + if (it != _coder_map.end()) { + return it->second; + } + return nullptr; + } + +private: + KeyCoderResolver() { + add_mapping(); + add_mapping(); + add_mapping(); + add_mapping(); + add_mapping(); + add_mapping(); + add_mapping(); + + add_mapping(); + add_mapping(); + add_mapping(); + add_mapping(); + } + + template + void add_mapping() { + _coder_map.emplace(field_type, new KeyCoder(KeyCoderTraits())); + } + + std::unordered_map _coder_map; +}; + +const KeyCoder* get_key_coder(FieldType type) { + return KeyCoderResolver::instance()->get_coder(type); +} + +} diff --git a/be/src/olap/key_coder.h b/be/src/olap/key_coder.h new file mode 100644 index 00000000000000..773c4e9ddb3188 --- /dev/null +++ b/be/src/olap/key_coder.h @@ -0,0 +1,224 @@ +// 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. + +#pragma once + +#include +#include + +#include "common/status.h" +#include "gutil/endian.h" +#include "gutil/strings/substitute.h" +#include "olap/types.h" +#include "util/arena.h" + +namespace doris { + +using strings::Substitute; + +using EncodeAscendingFunc = void (*)(const void* value, size_t index_size, std::string* buf); +using DecodeAscendingFunc = Status (*)(Slice* encoded_key, size_t index_size, uint8_t* cell_ptr, Arena* arena); + +// Helper class that is used to encode types of value in memory format +// into a sorted binary. For example, this class will encode unsigned +// integer to bit endian format which can compare with memcmp. +class KeyCoder { +public: + template + KeyCoder(TraitsType traits); + + void encode_ascending(const void* value, size_t index_size, std::string* buf) const { + _encode_ascending(value, index_size, buf); + } + Status decode_ascending(Slice* encoded_key, size_t index_size, uint8_t* cell_ptr, Arena* arena) const { + return _decode_ascending(encoded_key, index_size, cell_ptr, arena); + } + +private: + EncodeAscendingFunc _encode_ascending; + DecodeAscendingFunc _decode_ascending; +}; + +extern const KeyCoder* get_key_coder(FieldType type); + +template +class KeyCoderTraits { +}; + +template +class KeyCoderTraits::CppType>::value>::type> { +public: + using CppType = typename CppTypeTraits::CppType; + using UnsignedCppType = typename CppTypeTraits::UnsignedCppType; + +private: + // Swap value's endian from/to big endian + static UnsignedCppType swap_big_endian(UnsignedCppType val) { + switch (sizeof(UnsignedCppType)) { + case 1: return val; + case 2: return BigEndian::FromHost16(val); + case 4: return BigEndian::FromHost32(val); + case 8: return BigEndian::FromHost64(val); + case 16: return BigEndian::FromHost128(val); + default: LOG(FATAL) << "Invalid type to big endian, type=" << field_type + << ", size=" << sizeof(UnsignedCppType); + } + } + +public: + static void encode_ascending(const void* value, size_t index_size, std::string* buf) { + UnsignedCppType unsigned_val; + memcpy(&unsigned_val, value, sizeof(unsigned_val)); + // swap MSB to encode integer + if (std::is_signed::value) { + unsigned_val ^= (static_cast(1) << (sizeof(UnsignedCppType) * CHAR_BIT - 1)); + } + // make it bigendian + unsigned_val = swap_big_endian(unsigned_val); + + buf->append((char*)&unsigned_val, sizeof(unsigned_val)); + } + + static Status decode_ascending(Slice* encoded_key, size_t index_size, + uint8_t* cell_ptr, Arena* arena) { + if (encoded_key->size < sizeof(UnsignedCppType)) { + return Status::InvalidArgument( + Substitute("Key too short, need=$0 vs real=$1", + sizeof(UnsignedCppType), encoded_key->size)); + } + UnsignedCppType unsigned_val; + memcpy(&unsigned_val, encoded_key->data, sizeof(UnsignedCppType)); + unsigned_val = swap_big_endian(unsigned_val); + if (std::is_signed::value) { + unsigned_val ^= (static_cast(1) << (sizeof(UnsignedCppType) * CHAR_BIT - 1)); + } + memcpy(cell_ptr, &unsigned_val, sizeof(UnsignedCppType)); + encoded_key->remove_prefix(sizeof(UnsignedCppType)); + return Status::OK(); + } +}; + +template<> +class KeyCoderTraits { +public: + using CppType = typename CppTypeTraits::CppType; + using UnsignedCppType = typename CppTypeTraits::UnsignedCppType; + +public: + static void encode_ascending(const void* value, size_t index_size, std::string* buf) { + UnsignedCppType unsigned_val; + memcpy(&unsigned_val, value, sizeof(unsigned_val)); + // make it bigendian + unsigned_val = BigEndian::FromHost24(unsigned_val); + buf->append((char*)&unsigned_val, sizeof(unsigned_val)); + } + + static Status decode_ascending(Slice* encoded_key, size_t index_size, + uint8_t* cell_ptr, Arena* arena) { + if (encoded_key->size < sizeof(UnsignedCppType)) { + return Status::InvalidArgument( + Substitute("Key too short, need=$0 vs real=$1", + sizeof(UnsignedCppType), encoded_key->size)); + } + UnsignedCppType unsigned_val; + memcpy(&unsigned_val, encoded_key->data, sizeof(UnsignedCppType)); + unsigned_val = BigEndian::FromHost24(unsigned_val); + memcpy(cell_ptr, &unsigned_val, sizeof(UnsignedCppType)); + encoded_key->remove_prefix(sizeof(UnsignedCppType)); + return Status::OK(); + } +}; + +template<> +class KeyCoderTraits { +public: + static void encode_ascending(const void* value, size_t index_size, std::string* buf) { + decimal12_t decimal_val; + memcpy(&decimal_val, value, sizeof(decimal12_t)); + // encode integer + KeyCoderTraits::encode_ascending( + &decimal_val.integer, sizeof(decimal_val.integer), buf); + // encode integer + KeyCoderTraits::encode_ascending( + &decimal_val.fraction, sizeof(decimal_val.fraction), buf); + } + + static Status decode_ascending(Slice* encoded_key, size_t index_size, + uint8_t* cell_ptr, Arena* arena) { + decimal12_t decimal_val; + RETURN_IF_ERROR(KeyCoderTraits::decode_ascending( + encoded_key, sizeof(decimal_val.integer), (uint8_t*)&decimal_val.integer, arena)); + RETURN_IF_ERROR(KeyCoderTraits::decode_ascending( + encoded_key, sizeof(decimal_val.fraction), (uint8_t*)&decimal_val.fraction, arena)); + memcpy(cell_ptr, &decimal_val, sizeof(decimal12_t)); + return Status::OK(); + } +}; + +template<> +class KeyCoderTraits { +public: + static void encode_ascending(const void* value, size_t index_size, std::string* buf) { + const Slice* slice = (const Slice*)value; + CHECK(index_size <= slice->size) << "index size is larger than char size, index=" << index_size << ", char=" << slice->size; + buf->append(slice->data, index_size); + } + + static Status decode_ascending(Slice* encoded_key, size_t index_size, + uint8_t* cell_ptr, Arena* arena) { + if (encoded_key->size < index_size) { + return Status::InvalidArgument( + Substitute("Key too short, need=$0 vs real=$1", + index_size, encoded_key->size)); + } + Slice* slice = (Slice*)cell_ptr; + slice->data = arena->Allocate(index_size); + slice->size = index_size; + memcpy(slice->data, encoded_key->data, index_size); + encoded_key->remove_prefix(index_size); + return Status::OK(); + } +}; + +template<> +class KeyCoderTraits { +public: + static void encode_ascending(const void* value, size_t index_size, std::string* buf) { + const Slice* slice = (const Slice*)value; + size_t copy_size = std::min(index_size, slice->size); + buf->append(slice->data, copy_size); + } + + static Status decode_ascending(Slice* encoded_key, size_t index_size, + uint8_t* cell_ptr, Arena* arena) { + CHECK(encoded_key->size <= index_size) + << "encoded_key size is larger than index_size, key_size=" << encoded_key->size + << ", index_size=" << index_size; + auto copy_size = encoded_key->size; + Slice* slice = (Slice*)cell_ptr; + slice->data = arena->Allocate(copy_size); + slice->size = copy_size; + memcpy(slice->data, encoded_key->data, copy_size); + encoded_key->remove_prefix(copy_size); + return Status::OK(); + } +}; + +} diff --git a/be/src/olap/rowset/rowset_reader.h b/be/src/olap/rowset/rowset_reader.h index 0f60fc8e344a6b..57e3533b52e0f8 100644 --- a/be/src/olap/rowset/rowset_reader.h +++ b/be/src/olap/rowset/rowset_reader.h @@ -26,6 +26,7 @@ namespace doris { +class RowBlock; class RowsetReader; using RowsetReaderSharedPtr = std::shared_ptr; diff --git a/be/src/olap/rowset/rowset_reader_context.h b/be/src/olap/rowset/rowset_reader_context.h index 9d516242ee6ee2..f2c51100f75fde 100644 --- a/be/src/olap/rowset/rowset_reader_context.h +++ b/be/src/olap/rowset/rowset_reader_context.h @@ -18,17 +18,17 @@ #ifndef DORIS_BE_SRC_OLAP_ROWSET_ROWSET_READER_CONTEXT_H #define DORIS_BE_SRC_OLAP_ROWSET_ROWSET_READER_CONTEXT_H -#include "olap/schema.h" #include "olap/column_predicate.h" -#include "olap/row_cursor.h" -#include "olap/row_block.h" #include "olap/lru_cache.h" -#include "olap/olap_cond.h" -#include "olap/delete_handler.h" #include "runtime/runtime_state.h" namespace doris { +class RowCursor; +class Conditions; +class DeleteHandler; +class TabletSchema; + struct RowsetReaderContext { RowsetReaderContext() : reader_type(READER_QUERY), tablet_schema(nullptr), diff --git a/be/src/olap/rowset/rowset_writer.h b/be/src/olap/rowset/rowset_writer.h index 9d3eea05243f94..a637dae353e09c 100644 --- a/be/src/olap/rowset/rowset_writer.h +++ b/be/src/olap/rowset/rowset_writer.h @@ -20,14 +20,14 @@ #include "olap/rowset/rowset.h" #include "olap/rowset/rowset_writer_context.h" -#include "olap/schema.h" -#include "olap/row_block.h" #include "gen_cpp/types.pb.h" #include "runtime/mem_pool.h" +#include "olap/column_mapping.h" namespace doris { class ContiguousRow; +class RowCursor; class RowsetWriter; using RowsetWriterSharedPtr = std::shared_ptr; diff --git a/be/src/olap/rowset/segment_v2/column_writer.h b/be/src/olap/rowset/segment_v2/column_writer.h index d137059e2141b7..2e7d778d8f1541 100644 --- a/be/src/olap/rowset/segment_v2/column_writer.h +++ b/be/src/olap/rowset/segment_v2/column_writer.h @@ -58,6 +58,18 @@ class ColumnWriter { ~ColumnWriter(); Status init(); + + template + Status append(const CellType& cell) { + if (_is_nullable) { + uint8_t nullmap = 0; + BitmapChange(&nullmap, 0, cell.is_null()); + return append_nullable(&nullmap, cell.cell_ptr(), 1); + } else { + return append(cell.cell_ptr(), 1); + } + } + // Now we only support append one by one, we should support append // multi rows in one call Status append(bool is_null, void* data) { diff --git a/be/src/olap/rowset/segment_v2/segment.cpp b/be/src/olap/rowset/segment_v2/segment.cpp new file mode 100644 index 00000000000000..28c0b8f372371c --- /dev/null +++ b/be/src/olap/rowset/segment_v2/segment.cpp @@ -0,0 +1,183 @@ +// 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. + +#include "olap/rowset/segment_v2/segment.h" + +#include "common/logging.h" // LOG +#include "env/env.h" // RandomAccessFile +#include "gutil/strings/substitute.h" +#include "olap/rowset/segment_v2/column_reader.h" // ColumnReader +#include "olap/rowset/segment_v2/segment_writer.h" // k_segment_magic_length +#include "olap/rowset/segment_v2/segment_iterator.h" +#include "util/slice.h" // Slice +#include "olap/tablet_schema.h" + +namespace doris { +namespace segment_v2 { + +using strings::Substitute; + +Segment::Segment( + std::string fname, uint32_t segment_id, + const std::shared_ptr& tablet_schema, + size_t num_rows_per_block) + : _fname(std::move(fname)), + _segment_id(segment_id), + _tablet_schema(tablet_schema), + _num_rows_per_block(num_rows_per_block) { +} + +Segment::~Segment() { + for (auto reader : _column_readers) { + delete reader; + } +} + +Status Segment::open() { + RETURN_IF_ERROR(Env::Default()->new_random_access_file(_fname, &_input_file)); + RETURN_IF_ERROR(_input_file->size(&_file_size)); + + // 24: 1 * magic + 1 * checksum + 1 * footer length + if (_file_size < 12) { + return Status::Corruption( + Substitute("Bad segment, file size is too small, real=$0 vs need=$1", + _file_size, 12)); + } + + // check header's magic + RETURN_IF_ERROR(_check_magic(0)); + + // parse footer to get meta + RETURN_IF_ERROR(_parse_footer()); + // parse short key index + RETURN_IF_ERROR(_parse_index()); + // initial all column reader + RETURN_IF_ERROR(_initial_column_readers()); + return Status::OK(); +} + +Status Segment::new_iterator(const Schema& schema, std::unique_ptr* output) { + output->reset(new SegmentIterator(this->shared_from_this(), schema)); + return Status::OK(); +} + +// Read data at offset of input file, check if the file content match the magic +Status Segment::_check_magic(uint64_t offset) { + // read magic and length + uint8_t buf[k_segment_magic_length]; + Slice slice(buf, k_segment_magic_length); + RETURN_IF_ERROR(_input_file->read_at(offset, slice)); + + if (memcmp(slice.data, k_segment_magic, k_segment_magic_length) != 0) { + return Status::Corruption( + Substitute("Bad segment, file magic don't match, magic=$0 vs need=$1", + std::string((char*)buf, k_segment_magic_length), k_segment_magic)); + } + return Status::OK(); +} + +Status Segment::_parse_footer() { + uint64_t offset = _file_size - 8; + // read footer's length and checksum + uint8_t buf[8]; + Slice slice(buf, 8); + RETURN_IF_ERROR(_input_file->read_at(offset, slice)); + + uint32_t footer_length = decode_fixed32_le((uint8_t*)slice.data); + uint32_t checksum = decode_fixed32_le((uint8_t*)slice.data + 4); + + // check file size footer + if (offset < footer_length) { + return Status::Corruption( + Substitute("Bad segment, file size is too small, file_size=$0 vs footer_size=$1", + _file_size, footer_length)); + } + offset -= footer_length; + + std::string footer_buf; + footer_buf.resize(footer_length); + RETURN_IF_ERROR(_input_file->read_at(offset, footer_buf)); + + // TODO(zc): check footer's checksum + if (checksum != 0) { + return Status::Corruption( + Substitute("Bad segment, segment footer checksum not match, real=$0 vs expect=$1", + 0, checksum)); + } + + if (!_footer.ParseFromString(footer_buf)) { + return Status::Corruption("Bad segment, parse footer from PB failed"); + } + + return Status::OK(); +} + +// load and parse short key index +Status Segment::_parse_index() { + // read short key index content + _sk_index_buf.resize(_footer.short_key_index_page().size()); + Slice slice(_sk_index_buf.data(), _sk_index_buf.size()); + RETURN_IF_ERROR(_input_file->read_at(_footer.short_key_index_page().offset(), slice)); + + // Parse short key index + _sk_index_decoder.reset(new ShortKeyIndexDecoder(_sk_index_buf)); + RETURN_IF_ERROR(_sk_index_decoder->parse()); + return Status::OK(); +} + +Status Segment::_initial_column_readers() { + // Map from column unique id to column ordinal in footer's ColumnMetaPB + // If we can't find unique id, it means this segment is created + // with an old schema. So we should create a DefaultValueIterator + // for this column. + std::unordered_map unique_id_to_ordinal; + for (uint32_t ordinal = 0; ordinal < _footer.columns().size(); ++ordinal) { + auto& column_pb = _footer.columns(ordinal); + unique_id_to_ordinal.emplace(column_pb.unique_id(), ordinal); + } + // TODO(zc): Lazy init()? + // There may be too many columns, majority of them would not be used + // in query, so we should not init them here. + _column_readers.resize(_tablet_schema->columns().size(), nullptr); + + for (uint32_t ordinal = 0; ordinal < _tablet_schema->num_columns(); ++ordinal) { + auto& column = _tablet_schema->columns()[ordinal]; + auto iter = unique_id_to_ordinal.find(column.unique_id()); + if (iter == unique_id_to_ordinal.end()) { + continue; + } + + ColumnReaderOptions opts; + std::unique_ptr reader( + new ColumnReader(opts, _footer.columns(iter->second), _input_file.get())); + RETURN_IF_ERROR(reader->init()); + + _column_readers[ordinal] = reader.release(); + } + return Status::OK(); +} + +Status Segment::new_column_iterator(uint32_t cid, ColumnIterator** iter) { + if (_column_readers[cid] == nullptr) { + // TODO(zc): create a DefaultValueIterator for this column + // create + } + return _column_readers[cid]->new_iterator(iter); +} + +} +} diff --git a/be/src/olap/rowset/segment_v2/segment.h b/be/src/olap/rowset/segment_v2/segment.h new file mode 100644 index 00000000000000..e69a10e7513ad0 --- /dev/null +++ b/be/src/olap/rowset/segment_v2/segment.h @@ -0,0 +1,120 @@ +// 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. + +#pragma once + +#include +#include +#include // for unique_ptr +#include + +#include "common/status.h" // Status +#include "gen_cpp/segment_v2.pb.h" +#include "olap/rowset/segment_v2/common.h" // rowid_t +#include "olap/short_key_index.h" +#include "olap/tablet_schema.h" +#include "util/faststring.h" + +namespace doris { + +class RandomAccessFile; +class SegmentGroup; +class FieldInfo; +class TabletSchema; +class ShortKeyIndexDecoder; +class Schema; + +namespace segment_v2 { + +class ColumnReader; +class ColumnIterator; +class SegmentIterator; + +// A Segment is used to represent a segment in memory format. When segment is +// generated, it won't be modified, so this struct aimed to help read operation. +// It will prepare all ColumnReader to create ColumnIterator as needed. +// And user can create a SegmentIterator through new_iterator function. +// +// NOTE: This segment is used to a specified TabletSchema, when TabletSchema +// is changed, this segemnt can not be used any more. For eample, after a schema +// change finished, client should disalbe all cahced Segment for old TabletSchema. +class Segment : public std::enable_shared_from_this { +public: + Segment(std::string fname, uint32_t segment_id, + const std::shared_ptr& tablet_schema, + size_t num_rows_per_block); + ~Segment(); + + Status open(); + + Status new_iterator(const Schema& schema, std::unique_ptr* iter); + + uint64_t id() const { return _segment_id; } + + uint32_t num_rows() const { return _footer.num_rows(); } + +private: + friend class SegmentIterator; + + Status new_column_iterator(uint32_t cid, ColumnIterator** iter); + uint32_t num_rows_per_block() const { return _num_rows_per_block; } + size_t num_short_keys() const { return _tablet_schema->num_short_key_columns(); } + + Status _check_magic(uint64_t offset); + Status _parse_footer(); + Status _parse_index(); + Status _initial_column_readers(); + + ShortKeyIndexIterator lower_bound(const Slice& key) const { + return _sk_index_decoder->lower_bound(key); + } + ShortKeyIndexIterator upper_bound(const Slice& key) const { + return _sk_index_decoder->upper_bound(key); + } + + // This will return the last row block in this segment. + // NOTE: Before call this function , client should assure that + // this segment is not empty. + uint32_t last_block() const { + DCHECK(num_rows() > 0); + return _sk_index_decoder->num_items() - 1; + } + +private: + std::string _fname; + uint32_t _segment_id; + std::shared_ptr _tablet_schema; + uint32_t _num_rows_per_block; + + SegmentFooterPB _footer; + std::unique_ptr _input_file; + uint64_t _file_size = 0; + + // ColumnReader for each column in TabletSchema. If ColumnReader is nullptr, + // This means that this segment has no data for that column, which may be added + // after this segment is generated. + std::vector _column_readers; + + // used to store short key index + faststring _sk_index_buf; + + // short key index decoder + std::unique_ptr _sk_index_decoder; +}; + +} +} diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp b/be/src/olap/rowset/segment_v2/segment_iterator.cpp new file mode 100644 index 00000000000000..8ad70a1d525c34 --- /dev/null +++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp @@ -0,0 +1,258 @@ +// 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. + +#include "olap/rowset/segment_v2/segment_iterator.h" + +#include + +#include "gutil/strings/substitute.h" +#include "olap/rowset/segment_v2/segment.h" +#include "olap/rowset/segment_v2/column_reader.h" +#include "olap/row_block2.h" +#include "olap/row_cursor.h" +#include "olap/short_key_index.h" + +using strings::Substitute; + +namespace doris { +namespace segment_v2 { + +SegmentIterator::SegmentIterator(std::shared_ptr segment, + const Schema& schema) + : _segment(std::move(segment)), + _schema(schema), + _column_iterators(_schema.num_columns(), nullptr) { +} + +SegmentIterator::~SegmentIterator() { + for (auto iter : _column_iterators) { + delete iter; + } +} + +Status SegmentIterator::init(const StorageReadOptions& opts) { + _opts = opts; + RETURN_IF_ERROR(_init_short_key_range()); + RETURN_IF_ERROR(_init_column_iterators()); + return Status::OK(); +} + +// This function will use input key bounds to get a row range. +Status SegmentIterator::_init_short_key_range() { + _lower_rowid = 0; + _upper_rowid = num_rows(); + + // fast path for empty segment + if (_upper_rowid == 0) { + return Status::OK(); + } + + if (_opts.lower_bound == nullptr && _opts.upper_bound == nullptr) { + return Status::OK(); + } + + RETURN_IF_ERROR(_prepare_seek()); + + // init row range with short key range + if (_opts.upper_bound != nullptr) { + // If client want to read upper_bound, the include_upper_bound is true. So we + // should get the first ordinal at which key is larger than upper_bound. + // So we call _lookup_ordinal with include_upper_bound's negate + RETURN_IF_ERROR(_lookup_ordinal( + *_opts.upper_bound, !_opts.include_upper_bound, num_rows(), &_upper_rowid)); + } + if (_upper_rowid > 0 && _opts.lower_bound != nullptr) { + RETURN_IF_ERROR(_lookup_ordinal( + *_opts.lower_bound, _opts.include_lower_bound, _upper_rowid, &_lower_rowid)); + } + + return Status::OK(); +} + +// Set up environment for the following seek. +Status SegmentIterator::_prepare_seek() { + std::vector key_fields; + std::set column_set; + if (_opts.lower_bound != nullptr) { + for (auto cid : _opts.lower_bound->schema()->column_ids()) { + column_set.emplace(cid); + key_fields.push_back(*_opts.lower_bound->schema()->column(cid)); + } + } + if (_opts.upper_bound != nullptr) { + for (auto cid : _opts.upper_bound->schema()->column_ids()) { + if (column_set.count(cid) == 0) { + key_fields.push_back(*_opts.upper_bound->schema()->column(cid)); + column_set.emplace(cid); + } + } + } + _seek_schema.reset(new Schema(key_fields, key_fields.size())); + _seek_block.reset(new RowBlockV2(*_seek_schema, 1, &_arena)); + + // create used column iterator + for (auto cid : _seek_schema->column_ids()) { + if (_column_iterators[cid] == nullptr) { + RETURN_IF_ERROR(_create_column_iterator(cid, &_column_iterators[cid])); + } + } + + return Status::OK(); +} + +Status SegmentIterator::_init_column_iterators() { + _cur_rowid = _lower_rowid; + if (_cur_rowid >= num_rows()) { + return Status::OK(); + } + for (auto cid : _schema.column_ids()) { + if (_column_iterators[cid] == nullptr) { + RETURN_IF_ERROR(_create_column_iterator(cid, &_column_iterators[cid])); + } + + _column_iterators[cid]->seek_to_ordinal(_cur_rowid); + } + return Status::OK(); +} + +Status SegmentIterator::_create_column_iterator(uint32_t cid, ColumnIterator** iter) { + return _segment->new_column_iterator(cid, iter); +} + +// Schema of lhs and rhs are different. +// callers should assure that rhs' schema has all columns in lhs schema +template +int compare_row_with_lhs_columns(const LhsRowType& lhs, const RhsRowType& rhs) { + for (auto cid : lhs.schema()->column_ids()) { + auto res = lhs.schema()->column(cid)->compare_cell(lhs.cell(cid), rhs.cell(cid)); + if (res != 0) { + return res; + } + } + return 0; +} + +// look up one key to get its ordinal at which can get data. +// 'upper_bound' is defined the max ordinal the function will search. +// We use upper_bound to reduce search times. +// If we find a valid ordinal, it will be set in rowid and with Status::OK() +// If we can not find a valid key in this segment, we will set rowid to upper_bound +// Otherwise return error. +// 1. get [start, end) ordinal through short key index +// 2. binary search to find exact ordinal that match the input condition +// Make is_include template to reduce branch +Status SegmentIterator::_lookup_ordinal(const RowCursor& key, bool is_include, + rowid_t upper_bound, rowid_t* rowid) { + std::string index_key; + encode_key_with_padding(&index_key, key, _segment->num_short_keys(), is_include); + + uint32_t start_block_id = 0; + auto start_iter = _segment->lower_bound(index_key); + if (start_iter.valid()) { + // Because previous block may contain this key, so we should set rowid to + // last block's first row. + start_block_id = start_iter.ordinal(); + if (start_block_id > 0) { + start_block_id--; + } + } else { + // When we don't find a valid index item, which means all short key is + // smaller than input key, this means that this key may exist in the last + // row block. so we set the rowid to first row of last row block. + start_block_id = _segment->last_block(); + } + rowid_t start = start_block_id * _segment->num_rows_per_block(); + + rowid_t end = upper_bound; + auto end_iter = _segment->upper_bound(index_key); + if (end_iter.valid()) { + end = end_iter.ordinal() * _segment->num_rows_per_block(); + } + + // binary search to find the exact key + while (start < end) { + rowid_t mid = (start + end) / 2; + RETURN_IF_ERROR(_seek_and_peek(mid)); + int cmp = compare_row_with_lhs_columns(key, _seek_block->row(0)); + if (cmp > 0) { + start = mid + 1; + } else if (cmp == 0) { + if (is_include) { + // lower bound + end = mid; + } else { + // upper bound + start = mid + 1; + } + } else { + end = mid; + } + } + + *rowid = start; + return Status::OK(); +} + +// seek to the row and load that row to _key_cursor +Status SegmentIterator::_seek_and_peek(rowid_t rowid) { + for (auto cid : _seek_schema->column_ids()) { + _column_iterators[cid]->seek_to_ordinal(rowid); + } + size_t num_rows = 1; + _seek_block->resize(num_rows); + RETURN_IF_ERROR(_next_batch(_seek_block.get(), &num_rows)); + return Status::OK(); +} + +// Try to read data as much to block->num_rows(). The number of read rows +// will be set in rows_read when return OK. rows_read will small than +// block->num_rows() when reach the end of this segment +Status SegmentIterator::_next_batch(RowBlockV2* block, size_t* rows_read) { + bool has_read = false; + size_t first_read = 0; + for (int i = 0; i < block->schema()->column_ids().size(); ++i) { + auto cid = block->schema()->column_ids()[i]; + size_t num_rows = has_read ? first_read : block->num_rows(); + auto column_block = block->column_block(i); + RETURN_IF_ERROR(_column_iterators[cid]->next_batch(&num_rows, &column_block)); + if (!has_read) { + has_read = true; + first_read = num_rows; + } else if (num_rows != first_read) { + return Status::InternalError( + Substitute("Read different rows in different columns" + ", column($0) read $1 vs column($2) read $3", + block->schema()->column_ids()[0], first_read, cid, num_rows)); + } + } + *rows_read = first_read; + return Status::OK(); +} + +Status SegmentIterator::next_batch(RowBlockV2* block) { + size_t rows_to_read = std::min((rowid_t)block->capacity(), _upper_rowid - _cur_rowid); + block->resize(rows_to_read); + if (rows_to_read == 0) { + return Status::OK(); + } + RETURN_IF_ERROR(_next_batch(block, &rows_to_read)); + _cur_rowid += rows_to_read; + return Status::OK(); +} + +} +} diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.h b/be/src/olap/rowset/segment_v2/segment_iterator.h new file mode 100644 index 00000000000000..46a1e696b307a9 --- /dev/null +++ b/be/src/olap/rowset/segment_v2/segment_iterator.h @@ -0,0 +1,88 @@ +// 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. + +#pragma once + +#include +#include + +#include "common/status.h" +#include "olap/rowset/segment_v2/common.h" +#include "olap/rowset/segment_v2/segment.h" +#include "olap/iterators.h" +#include "olap/schema.h" +#include "util/arena.h" + +namespace doris { + +class RowCursor; +class RowBlockV2; +class ShortKeyIndexIterator; + +namespace segment_v2 { + +class ColumnIterator; + +class SegmentIterator : public RowwiseIterator { +public: + SegmentIterator(std::shared_ptr segment, const Schema& _schema); + ~SegmentIterator() override; + Status init(const StorageReadOptions& opts) override; + Status next_batch(RowBlockV2* row_block) override; + const Schema& schema() const override { return _schema; } +private: + Status _init_short_key_range(); + Status _prepare_seek(); + Status _init_column_iterators(); + Status _create_column_iterator(uint32_t cid, ColumnIterator** iter); + + Status _lookup_ordinal(const RowCursor& key, bool is_include, + rowid_t upper_bound, rowid_t* rowid); + Status _seek_and_peek(rowid_t rowid); + Status _next_batch(RowBlockV2* block, size_t* rows_read); + + uint32_t segment_id() const { return _segment->id(); } + uint32_t num_rows() const { return _segment->num_rows(); } + +private: + std::shared_ptr _segment; + // TODO(zc): rethink if we need copy it + Schema _schema; + + StorageReadOptions _opts; + + // Only used when init is called, help to finish seek_and_peek. + // Data will be saved in this batch + std::unique_ptr _seek_schema; + + // used to read data from columns when do bianry search to find + // oridnal for input bounds + std::unique_ptr _seek_block; + // helper to save row to compare with input bounds + std::unique_ptr _key_cursor; + + std::vector _column_iterators; + + rowid_t _lower_rowid; + rowid_t _upper_rowid; + rowid_t _cur_rowid; + + Arena _arena; +}; + +} +} diff --git a/be/src/olap/rowset/segment_v2/segment_writer.cpp b/be/src/olap/rowset/segment_v2/segment_writer.cpp new file mode 100644 index 00000000000000..c3975732c30e55 --- /dev/null +++ b/be/src/olap/rowset/segment_v2/segment_writer.cpp @@ -0,0 +1,172 @@ +// 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. + +#include "olap/rowset/segment_v2/segment_writer.h" + +#include "env/env.h" // Env +#include "olap/row_block.h" // RowBlock +#include "olap/row_cursor.h" // RowCursor +#include "olap/rowset/segment_v2/column_writer.h" // ColumnWriter +#include "olap/short_key_index.h" + +namespace doris { +namespace segment_v2 { + +const char* k_segment_magic = "D0R1"; +const uint32_t k_segment_magic_length = 4; + +SegmentWriter::SegmentWriter(std::string fname, uint32_t segment_id, + const std::shared_ptr& tablet_schema, + const SegmentWriterOptions& opts) + : _fname(std::move(fname)), + _segment_id(segment_id), + _tablet_schema(tablet_schema), + _opts(opts) { +} + +SegmentWriter::~SegmentWriter() { + for (auto writer : _column_writers) { + delete writer; + } +} + +Status SegmentWriter::init(uint32_t write_mbytes_per_sec) { + // create for write + RETURN_IF_ERROR(Env::Default()->new_writable_file(_fname, &_output_file)); + + uint32_t column_id = 0; + for (auto& column : _tablet_schema->columns()) { + ColumnMetaPB* column_meta = _footer.add_columns(); + // TODO(zc): Do we need this column_id?? + column_meta->set_column_id(column_id++); + column_meta->set_unique_id(column.unique_id()); + bool is_nullable = column.is_nullable(); + column_meta->set_is_nullable(is_nullable); + + // TODO(zc): we can add type_info into TabletColumn? + const TypeInfo* type_info = get_type_info(column.type()); + DCHECK(type_info != nullptr); + + ColumnWriterOptions opts; + std::unique_ptr writer(new ColumnWriter(opts, type_info, is_nullable, _output_file.get())); + RETURN_IF_ERROR(writer->init()); + _column_writers.push_back(writer.release()); + } + _index_builder.reset(new ShortKeyIndexBuilder(_segment_id, _opts.num_rows_per_block)); + return Status::OK(); +} + +template +Status SegmentWriter::append_row(const RowType& row) { + for (size_t cid = 0; cid < _column_writers.size(); ++cid) { + auto cell = row.cell(cid); + RETURN_IF_ERROR(_column_writers[cid]->append(cell)); + } + + if ((_row_count % _opts.num_rows_per_block) == 0) { + std::string encoded_key; + encode_key(&encoded_key, row, _tablet_schema->num_short_key_columns()); + RETURN_IF_ERROR(_index_builder->add_item(encoded_key)); + _block_count++; + } + _row_count++; + return Status::OK(); +} + +template Status SegmentWriter::append_row(const RowCursor& row); + +uint64_t SegmentWriter::estimate_segment_size() { + return 0; +} + +Status SegmentWriter::finalize(uint32_t* segment_file_size) { + for (auto column_writer : _column_writers) { + RETURN_IF_ERROR(column_writer->finish()); + } + RETURN_IF_ERROR(_write_raw_data({k_segment_magic})); + RETURN_IF_ERROR(_write_data()); + RETURN_IF_ERROR(_write_ordinal_index()); + RETURN_IF_ERROR(_write_short_key_index()); + RETURN_IF_ERROR(_write_footer()); + return Status::OK(); +} + +// write column data to file one by one +Status SegmentWriter::_write_data() { + for (auto column_writer : _column_writers) { + RETURN_IF_ERROR(column_writer->write_data()); + } + return Status::OK(); +} + +// write ordinal index after data has been written +Status SegmentWriter::_write_ordinal_index() { + for (auto column_writer : _column_writers) { + RETURN_IF_ERROR(column_writer->write_ordinal_index()); + } + return Status::OK(); +} + +Status SegmentWriter::_write_short_key_index() { + std::vector slices; + // TODO(zc): we should get segment_size + RETURN_IF_ERROR(_index_builder->finalize(_row_count * 100, _row_count, &slices)); + + uint64_t offset = _output_file->size(); + RETURN_IF_ERROR(_write_raw_data(slices)); + uint32_t written_bytes = _output_file->size() - offset; + + _footer.mutable_short_key_index_page()->set_offset(offset); + _footer.mutable_short_key_index_page()->set_size(written_bytes); + return Status::OK(); +} + +Status SegmentWriter::_write_footer() { + _footer.set_num_rows(_row_count); + // collect all + for (int i = 0; i < _column_writers.size(); ++i) { + _column_writers[i]->write_meta(_footer.mutable_columns(i)); + } + + // write footer + std::string footer_buf; + if (!_footer.SerializeToString(&footer_buf)) { + return Status::InternalError("failed to serialize segment footer"); + } + + std::string footer_info_buf; + // put footer's size + put_fixed32_le(&footer_info_buf, footer_buf.size()); + // TODO(zc): compute checksum for footer + uint32_t checksum = 0; + put_fixed32_le(&footer_info_buf, checksum); + + // I think we don't need to put a tail magic. + + std::vector slices{footer_buf, footer_info_buf}; + // write offset and length + RETURN_IF_ERROR(_write_raw_data(slices)); + return Status::OK(); +} + +Status SegmentWriter::_write_raw_data(const std::vector& slices) { + RETURN_IF_ERROR(_output_file->appendv(&slices[0], slices.size())); + return Status::OK(); +} + +} +} diff --git a/be/src/olap/rowset/segment_v2/segment_writer.h b/be/src/olap/rowset/segment_v2/segment_writer.h new file mode 100644 index 00000000000000..a6a1bd8a26b20a --- /dev/null +++ b/be/src/olap/rowset/segment_v2/segment_writer.h @@ -0,0 +1,88 @@ +// 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. + +#pragma once + +#include +#include // unique_ptr +#include +#include + +#include "common/logging.h" // LOG +#include "common/status.h" // Status +#include "gen_cpp/segment_v2.pb.h" +#include "olap/schema.h" + +namespace doris { + +class WritableFile; +class RowBlock; +class RowCursor; +class ShortKeyIndexBuilder; + +namespace segment_v2 { + +class ColumnWriter; + +extern const char* k_segment_magic; +extern const uint32_t k_segment_magic_length; + +struct SegmentWriterOptions { + uint32_t num_rows_per_block = 1024; +}; + +class SegmentWriter { +public: + explicit SegmentWriter(std::string file_name, + uint32_t segment_id, + const std::shared_ptr& tablet_schema, + const SegmentWriterOptions& opts); + + ~SegmentWriter(); + Status init(uint32_t write_mbytes_per_sec); + + template + Status append_row(const RowType& row); + + uint64_t estimate_segment_size(); + + Status finalize(uint32_t* segment_file_size); + +private: + Status _write_data(); + Status _write_ordinal_index(); + Status _write_short_key_index(); + Status _write_footer(); + Status _write_raw_data(const std::vector& slices); + +private: + std::string _fname; + uint32_t _segment_id; + std::shared_ptr _tablet_schema; + size_t _num_short_keys; + SegmentWriterOptions _opts; + + SegmentFooterPB _footer; + std::unique_ptr _index_builder; + std::unique_ptr _output_file; + std::vector _column_writers; + uint64_t _row_count = 0; + uint32_t _block_count = 0; +}; + +} +} diff --git a/be/src/olap/schema_change.cpp b/be/src/olap/schema_change.cpp index dcc84e504e7be5..7afeefccdbd262 100644 --- a/be/src/olap/schema_change.cpp +++ b/be/src/olap/schema_change.cpp @@ -45,6 +45,51 @@ using std::vector; namespace doris { +class RowBlockSorter { +public: + explicit RowBlockSorter(RowBlockAllocator* allocator); + virtual ~RowBlockSorter(); + + bool sort(RowBlock** row_block); + +private: + static bool _row_cursor_comparator(const RowCursor* a, const RowCursor* b) { + return compare_row(*a, *b) < 0; + } + + RowBlockAllocator* _row_block_allocator; + RowBlock* _swap_row_block; +}; + +class RowBlockMerger { +public: + explicit RowBlockMerger(TabletSharedPtr tablet); + virtual ~RowBlockMerger(); + + bool merge( + const std::vector& row_block_arr, + RowsetWriterSharedPtr rowset_writer, + uint64_t* merged_rows); + +private: + struct MergeElement { + bool operator<(const MergeElement& other) const { + return compare_row(*row_cursor, *other.row_cursor) > 0; + } + + const RowBlock* row_block; + RowCursor* row_cursor; + uint32_t row_block_index; + }; + + bool _make_heap(const std::vector& row_block_arr); + bool _pop_heap(); + + TabletSharedPtr _tablet; + std::priority_queue _heap; +}; + + RowBlockChanger::RowBlockChanger(const TabletSchema& tablet_schema, const TabletSharedPtr &base_tablet) { _schema_mapping.resize(tablet_schema.num_columns()); diff --git a/be/src/olap/schema_change.h b/be/src/olap/schema_change.h index 6081e6a6889e1b..7aa24f5d68a4c9 100644 --- a/be/src/olap/schema_change.h +++ b/be/src/olap/schema_change.h @@ -28,7 +28,6 @@ #include "olap/rowset/rowset_writer.h" #include "olap/tablet.h" #include "olap/column_mapping.h" -#include "olap/row.h" namespace doris { // defined in 'field.h' @@ -74,23 +73,6 @@ class RowBlockChanger { DISALLOW_COPY_AND_ASSIGN(RowBlockChanger); }; -class RowBlockAllocator; -class RowBlockSorter { -public: - explicit RowBlockSorter(RowBlockAllocator* allocator); - virtual ~RowBlockSorter(); - - bool sort(RowBlock** row_block); - -private: - static bool _row_cursor_comparator(const RowCursor* a, const RowCursor* b) { - return compare_row(*a, *b) < 0; - } - - RowBlockAllocator* _row_block_allocator; - RowBlock* _swap_row_block; -}; - class RowBlockAllocator { public: RowBlockAllocator(const TabletSchema& tablet_schema, size_t memory_limitation); @@ -107,34 +89,6 @@ class RowBlockAllocator { size_t _memory_limitation; }; -class RowBlockMerger { -public: - explicit RowBlockMerger(TabletSharedPtr tablet); - virtual ~RowBlockMerger(); - - bool merge( - const std::vector& row_block_arr, - RowsetWriterSharedPtr rowset_writer, - uint64_t* merged_rows); - -private: - struct MergeElement { - bool operator<(const MergeElement& other) const { - return compare_row(*row_cursor, *other.row_cursor) > 0; - } - - const RowBlock* row_block; - RowCursor* row_cursor; - uint32_t row_block_index; - }; - - bool _make_heap(const std::vector& row_block_arr); - bool _pop_heap(); - - TabletSharedPtr _tablet; - std::priority_queue _heap; -}; - class SchemaChange { public: SchemaChange() : _filtered_rows(0), _merged_rows(0) {} diff --git a/be/src/olap/short_key_index.cpp b/be/src/olap/short_key_index.cpp new file mode 100644 index 00000000000000..03d66dffbf2564 --- /dev/null +++ b/be/src/olap/short_key_index.cpp @@ -0,0 +1,117 @@ +// 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. + +#include "olap/short_key_index.h" + +#include + +#include "util/coding.h" +#include "gutil/strings/substitute.h" + +using strings::Substitute; + +namespace doris { + +Status ShortKeyIndexBuilder::add_item(const Slice& key) { + put_varint32(&_offset_buf, _key_buf.size()); + _footer.set_num_items(_footer.num_items() + 1); + _key_buf.append(key.data, key.size); + return Status::OK(); +} + +Status ShortKeyIndexBuilder::finalize(uint32_t segment_bytes, + uint32_t num_segment_rows, + std::vector* slices) { + _footer.set_num_segment_rows(num_segment_rows); + _footer.set_segment_bytes(segment_bytes); + _footer.set_key_bytes(_key_buf.size()); + _footer.set_offset_bytes(_offset_buf.size()); + + // encode header + if (!_footer.SerializeToString(&_footer_buf)) { + return Status::InternalError("Failed to serialize index footer"); + } + + put_fixed32_le(&_footer_buf, _footer_buf.size()); + // TODO(zc): checksum + uint32_t checksum = 0; + put_fixed32_le(&_footer_buf, checksum); + + slices->emplace_back(_key_buf); + slices->emplace_back(_offset_buf); + slices->emplace_back(_footer_buf); + return Status::OK(); +} + +Status ShortKeyIndexDecoder::parse() { + Slice data = _data; + + // 1. parse footer, get checksum and footer length + if (data.size < 2 * sizeof(uint32_t)) { + return Status::Corruption( + Substitute("Short key is too short, need=$0 vs real=$1", + 2 * sizeof(uint32_t), data.size)); + } + size_t offset = data.size - 2 * sizeof(uint32_t); + uint32_t footer_length = decode_fixed32_le((uint8_t*)data.data + offset); + uint32_t checksum = decode_fixed32_le((uint8_t*)data.data + offset + 4); + // TODO(zc): do checksum + if (checksum != 0) { + return Status::Corruption( + Substitute("Checksum not match, need=$0 vs read=$1", 0, checksum)); + } + // move offset to parse footer + offset -= footer_length; + std::string footer_buf(data.data + offset, footer_length); + if (!_footer.ParseFromString(footer_buf)) { + return Status::Corruption("Fail to parse index footer from string"); + } + + // check if real data size match footer's content + if (offset != _footer.key_bytes() + _footer.offset_bytes()) { + return Status::Corruption( + Substitute("Index size not match, need=$0, real=$1", + _footer.key_bytes() + _footer.offset_bytes(), offset)); + } + + // set index buffer + _key_data = Slice(_data.data, _footer.key_bytes()); + + // parse offset information + Slice offset_slice(_data.data + _footer.key_bytes(), _footer.offset_bytes()); + // +1 for record total length + _offsets.resize(_footer.num_items() + 1); + _offsets[_footer.num_items()] = _footer.key_bytes(); + for (uint32_t i = 0; i < _footer.num_items(); ++i) { + uint32_t offset = 0; + if (!get_varint32(&offset_slice, &offset)) { + return Status::Corruption("Fail to get varint from index offset buffer"); + } + DCHECK(offset <= _footer.key_bytes()) + << "Offset is larger than total bytes, offset=" << offset + << ", key_bytes=" << _footer.key_bytes(); + _offsets[i] = offset; + } + + if (offset_slice.size != 0) { + return Status::Corruption("Still has data after parse all key offset"); + } + + return Status::OK(); +} + +} diff --git a/be/src/olap/short_key_index.h b/be/src/olap/short_key_index.h new file mode 100644 index 00000000000000..5bc0374898f6ae --- /dev/null +++ b/be/src/olap/short_key_index.h @@ -0,0 +1,274 @@ +// 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. + +#pragma once + +#include +#include +#include +#include + +#include "common/status.h" +#include "gen_cpp/segment_v2.pb.h" +#include "util/faststring.h" +#include "util/slice.h" + +#include "util/debug_util.h" + +namespace doris { + +// In our system, we have more complicated situation. +// First, our keys can be NULL. +// Second, when key columns are not complete we want to distinguish GT and GE. For examle, +// there are two key columns a and b, we have only one condition a > 1. We can only encode +// a prefix key 1, which is less than 1|2. This will make our read more data than +// we actually need. So we want to add more marker. +// a > 1: will be encoded into 1|\xFF +// a >= 1: will be encoded into 1|\x00 +// a = 1 and b > 1: will be encoded into 1|\x02|1 +// a = 1 and b is null: will be encoded into 1|\x01 + +// Used to represent minimal value for that field +constexpr uint8_t KEY_MINIMAL_MARKER = 0x00; +// Used to represent a null field, which value is seemed as minimal than other values +constexpr uint8_t KEY_NULL_FIRST_MARKER = 0x01; +// Used to represent a normal field, which content is encoded after this marker +constexpr uint8_t KEY_NORMAL_MARKER = 0x02; +// Used to represent +constexpr uint8_t KEY_NULL_LAST_MARKER = 0xFE; +// Used to represent maximal value for that field +constexpr uint8_t KEY_MAXIMAL_MARKER = 0xFF; + +// Encode one row into binary according given num_keys. +// A cell will be encoded in the format of a marker and encoded content. +// When function encoding row, if any cell isn't found in row, this function will +// fill a marker and return. If padding_minimal is true, KEY_MINIMAL_MARKER will +// be added, if padding_minimal is false, KEY_MAXIMAL_MARKER will be added. +// If all num_keys are found in row, no marker will be added. +template +void encode_key_with_padding(std::string* buf, const RowType& row, + size_t num_keys, bool padding_minimal) { + for (auto cid = 0; cid < num_keys; cid++) { + auto field = row.schema()->column(cid); + if (field == nullptr) { + if (padding_minimal) { + buf->push_back(KEY_MINIMAL_MARKER); + } else { + buf->push_back(KEY_MAXIMAL_MARKER); + } + break; + } + + auto cell = row.cell(cid); + if (cell.is_null()) { + if (null_first) { + buf->push_back(KEY_NULL_FIRST_MARKER); + } else { + buf->push_back(KEY_NULL_LAST_MARKER); + } + continue; + } + buf->push_back(KEY_NORMAL_MARKER); + field->encode_ascending(cell.cell_ptr(), buf); + } +} + +// Encode one row into binary according given num_keys. +// Client call this function must assure that row contains the first +// num_keys columns. +template +void encode_key(std::string* buf, const RowType& row, size_t num_keys) { + for (auto cid = 0; cid < num_keys; cid++) { + auto cell = row.cell(cid); + if (cell.is_null()) { + if (null_first) { + buf->push_back(KEY_NULL_FIRST_MARKER); + } else { + buf->push_back(KEY_NULL_LAST_MARKER); + } + continue; + } + buf->push_back(KEY_NORMAL_MARKER); + row.schema()->column(cid)->encode_ascending(cell.cell_ptr(), buf); + } +} + +// Used to encode a segment short key indices to binary format. This version +// only accepts binary key, client should assure that input key is sorted, +// otherwise error could happens. This builder would arrange data in following +// format. +// index = encoded_keys + encoded_offsets + footer + footer_size + checksum +// encoded_keys = binary_key + [, ...] +// encoded_offsets = encoded_offset + [, ...] +// encoded_offset = variant32 +// footer = ShortKeyFooterPB +// footer_size = fixed32 +// checksum = fixed32 +// Usage: +// ShortKeyIndexBuilder builder(segment_id, num_rows_per_block); +// builder.add_item(key1); +// ... +// builder.add_item(keyN); +// builder.finalize(segment_size, num_rows, &slices); +// NOTE: This is used for BetaRowset and is not compatible with AlphaRowset's +// short key index format. +// TODO(zc): +// 1. If this can leverage binary page to save key and offset data +// 2. Extending this to save in a BTree like struct, which can index full key +// more than short key +class ShortKeyIndexBuilder { +public: + ShortKeyIndexBuilder(uint32_t segment_id, + uint32_t num_rows_per_block) { + _footer.set_segment_id(segment_id); + _footer.set_num_rows_per_block(num_rows_per_block); + } + + Status add_item(const Slice& key); + + Status finalize(uint32_t segment_size, uint32_t num_rows, std::vector* slices); + +private: + segment_v2::ShortKeyFooterPB _footer; + + faststring _key_buf; + faststring _offset_buf; + std::string _footer_buf; +}; + +class ShortKeyIndexDecoder; + +// An Iterator to iterate one short key index. +// Client can use this class to iterator all items in this index. +class ShortKeyIndexIterator { +public: + using iterator_category = std::random_access_iterator_tag; + using value_type = Slice; + using pointer = Slice*; + using reference = Slice&; + using difference_type = ssize_t; + + ShortKeyIndexIterator(const ShortKeyIndexDecoder* decoder, uint32_t ordinal = 0) + : _decoder(decoder), _ordinal(ordinal) { } + + ShortKeyIndexIterator& operator-=(ssize_t step) { + _ordinal -= step; + return *this; + } + + ShortKeyIndexIterator& operator+=(ssize_t step) { + _ordinal += step; + return *this; + } + + ShortKeyIndexIterator& operator++() { + _ordinal++; + return *this; + } + + bool operator!=(const ShortKeyIndexIterator& other) { + return _ordinal != other._ordinal || _decoder != other._decoder; + } + + bool operator==(const ShortKeyIndexIterator& other) { + return _ordinal == other._ordinal && _decoder == other._decoder; + } + + ssize_t operator-(const ShortKeyIndexIterator& other) const { + return _ordinal - other._ordinal; + } + + inline bool valid() const; + + Slice operator*() const; + + ssize_t ordinal() const { return _ordinal; } + +private: + const ShortKeyIndexDecoder* _decoder; + ssize_t _ordinal; +}; + +// Used to decode short key to header and encoded index data. +// Usage: +// MemIndex index; +// ShortKeyIndexDecoder decoder(slice) +// decoder.parse(); +// auto iter = decoder.lower_bound(key); +class ShortKeyIndexDecoder { +public: + // Client should assure that data is available when this class + // is used. + ShortKeyIndexDecoder(const Slice& data) : _data(data) { } + + Status parse(); + + ShortKeyIndexIterator begin() const { return {this, 0}; } + ShortKeyIndexIterator end() const { return {this, num_items()}; } + + // Return an iterator which locates at the first item who is + // equal with or greater than the given key. + // NOTE: If one key is the prefix of other key, this funciton thinks + // that longer key is greater than the shorter key. + ShortKeyIndexIterator lower_bound(const Slice& key) const { + return seek(key); + } + + // Return the iterator which locates the first item greater than the + // input key. + ShortKeyIndexIterator upper_bound(const Slice& key) const { + return seek(key); + } + + uint32_t num_items() const { return _footer.num_items(); } + + Slice key(ssize_t ordinal) const { + DCHECK(ordinal >= 0 && ordinal < num_items()); + return {_key_data.data + _offsets[ordinal], _offsets[ordinal + 1] - _offsets[ordinal]}; + } + +private: + template + ShortKeyIndexIterator seek(const Slice& key) const { + auto comparator = [this] (const Slice& lhs, const Slice& rhs) { + return lhs.compare(rhs) < 0; + }; + if (lower_bound) { + return std::lower_bound(begin(), end(), key, comparator); + } else { + return std::upper_bound(begin(), end(), key, comparator); + } + } + +private: + Slice _data; + + // All following fields are only valid after parse has been executed successfully + segment_v2::ShortKeyFooterPB _footer; + std::vector _offsets; + Slice _key_data; +}; + +inline Slice ShortKeyIndexIterator::operator*() const { + return _decoder->key(_ordinal); +} + +inline bool ShortKeyIndexIterator::valid() const { + return _ordinal >= 0 && _ordinal < _decoder->num_items(); +} + +} diff --git a/be/src/olap/storage_engine.h b/be/src/olap/storage_engine.h index 99cd6aa307c94a..f3718028834a38 100644 --- a/be/src/olap/storage_engine.h +++ b/be/src/olap/storage_engine.h @@ -43,7 +43,6 @@ #include "olap/tablet.h" #include "olap/olap_meta.h" #include "olap/options.h" -#include "olap/rowset/segment_group.h" #include "olap/tablet_manager.h" #include "olap/txn_manager.h" #include "olap/task/engine_task.h" @@ -299,7 +298,6 @@ class StorageEngine { static StorageEngine* _s_instance; - std::unordered_map> _gc_files; std::unordered_map _unused_rowsets; Mutex _gc_mutex; diff --git a/be/src/olap/tablet.h b/be/src/olap/tablet.h index fb34e1c381a8f1..fe26cca358c9cb 100644 --- a/be/src/olap/tablet.h +++ b/be/src/olap/tablet.h @@ -29,7 +29,6 @@ #include "gen_cpp/olap_file.pb.h" #include "olap/olap_define.h" #include "olap/tuple.h" -#include "olap/row_cursor.h" #include "olap/rowset_graph.h" #include "olap/rowset/rowset.h" #include "olap/rowset/rowset_reader.h" diff --git a/be/src/olap/types.h b/be/src/olap/types.h index d7597d36825601..0989d1a86599ce 100644 --- a/be/src/olap/types.h +++ b/be/src/olap/types.h @@ -104,24 +104,31 @@ struct CppTypeTraits { template<> struct CppTypeTraits { using CppType = bool; + using UnsignedCppType = bool; }; template<> struct CppTypeTraits { using CppType = int8_t; + using UnsignedCppType = uint8_t; }; template<> struct CppTypeTraits { using CppType = int16_t; + using UnsignedCppType = uint16_t; }; template<> struct CppTypeTraits { using CppType = int32_t; + using UnsignedCppType = uint32_t; }; template<> struct CppTypeTraits { using CppType = uint32_t; + using UnsignedCppType = uint32_t; }; template<> struct CppTypeTraits { using CppType = int64_t; + using UnsignedCppType = uint64_t; }; template<> struct CppTypeTraits { using CppType = int128_t; + using UnsignedCppType = unsigned int128_t; }; template<> struct CppTypeTraits { using CppType = float; @@ -131,12 +138,15 @@ template<> struct CppTypeTraits { }; template<> struct CppTypeTraits { using CppType = decimal12_t; + using UnsignedCppType = decimal12_t; }; template<> struct CppTypeTraits { using CppType = uint24_t; + using UnsignedCppType = uint24_t; }; template<> struct CppTypeTraits { using CppType = int64_t; + using UnsignedCppType = uint64_t; }; template<> struct CppTypeTraits { using CppType = Slice; diff --git a/be/src/olap/uint24.h b/be/src/olap/uint24.h index 7632b584c93d01..638ffae6e6a1ae 100644 --- a/be/src/olap/uint24.h +++ b/be/src/olap/uint24.h @@ -36,19 +36,26 @@ struct uint24_t { data[2] = value.data[2]; } - uint24_t(const int32_t& value) { + uint24_t(const uint32_t& value) { data[0] = static_cast(value); data[1] = static_cast(value >> 8); data[2] = static_cast(value >> 16); } + uint24_t& operator=(const uint32_t& value) { + data[0] = static_cast(value); + data[1] = static_cast(value >> 8); + data[2] = static_cast(value >> 16); + return *this; + } + uint24_t& operator+=(const uint24_t& value) { *this = static_cast(*this) + static_cast(value); return *this; } - operator int() const { - int value = static_cast(data[0]); + operator uint32_t() const { + uint32_t value = static_cast(data[0]); value += (static_cast(static_cast(data[1]))) << 8; value += (static_cast(static_cast(data[2]))) << 16; return value; diff --git a/be/src/util/CMakeLists.txt b/be/src/util/CMakeLists.txt index d2cc2b0152aec7..05cc4983a63f06 100644 --- a/be/src/util/CMakeLists.txt +++ b/be/src/util/CMakeLists.txt @@ -78,6 +78,7 @@ set(UTIL_FILES md5.cpp frontend_helper.cpp faststring.cc + slice.cpp ) if (WITH_MYSQL) diff --git a/be/src/util/slice.cpp b/be/src/util/slice.cpp new file mode 100644 index 00000000000000..fc583a2d228bef --- /dev/null +++ b/be/src/util/slice.cpp @@ -0,0 +1,28 @@ +// 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. + +#include "util/slice.h" + +#include "util/faststring.h" + +namespace doris { + +// NOTE(zc): we define this function here to make compile work. +Slice::Slice(const faststring& s) : // NOLINT(runtime/explicit) + data((char*)(s.data())), size(s.size()) { } + +} diff --git a/be/src/util/slice.h b/be/src/util/slice.h index 5c31072fe2e1e4..1eebe3a5f29fdd 100644 --- a/be/src/util/slice.h +++ b/be/src/util/slice.h @@ -29,6 +29,8 @@ namespace doris { +class faststring; + /// @brief A wrapper around externally allocated data. /// /// Slice is a simple structure containing a pointer into some external @@ -66,6 +68,8 @@ struct Slice { /// Create a slice that refers to the contents of the given string. Slice(const std::string& s) : // NOLINT(runtime/explicit) data(const_cast(s.data())), size(s.size()) { } + + Slice(const faststring& s); /// Create a slice that refers to a C-string s[0,strlen(s)-1]. Slice(const char* s) : // NOLINT(runtime/explicit) diff --git a/be/test/olap/CMakeLists.txt b/be/test/olap/CMakeLists.txt index 0a11d868f7466d..a258e0f73e4618 100644 --- a/be/test/olap/CMakeLists.txt +++ b/be/test/olap/CMakeLists.txt @@ -53,6 +53,7 @@ ADD_BE_TEST(rowset/segment_v2/encoding_info_test) ADD_BE_TEST(rowset/segment_v2/ordinal_page_index_test) ADD_BE_TEST(rowset/segment_v2/rle_page_test) ADD_BE_TEST(rowset/segment_v2/binary_dict_page_test) +ADD_BE_TEST(rowset/segment_v2/segment_test) ADD_BE_TEST(tablet_meta_manager_test) ADD_BE_TEST(tablet_mgr_test) ADD_BE_TEST(rowset/rowset_meta_manager_test) @@ -61,3 +62,5 @@ ADD_BE_TEST(rowset/alpha_rowset_test) ADD_BE_TEST(olap_snapshot_converter_test) ADD_BE_TEST(txn_manager_test) ADD_BE_TEST(generic_iterators_test) +ADD_BE_TEST(key_coder_test) +ADD_BE_TEST(short_key_index_test) diff --git a/be/test/olap/key_coder_test.cpp b/be/test/olap/key_coder_test.cpp new file mode 100644 index 00000000000000..9c6ddd6c34076a --- /dev/null +++ b/be/test/olap/key_coder_test.cpp @@ -0,0 +1,287 @@ +// 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. + +#include "olap/key_coder.h" + +#include +#include +#include + +#include "util/debug_util.h" + +namespace doris { + +class KeyCoderTest : public testing::Test { +public: + KeyCoderTest() { } + virtual ~KeyCoderTest() { + } +}; + +template +void test_integer_encode() { + using CppType = typename CppTypeTraits::CppType; + + auto key_coder = get_key_coder(type); + + { + std::string buf; + CppType val = std::numeric_limits::min(); + key_coder->encode_ascending(&val, 1, &buf); + + std::string result; + for (int i = 0; i < sizeof(CppType); ++i) { + result.append("00"); + } + + ASSERT_STREQ(result.c_str(), hexdump(buf.data(), buf.size()).c_str()); + + { + Slice slice(buf); + CppType check_val; + key_coder->decode_ascending(&slice, sizeof(CppType), (uint8_t*)&check_val, nullptr); + ASSERT_EQ(val, check_val); + } + } + + { + std::string buf; + CppType val = std::numeric_limits::max(); + key_coder->encode_ascending(&val, sizeof(CppType), &buf); + + std::string result; + for (int i = 0; i < sizeof(CppType); ++i) { + result.append("FF"); + } + + ASSERT_STREQ(result.c_str(), hexdump(buf.data(), buf.size()).c_str()); + { + Slice slice(buf); + CppType check_val; + key_coder->decode_ascending(&slice, sizeof(CppType), (uint8_t*)&check_val, nullptr); + ASSERT_EQ(val, check_val); + } + } + + for (auto i = 0; i < 100; ++i) { + CppType val1 = random(); + CppType val2 = random(); + + std::string buf1; + std::string buf2; + + key_coder->encode_ascending(&val1, sizeof(CppType), &buf1); + key_coder->encode_ascending(&val2, sizeof(CppType), &buf2); + + if (val1 < val2) { + ASSERT_TRUE(memcmp(buf1.c_str(), buf2.c_str(), buf1.size()) < 0); + } else if (val1 > val2) { + ASSERT_TRUE(memcmp(buf1.c_str(), buf2.c_str(), buf1.size()) > 0); + } else { + ASSERT_TRUE(memcmp(buf1.c_str(), buf2.c_str(), buf1.size()) == 0); + } + } +} + +TEST(KeyCoderTest, test_int) { + test_integer_encode(); + test_integer_encode(); + test_integer_encode(); + test_integer_encode(); + test_integer_encode(); + test_integer_encode(); + + test_integer_encode(); +} + +TEST(KeyCoderTest, test_date) { + using CppType = uint24_t; + auto key_coder = get_key_coder(OLAP_FIELD_TYPE_DATE); + + { + std::string buf; + CppType val = 0; + key_coder->encode_ascending(&val, 1, &buf); + + std::string result; + for (int i = 0; i < sizeof(uint24_t); ++i) { + result.append("00"); + } + + ASSERT_STREQ(result.c_str(), hexdump(buf.data(), buf.size()).c_str()); + + { + Slice slice(buf); + CppType check_val; + key_coder->decode_ascending(&slice, sizeof(CppType), (uint8_t*)&check_val, nullptr); + ASSERT_EQ(val, check_val); + } + } + + { + std::string buf; + CppType val = 10000; + key_coder->encode_ascending(&val, sizeof(CppType), &buf); + + std::string result("002710"); + + ASSERT_STREQ(result.c_str(), hexdump(buf.data(), buf.size()).c_str()); + { + Slice slice(buf); + CppType check_val; + key_coder->decode_ascending(&slice, sizeof(CppType), (uint8_t*)&check_val, nullptr); + ASSERT_EQ(val, check_val); + } + } + + for (auto i = 0; i < 100; ++i) { + CppType val1 = random(); + CppType val2 = random(); + + std::string buf1; + std::string buf2; + + key_coder->encode_ascending(&val1, sizeof(CppType), &buf1); + key_coder->encode_ascending(&val2, sizeof(CppType), &buf2); + + if (val1 < val2) { + ASSERT_TRUE(memcmp(buf1.c_str(), buf2.c_str(), buf1.size()) < 0); + } else if (val1 > val2) { + ASSERT_TRUE(memcmp(buf1.c_str(), buf2.c_str(), buf1.size()) > 0); + } else { + ASSERT_TRUE(memcmp(buf1.c_str(), buf2.c_str(), buf1.size()) == 0); + } + } +} + +TEST(KeyCoderTest, test_decimal) { + auto key_coder = get_key_coder(OLAP_FIELD_TYPE_DECIMAL); + + decimal12_t val1(1, 100000000); + std::string buf1; + + key_coder->encode_ascending(&val1, sizeof(decimal12_t), &buf1); + + decimal12_t check_val; + Slice slice1(buf1); + key_coder->decode_ascending(&slice1, sizeof(decimal12_t), (uint8_t*)&check_val, nullptr); + ASSERT_EQ(check_val, val1); + + { + decimal12_t val2(-1, -100000000); + std::string buf2; + key_coder->encode_ascending(&val2, sizeof(decimal12_t), &buf2); + ASSERT_TRUE(memcmp(buf1.c_str(), buf2.c_str(), buf1.size()) > 0); + } + { + decimal12_t val2(1, 100000001); + std::string buf2; + key_coder->encode_ascending(&val2, sizeof(decimal12_t), &buf2); + ASSERT_TRUE(memcmp(buf1.c_str(), buf2.c_str(), buf1.size()) < 0); + } + { + decimal12_t val2(0, 0); + std::string buf2; + key_coder->encode_ascending(&val2, sizeof(decimal12_t), &buf2); + ASSERT_TRUE(memcmp(buf1.c_str(), buf2.c_str(), buf1.size()) > 0); + + std::string result("80"); + for (int i = 0; i < sizeof(int64_t) - 1; ++i) { + result.append("00"); + } + result.append("80"); + for (int i = 0; i < sizeof(int32_t) - 1; ++i) { + result.append("00"); + } + + ASSERT_STREQ(result.c_str(), hexdump(buf2.data(), buf2.size()).c_str()); + } +} + +TEST(KeyCoderTest, test_char) { + auto key_coder = get_key_coder(OLAP_FIELD_TYPE_CHAR); + + char buf[] = "1234567890"; + Slice slice(buf, 10); + + { + std::string key; + key_coder->encode_ascending(&slice, 10, &key); + Slice encoded_key(key); + + Arena arena; + Slice check_slice; + auto st = key_coder->decode_ascending(&encoded_key, 10, (uint8_t*)&check_slice, &arena); + ASSERT_TRUE(st.ok()); + + ASSERT_STREQ("1234567890", check_slice.data); + } + + { + std::string key; + key_coder->encode_ascending(&slice, 5, &key); + Slice encoded_key(key); + + Arena arena; + Slice check_slice; + auto st = key_coder->decode_ascending(&encoded_key, 5, (uint8_t*)&check_slice, &arena); + ASSERT_TRUE(st.ok()); + + ASSERT_STREQ("12345", check_slice.data); + } +} + +TEST(KeyCoderTest, test_varchar) { + auto key_coder = get_key_coder(OLAP_FIELD_TYPE_VARCHAR); + + char buf[] = "1234567890"; + Slice slice(buf, 10); + + { + std::string key; + key_coder->encode_ascending(&slice, 15, &key); + Slice encoded_key(key); + + Arena arena; + Slice check_slice; + auto st = key_coder->decode_ascending(&encoded_key, 15, (uint8_t*)&check_slice, &arena); + ASSERT_TRUE(st.ok()); + + ASSERT_STREQ("1234567890", check_slice.data); + } + + { + std::string key; + key_coder->encode_ascending(&slice, 5, &key); + Slice encoded_key(key); + + Arena arena; + Slice check_slice; + auto st = key_coder->decode_ascending(&encoded_key, 5, (uint8_t*)&check_slice, &arena); + ASSERT_TRUE(st.ok()); + + ASSERT_STREQ("12345", check_slice.data); + } +} + + +} // namespace doris + +int main(int argc, char **argv) { + testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/be/test/olap/rowset/segment_v2/segment_test.cpp b/be/test/olap/rowset/segment_v2/segment_test.cpp new file mode 100644 index 00000000000000..c5c5e71a2c3d00 --- /dev/null +++ b/be/test/olap/rowset/segment_v2/segment_test.cpp @@ -0,0 +1,259 @@ +// 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. + +#include "olap/rowset/segment_v2/segment.h" +#include "olap/rowset/segment_v2/segment_writer.h" +#include "olap/rowset/segment_v2/segment_iterator.h" + +#include +#include + +#include "common/logging.h" +#include "olap/olap_common.h" +#include "olap/row_cursor.h" +#include "olap/tablet_schema.h" +#include "olap/row_block.h" +#include "olap/row_block2.h" +#include "olap/types.h" +#include "olap/tablet_schema_helper.h" +#include "util/file_utils.h" + +namespace doris { +namespace segment_v2 { + +class SegmentReaderWriterTest : public testing::Test { +public: + SegmentReaderWriterTest() { } + virtual ~SegmentReaderWriterTest() { + } +}; + +TEST_F(SegmentReaderWriterTest, normal) { + + size_t num_rows_per_block = 10; + + std::shared_ptr tablet_schema(new TabletSchema()); + tablet_schema->_num_columns = 4; + tablet_schema->_num_key_columns = 3; + tablet_schema->_num_short_key_columns = 2; + tablet_schema->_num_rows_per_row_block = num_rows_per_block; + tablet_schema->_cols.push_back(create_int_key(1)); + tablet_schema->_cols.push_back(create_int_key(2)); + tablet_schema->_cols.push_back(create_int_key(3)); + tablet_schema->_cols.push_back(create_int_value(4)); + + // segment write + std::string dname = "./ut_dir/segment_test"; + FileUtils::create_dir(dname); + + SegmentWriterOptions opts; + opts.num_rows_per_block = num_rows_per_block; + + std::string fname = dname + "/int_case"; + SegmentWriter writer(fname, 0, tablet_schema, opts); + auto st = writer.init(10); + ASSERT_TRUE(st.ok()); + + RowCursor row; + auto olap_st = row.init(*tablet_schema); + ASSERT_EQ(OLAP_SUCCESS, olap_st); + + // 0, 1, 2, 3 + // 10, 11, 12, 13 + // 20, 21, 22, 23 + for (int i = 0; i < 4096; ++i) { + for (int j = 0; j < 4; ++j) { + auto cell = row.cell(j); + cell.set_not_null(); + *(int*)cell.mutable_cell_ptr() = i * 10 + j; + } + writer.append_row(row); + } + + uint32_t file_size = 0; + st = writer.finalize(&file_size); + ASSERT_TRUE(st.ok()); + // reader + { + std::shared_ptr segment(new Segment(fname, 0, tablet_schema, num_rows_per_block)); + st = segment->open(); + LOG(INFO) << "segment open, msg=" << st.to_string(); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(4096, segment->num_rows()); + Schema schema(*tablet_schema); + // scan all rows + { + std::unique_ptr iter; + st = segment->new_iterator(schema, &iter); + ASSERT_TRUE(st.ok()); + + StorageReadOptions read_opts; + st = iter->init(read_opts); + ASSERT_TRUE(st.ok()); + + Arena arena; + RowBlockV2 block(schema, 1024, &arena); + + int left = 4096; + + int rowid = 0; + while (left > 0) { + int rows_read = left > 1024 ? 1024 : left; + st = iter->next_batch(&block); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(rows_read, block.num_rows()); + left -= rows_read; + + for (int j = 0; j < block.schema()->column_ids().size(); ++j) { + auto cid = block.schema()->column_ids()[j]; + auto column_block = block.column_block(j); + for (int i = 0; i < rows_read; ++i) { + int rid = rowid + i; + ASSERT_FALSE(BitmapTest(column_block.null_bitmap(), i)); + ASSERT_EQ(rid * 10 + cid, *(int*)column_block.cell_ptr(i)); + } + } + rowid += rows_read; + } + } + // test seek, key + { + std::unique_ptr iter; + st = segment->new_iterator(schema, &iter); + ASSERT_TRUE(st.ok()); + + // lower bound + StorageReadOptions read_opts; + read_opts.lower_bound.reset(new RowCursor()); + RowCursor* lower_bound = read_opts.lower_bound.get(); + lower_bound->init(*tablet_schema, 2); + { + auto cell = lower_bound->cell(0); + cell.set_not_null(); + *(int*)cell.mutable_cell_ptr() = 100; + } + { + auto cell = lower_bound->cell(1); + cell.set_not_null(); + *(int*)cell.mutable_cell_ptr() = 100; + } + read_opts.include_lower_bound = false; + + // upper bound + read_opts.upper_bound.reset(new RowCursor()); + RowCursor* upper_bound = read_opts.upper_bound.get(); + upper_bound->init(*tablet_schema, 1); + { + auto cell = upper_bound->cell(0); + cell.set_not_null(); + *(int*)cell.mutable_cell_ptr() = 200; + } + read_opts.include_upper_bound = true; + + st = iter->init(read_opts); + LOG(INFO) << "iterator init msg=" << st.to_string(); + ASSERT_TRUE(st.ok()); + + Arena arena; + RowBlockV2 block(schema, 100, &arena); + st = iter->next_batch(&block); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(11, block.num_rows()); + auto column_block = block.column_block(0); + for (int i = 0; i < 11; ++i) { + ASSERT_EQ(100 + i * 10, *(int*)column_block.cell_ptr(i)); + } + } + // test seek, key + { + std::unique_ptr iter; + st = segment->new_iterator(schema, &iter); + ASSERT_TRUE(st.ok()); + + StorageReadOptions read_opts; + + // lower bound + read_opts.lower_bound.reset(new RowCursor()); + RowCursor* lower_bound = read_opts.lower_bound.get(); + lower_bound->init(*tablet_schema, 1); + { + auto cell = lower_bound->cell(0); + cell.set_not_null(); + *(int*)cell.mutable_cell_ptr() = 40970; + } + read_opts.include_lower_bound = false; + + st = iter->init(read_opts); + LOG(INFO) << "iterator init msg=" << st.to_string(); + ASSERT_TRUE(st.ok()); + + Arena arena; + RowBlockV2 block(schema, 100, &arena); + st = iter->next_batch(&block); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(0, block.num_rows()); + } + // test seek, key (-2, -1) + { + std::unique_ptr iter; + st = segment->new_iterator(schema, &iter); + ASSERT_TRUE(st.ok()); + + StorageReadOptions read_opts; + + // lower bound + read_opts.lower_bound.reset(new RowCursor()); + RowCursor* lower_bound = read_opts.lower_bound.get(); + lower_bound->init(*tablet_schema, 1); + { + auto cell = lower_bound->cell(0); + cell.set_not_null(); + *(int*)cell.mutable_cell_ptr() = -2; + } + read_opts.include_lower_bound = false; + + read_opts.upper_bound.reset(new RowCursor()); + RowCursor* upper_bound = read_opts.upper_bound.get(); + upper_bound->init(*tablet_schema, 1); + { + auto cell = upper_bound->cell(0); + cell.set_not_null(); + *(int*)cell.mutable_cell_ptr() = -1; + } + read_opts.include_upper_bound = false; + + st = iter->init(read_opts); + LOG(INFO) << "iterator init msg=" << st.to_string(); + ASSERT_TRUE(st.ok()); + + Arena arena; + RowBlockV2 block(schema, 100, &arena); + st = iter->next_batch(&block); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(0, block.num_rows()); + } + } +} + +} +} + +int main(int argc, char** argv) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} + diff --git a/be/test/olap/short_key_index_test.cpp b/be/test/olap/short_key_index_test.cpp new file mode 100644 index 00000000000000..1b825e9320292c --- /dev/null +++ b/be/test/olap/short_key_index_test.cpp @@ -0,0 +1,161 @@ +// 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. + +#include "olap/short_key_index.h" + +#include + +#include "olap/tablet_schema_helper.h" +#include "olap/row_cursor.h" +#include "util/debug_util.h" + +namespace doris { + +class ShortKeyIndexTest : public testing::Test { +public: + ShortKeyIndexTest() { } + virtual ~ShortKeyIndexTest() { + } +}; + +TEST_F(ShortKeyIndexTest, buider) { + ShortKeyIndexBuilder builder(0, 1024); + + for (int i = 1000; i < 10000; i += 2) { + builder.add_item(std::to_string(i)); + } + std::vector slices; + auto st = builder.finalize(10000, 9000 * 1024, &slices); + ASSERT_TRUE(st.ok()); + + std::string buf; + for (auto& slice : slices) { + buf.append(slice.data, slice.size); + } + + ShortKeyIndexDecoder decoder(buf); + st = decoder.parse(); + ASSERT_TRUE(st.ok()); + + // find 1499 + { + auto iter = decoder.lower_bound("1499"); + ASSERT_TRUE(iter.valid()); + ASSERT_STREQ("1500", (*iter).to_string().c_str()); + } + // find 1500 lower bound + { + auto iter = decoder.lower_bound("1500"); + ASSERT_TRUE(iter.valid()); + ASSERT_STREQ("1500", (*iter).to_string().c_str()); + } + // find 1500 upper bound + { + auto iter = decoder.upper_bound("1500"); + ASSERT_TRUE(iter.valid()); + ASSERT_STREQ("1502", (*iter).to_string().c_str()); + } + // find prefix "87" + { + auto iter = decoder.lower_bound("87"); + ASSERT_TRUE(iter.valid()); + ASSERT_STREQ("8700", (*iter).to_string().c_str()); + } + // find prefix "87" + { + auto iter = decoder.upper_bound("87"); + ASSERT_TRUE(iter.valid()); + ASSERT_STREQ("8700", (*iter).to_string().c_str()); + } + + // find prefix "9999" + { + auto iter = decoder.upper_bound("9999"); + ASSERT_FALSE(iter.valid()); + } +} + + +TEST_F(ShortKeyIndexTest, enocde) { + TabletSchema tablet_schema; + tablet_schema._cols.push_back(create_int_key(0)); + tablet_schema._cols.push_back(create_int_key(1)); + tablet_schema._cols.push_back(create_int_key(2)); + tablet_schema._cols.push_back(create_int_value(3)); + tablet_schema._num_columns = 4; + tablet_schema._num_key_columns = 3; + tablet_schema._num_short_key_columns = 3; + + // test encoding with padding + { + RowCursor row; + row.init(tablet_schema, 2); + + { + // test padding + { + auto cell = row.cell(0); + cell.set_is_null(false); + *(int*)cell.mutable_cell_ptr() = 12345; + } + { + auto cell = row.cell(1); + cell.set_is_null(false); + *(int*)cell.mutable_cell_ptr() = 54321; + } + std::string buf; + encode_key_with_padding(&buf, row, 3, true); + // should be \x02\x80\x00\x30\x39\x02\x80\x00\xD4\x31\x00 + ASSERT_STREQ("0280003039028000D43100", hexdump(buf.c_str(), buf.size()).c_str()); + } + // test with null + { + { + auto cell = row.cell(0); + cell.set_is_null(false); + *(int*)cell.mutable_cell_ptr() = 54321; + } + { + auto cell = row.cell(1); + cell.set_is_null(true); + *(int*)cell.mutable_cell_ptr() = 54321; + } + + { + std::string buf; + encode_key_with_padding(&buf, row, 3, false); + // should be \x02\x80\x00\xD4\x31\x01\xff + ASSERT_STREQ("028000D43101FF", hexdump(buf.c_str(), buf.size()).c_str()); + } + // encode key + { + std::string buf; + encode_key(&buf, row, 2); + // should be \x02\x80\x00\xD4\x31\x01 + ASSERT_STREQ("028000D43101", hexdump(buf.c_str(), buf.size()).c_str()); + } + } + } +} + +} + +int main(int argc, char** argv) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} + diff --git a/be/test/olap/tablet_schema_helper.h b/be/test/olap/tablet_schema_helper.h new file mode 100644 index 00000000000000..01f43bc0eb7a6e --- /dev/null +++ b/be/test/olap/tablet_schema_helper.h @@ -0,0 +1,54 @@ +// 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. + +#pragma once + +#include +#include "olap/tablet_schema.h" + +namespace doris { + +TabletColumn create_int_key(int32_t id, bool is_nullable = true) { + TabletColumn column; + column._unique_id = id; + column._col_name = std::to_string(id); + column._type = OLAP_FIELD_TYPE_INT; + column._is_key = true; + column._is_nullable = is_nullable; + column._length = 4; + column._index_length = 4; + return column; +} + + +TabletColumn create_int_value( + int32_t id, + FieldAggregationMethod agg_method = OLAP_FIELD_AGGREGATION_SUM, + bool is_nullable = true) { + TabletColumn column; + column._unique_id = id; + column._col_name = std::to_string(id); + column._type = OLAP_FIELD_TYPE_INT; + column._is_key = false; + column._aggregation = agg_method; + column._is_nullable = is_nullable; + column._length = 4; + column._index_length = 4; + return column; +} + +} diff --git a/gensrc/proto/segment_v2.proto b/gensrc/proto/segment_v2.proto index c66c2d88e023be..edc7b23ff832d5 100644 --- a/gensrc/proto/segment_v2.proto +++ b/gensrc/proto/segment_v2.proto @@ -72,17 +72,21 @@ message ZoneMapPB { } message ColumnMetaPB { + // column id in table schema + optional uint32 column_id = 1; + // unique column id + optional uint32 unique_id = 2; // this field is FieldType's value - optional int32 type = 1; - optional EncodingTypePB encoding = 2; + optional int32 type = 3; + optional EncodingTypePB encoding = 4; // compress type for column - optional CompressionTypePB compression = 3; + optional CompressionTypePB compression = 5; // if this column can be nullable - optional bool is_nullable = 4; + optional bool is_nullable = 6; // if this column has checksum for each page - optional bool has_checksum = 5; + optional bool has_checksum = 7; // ordinal index page - optional PagePointerPB ordinal_index_page = 6; + optional PagePointerPB ordinal_index_page = 8; // // dictionary page for DICT_ENCODING // optional PagePointerPB dict_page = 2; @@ -117,3 +121,36 @@ message FileFooterPB { repeated MetadataPairPB file_meta_datas = 8; // meta data of file optional PagePointerPB key_index_page = 9; // short key index page } + +message ShortKeyFooterPB { + // How many index item in this index. + optional uint32 num_items = 1; + // The total bytes occupied by the index key + optional uint32 key_bytes = 2; + // The total bytes occupied by the key offsets + optional uint32 offset_bytes = 3; + // Segment id which this index is belong to + optional uint32 segment_id = 4; + // number rows in each block + optional uint32 num_rows_per_block = 5; + // How many rows in this segment + optional uint32 num_segment_rows = 6; + // Total bytes for this segment + optional uint32 segment_bytes = 7; +} + +message SegmentFooterPB { + optional uint32 version = 1 [default = 1]; // file version + repeated ColumnMetaPB columns = 2; // tablet schema + optional uint64 num_rows = 3; // number of values + optional uint64 index_footprint = 4; // total idnex footprint of all columns + optional uint64 data_footprint = 5; // total data footprint of all columns + optional uint64 raw_data_footprint = 6; // raw data footprint + + optional CompressionTypePB compress_type = 7 [default = LZ4]; // default compression type for file columns + repeated MetadataPairPB file_meta_datas = 8; // meta data of file + + // Short key index's page + optional PagePointerPB short_key_index_page = 9; +} + diff --git a/run-ut.sh b/run-ut.sh index 5c3a4a4bc6b721..fdf7b32741de64 100755 --- a/run-ut.sh +++ b/run-ut.sh @@ -248,10 +248,13 @@ ${DORIS_TEST_BINARY_DIR}/olap/rowset/segment_v2/binary_plain_page_test ${DORIS_TEST_BINARY_DIR}/olap/rowset/segment_v2/column_reader_writer_test ${DORIS_TEST_BINARY_DIR}/olap/rowset/segment_v2/rle_page_test ${DORIS_TEST_BINARY_DIR}/olap/rowset/segment_v2/binary_dict_page_test +${DORIS_TEST_BINARY_DIR}/olap/rowset/segment_v2/segment_test ${DORIS_TEST_BINARY_DIR}/olap/txn_manager_test ${DORIS_TEST_BINARY_DIR}/olap/storage_types_test ${DORIS_TEST_BINARY_DIR}/olap/generic_iterators_test ${DORIS_TEST_BINARY_DIR}/olap/aggregate_func_test +${DORIS_TEST_BINARY_DIR}/olap/short_key_index_test +${DORIS_TEST_BINARY_DIR}/olap/key_coder_test # Running routine load test ${DORIS_TEST_BINARY_DIR}/runtime/kafka_consumer_pipe_test From 343b913f0d1aeca66a86db1e01979dd4e52578b7 Mon Sep 17 00:00:00 2001 From: Mingyu Chen Date: Tue, 6 Aug 2019 19:23:53 +0800 Subject: [PATCH 13/26] Fix a serious bug that will cause all replicas being deleted. (#1589) Revert commit: eda55a7394fcec2f7b6c0aefd1628f9d63911815 --- .../apache/doris/master/ReportHandler.java | 29 ++++++++++--------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/fe/src/main/java/org/apache/doris/master/ReportHandler.java b/fe/src/main/java/org/apache/doris/master/ReportHandler.java index b2e85ceacf20be..a040ad70a70d35 100644 --- a/fe/src/main/java/org/apache/doris/master/ReportHandler.java +++ b/fe/src/main/java/org/apache/doris/master/ReportHandler.java @@ -631,19 +631,22 @@ private static void deleteFromBackend(Map backendTablets, for (Long tabletId : backendTablets.keySet()) { TTablet backendTablet = backendTablets.get(tabletId); for (TTabletInfo backendTabletInfo : backendTablet.getTablet_infos()) { - boolean needDelete = true; - if (!foundTabletsWithValidSchema.contains(tabletId) - && isBackendReplicaHealthy(backendTabletInfo)) { - // if this tablet is not in meta. try adding it. - // if add failed. delete this tablet from backend. - try { - addReplica(tabletId, backendTabletInfo, backendId); - // update counter - needDelete = false; - ++addToMetaCounter; - } catch (MetaNotFoundException e) { - LOG.warn("failed add to meta. tablet[{}], backend[{}]. {}", - tabletId, backendId, e.getMessage()); + boolean needDelete = false; + if (!foundTabletsWithValidSchema.contains(tabletId)) { + if (isBackendReplicaHealthy(backendTabletInfo)) { + // if this tablet is not in meta. try adding it. + // if add failed. delete this tablet from backend. + try { + addReplica(tabletId, backendTabletInfo, backendId); + // update counter + needDelete = false; + ++addToMetaCounter; + } catch (MetaNotFoundException e) { + LOG.warn("failed add to meta. tablet[{}], backend[{}]. {}", + tabletId, backendId, e.getMessage()); + needDelete = true; + } + } else { needDelete = true; } } From f7a05d85802a3bd79677b4194b539eeb499d7c98 Mon Sep 17 00:00:00 2001 From: Youngwb Date: Wed, 7 Aug 2019 09:25:26 +0800 Subject: [PATCH 14/26] Support setting timezone variable in FE (#1587) --- .../org/apache/doris/common/ErrorCode.java | 1 + .../apache/doris/common/util/TimeUtils.java | 36 +++++++++++++++++ .../org/apache/doris/qe/GlobalVariable.java | 4 +- .../java/org/apache/doris/qe/VariableMgr.java | 6 +++ .../org/apache/doris/qe/VariableMgrTest.java | 39 +++++++++++++++++++ 5 files changed, 85 insertions(+), 1 deletion(-) diff --git a/fe/src/main/java/org/apache/doris/common/ErrorCode.java b/fe/src/main/java/org/apache/doris/common/ErrorCode.java index 88311987b6e39f..67cf454b2a5201 100644 --- a/fe/src/main/java/org/apache/doris/common/ErrorCode.java +++ b/fe/src/main/java/org/apache/doris/common/ErrorCode.java @@ -81,6 +81,7 @@ public enum ErrorCode { ERR_NOT_SUPPORTED_AUTH_MODE(1251, new byte[] {'0', '8', '0', '0', '4'}, "Client does not support authentication protocol requested by server; consider upgrading MySQL client"), ERR_UNKNOWN_STORAGE_ENGINE(1286, new byte[] {'4', '2', '0', '0', '0'}, "Unknown storage engine '%s'"), + ERR_UNKNOWN_TIME_ZONE(1298, new byte[] {'H', 'Y', '0', '0', '0'}, "Unknown or incorrect time zone: '%s'"), ERR_WRONG_OBJECT(1347, new byte[] {'H', 'Y', '0', '0', '0'}, "'%s'.'%s' is not '%s'"), ERR_VIEW_WRONG_LIST(1353, new byte[] {'H', 'Y', '0', '0', '0'}, "View's SELECT and view's field list have different column counts"), diff --git a/fe/src/main/java/org/apache/doris/common/util/TimeUtils.java b/fe/src/main/java/org/apache/doris/common/util/TimeUtils.java index 7adae22a674572..fa15d0a1e40ec3 100644 --- a/fe/src/main/java/org/apache/doris/common/util/TimeUtils.java +++ b/fe/src/main/java/org/apache/doris/common/util/TimeUtils.java @@ -22,13 +22,19 @@ import org.apache.doris.common.AnalysisException; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import java.text.ParseException; import java.text.ParsePosition; import java.text.SimpleDateFormat; +import java.time.DateTimeException; +import java.time.ZoneId; import java.util.Calendar; import java.util.Date; import java.util.SimpleTimeZone; @@ -42,6 +48,9 @@ public class TimeUtils { private static final TimeZone TIME_ZONE; + // set CST to +08:00 instead of America/Chicago + public static final ImmutableMap timeZoneAliasMap = ImmutableMap.of("CST", "Asia/Shanghai"); + // NOTICE: Date formats are not synchronized. // it must be used as synchronized externally. private static final SimpleDateFormat DATE_FORMAT; @@ -56,6 +65,8 @@ public class TimeUtils { + "[\\-\\/\\s]?((0?[1-9])|([1-2][0-9])|(3[01])))|(((0?[469])|(11))[\\-\\/\\s]?" + "((0?[1-9])|([1-2][0-9])|(30)))|(0?2[\\-\\/\\s]?((0?[1-9])|(1[0-9])|(2[0-8]))))))" + "(\\s(((0?[0-9])|([1][0-9])|([2][0-3]))\\:([0-5]?[0-9])((\\s)|(\\:([0-5]?[0-9])))))?$"); + + private static final Pattern TIMEZONE_OFFSET_FORMAT_REG = Pattern.compile("^[+-]{1}\\d{2}\\:\\d{2}$"); public static Date MIN_DATE = null; public static Date MAX_DATE = null; @@ -205,4 +216,29 @@ public static long timeStringToLong(String timeStr) { } return d.getTime(); } + + // Check if the time zone_value is valid + public static void checkTimeZoneValid(String value) throws DdlException { + try { + // match offset type, such as +08:00, -07:00 + Matcher matcher = TIMEZONE_OFFSET_FORMAT_REG.matcher(value); + // it supports offset and region timezone type, "CST" use here is compatibility purposes. + boolean match = matcher.matches(); + if (!value.contains("/") && !value.equals("CST") && !match) { + ErrorReport.reportDdlException(ErrorCode.ERR_UNKNOWN_TIME_ZONE, value); + } + if (match) { + // timezone offsets around the world extended from -12:00 to +14:00 + int tz = Integer.parseInt(value.substring(1, 3)) * 100 + Integer.parseInt(value.substring(4, 6)); + if (value.charAt(0) == '-' && tz > 1200) { + ErrorReport.reportDdlException(ErrorCode.ERR_UNKNOWN_TIME_ZONE, value); + } else if (value.charAt(0) == '+' && tz > 1400) { + ErrorReport.reportDdlException(ErrorCode.ERR_UNKNOWN_TIME_ZONE, value); + } + } + ZoneId.of(value, timeZoneAliasMap); + } catch (DateTimeException ex) { + ErrorReport.reportDdlException(ErrorCode.ERR_UNKNOWN_TIME_ZONE, value); + } + } } diff --git a/fe/src/main/java/org/apache/doris/qe/GlobalVariable.java b/fe/src/main/java/org/apache/doris/qe/GlobalVariable.java index 6b8bd8fcc20941..70105aed32fd55 100644 --- a/fe/src/main/java/org/apache/doris/qe/GlobalVariable.java +++ b/fe/src/main/java/org/apache/doris/qe/GlobalVariable.java @@ -19,6 +19,8 @@ import org.apache.doris.common.Version; +import java.time.ZoneId; + // You can place your global variable in this class with public and VariableMgr.VarAttr annotation. // You can get this variable from MySQL client with statement `SELECT @@variable_name`, // and change its value through `SET variable_name = xxx` @@ -48,7 +50,7 @@ public final class GlobalVariable { // A string to be executed by the server for each client that connects @VariableMgr.VarAttr(name = "system_time_zone", flag = VariableMgr.READ_ONLY) - private static String systemTimeZone = "CST"; + public static String systemTimeZone = ZoneId.systemDefault().normalized().toString(); // The amount of memory allocated for caching query results @VariableMgr.VarAttr(name = "query_cache_size") diff --git a/fe/src/main/java/org/apache/doris/qe/VariableMgr.java b/fe/src/main/java/org/apache/doris/qe/VariableMgr.java index 31579feb2f5f43..8754103cac9143 100644 --- a/fe/src/main/java/org/apache/doris/qe/VariableMgr.java +++ b/fe/src/main/java/org/apache/doris/qe/VariableMgr.java @@ -27,6 +27,7 @@ import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.PatternMatcher; +import org.apache.doris.common.util.TimeUtils; import org.apache.doris.persist.EditLog; import com.google.common.collect.ImmutableMap; @@ -203,6 +204,7 @@ private static void checkUpdate(SetVar setVar, int flag) throws DdlException { } } + // Get from show name to field public static void setVar(SessionVariable sessionVariable, SetVar setVar) throws DdlException { VarContext ctx = ctxByVarName.get(setVar.getVariable()); @@ -211,6 +213,10 @@ public static void setVar(SessionVariable sessionVariable, SetVar setVar) throws } // Check variable attribute and setVar checkUpdate(setVar, ctx.getFlag()); + // Check variable time_zone value is valid + if (setVar.getVariable().toLowerCase().equals("time_zone")) { + TimeUtils.checkTimeZoneValid(setVar.getValue().getStringValue()); + } // To modify to default value. VarAttr attr = ctx.getField().getAnnotation(VarAttr.class); diff --git a/fe/src/test/java/org/apache/doris/qe/VariableMgrTest.java b/fe/src/test/java/org/apache/doris/qe/VariableMgrTest.java index 5053e7d9f08a31..355440cf94f74d 100644 --- a/fe/src/test/java/org/apache/doris/qe/VariableMgrTest.java +++ b/fe/src/test/java/org/apache/doris/qe/VariableMgrTest.java @@ -99,11 +99,21 @@ public void testNormal() throws IllegalAccessException, DdlException, NoSuchFiel var = VariableMgr.newSessionVariable(); Assert.assertEquals(5L, var.getParallelExecInstanceNum()); + SetVar setVar3 = new SetVar(SetType.GLOBAL, "time_zone", new StringLiteral("Asia/Shanghai")); + VariableMgr.setVar(var, setVar3); + Assert.assertEquals("CST", var.getTimeZone()); + var = VariableMgr.newSessionVariable(); + Assert.assertEquals("Asia/Shanghai", var.getTimeZone()); + // Set session variable setVar = new SetVar(SetType.GLOBAL, "exec_mem_limit", new IntLiteral(1234L)); VariableMgr.setVar(var, setVar); Assert.assertEquals(1234L, var.getMaxExecMemByte()); + setVar3 = new SetVar(SetType.SESSION, "time_zone", new StringLiteral("Asia/Jakarta")); + VariableMgr.setVar(var, setVar3); + Assert.assertEquals("Asia/Jakarta", var.getTimeZone()); + // Get from name SysVariableDesc desc = new SysVariableDesc("exec_mem_limit"); Assert.assertEquals(var.getMaxExecMemByte() + "", VariableMgr.getValue(var, desc)); @@ -123,6 +133,35 @@ public void testInvalidType() throws DdlException { Assert.fail("No exception throws."); } + @Test(expected = DdlException.class) + public void testInvalidTimeZoneRegion() throws DdlException { + // Set global variable + SetVar setVar = new SetVar(SetType.GLOBAL, "time_zone", new StringLiteral("Hongkong")); + SessionVariable var = VariableMgr.newSessionVariable(); + try { + VariableMgr.setVar(var, setVar); + } catch (DdlException e) { + LOG.warn("VariableMgr throws", e); + throw e; + } + Assert.fail("No exception throws."); + } + + @Test(expected = DdlException.class) + public void testInvalidTimeZoneOffset() throws DdlException { + // Set global variable + SetVar setVar = new SetVar(SetType.GLOBAL, "time_zone", new StringLiteral("+15:00")); + SessionVariable var = VariableMgr.newSessionVariable(); + try { + VariableMgr.setVar(var, setVar); + } catch (DdlException e) { + LOG.warn("VariableMgr throws", e); + throw e; + } + Assert.fail("No exception throws."); + } + + @Test(expected = DdlException.class) public void testReadOnly() throws AnalysisException, DdlException { SysVariableDesc desc = new SysVariableDesc("version_comment"); From 9402456f5b86cd9067ea40afe1d7ac19beef0432 Mon Sep 17 00:00:00 2001 From: HangyuanLiu <460660596@qq.com> Date: Wed, 7 Aug 2019 15:08:22 +0800 Subject: [PATCH 15/26] Fix parquet directory have empty file (#1593) --- be/src/exec/parquet_reader.cpp | 3 +++ be/src/exec/parquet_scanner.cpp | 7 ++++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/be/src/exec/parquet_reader.cpp b/be/src/exec/parquet_reader.cpp index a60295b0fab111..502fa0ef51c53f 100644 --- a/be/src/exec/parquet_reader.cpp +++ b/be/src/exec/parquet_reader.cpp @@ -55,6 +55,9 @@ Status ParquetReaderWrap::init_parquet_reader(const std::vector _file_metadata = _reader->parquet_reader()->metadata(); // initial members _total_groups = _file_metadata->num_row_groups(); + if (_total_groups == 0) { + return Status::EndOfFile("Empty Parquet File"); + } _rows_of_group = _file_metadata->RowGroup(0)->num_rows(); // map diff --git a/be/src/exec/parquet_scanner.cpp b/be/src/exec/parquet_scanner.cpp index c69d19940d1e75..2ce43a121956a1 100644 --- a/be/src/exec/parquet_scanner.cpp +++ b/be/src/exec/parquet_scanner.cpp @@ -142,7 +142,12 @@ Status ParquetScanner::open_next_reader() { continue; } _cur_file_reader = new ParquetReaderWrap(file_reader.release()); - return _cur_file_reader->init_parquet_reader(_src_slot_descs); + Status status = _cur_file_reader->init_parquet_reader(_src_slot_descs); + if (status.is_end_of_file()) { + continue; + } else { + return status; + } } } From dc4a5e6c10e040a46975f6512004540d9bd5c9f4 Mon Sep 17 00:00:00 2001 From: worker24h Date: Wed, 7 Aug 2019 06:52:23 -0500 Subject: [PATCH 16/26] Support Decimal Type when load Parquet File (#1595) --- be/src/exec/parquet_reader.cpp | 72 +++++++++++++++++++--------------- 1 file changed, 41 insertions(+), 31 deletions(-) diff --git a/be/src/exec/parquet_reader.cpp b/be/src/exec/parquet_reader.cpp index 502fa0ef51c53f..b91d14713364fd 100644 --- a/be/src/exec/parquet_reader.cpp +++ b/be/src/exec/parquet_reader.cpp @@ -170,7 +170,7 @@ Status ParquetReaderWrap::read_record_batch(const std::vector& return Status::OK(); } -Status ParquetReaderWrap::handle_timestamp(const std::shared_ptr& ts_array, uint8_t *buf, int32_t *wbtyes) { +Status ParquetReaderWrap::handle_timestamp(const std::shared_ptr& ts_array, uint8_t *buf, int32_t *wbytes) { const auto type = std::dynamic_pointer_cast(ts_array->type()); // Doris only supports seconds time_t timestamp = 0; @@ -196,13 +196,13 @@ Status ParquetReaderWrap::handle_timestamp(const std::shared_ptr& tuple_slot_descs, MemPool* mem_pool, bool* eof) { uint8_t tmp_buf[128] = {0}; - int32_t wbtyes = 0; + int32_t wbytes = 0; const uint8_t *value = nullptr; int column_index = 0; try { @@ -216,8 +216,8 @@ Status ParquetReaderWrap::read(Tuple* tuple, const std::vector& if (str_array->IsNull(_current_line_of_group)) { RETURN_IF_ERROR(set_field_null(tuple, slot_desc)); } else { - value = str_array->GetValue(_current_line_of_group, &wbtyes); - fill_slot(tuple, slot_desc, mem_pool, value, wbtyes); + value = str_array->GetValue(_current_line_of_group, &wbytes); + fill_slot(tuple, slot_desc, mem_pool, value, wbytes); } break; } @@ -227,8 +227,8 @@ Status ParquetReaderWrap::read(Tuple* tuple, const std::vector& RETURN_IF_ERROR(set_field_null(tuple, slot_desc)); } else { int32_t value = int32_array->Value(_current_line_of_group); - wbtyes = sprintf((char*)tmp_buf, "%d", value); - fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbtyes); + wbytes = sprintf((char*)tmp_buf, "%d", value); + fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbytes); } break; } @@ -238,8 +238,8 @@ Status ParquetReaderWrap::read(Tuple* tuple, const std::vector& RETURN_IF_ERROR(set_field_null(tuple, slot_desc)); } else { int64_t value = int64_array->Value(_current_line_of_group); - wbtyes = sprintf((char*)tmp_buf, "%ld", value); - fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbtyes); + wbytes = sprintf((char*)tmp_buf, "%ld", value); + fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbytes); } break; } @@ -249,8 +249,8 @@ Status ParquetReaderWrap::read(Tuple* tuple, const std::vector& RETURN_IF_ERROR(set_field_null(tuple, slot_desc)); } else { uint32_t value = uint32_array->Value(_current_line_of_group); - wbtyes = sprintf((char*)tmp_buf, "%u", value); - fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbtyes); + wbytes = sprintf((char*)tmp_buf, "%u", value); + fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbytes); } break; } @@ -260,8 +260,8 @@ Status ParquetReaderWrap::read(Tuple* tuple, const std::vector& RETURN_IF_ERROR(set_field_null(tuple, slot_desc)); } else { uint64_t value = uint64_array->Value(_current_line_of_group); - wbtyes = sprintf((char*)tmp_buf, "%lu", value); - fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbtyes); + wbytes = sprintf((char*)tmp_buf, "%lu", value); + fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbytes); } break; } @@ -270,8 +270,8 @@ Status ParquetReaderWrap::read(Tuple* tuple, const std::vector& if (str_array->IsNull(_current_line_of_group)) { RETURN_IF_ERROR(set_field_null(tuple, slot_desc)); } else { - value = str_array->GetValue(_current_line_of_group, &wbtyes); - fill_slot(tuple, slot_desc, mem_pool, value, wbtyes); + value = str_array->GetValue(_current_line_of_group, &wbytes); + fill_slot(tuple, slot_desc, mem_pool, value, wbytes); } break; } @@ -305,8 +305,8 @@ Status ParquetReaderWrap::read(Tuple* tuple, const std::vector& RETURN_IF_ERROR(set_field_null(tuple, slot_desc)); } else { uint8_t value = uint8_array->Value(_current_line_of_group); - wbtyes = sprintf((char*)tmp_buf, "%d", value); - fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbtyes); + wbytes = sprintf((char*)tmp_buf, "%d", value); + fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbytes); } break; } @@ -316,8 +316,8 @@ Status ParquetReaderWrap::read(Tuple* tuple, const std::vector& RETURN_IF_ERROR(set_field_null(tuple, slot_desc)); } else { int8_t value = int8_array->Value(_current_line_of_group); - wbtyes = sprintf((char*)tmp_buf, "%d", value); - fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbtyes); + wbytes = sprintf((char*)tmp_buf, "%d", value); + fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbytes); } break; } @@ -327,8 +327,8 @@ Status ParquetReaderWrap::read(Tuple* tuple, const std::vector& RETURN_IF_ERROR(set_field_null(tuple, slot_desc)); } else { uint16_t value = uint16_array->Value(_current_line_of_group); - wbtyes = sprintf((char*)tmp_buf, "%d", value); - fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbtyes); + wbytes = sprintf((char*)tmp_buf, "%d", value); + fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbytes); } break; } @@ -338,8 +338,8 @@ Status ParquetReaderWrap::read(Tuple* tuple, const std::vector& RETURN_IF_ERROR(set_field_null(tuple, slot_desc)); } else { int16_t value = int16_array->Value(_current_line_of_group); - wbtyes = sprintf((char*)tmp_buf, "%d", value); - fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbtyes); + wbytes = sprintf((char*)tmp_buf, "%d", value); + fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbytes); } break; } @@ -349,8 +349,8 @@ Status ParquetReaderWrap::read(Tuple* tuple, const std::vector& RETURN_IF_ERROR(set_field_null(tuple, slot_desc)); } else { float value = half_float_array->Value(_current_line_of_group); - wbtyes = sprintf((char*)tmp_buf, "%f", value); - fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbtyes); + wbytes = sprintf((char*)tmp_buf, "%f", value); + fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbytes); } break; } @@ -360,8 +360,8 @@ Status ParquetReaderWrap::read(Tuple* tuple, const std::vector& RETURN_IF_ERROR(set_field_null(tuple, slot_desc)); } else { float value = float_array->Value(_current_line_of_group); - wbtyes = sprintf((char*)tmp_buf, "%f", value); - fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbtyes); + wbytes = sprintf((char*)tmp_buf, "%f", value); + fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbytes); } break; } @@ -371,8 +371,8 @@ Status ParquetReaderWrap::read(Tuple* tuple, const std::vector& RETURN_IF_ERROR(set_field_null(tuple, slot_desc)); } else { float value = double_array->Value(_current_line_of_group); - wbtyes = sprintf((char*)tmp_buf, "%f", value); - fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbtyes); + wbytes = sprintf((char*)tmp_buf, "%f", value); + fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbytes); } break; } @@ -381,8 +381,18 @@ Status ParquetReaderWrap::read(Tuple* tuple, const std::vector& if (ts_array->IsNull(_current_line_of_group)) { RETURN_IF_ERROR(set_field_null(tuple, slot_desc)); } else { - RETURN_IF_ERROR(handle_timestamp(ts_array, tmp_buf, &wbtyes));// convert timestamp to string time - fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbtyes); + RETURN_IF_ERROR(handle_timestamp(ts_array, tmp_buf, &wbytes));// convert timestamp to string time + fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbytes); + } + break; + } + case arrow::Type::type::DECIMAL: { + auto decimal_array = std::dynamic_pointer_cast(_batch->column(column_index)); + if (decimal_array->IsNull(_current_line_of_group)) { + RETURN_IF_ERROR(set_field_null(tuple, slot_desc)); + } else { + std::string value = decimal_array->FormatValue(_current_line_of_group); + fill_slot(tuple, slot_desc, mem_pool, (const uint8_t*)value.c_str(), value.length()); } break; } From 41cbedf57d08a35f5bf53cd4add4ad36fc098dbb Mon Sep 17 00:00:00 2001 From: yiguolei Date: Wed, 7 Aug 2019 20:54:50 +0800 Subject: [PATCH 17/26] Manage tablet by partition id (#1591) --- be/src/olap/tablet.cpp | 4 ++++ be/src/olap/tablet.h | 2 ++ be/src/olap/tablet_manager.cpp | 12 ++++++++++++ be/src/olap/tablet_manager.h | 4 ++-- 4 files changed, 20 insertions(+), 2 deletions(-) diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp index 06097ce5bf494b..6ad96794937181 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -933,4 +933,8 @@ OLAPStatus Tablet::set_partition_id(int64_t partition_id) { return _tablet_meta->set_partition_id(partition_id); } +TabletInfo Tablet::get_tablet_info() { + return TabletInfo(tablet_id(), schema_hash(), tablet_uid()); +} + } // namespace doris diff --git a/be/src/olap/tablet.h b/be/src/olap/tablet.h index fe26cca358c9cb..e4cf7e42be208e 100644 --- a/be/src/olap/tablet.h +++ b/be/src/olap/tablet.h @@ -230,6 +230,8 @@ class Tablet : public std::enable_shared_from_this { return _tablet_meta->initial_end_rowset_id(); } + TabletInfo get_tablet_info(); + private: void _print_missed_versions(const std::vector& missed_versions) const; OLAPStatus _check_added_rowset(const RowsetSharedPtr& rowset); diff --git a/be/src/olap/tablet_manager.cpp b/be/src/olap/tablet_manager.cpp index 833541556659e1..28255de0580681 100755 --- a/be/src/olap/tablet_manager.cpp +++ b/be/src/olap/tablet_manager.cpp @@ -200,6 +200,10 @@ OLAPStatus TabletManager::_add_tablet_to_map(TTabletId tablet_id, SchemaHash sch } _tablet_map[tablet_id].table_arr.push_back(tablet); _tablet_map[tablet_id].table_arr.sort(_sort_tablet_by_creation_time); + + // add the tablet id to partition map + _partition_tablet_map[tablet->partition_id()].insert(tablet->get_tablet_info()); + VLOG(3) << "add tablet to map successfully" << " tablet_id = " << tablet_id << " schema_hash = " << schema_hash; @@ -598,6 +602,10 @@ OLAPStatus TabletManager::drop_tablets_on_error_root_path( it != _tablet_map[tablet_id].table_arr.end();) { if ((*it)->equal(tablet_id, schema_hash)) { it = _tablet_map[tablet_id].table_arr.erase(it); + _partition_tablet_map[(*it)->partition_id()].erase((*it)->get_tablet_info()); + if (_partition_tablet_map[(*it)->partition_id()].empty()) { + _partition_tablet_map.erase((*it)->partition_id()); + } } else { ++it; } @@ -1322,6 +1330,10 @@ OLAPStatus TabletManager::_drop_tablet_directly_unlocked( it != _tablet_map[tablet_id].table_arr.end();) { if ((*it)->equal(tablet_id, schema_hash)) { TabletSharedPtr tablet = *it; + _partition_tablet_map[(*it)->partition_id()].erase((*it)->get_tablet_info()); + if (_partition_tablet_map[(*it)->partition_id()].empty()) { + _partition_tablet_map.erase((*it)->partition_id()); + } it = _tablet_map[tablet_id].table_arr.erase(it); if (!keep_files) { // drop tablet will update tablet meta, should lock diff --git a/be/src/olap/tablet_manager.h b/be/src/olap/tablet_manager.h index 6394ace69a5530..85c4a39122f884 100644 --- a/be/src/olap/tablet_manager.h +++ b/be/src/olap/tablet_manager.h @@ -196,8 +196,8 @@ class TabletManager { std::vector _shutdown_tablets; - // a map from partition id to - std::map> partition_tablet_map; + // map from partition id to tablet_id + std::map> _partition_tablet_map; DISALLOW_COPY_AND_ASSIGN(TabletManager); }; From 4c2a3d6da4aaed2060bf34453c6864fc90f8f8d8 Mon Sep 17 00:00:00 2001 From: xy720 <22125576+xy720@users.noreply.github.com> Date: Wed, 7 Aug 2019 21:31:53 +0800 Subject: [PATCH 18/26] Merge Help document to documentation (#1586) Help document collation (integration of help and documentation documents) --- docs/Makefile | 8 +- .../sql-functions/aggregate-functions/avg.md | 9 +- .../aggregate-functions/count.md | 9 +- .../aggregate-functions/count_distinct.md | 9 +- .../aggregate-functions/group_concat.md | 21 - .../aggregate-functions/hll_union_agg.md | 9 +- .../sql-functions/aggregate-functions/max.md | 9 +- .../sql-functions/aggregate-functions/min.md | 9 +- .../sql-functions/aggregate-functions/ndv.md | 9 +- .../aggregate-functions/percentile_approx.md | 10 +- .../aggregate-functions/stddev.md | 9 +- .../aggregate-functions/stddev_samp.md | 9 +- .../sql-functions/aggregate-functions/sum.md | 9 +- .../aggregate-functions/var_samp.md | 10 +- .../aggregate-functions/variance.md | 10 +- .../cn/sql-reference/sql-functions/cast.md | 18 +- .../date-time-functions/current_timestamp.md | 10 +- .../date-time-functions/date_add.md | 10 +- .../date-time-functions/date_format.md | 9 +- .../date-time-functions/date_sub.md | 10 +- .../date-time-functions/datediff.md | 9 +- .../sql-functions/date-time-functions/day.md | 10 +- .../date-time-functions/dayname.md | 10 +- .../date-time-functions/dayofmonth.md | 10 +- .../date-time-functions/dayofweek.md | 10 +- .../date-time-functions/dayofyear.md | 10 +- .../date-time-functions/from_days.md | 10 +- .../date-time-functions/from_unixtime.md | 10 +- .../date-time-functions/month.md | 10 +- .../date-time-functions/monthname.md | 10 +- .../sql-functions/date-time-functions/now.md | 10 +- .../date-time-functions/str_to_date.md | 10 +- .../date-time-functions/timediff.md | 9 +- .../date-time-functions/to_days.md | 10 +- .../date-time-functions/unix_timestamp.md | 10 +- .../date-time-functions/utc_timestamp.md | 10 +- .../date-time-functions/workofyear.md | 10 +- .../sql-functions/date-time-functions/year.md | 10 +- .../spatial-functions/st_astext.md | 9 +- .../spatial-functions/st_circle.md | 9 +- .../spatial-functions/st_contains.md | 9 +- .../spatial-functions/st_distance_sphere.md | 9 +- .../spatial-functions/st_geometryfromtext.md | 9 +- .../spatial-functions/st_linefromtext.md | 9 +- .../spatial-functions/st_point.md | 9 +- .../spatial-functions/st_polygon.md | 9 +- .../sql-functions/spatial-functions/st_x.md | 9 +- .../sql-functions/spatial-functions/st_y.md | 9 +- .../sql-functions/string-functions/ascii.md | 9 +- .../sql-functions/string-functions/concat.md | 9 +- .../string-functions/concat_ws.md | 9 +- .../string-functions/find_in_set.md | 9 +- .../string-functions/get_json_double.md | 9 +- .../string-functions/get_json_int.md | 9 +- .../string-functions/get_json_string.md | 9 +- .../string-functions/group_concat.md | 9 +- .../sql-functions/string-functions/instr.md | 9 +- .../sql-functions/string-functions/lcase.md | 7 +- .../sql-functions/string-functions/left.md | 9 +- .../sql-functions/string-functions/length.md | 9 +- .../sql-functions/string-functions/locate.md | 9 +- .../sql-functions/string-functions/lower.md | 9 +- .../sql-functions/string-functions/lpad.md | 9 +- .../sql-functions/string-functions/ltrim.md | 9 +- .../string-functions/money_format.md | 9 +- .../string-functions/regexp_extract.md | 9 +- .../string-functions/regexp_replace.md | 9 +- .../sql-functions/string-functions/repeat.md | 9 +- .../sql-functions/string-functions/right.md | 9 +- .../string-functions/split_part.md | 9 +- .../sql-functions/string-functions/strleft.md | 9 +- .../string-functions/strright.md | 9 +- .../Account Management/CREATE ROLE.md | 19 + .../Account Management/CREATE USER.md | 49 + .../Account Management/DROP ROLE.md | 18 + .../Account Management/DROP USER.md | 18 + .../Account Management/GRANT.md | 55 + .../Account Management/REVOKE.md | 23 + .../Account Management/SET PASSWORD.md | 30 + .../Account Management/SET PROPERTY.md | 56 + .../Account Management/SHOW GRANTS.md | 30 + .../Account Management/SHOW ROLES.md | 16 + .../Administration/ADMIN CANCEL REPAIR.md | 22 + .../Administration/ADMIN REPAIR.md} | 22 - .../Administration/ADMIN SET CONFIG.md | 18 + .../Administration/ADMIN SHOW CONFIG.md | 27 + .../ADMIN SHOW REPLICA DISTRIBUTION.md | 26 + .../ADMIN SHOW REPLICA STATUS.md | 39 + .../Administration/ALTER CLUSTER.md | 28 + .../Administration/ALTER SYSTEM.md | 94 ++ .../Administration/CANCEL DECOMMISSION.md | 15 + .../Administration/CREATE CLUSTER.md | 36 + .../Administration/CREATE FILE.md} | 52 - .../Administration/DROP CLUSTER.md | 18 + .../Administration/DROP FILE.md | 25 + .../sql-statements/Administration/ENTER.md | 18 + .../Administration/LINK DATABASE.md | 24 + .../Administration/MIGRATE DATABASE.md | 20 + .../Administration/SHOW BACKENDS.md | 22 + .../Administration/SHOW BROKER.md | 15 + .../Administration/SHOW FILE.md | 27 + .../Administration/SHOW FRONTENDS.md | 18 + .../Administration/SHOW MIGRATIONS.md | 12 + .../Data Definition/ALTER DATABASE.md | 29 + .../Data Definition/ALTER TABLE.md | 240 ++++ .../sql-statements/Data Definition/BACKUP.md | 39 + .../Data Definition/CANCEL ALTER.md | 32 + .../Data Definition/CANCEL BACKUP.md | 13 + .../Data Definition/CANCEL RESTORE.md | 16 + .../Data Definition/CREATE DATABASE.md | 13 + .../Data Definition/CREATE REPOSITORY.md | 50 + .../Data Definition/CREATE TABLE.md | 334 +++++ .../Data Definition/CREATE VIEW.md | 22 + .../Data Definition/Colocate Join.md | 72 + .../Data Definition/DROP DATABASE.md | 16 + .../Data Definition/DROP REPOSITORY.md | 16 + .../Data Definition/DROP TABLE.md | 19 + .../Data Definition/DROP VIEW.md | 14 + .../sql-statements/Data Definition/HLL.md | 80 ++ .../sql-statements/Data Definition/RECOVER.md | 28 + .../sql-statements/Data Definition/RESTORE.md | 52 + .../Data Definition/TRUNCATE TABLE.md | 27 + .../{ => Data Definition}/create-function.md | 21 +- .../{ => Data Definition}/drop-function.md | 17 +- .../{ => Data Definition}/show-function.md | 17 +- .../Data Manipulation/CANCEL DELETE.md | 10 + .../Data Manipulation/CANCEL LABEL.md | 33 + .../Data Manipulation/CANCEL LOAD.md | 20 + .../Data Manipulation/DELETE.md | 36 + .../Data Manipulation/EXPORT.md | 58 + .../Data Manipulation/GET LABEL STATE.md | 33 + .../sql-statements/Data Manipulation/LOAD.md | 284 ++++ .../Data Manipulation/MINI LOAD.md | 104 ++ .../Data Manipulation/MULTI LOAD.md | 82 ++ .../Data Manipulation/PAUSE ROUTINE LOAD.md | 10 + .../Data Manipulation/RESTORE TABLET.md | 15 + .../Data Manipulation/RESUME ROUTINE LOAD.md | 10 + .../Data Manipulation/ROUTINE LOAD.md} | 194 --- .../Data Manipulation/SHOW ALTER.md | 25 + .../Data Manipulation/SHOW BACKUP.md | 37 + .../Data Manipulation/SHOW DATA.md | 21 + .../Data Manipulation/SHOW DATABASES.md | 9 + .../Data Manipulation/SHOW DELETE.md | 13 + .../Data Manipulation/SHOW EXPORT.md | 36 + .../Data Manipulation/SHOW LOAD.md | 49 + .../Data Manipulation/SHOW PARTITIONS.md | 16 + .../Data Manipulation/SHOW PROPERTY.md | 16 + .../Data Manipulation/SHOW REPOSITORIES.md | 23 + .../Data Manipulation/SHOW RESTORE.md | 41 + .../SHOW ROUTINE LOAD TASK.md | 9 + .../Data Manipulation/SHOW ROUTINE LOAD.md | 32 + .../Data Manipulation/SHOW SNAPSHOT.md | 31 + .../Data Manipulation/SHOW TABLES.md | 9 + .../Data Manipulation/SHOW TABLET.md | 17 + .../Data Manipulation/STOP ROUTINE LOAD.md | 10 + .../Data Manipulation/STREAM LOAD.md} | 66 - .../Data Manipulation/broker_load.md | 2 +- .../{ => Data Manipulation}/insert.md | 12 +- .../sql-statements/Data Types/BIGINT.md | 7 + .../sql-statements/Data Types/BOOLEAN.md | 7 + .../sql-statements/Data Types/CHAR.md | 7 + .../sql-statements/Data Types/DATE.md | 15 + .../sql-statements/Data Types/DATETIME.md | 8 + .../sql-statements/Data Types/DECIMAL.md | 8 + .../sql-statements/Data Types/DOUBLE.md | 7 + .../sql-statements/Data Types/FLOAT.md | 7 + .../Data Types/HLL(HyperLogLog).md | 9 + .../sql-statements/Data Types/INT.md | 7 + .../sql-statements/Data Types/SMALLINT.md | 7 + .../sql-statements/Data Types/TINYINT.md | 7 + .../sql-statements/Data Types/VARCHAR.md | 7 + .../sql-statements}/Utility/util_stmt.md | 26 +- docs/help/Contents/Account Management/help.md | 335 ----- .../Administration/admin_show_stmt.md | 92 -- .../Contents/Administration/admin_stmt.md | 338 ----- .../help/Contents/Data Definition/ddl_stmt.md | 1153 ----------------- .../Data Manipulation/manipulation_stmt.md | 950 -------------- docs/help/Contents/Data Types/help.md | 77 -- .../Functions/Date and Time Functions/help.md | 310 ----- .../apache/doris/common/MarkDownParser.java | 13 +- .../doris/common/MarkDownParserTest.java | 39 +- 181 files changed, 3576 insertions(+), 3970 deletions(-) delete mode 100755 docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/group_concat.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Account Management/CREATE ROLE.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Account Management/CREATE USER.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Account Management/DROP ROLE.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Account Management/DROP USER.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Account Management/GRANT.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Account Management/REVOKE.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Account Management/SET PASSWORD.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Account Management/SET PROPERTY.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Account Management/SHOW GRANTS.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Account Management/SHOW ROLES.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Administration/ADMIN CANCEL REPAIR.md rename docs/{help/Contents/Administration/admin_repair_stmt.md => documentation/cn/sql-reference/sql-statements/Administration/ADMIN REPAIR.md} (60%) create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Administration/ADMIN SET CONFIG.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Administration/ADMIN SHOW CONFIG.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Administration/ADMIN SHOW REPLICA DISTRIBUTION.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Administration/ADMIN SHOW REPLICA STATUS.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Administration/ALTER CLUSTER.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Administration/ALTER SYSTEM.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Administration/CANCEL DECOMMISSION.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Administration/CREATE CLUSTER.md rename docs/{help/Contents/Administration/small_files.md => documentation/cn/sql-reference/sql-statements/Administration/CREATE FILE.md} (63%) create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Administration/DROP CLUSTER.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Administration/DROP FILE.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Administration/ENTER.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Administration/LINK DATABASE.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Administration/MIGRATE DATABASE.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Administration/SHOW BACKENDS.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Administration/SHOW BROKER.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Administration/SHOW FILE.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Administration/SHOW FRONTENDS.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Administration/SHOW MIGRATIONS.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Definition/ALTER DATABASE.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Definition/ALTER TABLE.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Definition/BACKUP.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Definition/CANCEL ALTER.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Definition/CANCEL BACKUP.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Definition/CANCEL RESTORE.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Definition/CREATE DATABASE.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Definition/CREATE REPOSITORY.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Definition/CREATE TABLE.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Definition/CREATE VIEW.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Definition/Colocate Join.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Definition/DROP DATABASE.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Definition/DROP REPOSITORY.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Definition/DROP TABLE.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Definition/DROP VIEW.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Definition/HLL.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Definition/RECOVER.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Definition/RESTORE.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Definition/TRUNCATE TABLE.md rename docs/documentation/cn/sql-reference/sql-statements/{ => Data Definition}/create-function.md (97%) rename docs/documentation/cn/sql-reference/sql-statements/{ => Data Definition}/drop-function.md (81%) rename docs/documentation/cn/sql-reference/sql-statements/{ => Data Definition}/show-function.md (94%) create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/CANCEL DELETE.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/CANCEL LABEL.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/CANCEL LOAD.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/DELETE.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/EXPORT.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/GET LABEL STATE.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/LOAD.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/MINI LOAD.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/MULTI LOAD.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/PAUSE ROUTINE LOAD.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/RESTORE TABLET.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/RESUME ROUTINE LOAD.md rename docs/{help/Contents/Data Manipulation/routine_load.md => documentation/cn/sql-reference/sql-statements/Data Manipulation/ROUTINE LOAD.md} (62%) create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW ALTER.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW BACKUP.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW DATA.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW DATABASES.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW DELETE.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW EXPORT.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW LOAD.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW PARTITIONS.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW PROPERTY.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW REPOSITORIES.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW RESTORE.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW ROUTINE LOAD TASK.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW ROUTINE LOAD.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW SNAPSHOT.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW TABLES.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW TABLET.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/STOP ROUTINE LOAD.md rename docs/{help/Contents/Data Manipulation/streaming.md => documentation/cn/sql-reference/sql-statements/Data Manipulation/STREAM LOAD.md} (78%) rename docs/{help/Contents => documentation/cn/sql-reference/sql-statements}/Data Manipulation/broker_load.md (99%) rename docs/documentation/cn/sql-reference/sql-statements/{ => Data Manipulation}/insert.md (97%) create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Types/BIGINT.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Types/BOOLEAN.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Types/CHAR.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Types/DATE.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Types/DATETIME.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Types/DECIMAL.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Types/DOUBLE.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Types/FLOAT.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Types/HLL(HyperLogLog).md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Types/INT.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Types/SMALLINT.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Types/TINYINT.md create mode 100644 docs/documentation/cn/sql-reference/sql-statements/Data Types/VARCHAR.md rename docs/{help/Contents => documentation/cn/sql-reference/sql-statements}/Utility/util_stmt.md (81%) delete mode 100644 docs/help/Contents/Account Management/help.md delete mode 100644 docs/help/Contents/Administration/admin_show_stmt.md delete mode 100644 docs/help/Contents/Administration/admin_stmt.md delete mode 100644 docs/help/Contents/Data Definition/ddl_stmt.md delete mode 100644 docs/help/Contents/Data Manipulation/manipulation_stmt.md delete mode 100644 docs/help/Contents/Data Types/help.md delete mode 100644 docs/help/Contents/Functions/Date and Time Functions/help.md mode change 100644 => 100755 fe/src/main/java/org/apache/doris/common/MarkDownParser.java mode change 100644 => 100755 fe/src/test/java/org/apache/doris/common/MarkDownParserTest.java diff --git a/docs/Makefile b/docs/Makefile index d337c2091bdeeb..6d971114bc8806 100644 --- a/docs/Makefile +++ b/docs/Makefile @@ -20,17 +20,21 @@ # BUILD_DIR = ${CURDIR}/build +HELP_DIR = ${BUILD_DIR}/contents all: zip_help .PHONY: all ${BUILD_DIR}: mkdir -p ${BUILD_DIR} +${HELP_DIR}: + mkdir -p ${HELP_DIR} # build help zip HELP_OUTPUT = ${BUILD_DIR}/help-resource.zip -${HELP_OUTPUT}: help | ${BUILD_DIR} - zip -r $@ $< +${HELP_OUTPUT}: documentation/cn/sql-reference ${BUILD_DIR} ${HELP_DIR} + cp -r $ SELECT datetime, AVG(cost_time) FROM log_statis group by datetime; @@ -28,3 +27,5 @@ mysql> SELECT datetime, AVG(distinct cost_time) FROM log_statis group by datetim +---------------------+---------------------------+ ``` +##keyword +AVG diff --git a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/count.md b/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/count.md index 507ac1c649b8fb..b8bdef43a55e96 100755 --- a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/count.md +++ b/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/count.md @@ -1,14 +1,13 @@ # COUNT - -## Syntax +## description +### Syntax `COUNT([DISTINCT] expr)` -## Description 用于返回满足要求的行的数目 -## Examples +## example ``` MySQL > select count(*) from log_statis group by datetime; @@ -32,3 +31,5 @@ MySQL > select count(distinct datetime) from log_statis group by datetime; | 71045 | +-------------------------------+ ``` +##keyword +COUNT diff --git a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/count_distinct.md b/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/count_distinct.md index ecaee4400eafd4..a5d21e604be4a4 100755 --- a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/count_distinct.md +++ b/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/count_distinct.md @@ -1,14 +1,13 @@ # COUNT_DISTINCT - -## Syntax +## description +### Syntax `COUNT_DISTINCT(expr)` -## Description 用于返回满足要求的行的数目,或者非NULL行的数目 -## Examples +## example ``` MySQL > select count_distinct(query_id) from log_statis group by datetime; @@ -18,3 +17,5 @@ MySQL > select count_distinct(query_id) from log_statis group by datetime; | 577 | +----------------------------+ ``` +##keyword +COUNT_DISTINCT,COUNT,DISTINCT diff --git a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/group_concat.md b/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/group_concat.md deleted file mode 100755 index c326dbb10a9651..00000000000000 --- a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/group_concat.md +++ /dev/null @@ -1,21 +0,0 @@ -# GROUP_CONCAT - -## Syntax - -`GROUP_CONCAT(expr)` - -## Description - -用于返回选中字段字符串连接起来的新字符串 - -使用逗号连接 - -## Examples -``` -MySQL> select group_concat(`query_id`) from log_statis group by datetime; -+---------------------------------------------------------------------------------------------------------+ -| group_concat(`query_id`) | -+---------------------------------------------------------------------------------------------------------+ -| 445f5875f8854dfa:b9358d5cd86488a2, d5055534e6c04fa0:9b3f645913c4973c, a55801d0764d47fe:a98f3710649dc558 | -+------------------------------------------------------------------------------------------------ --------+ -``` diff --git a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/hll_union_agg.md b/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/hll_union_agg.md index 69253c8c7289a0..6d3b1ebdda7058 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/hll_union_agg.md +++ b/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/hll_union_agg.md @@ -1,10 +1,9 @@ # HLL_UNION_AGG - -## Syntax +## description +### Syntax `HLL_UNION_AGG(hll)` -## Description HLL是基于HyperLogLog算法的工程实现,用于保存HyperLogLog计算过程的中间结果 @@ -14,7 +13,7 @@ HLL是基于HyperLogLog算法的工程实现,用于保存HyperLogLog计算过 导入的时候通过hll_hash函数来指定数据中哪一列用于生成hll列,它常用于替代count distinct,通过结合rollup在业务上用于快速计算uv等 -## Examples +## example ``` MySQL > select HLL_UNION_AGG(uv_set) from test_uv;; +-------------------------+ @@ -23,3 +22,5 @@ MySQL > select HLL_UNION_AGG(uv_set) from test_uv;; | 17721 | +-------------------------+ ``` +##keyword +HLL_UNION_AGG,HLL,UNION,AGG diff --git a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/max.md b/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/max.md index d4ca2506471650..fe4ecdadc63254 100755 --- a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/max.md +++ b/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/max.md @@ -1,14 +1,13 @@ # MAX - -## Syntax +## description +### Syntax `MAX(expr)` -## Description 返回expr表达式的最大值 -## Examples +## example ``` MySQL > select max(scan_rows) from log_statis group by datetime; +------------------+ @@ -17,3 +16,5 @@ MySQL > select max(scan_rows) from log_statis group by datetime; | 4671587 | +------------------+ ``` +##keyword +MAX diff --git a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/min.md b/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/min.md index e6c8f18516e4e3..3a6ce810f91f1f 100755 --- a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/min.md +++ b/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/min.md @@ -1,14 +1,13 @@ # MIN - -## Syntax +## description +### Syntax `MIN(expr)` -## Description 返回expr表达式的最小值 -## Examples +## example ``` MySQL > select min(scan_rows) from log_statis group by datetime; +------------------+ @@ -17,3 +16,5 @@ MySQL > select min(scan_rows) from log_statis group by datetime; | 0 | +------------------+ ``` +##keyword +MIN diff --git a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/ndv.md b/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/ndv.md index ff3a08753421e4..01a46f5b9a70d0 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/ndv.md +++ b/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/ndv.md @@ -1,16 +1,15 @@ # NDV - -## Syntax +## description +### Syntax `NDV(expr)` -## Description 返回类似于 COUNT(DISTINCT col) 结果的近似值聚合函数。 它比 COUNT 和 DISTINCT 组合的速度更快,并使用固定大小的内存,因此对于高基数的列可以使用更少的内存。 -## Examples +## example ``` MySQL > select ndv(query_id) from log_statis group by datetime; +-----------------+ @@ -19,3 +18,5 @@ MySQL > select ndv(query_id) from log_statis group by datetime; | 17721 | +-----------------+ ``` +##keyword +NDV diff --git a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/percentile_approx.md b/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/percentile_approx.md index 329f6b104a0071..f47ce1ce1430b3 100755 --- a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/percentile_approx.md +++ b/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/percentile_approx.md @@ -1,16 +1,15 @@ # PERCENTILE_APPROX - -## Syntax +## description +### Syntax `PERCENTILE_APPROX(expr, DOUBLE p)` -## Description 返回第p个百分位点的近似值,p的值介于0到1之间 该函数使用固定大小的内存,因此对于高基数的列可以使用更少的内存,可用于计算tp99等统计值 -## Examples +## example ``` MySQL > select `table`, percentile_approx(cost_time,0.99) from log_statis group by `table`; +---------------------+---------------------------+ @@ -18,4 +17,5 @@ MySQL > select `table`, percentile_approx(cost_time,0.99) from log_statis group +----------+--------------------------------------+ | test | 54.22 | +----------+--------------------------------------+ -``` \ No newline at end of file +##keyword +PERCENTILE_APPROX,PERCENTILE,APPROX diff --git a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/stddev.md b/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/stddev.md index 393ffdf1419925..3a84758a2783fc 100755 --- a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/stddev.md +++ b/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/stddev.md @@ -1,14 +1,13 @@ # STDDEV,STDDEV_POP - -## Syntax +## description +### Syntax `STDDEV(expr)` -## Description 返回expr表达式的标准差 -## Examples +## example ``` MySQL > select stddev(scan_rows) from log_statis group by datetime; +---------------------+ @@ -24,3 +23,5 @@ MySQL > select stddev_pop(scan_rows) from log_statis group by datetime; | 2.3722760595994914 | +-------------------------+ ``` +##keyword +STDDEV,STDDEV_POP,POP diff --git a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/stddev_samp.md b/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/stddev_samp.md index 5243a99b90d12b..2af5308e97d032 100755 --- a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/stddev_samp.md +++ b/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/stddev_samp.md @@ -1,14 +1,13 @@ # STDDEV_SAMP - -## Syntax +## description +### Syntax `STDDEV_SAMP(expr)` -## Description 返回expr表达式的样本标准差 -## Examples +## example ``` MySQL > select stddev_samp(scan_rows) from log_statis group by datetime; +--------------------------+ @@ -17,3 +16,5 @@ MySQL > select stddev_samp(scan_rows) from log_statis group by datetime; | 2.372044195280762 | +--------------------------+ ``` +##keyword +STDDEV_SAMP,STDDEV,SAMP diff --git a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/sum.md b/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/sum.md index ad4fe7c9932b59..4d0959af1ddee5 100755 --- a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/sum.md +++ b/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/sum.md @@ -1,14 +1,13 @@ # SUM - -## Syntax +## description +### Syntax `SUM(expr)` -## Description 用于返回选中字段所有值的和 -## Examples +## example ``` MySQL > select sum(scan_rows) from log_statis group by datetime; +------------------+ @@ -17,3 +16,5 @@ MySQL > select sum(scan_rows) from log_statis group by datetime; | 8217360135 | +------------------+ ``` +##keyword +SUM diff --git a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/var_samp.md b/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/var_samp.md index 5da00ad73daeb3..ac105c257f2447 100755 --- a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/var_samp.md +++ b/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/var_samp.md @@ -1,14 +1,13 @@ # VAR_SAMP,VARIANCE_SAMP - -## Syntax +## description +### Syntax `VAR_SAMP(expr)` -## Description 返回expr表达式的样本方差 -## Examples +## example ``` MySQL > select var_samp(scan_rows) from log_statis group by datetime; +-----------------------+ @@ -16,4 +15,5 @@ MySQL > select var_samp(scan_rows) from log_statis group by datetime; +-----------------------+ | 5.6227132145741789 | +-----------------------+ -``` \ No newline at end of file +##keyword +VAR_SAMP,VARIANCE_SAMP,VAR,SAMP,VARIANCE diff --git a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/variance.md b/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/variance.md index 27e948f3aec453..20c3a6fa152152 100755 --- a/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/variance.md +++ b/docs/documentation/cn/sql-reference/sql-functions/aggregate-functions/variance.md @@ -1,14 +1,13 @@ # VARIANCE,VAR_POP,VARIANCE_POP - -## Syntax +## description +### Syntax `VARIANCE(expr)` -## Description 返回expr表达式的方差 -## Examples +## example ``` MySQL > select variance(scan_rows) from log_statis group by datetime; +-----------------------+ @@ -23,4 +22,5 @@ MySQL > select var_pop(scan_rows) from log_statis group by datetime; +----------------------+ | 5.6230744719006163 | +----------------------+ -``` \ No newline at end of file +##keyword +VARIANCE,VAR_POP,VARIANCE_POP,VAR,POP diff --git a/docs/documentation/cn/sql-reference/sql-functions/cast.md b/docs/documentation/cn/sql-reference/sql-functions/cast.md index b9898a2f710693..9b08ba6ea6cb04 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/cast.md +++ b/docs/documentation/cn/sql-reference/sql-functions/cast.md @@ -1,26 +1,24 @@ # CAST - -## Syntax +## description +### Syntax ``` cast (input as type) ``` -## Description +### BIGINT type -将 input 转成 指定的 type +### Syntax(BIGINT) -## BIGINT type +``` cast (input as BIGINT) ``` -### Syntax -``` cast (input as BIGINT) ``` +将 input 转成 指定的 type -### Description 将当前列 input 转换为 BIGINT 类型 -### Examples +## example 1. 转常量,或表中某列 @@ -54,3 +52,5 @@ mysql> select cast(cast ("11.2" as double) as bigint); +----------------------------------------+ 1 row in set (0.00 sec) ``` +##keyword +CAST diff --git a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/current_timestamp.md b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/current_timestamp.md index c73c10a79d579f..85afc0f01b4f7b 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/current_timestamp.md +++ b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/current_timestamp.md @@ -1,14 +1,13 @@ # current_timestamp - -## Syntax +## description +### Syntax `DATETIME CURRENT_TIMESTAMP()` -## Description 获得当前的时间,以Datetime类型返回 -## Examples +## example ``` mysql> select current_timestamp(); @@ -17,4 +16,5 @@ mysql> select current_timestamp(); +---------------------+ | 2019-05-27 15:59:33 | +---------------------+ -``` \ No newline at end of file +##keyword +CURRENT_TIMESTAMP,CURRENT,TIMESTAMP diff --git a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/date_add.md b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/date_add.md index 1ae067295caeab..ecb45512c7ff18 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/date_add.md +++ b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/date_add.md @@ -1,10 +1,9 @@ # date_add - -## Syntax +## description +### Syntax `INT DATE_ADD(DATETIME date,INTERVAL expr type)` -## Description 向日期添加指定的时间间隔。 @@ -14,7 +13,7 @@ expr 参数是您希望添加的时间间隔。 type 参数可以是下列值:YEAR, MONTH, DAY, HOUR, MINUTE, SECOND -## Examples +## example ``` mysql> select date_add('2010-11-30 23:59:59', INTERVAL 2 DAY); @@ -23,4 +22,5 @@ mysql> select date_add('2010-11-30 23:59:59', INTERVAL 2 DAY); +-------------------------------------------------+ | 2010-12-02 23:59:59 | +-------------------------------------------------+ -``` \ No newline at end of file +##keyword +DATE_ADD,DATE,ADD diff --git a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/date_format.md b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/date_format.md index 95be89645e3066..8a05cf0d8f124b 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/date_format.md +++ b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/date_format.md @@ -1,10 +1,9 @@ # date_format - -## Syntax +## description +### Syntax `VARCHAR DATE_FORMAT(DATETIME date, VARCHAR format)` -## Description 将日期类型按照format的类型转化位字符串, 当前支持最大128字节的字符串,如果返回值长度超过128,则返回NULL @@ -75,7 +74,7 @@ date 参数是合法的日期。format 规定日期/时间的输出格式。 %y | 年,2 位 -## Examples +## example ``` mysql> select date_format('2009-10-04 22:23:00', '%W %M %Y'); @@ -120,3 +119,5 @@ mysql> select date_format('2006-06-01', '%d'); | 01 | +------------------------------------------+ ``` +##keyword +DATE_FORMAT,DATE,FORMAT diff --git a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/date_sub.md b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/date_sub.md index 20150ad9b2326c..6d58475ca9985f 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/date_sub.md +++ b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/date_sub.md @@ -1,10 +1,9 @@ # date_sub - -## Syntax +## description +### Syntax `INT DATE_SUB(DATETIME date,INTERVAL expr type)` -## Description 从日期减去指定的时间间隔 @@ -14,7 +13,7 @@ expr 参数是您希望添加的时间间隔。 type 参数可以是下列值:YEAR, MONTH, DAY, HOUR, MINUTE, SECOND -## Examples +## example ``` mysql> select date_sub('2010-11-30 23:59:59', INTERVAL 2 DAY); @@ -23,4 +22,5 @@ mysql> select date_sub('2010-11-30 23:59:59', INTERVAL 2 DAY); +-------------------------------------------------+ | 2010-11-28 23:59:59 | +-------------------------------------------------+ -``` \ No newline at end of file +##keyword +DATE_SUB,DATE,SUB diff --git a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/datediff.md b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/datediff.md index 085d1c8930f388..67a8fedeba25b3 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/datediff.md +++ b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/datediff.md @@ -1,10 +1,9 @@ # datediff - -## Syntax +## description +### Syntax `DATETIME DATEDIFF(DATETIME expr1,DATETIME expr2)` -## Description 计算expr1 - expr2,结果精确到天。 @@ -12,7 +11,7 @@ expr1 和 expr2 参数是合法的日期或日期/时间表达式。 注释:只有值的日期部分参与计算。 -## Examples +## example ``` mysql> select datediff(CAST('2007-12-31 23:59:59' AS DATETIME), CAST('2007-12-30' AS DATETIME)); @@ -29,3 +28,5 @@ mysql> select datediff(CAST('2010-11-30 23:59:59' AS DATETIME), CAST('2010-12-31 | -31 | +-----------------------------------------------------------------------------------+ ``` +##keyword +DATEDIFF diff --git a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/day.md b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/day.md index 25852adde340c2..cd926a4287d54e 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/day.md +++ b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/day.md @@ -1,16 +1,15 @@ # day - -## Syntax +## description +### Syntax `INT DAY(DATETIME date)` -## Description 获得日期中的天信息,返回值范围从1-31。 参数为Date或者Datetime类型 -## Examples +## example ``` mysql> select day('1987-01-31'); @@ -19,4 +18,5 @@ mysql> select day('1987-01-31'); +----------------------------+ | 31 | +----------------------------+ -``` \ No newline at end of file +##keyword +DAY diff --git a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/dayname.md b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/dayname.md index b6353f609192ab..872daa7cdc0a9c 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/dayname.md +++ b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/dayname.md @@ -1,16 +1,15 @@ # dayname - -## Syntax +## description +### Syntax `VARCHAR DAYNAME(DATE)` -## Description 返回日期对应的日期名字 参数为Date或者Datetime类型 -## Examples +## example ``` mysql> select dayname('2007-02-03 00:00:00'); @@ -19,4 +18,5 @@ mysql> select dayname('2007-02-03 00:00:00'); +--------------------------------+ | Saturday | +--------------------------------+ -``` \ No newline at end of file +##keyword +DAYNAME diff --git a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/dayofmonth.md b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/dayofmonth.md index 40fe41767c0844..7071b7d20bfda9 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/dayofmonth.md +++ b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/dayofmonth.md @@ -1,16 +1,15 @@ # dayofmonth - -## Syntax +## description +### Syntax `INT DAYOFMONTH(DATETIME date)` -## Description 获得日期中的天信息,返回值范围从1-31。 参数为Date或者Datetime类型 -## Examples +## example ``` mysql> select dayofmonth('1987-01-31'); @@ -19,4 +18,5 @@ mysql> select dayofmonth('1987-01-31'); +-----------------------------------+ | 31 | +-----------------------------------+ -``` \ No newline at end of file +##keyword +DAYOFMONTH diff --git a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/dayofweek.md b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/dayofweek.md index 43e6511600d016..63f8853e54edd6 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/dayofweek.md +++ b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/dayofweek.md @@ -1,16 +1,15 @@ # dayofweek - -## Syntax +## description +### Syntax `INT dayofweek(DATETIME date)` -## Description DAYOFWEEK函数返回日期的工作日索引值,即星期日为1,星期一为2,星期六为7 参数为Date或者Datetime类型 -## Examples +## example ``` mysql> select dayofweek('2019-06-25'); @@ -19,4 +18,5 @@ mysql> select dayofweek('2019-06-25'); +----------------------------------+ | 3 | +----------------------------------+ -``` \ No newline at end of file +##keyword +DAYOFWEEK diff --git a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/dayofyear.md b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/dayofyear.md index 6f91d92e3dd1e4..2d0bae42f6fb3c 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/dayofyear.md +++ b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/dayofyear.md @@ -1,16 +1,15 @@ # dayofyear - -## Syntax +## description +### Syntax `INT DAYOFYEAR(DATETIME date)` -## Description 获得日期中对应当年中的哪一天。 参数为Date或者Datetime类型 -## Examples +## example ``` mysql> select dayofyear('2007-02-03 00:00:00'); @@ -19,4 +18,5 @@ mysql> select dayofyear('2007-02-03 00:00:00'); +----------------------------------+ | 34 | +----------------------------------+ -``` \ No newline at end of file +##keyword +DAYOFYEAR diff --git a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/from_days.md b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/from_days.md index abfbdb7216f699..741e9ee4b2527d 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/from_days.md +++ b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/from_days.md @@ -1,14 +1,13 @@ # from_days - -## Syntax +## description +### Syntax `DATE FROM_DAYS(INT N)` -## Description 通过距离0000-01-01日的天数计算出哪一天 -## Examples +## example ``` mysql> select from_days(730669); @@ -17,4 +16,5 @@ mysql> select from_days(730669); +-------------------+ | 2000-07-03 | +-------------------+ -``` \ No newline at end of file +##keyword +FROM_DAYS,FROM,DAYS diff --git a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/from_unixtime.md b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/from_unixtime.md index 401362d83578ae..9ac15232858f67 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/from_unixtime.md +++ b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/from_unixtime.md @@ -1,10 +1,9 @@ # from_unixtime - -## Syntax +## description +### Syntax `DATETIME FROM_UNIXTIME(INT unix_timestamp[, VARCHAR string_format])` -## Description 将unix时间戳转化位对应的time格式,返回的格式由string_format指定 @@ -16,7 +15,7 @@ 其余string_format格式是非法的,返回NULL -## Examples +## example ``` mysql> select from_unixtime(1196440219); @@ -39,4 +38,5 @@ mysql> select from_unixtime(1196440219, 'yyyy-MM-dd HH:mm:ss'); +--------------------------------------------------+ | 2007-12-01 00:30:19 | +--------------------------------------------------+ -``` \ No newline at end of file +##keyword +FROM_UNIXTIME,FROM,UNIXTIME diff --git a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/month.md b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/month.md index b85715b4784bbe..7588b1f63cd8fb 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/month.md +++ b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/month.md @@ -1,16 +1,15 @@ # month - -## Syntax +## description +### Syntax `INT MONTH(DATETIME date)` -## Description 返回时间类型中的月份信息,范围是1, 12 参数为Date或者Datetime类型 -## Examples +## example ``` mysql> select month('1987-01-01'); @@ -19,4 +18,5 @@ mysql> select month('1987-01-01'); +-----------------------------+ | 1 | +-----------------------------+ -``` \ No newline at end of file +##keyword +MONTH diff --git a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/monthname.md b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/monthname.md index 6ba49bf6afb03c..8e0a6e0901666c 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/monthname.md +++ b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/monthname.md @@ -1,16 +1,15 @@ # monthname - -## Syntax +## description +### Syntax `VARCHAR MONTHNAME(DATE)` -## Description 返回日期对应的月份名字 参数为Date或者Datetime类型 -## Examples +## example ``` mysql> select monthname('2008-02-03 00:00:00'); @@ -19,4 +18,5 @@ mysql> select monthname('2008-02-03 00:00:00'); +----------------------------------+ | February | +----------------------------------+ -``` \ No newline at end of file +##keyword +MONTHNAME diff --git a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/now.md b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/now.md index ff2a365a92bec7..71afb0f0f99be4 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/now.md +++ b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/now.md @@ -1,14 +1,13 @@ # now - -## Syntax +## description +### Syntax `DATETIME NOW()` -## Description 获得当前的时间,以Datetime类型返回 -## Examples +## example ``` mysql> select now(); @@ -17,4 +16,5 @@ mysql> select now(); +---------------------+ | 2019-05-27 15:58:25 | +---------------------+ -``` \ No newline at end of file +##keyword +NOW diff --git a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/str_to_date.md b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/str_to_date.md index b44fc84a46cb93..1f7ca178525e46 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/str_to_date.md +++ b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/str_to_date.md @@ -1,16 +1,15 @@ # str_to_date - -## Syntax +## description +### Syntax `DATETIME STR_TO_DATE(VARCHAR str, VARCHAR format)` -## Description 通过format指定的方式将str转化为DATE类型,如果转化结果不对返回NULL 支持的format格式与date_format一致 -## Examples +## example ``` mysql> select str_to_date('2014-12-21 12:34:56', '%Y-%m-%d %H:%i:%s'); @@ -26,4 +25,5 @@ mysql> select str_to_date('200442 Monday', '%X%V %W'); +-----------------------------------------+ | 2004-10-18 | +-----------------------------------------+ -``` \ No newline at end of file +##keyword +STR_TO_DATE,STR,TO,DATE diff --git a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/timediff.md b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/timediff.md index d45e6b4ef34232..7ab169a3c57f8f 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/timediff.md +++ b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/timediff.md @@ -1,10 +1,9 @@ # timediff - -## Syntax +## description +### Syntax `TIME TIMEDIFF(DATETIME expr1, DATETIME expr2)` -## Description TIMEDIFF返回两个DATETIME之间的差值 @@ -12,7 +11,7 @@ TIMEDIFF函数返回表示为时间值的expr1 - expr2的结果,返回值为TI 其结果被限制在从-838:59:59到838:59:59之间的TIME值范围内 -## Examples +## example ``` mysql> SELECT TIMEDIFF(now(),utc_timestamp()); @@ -43,3 +42,5 @@ mysql> SELECT TIMEDIFF('2019-01-01 00:00:00', NULL); | NULL | +---------------------------------------+ ``` +##keyword +TIMEDIFF diff --git a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/to_days.md b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/to_days.md index 2deb3cc5809684..4dc6894696c849 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/to_days.md +++ b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/to_days.md @@ -1,16 +1,15 @@ # to_days - -## Syntax +## description +### Syntax `INT TO_DAYS(DATETIME date)` -## Description 返回date距离0000-01-01的天数 参数为Date或者Datetime类型 -## Examples +## example ``` mysql> select to_days('2007-10-07'); @@ -19,4 +18,5 @@ mysql> select to_days('2007-10-07'); +-----------------------+ | 733321 | +-----------------------+ -``` \ No newline at end of file +##keyword +TO_DAYS,TO,DAYS diff --git a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/unix_timestamp.md b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/unix_timestamp.md index dfdb0461103973..fc63e69e07fb42 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/unix_timestamp.md +++ b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/unix_timestamp.md @@ -1,10 +1,9 @@ # unix_timestamp - -## Syntax +## description +### Syntax `INT UNIX_TIMESTAMP(), UNIX_TIMESTAMP(DATETIME date)` -## Description 将Date或者Datetime类型转化为unix时间戳 @@ -12,7 +11,7 @@ 参数需要是Date或者Datetime类型 -## Examples +## example ``` mysql> select unix_timestamp(); @@ -28,4 +27,5 @@ mysql> select unix_timestamp('2007-11-30 10:30:19'); +---------------------------------------+ | 1196389819 | +---------------------------------------+ -``` \ No newline at end of file +##keyword +UNIX_TIMESTAMP,UNIX,TIMESTAMP diff --git a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/utc_timestamp.md b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/utc_timestamp.md index a93ed98a139c86..c61d7a2da0ec55 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/utc_timestamp.md +++ b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/utc_timestamp.md @@ -1,10 +1,9 @@ # utc_timestamp - -## Syntax +## description +### Syntax `DATETIME UTC_TIMESTAMP()` -## Description 返回当前UTC日期和时间在 "YYYY-MM-DD HH:MM:SS" 或 @@ -12,7 +11,7 @@ 根据该函数是否用在字符串或数字语境中 -## Examples +## example ``` mysql> select utc_timestamp(),utc_timestamp() + 1; @@ -21,4 +20,5 @@ mysql> select utc_timestamp(),utc_timestamp() + 1; +---------------------+---------------------+ | 2019-07-10 12:31:18 | 20190710123119 | +---------------------+---------------------+ -``` \ No newline at end of file +##keyword +UTC_TIMESTAMP,UTC,TIMESTAMP diff --git a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/workofyear.md b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/workofyear.md index e04f2a45ad84c3..3377af454e0309 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/workofyear.md +++ b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/workofyear.md @@ -1,17 +1,16 @@ # weekofyear - -## Syntax +## description +### Syntax `INT WEEKOFYEAR(DATETIME date)` -## Description 获得一年中的第几周 参数为Date或者Datetime类型 -## Examples +## example ``` mysql> select weekofyear('2008-02-20 00:00:00'); @@ -20,4 +19,5 @@ mysql> select weekofyear('2008-02-20 00:00:00'); +-----------------------------------+ | 8 | +-----------------------------------+ -``` \ No newline at end of file +##keyword +WEEKOFYEAR diff --git a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/year.md b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/year.md index e2511edf4d43cf..1ed81559f893dc 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/year.md +++ b/docs/documentation/cn/sql-reference/sql-functions/date-time-functions/year.md @@ -1,16 +1,15 @@ # year - -## Syntax +## description +### Syntax `INT YEAR(DATETIME date)` -## Description 返回date类型的year部分,范围从1000-9999 参数为Date或者Datetime类型 -## Examples +## example ``` mysql> select year('1987-01-01'); @@ -19,4 +18,5 @@ mysql> select year('1987-01-01'); +-----------------------------+ | 1987 | +-----------------------------+ -``` \ No newline at end of file +##keyword +YEAR diff --git a/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_astext.md b/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_astext.md index 4b37351235b6f0..7e8c47c795d645 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_astext.md +++ b/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_astext.md @@ -1,14 +1,13 @@ # `ST_AsText`,`ST_AsWKT` - -## Syntax +## description +### Syntax `VARCHAR ST_AsText(GEOMETRY geo)` -## Description 将一个几何图形转化为WKT(Well Known Text)的表示形式 -## Examples +## example ``` mysql> SELECT ST_AsText(ST_Point(24.7, 56.7)); @@ -18,3 +17,5 @@ mysql> SELECT ST_AsText(ST_Point(24.7, 56.7)); | POINT (24.7 56.7) | +---------------------------------+ ``` +##keyword +ST_ASTEXT,ST_ASWKT,ST,ASTEXT,ASWKT diff --git a/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_circle.md b/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_circle.md index 073d49f6c80f86..b244e04366deb1 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_circle.md +++ b/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_circle.md @@ -1,15 +1,14 @@ # `ST_Circle` - -## Syntax +## description +### Syntax `GEOMETRY ST_Circle(DOUBLE center_lng, DOUBLE center_lat, DOUBLE radius)` -## Description 将一个WKT(Well Known Text)转化为地球球面上的一个圆。其中`center_lng`表示的圆心的经度, `center_lat`表示的是圆心的纬度,`radius`表示的是圆的半径,单位是米 -## Examples +## example ``` mysql> SELECT ST_AsText(ST_Circle(111, 64, 10000)); @@ -19,3 +18,5 @@ mysql> SELECT ST_AsText(ST_Circle(111, 64, 10000)); | CIRCLE ((111 64), 10000) | +--------------------------------------------+ ``` +##keyword +ST_CIRCLE,ST,CIRCLE diff --git a/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_contains.md b/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_contains.md index 47e939f4099609..ce47c52a45f148 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_contains.md +++ b/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_contains.md @@ -1,14 +1,13 @@ # `ST_Contains` - -## Syntax +## description +### Syntax `BOOL ST_Contains(GEOMETRY shape1, GEOMETRY shape2)` -## Description 判断几何图形shape1是否完全能够包含几何图形shape2 -## Examples +## example ``` mysql> SELECT ST_Contains(ST_Polygon("POLYGON ((0 0, 10 0, 10 10, 0 10, 0 0))"), ST_Point(5, 5)); @@ -25,3 +24,5 @@ mysql> SELECT ST_Contains(ST_Polygon("POLYGON ((0 0, 10 0, 10 10, 0 10, 0 0))"), | 0 | +------------------------------------------------------------------------------------------+ ``` +##keyword +ST_CONTAINS,ST,CONTAINS diff --git a/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_distance_sphere.md b/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_distance_sphere.md index 4b5b034f07e814..27b085861463b0 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_distance_sphere.md +++ b/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_distance_sphere.md @@ -1,14 +1,13 @@ # `ST_Distance_Sphere` - -## Syntax +## description +### Syntax `DOUBLE ST_Distance_Sphere(DOUBLE x_lng, DOUBLE x_lat, DOUBLE y_lng, DOUBLE x_lat)` -## Description 计算地球两点之间的球面距离,单位为 米。传入的参数分别为X点的经度,X点的纬度,Y点的经度,Y点的纬度。 -## Examples +## example ``` mysql> select st_distance_sphere(116.35620117, 39.939093, 116.4274406433, 39.9020987219); @@ -18,3 +17,5 @@ mysql> select st_distance_sphere(116.35620117, 39.939093, 116.4274406433, 39.902 | 7336.9135549995917 | +----------------------------------------------------------------------------+ ``` +##keyword +ST_DISTANCE_SPHERE,ST,DISTANCE,SPHERE diff --git a/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_geometryfromtext.md b/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_geometryfromtext.md index 577d1558a09ccb..72dd9d7dfbaa58 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_geometryfromtext.md +++ b/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_geometryfromtext.md @@ -1,14 +1,13 @@ # `ST_GeometryFromText`,`ST_GeomFromText` - -## Syntax +## description +### Syntax `GEOMETRY ST_GeometryFromText(VARCHAR wkt)` -## Description 将一个WKT(Well Known Text)转化为对应的内存的几何形式 -## Examples +## example ``` mysql> SELECT ST_AsText(ST_GeometryFromText("LINESTRING (1 1, 2 2)")); @@ -18,3 +17,5 @@ mysql> SELECT ST_AsText(ST_GeometryFromText("LINESTRING (1 1, 2 2)")); | LINESTRING (1 1, 2 2) | +---------------------------------------------------------+ ``` +##keyword +ST_GEOMETRYFROMTEXT,ST_GEOMFROMTEXT,ST,GEOMETRYFROMTEXT,GEOMFROMTEXT diff --git a/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_linefromtext.md b/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_linefromtext.md index 40201909932ad3..7f7165cd9f5d65 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_linefromtext.md +++ b/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_linefromtext.md @@ -1,14 +1,13 @@ # `ST_LineFromText`,`ST_LineStringFromText` - -## Syntax +## description +### Syntax `GEOMETRY ST_LineFromText(VARCHAR wkt)` -## Description 将一个WKT(Well Known Text)转化为一个Line形式的内存表现形式 -## Examples +## example ``` mysql> SELECT ST_AsText(ST_LineFromText("LINESTRING (1 1, 2 2)")); @@ -18,3 +17,5 @@ mysql> SELECT ST_AsText(ST_LineFromText("LINESTRING (1 1, 2 2)")); | LINESTRING (1 1, 2 2) | +---------------------------------------------------------+ ``` +##keyword +ST_LINEFROMTEXT,ST_LINESTRINGFROMTEXT,ST,LINEFROMTEXT,LINESTRINGFROMTEXT diff --git a/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_point.md b/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_point.md index 9bf5e30f2178f8..876ac0035b12e3 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_point.md +++ b/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_point.md @@ -1,15 +1,14 @@ # `ST_Point` - -## Syntax +## description +### Syntax `POINT ST_Point(DOUBLE x, DOUBLE y)` -## Description 通过给定的X坐标值,Y坐标值返回对应的Point。 当前这个值只是在球面集合上有意义,X/Y对应的是经度/纬度(longitude/latitude) -## Examples +## example ``` mysql> SELECT ST_AsText(ST_Point(24.7, 56.7)); @@ -19,3 +18,5 @@ mysql> SELECT ST_AsText(ST_Point(24.7, 56.7)); | POINT (24.7 56.7) | +---------------------------------+ ``` +##keyword +ST_POINT,ST,POINT diff --git a/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_polygon.md b/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_polygon.md index 6e5765e3e61918..d1b810199e5fd6 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_polygon.md +++ b/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_polygon.md @@ -1,14 +1,13 @@ # `ST_Polygon`,`ST_PolyFromText`,`ST_PolygonFromText` - -## Syntax +## description +### Syntax `GEOMETRY ST_Polygon(VARCHAR wkt)` -## Description 将一个WKT(Well Known Text)转化为对应的多边形内存形式 -## Examples +## example ``` mysql> SELECT ST_AsText(ST_Polygon("POLYGON ((0 0, 10 0, 10 10, 0 10, 0 0))")); @@ -18,3 +17,5 @@ mysql> SELECT ST_AsText(ST_Polygon("POLYGON ((0 0, 10 0, 10 10, 0 10, 0 0))")); | POLYGON ((0 0, 10 0, 10 10, 0 10, 0 0)) | +------------------------------------------------------------------+ ``` +##keyword +ST_POLYGON,ST_POLYFROMTEXT,ST_POLYGONFROMTEXT,ST,POLYGON,POLYFROMTEXT,POLYGONFROMTEXT diff --git a/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_x.md b/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_x.md index 644b9ee11dc1a6..f819e697c4f876 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_x.md +++ b/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_x.md @@ -1,14 +1,13 @@ # `ST_X` - -## Syntax +## description +### Syntax `DOUBLE ST_X(POINT point)` -## Description 当point是一个合法的POINT类型时,返回对应的X坐标值 -## Examples +## example ``` mysql> SELECT ST_X(ST_Point(24.7, 56.7)); @@ -18,3 +17,5 @@ mysql> SELECT ST_X(ST_Point(24.7, 56.7)); | 24.7 | +----------------------------+ ``` +##keyword +ST_X,ST,X diff --git a/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_y.md b/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_y.md index fb02ddb102d4c0..a8bc23b257ae28 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_y.md +++ b/docs/documentation/cn/sql-reference/sql-functions/spatial-functions/st_y.md @@ -1,14 +1,13 @@ # `ST_Y` - -## Syntax +## description +### Syntax `DOUBLE ST_Y(POINT point)` -## Description 当point是一个合法的POINT类型时,返回对应的Y坐标值 -## Examples +## example ``` mysql> SELECT ST_Y(ST_Point(24.7, 56.7)); @@ -18,3 +17,5 @@ mysql> SELECT ST_Y(ST_Point(24.7, 56.7)); | 56.7 | +----------------------------+ ``` +##keyword +ST_Y,ST,Y diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/ascii.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/ascii.md index 95debfd84a0a0d..49f96fa28409b7 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/string-functions/ascii.md +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/ascii.md @@ -1,14 +1,13 @@ # ascii - -## Syntax +## description +### Syntax `INT ascii(VARCHAR str)` -## Description 返回字符串第一个字符对应的 ascii 码 -## Examples +## example ``` mysql> select ascii('1'); @@ -25,3 +24,5 @@ mysql> select ascii('234'); | 50 | +--------------+ ``` +##keyword +ASCII diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/concat.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/concat.md index 92ba559f613bf5..4bcae3df62183c 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/string-functions/concat.md +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/concat.md @@ -1,14 +1,13 @@ # concat - -## Syntax +## description +### Syntax `VARCHAR concat(VARCHAR,...)` -## Description 将多个字符串连接起来, 如果参数中任意一个值是 NULL,那么返回的结果就是 NULL -## Examples +## example ``` mysql> select concat("a", "b"); @@ -32,3 +31,5 @@ mysql> select concat("a", null, "c"); | NULL | +------------------------+ ``` +##keyword +CONCAT diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/concat_ws.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/concat_ws.md index d72b7fd7e6b816..44784208fae6ad 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/string-functions/concat_ws.md +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/concat_ws.md @@ -1,16 +1,15 @@ # concat_ws - -## Syntax +## description +### Syntax `VARCHAR concat_ws(VARCHAR sep, VARCHAR str,...)` -## Description 使用第一个参数 sep 作为连接符,将第二个参数以及后续所有参数拼接成一个字符串. 如果分隔符是 NULL,返回 NULL。 `concat_ws`函数不会跳过空字符串,会跳过 NULL 值 -## Examples +## example ``` mysql> select concat_ws("or", "d", "is"); @@ -34,3 +33,5 @@ mysql> select concat_ws("or", "d", NULL,"is"); | doris | +---------------------------------+ ``` +##keyword +CONCAT_WS,CONCAT,WS diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/find_in_set.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/find_in_set.md index 121a6672e297f1..e66d41d7ac7d6a 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/string-functions/find_in_set.md +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/find_in_set.md @@ -1,14 +1,13 @@ # find_in_set - -## Syntax +## description +### Syntax `INT find_in_set(VARCHAR str, VARCHAR strlist)` -## Description 返回 strlist 中第一次出现 str 的位置(从1开始计数)。strlist 是用逗号分隔的字符串。如果没有找到,返回0。任意参数为 NULL ,返回 NULL。 -## Examples +## example ``` mysql> select find_in_set("b", "a,b,c"); @@ -18,3 +17,5 @@ mysql> select find_in_set("b", "a,b,c"); | 2 | +---------------------------+ ``` +##keyword +FIND_IN_SET,FIND,IN,SET diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/get_json_double.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/get_json_double.md index a7b1bd198c7bcd..c6f766e6fe515f 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/string-functions/get_json_double.md +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/get_json_double.md @@ -1,10 +1,9 @@ # get_json_double - -## Syntax +## description +### Syntax `DOUBLE get_json_double(VARCHAR json_str, VARCHAR json_path) -## Description 解析并获取 json 字符串内指定路径的浮点型内容。 其中 json_path 必须以 $ 符号作为开头,使用 . 作为路径分割符。如果路径中包含 . ,则可以使用双引号包围。 @@ -12,7 +11,7 @@ path 的内容不能包含 ", [ 和 ]。 如果 json_string 格式不对,或 json_path 格式不对,或无法找到匹配项,则返回 NULL。 -## Examples +## example 1. 获取 key 为 "k1" 的 value @@ -45,3 +44,5 @@ mysql> SELECT get_json_double('{"k1.key":{"k2":[1.1, 2.2]}}', '$."k1.key".k2[0]' | 1.1 | +---------------------------------------------------------------------+ ``` +##keyword +GET_JSON_DOUBLE,GET,JSON,DOUBLE diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/get_json_int.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/get_json_int.md index 005fad597c9c48..56366e4b956b16 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/string-functions/get_json_int.md +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/get_json_int.md @@ -1,10 +1,9 @@ # get_json_int - -## Syntax +## description +### Syntax `INT get_json_int(VARCHAR json_str, VARCHAR json_path) -## Description 解析并获取 json 字符串内指定路径的整型内容。 其中 json_path 必须以 $ 符号作为开头,使用 . 作为路径分割符。如果路径中包含 . ,则可以使用双引号包围。 @@ -12,7 +11,7 @@ path 的内容不能包含 ", [ 和 ]。 如果 json_string 格式不对,或 json_path 格式不对,或无法找到匹配项,则返回 NULL。 -## Examples +## example 1. 获取 key 为 "k1" 的 value @@ -45,3 +44,5 @@ mysql> SELECT get_json_int('{"k1.key":{"k2":[1, 2]}}', '$."k1.key".k2[0]'); | 1 | +--------------------------------------------------------------+ ``` +##keyword +GET_JSON_INT,GET,JSON,INT diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/get_json_string.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/get_json_string.md index ab576a8ac189d3..504de06f92c318 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/string-functions/get_json_string.md +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/get_json_string.md @@ -1,10 +1,9 @@ # get_json_string - -## Syntax +## description +### Syntax `VARCHAR get_json_string(VARCHAR json_str, VARCHAR json_path) -## Description 解析并获取 json 字符串内指定路径的字符串内容。 其中 json_path 必须以 $ 符号作为开头,使用 . 作为路径分割符。如果路径中包含 . ,则可以使用双引号包围。 @@ -12,7 +11,7 @@ path 的内容不能包含 ", [ 和 ]。 如果 json_string 格式不对,或 json_path 格式不对,或无法找到匹配项,则返回 NULL。 -## Examples +## example 1. 获取 key 为 "k1" 的 value @@ -55,3 +54,5 @@ mysql> SELECT get_json_string('[{"k1":"v1"}, {"k2":"v2"}, {"k1":"v3"}, {"k1":"v4 | ["v1","v3","v4"] | +---------------------------------------------------------------------------------+ ``` +##keyword +GET_JSON_STRING,GET,JSON,STRING diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/group_concat.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/group_concat.md index f1a3151516588c..721f820543d25e 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/string-functions/group_concat.md +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/group_concat.md @@ -1,14 +1,13 @@ # group_concat - -## Syntax +## description +### Syntax `VARCHAR group_concat(VARCHAR str[, VARCHAR sep])` -## Description 该函数是类似于 sum() 的聚合函数,group_concat 将结果集中的多行结果连接成一个字符串。第二个参数 sep 为字符串之间的连接符号,该参数可以省略。该函数通常需要和 group by 语句一起使用。 -## Examples +## example ``` mysql> select value from test; @@ -34,3 +33,5 @@ mysql> select group_concat(value, " ") from test; | a b c | +----------------------------+ ``` +##keyword +GROUP_CONCAT,GROUP,CONCAT diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/instr.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/instr.md index 58276bf4c4c098..65db339d3b7592 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/string-functions/instr.md +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/instr.md @@ -1,14 +1,13 @@ # instr - -## Syntax +## description +### Syntax `INT instr(VARCHAR str, VARCHAR substr)` -## Description 返回 substr 在 str 中第一次出现的位置(从1开始计数)。如果 substr 不在 str 中出现,则返回0。 -## Examples +## example ``` mysql> select instr("abc", "b"); @@ -25,3 +24,5 @@ mysql> select instr("abc", "d"); | 0 | +-------------------+ ``` +##keyword +INSTR diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/lcase.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/lcase.md index be8464ba14c674..7acb6840be8b18 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/string-functions/lcase.md +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/lcase.md @@ -1,10 +1,11 @@ # lcase - -## Syntax +## description +### Syntax `INT lcase(VARCHAR str)` -## Description 与`lower`一致 +##keyword +LCASE diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/left.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/left.md index b8376bfbca0e9e..705b3f88655146 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/string-functions/left.md +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/left.md @@ -1,14 +1,13 @@ # left - -## Syntax +## description +### Syntax `VARCHAR left(VARCHAR str)` -## Description 它返回具有指定长度的字符串的左边部分 -## Examples +## example ``` mysql> select left("Hello doris",5); @@ -18,3 +17,5 @@ mysql> select left("Hello doris",5); | Hello | +------------------------+ ``` +##keyword +LEFT diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/length.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/length.md index 03b3ce42bbf534..fa852edd50e67d 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/string-functions/length.md +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/length.md @@ -1,14 +1,13 @@ # length - -## Syntax +## description +### Syntax `INT length(VARCHAR str)` -## Description 返回字符串的长度,对于多字节字符,返回的字符数。比如5个两字节宽度字,返回的长度是10。 -## Examples +## example ``` mysql> select length("abc"); @@ -25,3 +24,5 @@ mysql> select length("中国"); | 6 | +------------------+ ``` +##keyword +LENGTH diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/locate.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/locate.md index e6a7704381335f..e239904e014287 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/string-functions/locate.md +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/locate.md @@ -1,14 +1,13 @@ # locate - -## Syntax +## description +### Syntax `INT locate(VARCHAR substr, VARCHAR str[, INT pos])` -## Description 返回 substr 在 str 中出现的位置(从1开始计数)。如果指定第3个参数 pos,则从 str 以 pos 下标开始的字符串处开始查找 substr 出现的位置。如果没有找到,返回0 -## Examples +## example ``` mysql> SELECT LOCATE('bar', 'foobarbar'); @@ -32,3 +31,5 @@ mysql> SELECT LOCATE('bar', 'foobarbar', 5); | 7 | +-------------------------------+ ``` +##keyword +LOCATE diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/lower.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/lower.md index 776d559c199f84..00f3e20313aa64 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/string-functions/lower.md +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/lower.md @@ -1,14 +1,13 @@ # lower - -## Syntax +## description +### Syntax `INT lower(VARCHAR str)` -## Description 将参数中所有的字符串都转换成小写 -## Examples +## example ``` mysql> SELECT lower("AbC123"); @@ -18,3 +17,5 @@ mysql> SELECT lower("AbC123"); | abc123 | +-----------------+ ``` +##keyword +LOWER diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/lpad.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/lpad.md index 8ba1176ea5533c..41c836a168b047 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/string-functions/lpad.md +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/lpad.md @@ -1,14 +1,13 @@ # lpad - -## Syntax +## description +### Syntax `VARCHAR lpad(VARCHAR str, INT len, VARCHAR pad)` -## Description 返回 str 中长度为 len(从首字母开始算起)的字符串。如果 len 大于 str 的长度,则在 str 的前面不断补充 pad 字符,直到该字符串的长度达到 len 为止。如果 len 小于 str 的长度,该函数相当于截断 str 字符串,只返回长度为 len 的字符串。 -## Examples +## example ``` mysql> SELECT lpad("hi", 5, "xy"); @@ -25,3 +24,5 @@ mysql> SELECT lpad("hi", 1, "xy"); | h | +---------------------+ ``` +##keyword +LPAD diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/ltrim.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/ltrim.md index 4a265d0868f9f5..c3c8f04c383a7a 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/string-functions/ltrim.md +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/ltrim.md @@ -1,14 +1,13 @@ # ltrim - -## Syntax +## description +### Syntax `VARCHAR ltrim(VARCHAR str)` -## Description 将参数 str 中从开始部分连续出现的空格去掉 -## Examples +## example ``` mysql> SELECT ltrim(' ab d'); @@ -18,3 +17,5 @@ mysql> SELECT ltrim(' ab d'); | ab d | +------------------+ ``` +##keyword +LTRIM diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/money_format.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/money_format.md index 98da921327d4dd..6abc5de94f408d 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/string-functions/money_format.md +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/money_format.md @@ -1,14 +1,13 @@ # money_format - -## Syntax +## description +### Syntax VARCHAR money_format(Number) -## Description 将数字按照货币格式输出,整数部分每隔3位用逗号分隔,小数部分保留2位 -## Examples +## example ``` mysql> select money_format(17014116); @@ -32,3 +31,5 @@ mysql> select money_format(1123.4); | 1,123.40 | +----------------------+ ``` +##keyword +MONEY_FORMAT,MONEY,FORMAT diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/regexp_extract.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/regexp_extract.md index 8cbf690eea9cb5..5746b7b7b0cdd8 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/string-functions/regexp_extract.md +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/regexp_extract.md @@ -1,14 +1,13 @@ # regexp_extract - -## Syntax +## description +### Syntax `VARCHAR regexp_extract(VARCHAR str, VARCHAR pattern, int pos)` -## Description 对字符串 str 进行正则匹配,抽取符合 pattern 的第 pos 个匹配部分。需要 pattern 完全匹配 str 中的某部分,这样才能返回 pattern 部分中需匹配部分。如果没有匹配,返回空字符串。 -## Examples +## example ``` mysql> SELECT regexp_extract('AbCdE', '([[:lower:]]+)C([[:lower:]]+)', 1); @@ -25,3 +24,5 @@ mysql> SELECT regexp_extract('AbCdE', '([[:lower:]]+)C([[:lower:]]+)', 2); | d | +-------------------------------------------------------------+ ``` +##keyword +REGEXP_EXTRACT,REGEXP,EXTRACT diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/regexp_replace.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/regexp_replace.md index c131996cdc4760..c6bdae2d96e723 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/string-functions/regexp_replace.md +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/regexp_replace.md @@ -1,14 +1,13 @@ # regexp_replace - -## Syntax +## description +### Syntax `VARCHAR regexp_replace(VARCHAR str, VARCHAR pattern, VARCHAR repl) -## Description 对字符串 str 进行正则匹配, 将命中 pattern 的部分使用 repl 来进行替换 -## Examples +## example ``` mysql> SELECT regexp_replace('a b c', " ", "-"); @@ -25,3 +24,5 @@ mysql> SELECT regexp_replace('a b c','(b)','<\\1>'); | a c | +----------------------------------------+ ``` +##keyword +REGEXP_REPLACE,REGEXP,REPLACE diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/repeat.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/repeat.md index 4ef1eeb2f70acb..71885baaba7a39 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/string-functions/repeat.md +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/repeat.md @@ -1,14 +1,13 @@ # repeat - -## Syntax +## description +### Syntax `VARCHAR repeat(VARCHAR str, INT count) -## Description 将字符串 str 重复 count 次输出,count 小于1时返回空串,str,count 任一为NULL时,返回 NULL -## Examples +## example ``` mysql> SELECT repeat("a", 3); @@ -25,3 +24,5 @@ mysql> SELECT repeat("a", -1); | | +-----------------+ ``` +##keyword +REPEAT, diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/right.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/right.md index 832a124619d9f2..e5fb9a16cd7a04 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/string-functions/right.md +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/right.md @@ -1,14 +1,13 @@ # right - -## Syntax +## description +### Syntax `VARCHAR right(VARCHAR str)` -## Description 它返回具有指定长度的字符串的右边部分 -## Examples +## example ``` mysql> select right("Hello doris",5); @@ -18,3 +17,5 @@ mysql> select right("Hello doris",5); | doris | +-------------------------+ ``` +##keyword +RIGHT diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/split_part.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/split_part.md index 142a567c4104f8..80c8c25292771a 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/string-functions/split_part.md +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/split_part.md @@ -1,14 +1,13 @@ # split_part - -## Syntax +## description +### Syntax `VARCHAR split_part(VARCHAR content, VARCHAR delimiter, INT field)` -## Description 根据分割符拆分字符串, 返回指定的分割部分(从一开始计数)。 -## Examples +## example ``` mysql> select split_part("hello world", " ", 1); @@ -40,3 +39,5 @@ mysql> select split_part("abca", "a", 1); | | +----------------------------+ ``` +##keyword +SPLIT_PART,SPLIT,PART diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/strleft.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/strleft.md index 2f7f62027e581a..c2c4d904a81d91 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/string-functions/strleft.md +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/strleft.md @@ -1,14 +1,13 @@ # strleft - -## Syntax +## description +### Syntax `VARCHAR strleft(VARCHAR str)` -## Description 它返回具有指定长度的字符串的左边部分 -## Examples +## example ``` mysql> select strleft("Hello doris",5); @@ -18,3 +17,5 @@ mysql> select strleft("Hello doris",5); | Hello | +------------------------+ ``` +##keyword +STRLEFT diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/strright.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/strright.md index aeea95ee3ada3d..7976d9288c4dbd 100644 --- a/docs/documentation/cn/sql-reference/sql-functions/string-functions/strright.md +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/strright.md @@ -1,14 +1,13 @@ # strright - -## Syntax +## description +### Syntax `VARCHAR strright(VARCHAR str)` -## Description 它返回具有指定长度的字符串的右边部分 -## Examples +## example ``` mysql> select strright("Hello doris",5); @@ -18,3 +17,5 @@ mysql> select strright("Hello doris",5); | doris | +-------------------------+ ``` +##keyword +STRRIGHT diff --git a/docs/documentation/cn/sql-reference/sql-statements/Account Management/CREATE ROLE.md b/docs/documentation/cn/sql-reference/sql-statements/Account Management/CREATE ROLE.md new file mode 100644 index 00000000000000..5af07a5342ba86 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Account Management/CREATE ROLE.md @@ -0,0 +1,19 @@ +# CREATE ROLE +## description + 该语句用户创建一个角色 + + 语法: + CREATE ROLE role1; + + 该语句创建一个无权限的角色,可以后续通过 GRANT 命令赋予该角色权限。 + +## example + + 1. 创建一个角色 + + CREATE ROLE role1; + +## keyword + CREATE, ROLE + + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Account Management/CREATE USER.md b/docs/documentation/cn/sql-reference/sql-statements/Account Management/CREATE USER.md new file mode 100644 index 00000000000000..d3f891c4be4e55 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Account Management/CREATE USER.md @@ -0,0 +1,49 @@ +# CREATE USER +## description + +Syntax: + + CREATE USER user_identity [IDENTIFIED BY 'password'] [DEFAULT ROLE 'role_name'] + + user_identity: + 'user_name'@'host' + +CREATE USER 命令用于创建一个 Doris 用户。在 Doris 中,一个 user_identity 唯一标识一个用户。user_identity 由两部分组成,user_name 和 host,其中 username 为用户名。host 标识用户端连接所在的主机地址。host 部分可以使用 % 进行模糊匹配。如果不指定 host,默认为 '%',即表示该用户可以从任意 host 连接到 Doris。 + +host 部分也可指定为 domain,语法为:'user_name'@['domain'],即使用中括号包围,则 Doris 会认为这个是一个 domain,并尝试解析其 ip 地址。目前仅支持百度内部的 BNS 解析。 + +如果指定了角色(ROLE),则会自动将该角色所拥有的权限赋予新创建的这个用户。如果不指定,则该用户默认没有任何权限。指定的 ROLE 必须已经存在。 + +## example + +1. 创建一个无密码用户(不指定 host,则等价于 jack@'%') + + CREATE USER 'jack'; + +2. 创建一个有密码用户,允许从 '172.10.1.10' 登陆 + + CREATE USER jack@'172.10.1.10' IDENTIFIED BY '123456'; + +3. 为了避免传递明文,用例2也可以使用下面的方式来创建 + + CREATE USER jack@'172.10.1.10' IDENTIFIED BY PASSWORD '*6BB4837EB74329105EE4568DDA7DC67ED2CA2AD9'; + + 后面加密的内容可以通过PASSWORD()获得到,例如: + + SELECT PASSWORD('123456'); + +4. 创建一个允许从 '192.168' 子网登陆的用户,同时指定其角色为 example_role + + CREATE USER 'jack'@'192.168.%' DEFAULT ROLE 'example_role'; + +5. 创建一个允许从域名 'example_domain' 登陆的用户 + + CREATE USER 'jack'@['example_domain'] IDENTIFIED BY '12345'; + +6. 创建一个用户,并指定一个角色 + + CREATE USER 'jack'@'%' IDENTIFIED BY '12345' DEFAULT ROLE 'my_role'; + +## keyword + CREATE, USER + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Account Management/DROP ROLE.md b/docs/documentation/cn/sql-reference/sql-statements/Account Management/DROP ROLE.md new file mode 100644 index 00000000000000..b7d7e248b96843 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Account Management/DROP ROLE.md @@ -0,0 +1,18 @@ +# DROP ROLE +## description + 该语句用户删除一个角色 + + 语法: + DROP ROLE role1; + + 删除一个角色,不会影响之前属于该角色的用户的权限。仅相当于将该角色与用户解耦。用户已经从该角色中获取到的权限,不会改变。 + +## example + + 1. 删除一个角色 + + DROP ROLE role1; + +## keyword + DROP, ROLE + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Account Management/DROP USER.md b/docs/documentation/cn/sql-reference/sql-statements/Account Management/DROP USER.md new file mode 100644 index 00000000000000..3711947d5ce9cf --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Account Management/DROP USER.md @@ -0,0 +1,18 @@ +# DROP USER +## description + +Syntax: + + DROP USER 'user_name' + + DROP USER 命令会删除一个 palo 用户。这里 Doris 不支持删除指定的 user_identity。当删除一个指定用户后,该用户所对应的所有 user_identity 都会被删除。比如之前通过 CREATE USER 语句创建了 jack@'192.%' 以及 jack@['domain'] 两个用户,则在执行 DROP USER 'jack' 后,jack@'192.%' 以及 jack@['domain'] 都将被删除。 + +## example + +1. 删除用户 jack + + DROP USER 'jack' + +## keyword + DROP, USER + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Account Management/GRANT.md b/docs/documentation/cn/sql-reference/sql-statements/Account Management/GRANT.md new file mode 100644 index 00000000000000..725ee855335a03 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Account Management/GRANT.md @@ -0,0 +1,55 @@ +# GRANT +## description + +GRANT 命令用于赋予指定用户或角色指定的权限。 + +Syntax: + + GRANT privilege_list ON db_name[.tbl_name] TO user_identity [ROLE role_name] + + +privilege_list 是需要赋予的权限列表,以逗号分隔。当前 Doris 支持如下权限: + + NODE_PRIV:集群节点操作权限,包括节点上下线等操作,只有 root 用户有该权限,不可赋予其他用户。 + ADMIN_PRIV:除 NODE_PRIV 以外的所有权限。 + GRANT_PRIV: 操作权限的权限。包括创建删除用户、角色,授权和撤权,设置密码等。 + SELECT_PRIV:对指定的库或表的读取权限 + LOAD_PRIV:对指定的库或表的导入权限 + ALTER_PRIV:对指定的库或表的schema变更权限 + CREATE_PRIV:对指定的库或表的创建权限 + DROP_PRIV:对指定的库或表的删除权限 + + 旧版权限中的 ALL 和 READ_WRITE 会被转换成:SELECT_PRIV,LOAD_PRIV,ALTER_PRIV,CREATE_PRIV,DROP_PRIV; + READ_ONLY 会被转换为 SELECT_PRIV。 + +db_name[.tbl_name] 支持以下三种形式: + + 1. *.* 权限可以应用于所有库及其中所有表 + 2. db.* 权限可以应用于指定库下的所有表 + 3. db.tbl 权限可以应用于指定库下的指定表 + + 这里指定的库或表可以是不存在的库和表。 + +user_identity: + + 这里的 user_identity 语法同 CREATE USER。且必须为使用 CREATE USER 创建过的 user_identity。user_identity 中的host可以是域名,如果是域名的话,权限的生效时间可能会有1分钟左右的延迟。 + + 也可以将权限赋予指定的 ROLE,如果指定的 ROLE 不存在,则会自动创建。 + +## example + + 1. 授予所有库和表的权限给用户 + + GRANT SELECT_PRIV ON *.* TO 'jack'@'%'; + + 2. 授予指定库表的权限给用户 + + GRANT SELECT_PRIV,ALTER_PRIVS,LOAD_PRIV ON db1.tbl1 TO 'jack'@'192.8.%'; + + 3. 授予指定库表的权限给角色 + + GRANT LOAD_PRIV ON db1.* TO ROLE 'my_role'; + +## keyword + GRANT + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Account Management/REVOKE.md b/docs/documentation/cn/sql-reference/sql-statements/Account Management/REVOKE.md new file mode 100644 index 00000000000000..bb377051d69829 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Account Management/REVOKE.md @@ -0,0 +1,23 @@ +# REVOKE +## description + + REVOKE 命令用于撤销指定用户或角色指定的权限。 + Syntax: + REVOKE privilege_list ON db_name[.tbl_name] FROM user_identity [ROLE role_name] + + user_identity: + + 这里的 user_identity 语法同 CREATE USER。且必须为使用 CREATE USER 创建过的 user_identity。user_identity 中的host可以是域名,如果是域名的话,权限的撤销时间可能会有1分钟左右的延迟。 + + 也可以撤销指定的 ROLE 的权限,执行的 ROLE 必须存在。 + +## example + + 1. 撤销用户 jack 数据库 testDb 的权限 + + REVOKE SELECT_PRIV ON db1.* FROM 'jack'@'192.%'; + +## keyword + + REVOKE + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Account Management/SET PASSWORD.md b/docs/documentation/cn/sql-reference/sql-statements/Account Management/SET PASSWORD.md new file mode 100644 index 00000000000000..27eac65537f586 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Account Management/SET PASSWORD.md @@ -0,0 +1,30 @@ +# SET PASSWORD +## description + +Syntax: + + SET PASSWORD [FOR user_identity] = + [PASSWORD('plain password')]|['hashed password'] + + SET PASSWORD 命令可以用于修改一个用户的登录密码。如果 [FOR user_identity] 字段不存在,那么修改当前用户的密码。 + + 注意这里的 user_identity 必须完全匹配在使用 CREATE USER 创建用户时指定的 user_identity,否则会报错用户不存在。如果不指定 user_identity,则当前用户为 'username'@'ip',这个当前用户,可能无法匹配任何 user_identity。可以通过 SHOW GRANTS 查看当前用户。 + + PASSWORD() 方式输入的是明文密码; 而直接使用字符串,需要传递的是已加密的密码。 + 如果修改其他用户的密码,需要具有管理员权限。 + +## example + +1. 修改当前用户的密码 + + SET PASSWORD = PASSWORD('123456') + SET PASSWORD = '*6BB4837EB74329105EE4568DDA7DC67ED2CA2AD9' + +2. 修改指定用户密码 + + SET PASSWORD FOR 'jack'@'192.%' = PASSWORD('123456') + SET PASSWORD FOR 'jack'@['domain'] = '*6BB4837EB74329105EE4568DDA7DC67ED2CA2AD9' + +## keyword + SET, PASSWORD + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Account Management/SET PROPERTY.md b/docs/documentation/cn/sql-reference/sql-statements/Account Management/SET PROPERTY.md new file mode 100644 index 00000000000000..f324ffaacfa786 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Account Management/SET PROPERTY.md @@ -0,0 +1,56 @@ +# SET PROPERTY +## description + + Syntax: + + SET PROPERTY [FOR 'user'] 'key' = 'value' [, 'key' = 'value'] + + 设置用户的属性,包括分配给用户的资源、导入cluster等。这里设置的用户属性,是针对 user 的,而不是 user_identity。即假设通过 CREATE USER 语句创建了两个用户 'jack'@'%' 和 'jack'@'192.%',则使用 SET PROPERTY 语句,只能针对 jack 这个用户,而不是 'jack'@'%' 或 'jack'@'192.%' + + 导入 cluster 仅适用于百度内部用户。 + + key: + + 超级用户权限: + max_user_connections: 最大连接数。 + resource.cpu_share: cpu资源分配。 + load_cluster.{cluster_name}.priority: 为指定的cluster分配优先级,可以为 HIGH 或 NORMAL + + 普通用户权限: + quota.normal: normal级别的资源分配。 + quota.high: high级别的资源分配。 + quota.low: low级别的资源分配。 + + load_cluster.{cluster_name}.hadoop_palo_path: palo使用的hadoop目录,需要存放etl程序及etl生成的中间数据供palo导入。导入完成后会自动清理中间数据,etl程序自动保留下次使用。 + load_cluster.{cluster_name}.hadoop_configs: hadoop的配置,其中fs.default.name、mapred.job.tracker、hadoop.job.ugi必须填写。 + load_cluster.{cluster_name}.hadoop_http_port: hadoop hdfs name node http端口。其中 hdfs 默认为8070,afs 默认 8010。 + default_load_cluster: 默认的导入cluster。 + +## example + + 1. 修改用户 jack 最大连接数为1000 + SET PROPERTY FOR 'jack' 'max_user_connections' = '1000'; + + 2. 修改用户 jack 的cpu_share为1000 + SET PROPERTY FOR 'jack' 'resource.cpu_share' = '1000'; + + 3. 修改 jack 用户的normal组的权重 + SET PROPERTY FOR 'jack' 'quota.normal' = '400'; + + 4. 为用户 jack 添加导入cluster + SET PROPERTY FOR 'jack' + 'load_cluster.{cluster_name}.hadoop_palo_path' = '/user/palo/palo_path', + 'load_cluster.{cluster_name}.hadoop_configs' = 'fs.default.name=hdfs://dpp.cluster.com:port;mapred.job.tracker=dpp.cluster.com:port;hadoop.job.ugi=user,password;mapred.job.queue.name=job_queue_name_in_hadoop;mapred.job.priority=HIGH;'; + + 5. 删除用户 jack 下的导入cluster。 + SET PROPERTY FOR 'jack' 'load_cluster.{cluster_name}' = ''; + + 6. 修改用户 jack 默认的导入cluster + SET PROPERTY FOR 'jack' 'default_load_cluster' = '{cluster_name}'; + + 7. 修改用户 jack 的集群优先级为 HIGH + SET PROPERTY FOR 'jack' 'load_cluster.{cluster_name}.priority' = 'HIGH'; + +## keyword + SET, PROPERTY + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Account Management/SHOW GRANTS.md b/docs/documentation/cn/sql-reference/sql-statements/Account Management/SHOW GRANTS.md new file mode 100644 index 00000000000000..16d65357292ce1 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Account Management/SHOW GRANTS.md @@ -0,0 +1,30 @@ +# SHOW GRANTS +## description + + 该语句用于查看用户权限。 + + 语法: + SHOW [ALL] GRANTS [FOR user_identity]; + + 说明: + 1. SHOW ALL GRANTS 可以查看所有用户的权限。 + 2. 如果指定 user_identity,则查看该指定用户的权限。且该 user_identity 必须为通过 CREATE USER 命令创建的。 + 3. 如果不指定 user_identity,则查看当前用户的权限。 + + +## example + + 1. 查看所有用户权限信息 + + SHOW ALL GRANTS; + + 2. 查看指定 user 的权限 + + SHOW GRANTS FOR jack@'%'; + + 3. 查看当前用户的权限 + + SHOW GRANTS; + +## keyword + SHOW, GRANTS diff --git a/docs/documentation/cn/sql-reference/sql-statements/Account Management/SHOW ROLES.md b/docs/documentation/cn/sql-reference/sql-statements/Account Management/SHOW ROLES.md new file mode 100644 index 00000000000000..b25119cedfa1dc --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Account Management/SHOW ROLES.md @@ -0,0 +1,16 @@ +# SHOW ROLES +## description + 该语句用于展示所有已创建的角色信息,包括角色名称,包含的用户以及权限。 + + 语法: + SHOW ROLES; + +## example + + 1. 查看已创建的角色: + + SHOW ROLES; + +## keyword + SHOW,ROLES + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Administration/ADMIN CANCEL REPAIR.md b/docs/documentation/cn/sql-reference/sql-statements/Administration/ADMIN CANCEL REPAIR.md new file mode 100644 index 00000000000000..52e521b254979e --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Administration/ADMIN CANCEL REPAIR.md @@ -0,0 +1,22 @@ +# ADMIN CANCEL REPAIR +## description + + 该语句用于取消以高优先级修复指定表或分区 + + 语法: + + ADMIN CANCEL REPAIR TABLE table_name[ PARTITION (p1,...)]; + + 说明: + + 1. 该语句仅表示系统不再以高优先级修复指定表或分区的分片副本。系统仍会以默认调度方式修复副本。 + +## example + + 1. 取消高优先级修复 + + ADMIN CANCEL REPAIR TABLE tbl PARTITION(p1); + +## keyword + ADMIN,CANCEL,REPAIR + diff --git a/docs/help/Contents/Administration/admin_repair_stmt.md b/docs/documentation/cn/sql-reference/sql-statements/Administration/ADMIN REPAIR.md similarity index 60% rename from docs/help/Contents/Administration/admin_repair_stmt.md rename to docs/documentation/cn/sql-reference/sql-statements/Administration/ADMIN REPAIR.md index 1549c65b539bdd..2ae44aed4a7412 100644 --- a/docs/help/Contents/Administration/admin_repair_stmt.md +++ b/docs/documentation/cn/sql-reference/sql-statements/Administration/ADMIN REPAIR.md @@ -25,25 +25,3 @@ ## keyword ADMIN,REPAIR -# ADMIN CANCEL REPAIR -## description - - 该语句用于取消以高优先级修复指定表或分区 - - 语法: - - ADMIN CANCEL REPAIR TABLE table_name[ PARTITION (p1,...)]; - - 说明: - - 1. 该语句仅表示系统不再以高优先级修复指定表或分区的分片副本。系统仍会以默认调度方式修复副本。 - -## example - - 1. 取消高优先级修复 - - ADMIN CANCEL REPAIR TABLE tbl PARTITION(p1); - -## keyword - ADMIN,CANCEL,REPAIR - diff --git a/docs/documentation/cn/sql-reference/sql-statements/Administration/ADMIN SET CONFIG.md b/docs/documentation/cn/sql-reference/sql-statements/Administration/ADMIN SET CONFIG.md new file mode 100644 index 00000000000000..b6e277dcaba36b --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Administration/ADMIN SET CONFIG.md @@ -0,0 +1,18 @@ +# ADMIN SET CONFIG +## description + + 该语句用于设置集群的配置项(当前仅支持设置FE的配置项)。 + 可设置的配置项,可以通过 AMDIN SHOW FRONTEND CONFIG; 命令查看。 + + 语法: + + ADMIN SET FRONTEND CONFIG ("key" = "value"); + +## example + + 1. 设置 'disable_balance' 为 true + + ADMIN SET FRONTEND CONFIG ("disable_balance" = "true"); + +## keyword + ADMIN,SET,CONFIG diff --git a/docs/documentation/cn/sql-reference/sql-statements/Administration/ADMIN SHOW CONFIG.md b/docs/documentation/cn/sql-reference/sql-statements/Administration/ADMIN SHOW CONFIG.md new file mode 100644 index 00000000000000..c8ee9e6672dc95 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Administration/ADMIN SHOW CONFIG.md @@ -0,0 +1,27 @@ +# ADMIN SHOW CONFIG +## description + + 该语句用于展示当前集群的配置(当前仅支持展示 FE 的配置项) + + 语法: + + ADMIN SHOW FRONTEND CONFIG; + + 说明: + + 结果中的各列含义如下: + 1. Key: 配置项名称 + 2. Value: 配置项值 + 3. Type: 配置项类型 + 4. IsMutable: 是否可以通过 ADMIN SET CONFIG 命令设置 + 5. MasterOnly: 是否仅适用于 Master FE + 6. Comment: 配置项说明 + +## example + + 1. 查看当前FE节点的配置 + + ADMIN SHOW FRONTEND CONFIG; + +## keyword + ADMIN,SHOW,CONFIG diff --git a/docs/documentation/cn/sql-reference/sql-statements/Administration/ADMIN SHOW REPLICA DISTRIBUTION.md b/docs/documentation/cn/sql-reference/sql-statements/Administration/ADMIN SHOW REPLICA DISTRIBUTION.md new file mode 100644 index 00000000000000..c948e0cf2f238c --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Administration/ADMIN SHOW REPLICA DISTRIBUTION.md @@ -0,0 +1,26 @@ +# ADMIN SHOW REPLICA DISTRIBUTION +## description + + 该语句用于展示一个表或分区副本分布状态 + + 语法: + + ADMIN SHOW REPLICA DISTRIBUTION FROM [db_name.]tbl_name [PARTITION (p1, ...)]; + + 说明: + + 结果中的 Graph 列以图形的形式展示副本分布比例 + +## example + + 1. 查看表的副本分布 + + ADMIN SHOW REPLICA DISTRIBUTION FROM tbl1; + + 2. 查看表的分区的副本分布 + + ADMIN SHOW REPLICA DISTRIBUTION FROM db1.tbl1 PARTITION(p1, p2); + +## keyword + ADMIN,SHOW,REPLICA,DISTRIBUTION + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Administration/ADMIN SHOW REPLICA STATUS.md b/docs/documentation/cn/sql-reference/sql-statements/Administration/ADMIN SHOW REPLICA STATUS.md new file mode 100644 index 00000000000000..93e00335d37a96 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Administration/ADMIN SHOW REPLICA STATUS.md @@ -0,0 +1,39 @@ +# ADMIN SHOW REPLICA STATUS +## description + + 该语句用于展示一个表或分区的副本状态信息 + + 语法: + + ADMIN SHOW REPLICA STATUS FROM [db_name.]tbl_name [PARTITION (p1, ...)] + [where_clause]; + + where_clause: + WHERE STATUS [!]= "replica_status" + + replica_status: + OK: replica 处于健康状态 + DEAD: replica 所在 Backend 不可用 + VERSION_ERROR: replica 数据版本有缺失 + SCHEMA_ERROR: replica 的 schema hash 不正确 + MISSING: replica 不存在 + +## example + + 1. 查看表全部的副本状态 + + ADMIN SHOW REPLICA STATUS FROM db1.tbl1; + + 2. 查看表某个分区状态为 VERSION_ERROR 的副本 + + ADMIN SHOW REPLICA STATUS FROM tbl1 PARTITION (p1, p2) + WHERE STATUS = "VERSION_ERROR"; + + 3. 查看表所有状态不健康的副本 + + ADMIN SHOW REPLICA STATUS FROM tbl1 + WHERE STATUS != "OK"; + +## keyword + ADMIN,SHOW,REPLICA,STATUS + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Administration/ALTER CLUSTER.md b/docs/documentation/cn/sql-reference/sql-statements/Administration/ALTER CLUSTER.md new file mode 100644 index 00000000000000..b9de2e3a683b9f --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Administration/ALTER CLUSTER.md @@ -0,0 +1,28 @@ +# ALTER CLUSTER +## description + + 该语句用于更新逻辑集群。需要有管理员权限 + + 语法 + + ALTER CLUSTER cluster_name PROPERTIES ("key"="value", ...); + + 1. 缩容,扩容 (根据集群现有的be数目,大则为扩容,小则为缩容), 扩容为同步操作,缩容为异步操作,通过backend的状态可以得知是否缩容完成 + + PROERTIES ("instance_num" = "3") + + instance_num 逻辑集群节点树 + +## example + + 1. 缩容,减少含有3个be的逻辑集群test_cluster的be数为2 + + ALTER CLUSTER test_cluster PROPERTIES ("instance_num"="2"); + + 2. 扩容,增加含有3个be的逻辑集群test_cluster的be数为4 + + ALTER CLUSTER test_cluster PROPERTIES ("instance_num"="4"); + +## keyword + ALTER,CLUSTER + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Administration/ALTER SYSTEM.md b/docs/documentation/cn/sql-reference/sql-statements/Administration/ALTER SYSTEM.md new file mode 100644 index 00000000000000..d6a35691951061 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Administration/ALTER SYSTEM.md @@ -0,0 +1,94 @@ +# ALTER SYSTEM +## description + + 该语句用于操作一个系统内的节点。(仅管理员使用!) + 语法: + 1) 增加节点(不使用多租户功能则按照此方法添加) + ALTER SYSTEM ADD BACKEND "host:heartbeat_port"[,"host:heartbeat_port"...]; + 2) 增加空闲节点(即添加不属于任何cluster的BACKEND) + ALTER SYSTEM ADD FREE BACKEND "host:heartbeat_port"[,"host:heartbeat_port"...]; + 3) 增加节点到某个cluster + ALTER SYSTEM ADD BACKEND TO cluster_name "host:heartbeat_port"[,"host:heartbeat_port"...]; + 4) 删除节点 + ALTER SYSTEM DROP BACKEND "host:heartbeat_port"[,"host:heartbeat_port"...]; + 5) 节点下线 + ALTER SYSTEM DECOMMISSION BACKEND "host:heartbeat_port"[,"host:heartbeat_port"...]; + 6) 增加Broker + ALTER SYSTEM ADD BROKER broker_name "host:port"[,"host:port"...]; + 7) 减少Broker + ALTER SYSTEM DROP BROKER broker_name "host:port"[,"host:port"...]; + 8) 删除所有Broker + ALTER SYSTEM DROP ALL BROKER broker_name + 9) 设置一个 Load error hub,用于集中展示导入时的错误信息 + ALTER SYSTEM SET LOAD ERRORS HUB PROPERTIES ("key" = "value"[, ...]); + + 说明: + 1) host 可以是主机名或者ip地址 + 2) heartbeat_port 为该节点的心跳端口 + 3) 增加和删除节点为同步操作。这两种操作不考虑节点上已有的数据,节点直接从元数据中删除,请谨慎使用。 + 4) 节点下线操作用于安全下线节点。该操作为异步操作。如果成功,节点最终会从元数据中删除。如果失败,则不会完成下线。 + 5) 可以手动取消节点下线操作。详见 CANCEL DECOMMISSION + 6) Load error hub: + 当前支持两种类型的 Hub:Mysql 和 Broker。需在 PROPERTIES 中指定 "type" = "mysql" 或 "type" = "broker"。 + 如果需要删除当前的 load error hub,可以将 type 设为 null。 + 1) 当使用 Mysql 类型时,导入时产生的错误信息将会插入到指定的 mysql 库表中,之后可以通过 show load warnings 语句直接查看错误信息。 + + Mysql 类型的 Hub 需指定以下参数: + host:mysql host + port:mysql port + user:mysql user + password:mysql password + database:mysql database + table:mysql table + + 2) 当使用 Broker 类型时,导入时产生的错误信息会形成一个文件,通过 broker,写入到指定的远端存储系统中。须确保已经部署对应的 broker + Broker 类型的 Hub 需指定以下参数: + broker: broker 的名称 + path: 远端存储路径 + other properties: 其他访问远端存储所必须的信息,比如认证信息等。 + +## example + + 1. 增加一个节点 + ALTER SYSTEM ADD BACKEND "host:port"; + + 2. 增加一个空闲节点 + ALTER SYSTEM ADD FREE BACKEND "host:port"; + + 3. 删除两个节点 + ALTER SYSTEM DROP BACKEND "host1:port", "host2:port"; + + 4. 下线两个节点 + ALTER SYSTEM DECOMMISSION BACKEND "host1:port", "host2:port"; + + 5. 增加两个Hdfs Broker + ALTER SYSTEM ADD BROKER hdfs "host1:port", "host2:port"; + + 6. 添加一个 Mysql 类型的 load error hub + ALTER SYSTEM SET LOAD ERRORS HUB PROPERTIES + ("type"= "mysql", + "host" = "192.168.1.17" + "port" = "3306", + "user" = "my_name", + "password" = "my_passwd", + "database" = "doris_load", + "table" = "load_errors" + ); + + 7. 添加一个 Broker 类型的 load error hub + ALTER SYSTEM SET LOAD ERRORS HUB PROPERTIES + ("type"= "broker", + "name" = "bos", + "path" = "bos://backup-cmy/logs", + "bos_endpoint" = "http://gz.bcebos.com", + "bos_accesskey" = "069fc278xxxxxx24ddb522", + "bos_secret_accesskey"="700adb0c6xxxxxx74d59eaa980a" + ); + + 8. 删除当前的 load error hub + ALTER SYSTEM SET LOAD ERRORS HUB PROPERTIES + ("type"= "null"); + +## keyword + ALTER,SYSTEM,BACKEND,BROKER,FREE + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Administration/CANCEL DECOMMISSION.md b/docs/documentation/cn/sql-reference/sql-statements/Administration/CANCEL DECOMMISSION.md new file mode 100644 index 00000000000000..54d76fa7292ba2 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Administration/CANCEL DECOMMISSION.md @@ -0,0 +1,15 @@ +# CANCEL DECOMMISSION +## description + + 该语句用于撤销一个节点下线操作。(仅管理员使用!) + 语法: + CANCEL DECOMMISSION BACKEND "host:heartbeat_port"[,"host:heartbeat_port"...]; + +## example + + 1. 取消两个节点的下线操作: + CANCEL DECOMMISSION BACKEND "host1:port", "host2:port"; + +## keyword + CANCEL,DECOMMISSION,BACKEND + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Administration/CREATE CLUSTER.md b/docs/documentation/cn/sql-reference/sql-statements/Administration/CREATE CLUSTER.md new file mode 100644 index 00000000000000..6ff866f6c78df7 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Administration/CREATE CLUSTER.md @@ -0,0 +1,36 @@ +# CREATE CLUSTER +## description + + 该语句用于新建逻辑集群 (cluster), 需要管理员权限。如果不使用多租户,直接创建一个名称为default_cluster的cluster。否则创建一个自定义名称的cluster。 + + 语法 + + CREATE CLUSTER [IF NOT EXISTS] cluster_name + + PROPERTIES ("key"="value", ...) + + IDENTIFIED BY 'password' + + 1. PROPERTIES + + 指定逻辑集群的属性 + + PROPERTIES ("instance_num" = "3") + + instance_num 逻辑集群节点树 + + 2. identified by ‘password' 每个逻辑集群含有一个superuser,创建逻辑集群时必须指定其密码 + +## example + + 1. 新建一个含有3个be节点逻辑集群 test_cluster, 并指定其superuser用户密码 + + CREATE CLUSTER test_cluster PROPERTIES("instance_num"="3") IDENTIFIED BY 'test'; + + 2. 新建一个含有3个be节点逻辑集群 default_cluster(不使用多租户), 并指定其superuser用户密码 + + CREATE CLUSTER default_cluster PROPERTIES("instance_num"="3") IDENTIFIED BY 'test'; + +## keyword + CREATE,CLUSTER + diff --git a/docs/help/Contents/Administration/small_files.md b/docs/documentation/cn/sql-reference/sql-statements/Administration/CREATE FILE.md similarity index 63% rename from docs/help/Contents/Administration/small_files.md rename to docs/documentation/cn/sql-reference/sql-statements/Administration/CREATE FILE.md index 631c125d75769a..bc3bc395e4e81b 100644 --- a/docs/help/Contents/Administration/small_files.md +++ b/docs/documentation/cn/sql-reference/sql-statements/Administration/CREATE FILE.md @@ -49,55 +49,3 @@ ## keyword CREATE,FILE -# SHOW FILE -## description - - 该语句用于展示一个 database 内创建的文件 - - 语法: - - SHOW FILE [FROM database]; - - 说明: - - FileId: 文件ID,全局唯一 - DbName: 所属数据库名称 - Catalog: 自定义分类 - FileName: 文件名 - FileSize: 文件大小,单位字节 - MD5: 文件的 MD5 - -## example - - 1. 查看数据库 my_database 中已上传的文件 - - SHOW FILE FROM my_database; - -## keyword - SHOW,FILE - -# DROP FILE -## description - - 该语句用于删除一个已上传的文件。 - - 语法: - - DROP FILE "file_name" [FROM database] - [properties] - - 说明: - file_name: 文件名。 - database: 文件归属的某一个 db,如果没有指定,则使用当前 session 的 db。 - properties 支持以下参数: - - catalog: 必须。文件所属分类。 - -## example - - 1. 删除文件 ca.pem - - DROP FILE "ca.pem" properties("catalog" = "kafka"); - -## keyword - DROP,FILE diff --git a/docs/documentation/cn/sql-reference/sql-statements/Administration/DROP CLUSTER.md b/docs/documentation/cn/sql-reference/sql-statements/Administration/DROP CLUSTER.md new file mode 100644 index 00000000000000..d755037b5e7537 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Administration/DROP CLUSTER.md @@ -0,0 +1,18 @@ +# DROP CLUSTER +## description + + 该语句用于删除逻辑集群,成功删除逻辑集群需要首先删除集群内的db,需要管理员权限 + + 语法 + + DROP CLUSTER [IF EXISTS] cluster_name + +## example + + 删除逻辑集群 test_cluster + + DROP CLUSTER test_cluster; + +## keyword + DROP,CLUSTER + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Administration/DROP FILE.md b/docs/documentation/cn/sql-reference/sql-statements/Administration/DROP FILE.md new file mode 100644 index 00000000000000..205f4a53167f35 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Administration/DROP FILE.md @@ -0,0 +1,25 @@ +# DROP FILE +## description + + 该语句用于删除一个已上传的文件。 + + 语法: + + DROP FILE "file_name" [FROM database] + [properties] + + 说明: + file_name: 文件名。 + database: 文件归属的某一个 db,如果没有指定,则使用当前 session 的 db。 + properties 支持以下参数: + + catalog: 必须。文件所属分类。 + +## example + + 1. 删除文件 ca.pem + + DROP FILE "ca.pem" properties("catalog" = "kafka"); + +## keyword + DROP,FILE diff --git a/docs/documentation/cn/sql-reference/sql-statements/Administration/ENTER.md b/docs/documentation/cn/sql-reference/sql-statements/Administration/ENTER.md new file mode 100644 index 00000000000000..cf21a7e041ad55 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Administration/ENTER.md @@ -0,0 +1,18 @@ +# ENTER +## description + + 该语句用于进入一个逻辑集群, 所有创建用户、创建数据库都需要在一个逻辑集群内执行,创建后并且隶属于这个逻 + + 辑集群,需要管理员权限 + + ENTER cluster_name + +## example + + 1. 进入逻辑集群test_cluster + + ENTER test_cluster; + +## keyword + ENTER + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Administration/LINK DATABASE.md b/docs/documentation/cn/sql-reference/sql-statements/Administration/LINK DATABASE.md new file mode 100644 index 00000000000000..580d0a952dd884 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Administration/LINK DATABASE.md @@ -0,0 +1,24 @@ +# LINK DATABASE +## description + + 该语句用户链接一个逻辑集群的数据库到另外一个逻辑集群, 一个数据库只允许同时被链接一次,删除链接的数据库 + + 并不会删除数据,并且被链接的数据库不能被删除, 需要管理员权限 + + 语法 + + LINK DATABASE src_cluster_name.src_db_name des_cluster_name.des_db_name + +## example + + 1. 链接test_clusterA中的test_db到test_clusterB,并命名为link_test_db + + LINK DATABASE test_clusterA.test_db test_clusterB.link_test_db; + + 2. 删除链接的数据库link_test_db + + DROP DATABASE link_test_db; + +## keyword + LINK,DATABASE + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Administration/MIGRATE DATABASE.md b/docs/documentation/cn/sql-reference/sql-statements/Administration/MIGRATE DATABASE.md new file mode 100644 index 00000000000000..fce717142514c9 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Administration/MIGRATE DATABASE.md @@ -0,0 +1,20 @@ +# MIGRATE DATABASE +## description + + 该语句用于迁移一个逻辑集群的数据库到另外一个逻辑集群,执行此操作前数据库必须已经处于链接状态, 需要管理 + + 员权限 + + 语法 + + MIGRATE DATABASE src_cluster_name.src_db_name des_cluster_name.des_db_name + +## example + + 1. 迁移test_clusterA中的test_db到test_clusterB + + MIGRATE DATABASE test_clusterA.test_db test_clusterB.link_test_db; + +## keyword + MIGRATE,DATABASE + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Administration/SHOW BACKENDS.md b/docs/documentation/cn/sql-reference/sql-statements/Administration/SHOW BACKENDS.md new file mode 100644 index 00000000000000..ae96e8d46e7163 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Administration/SHOW BACKENDS.md @@ -0,0 +1,22 @@ +# SHOW BACKENDS +## description + 该语句用于查看 cluster 内的 BE 节点 + 语法: + SHOW BACKENDS; + + 说明: + 1. LastStartTime 表示最近一次 BE 启动时间。 + 2. LastHeartbeat 表示最近一次心跳。 + 3. Alive 表示节点是否存活。 + 4. SystemDecommissioned 为 true 表示节点正在安全下线中。 + 5. ClusterDecommissioned 为 true 表示节点正在冲当前cluster中下线。 + 6. TabletNum 表示该节点上分片数量。 + 7. DataUsedCapacity 表示实际用户数据所占用的空间。 + 8. AvailCapacity 表示磁盘的可使用空间。 + 9. TotalCapacity 表示总磁盘空间。TotalCapacity = AvailCapacity + DataUsedCapacity + 其他非用户数据文件占用空间。 + 10. UsedPct 表示磁盘已使用量百分比。 + 11. ErrMsg 用于显示心跳失败时的错误信息。 + +## keyword + SHOW, BACKENDS + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Administration/SHOW BROKER.md b/docs/documentation/cn/sql-reference/sql-statements/Administration/SHOW BROKER.md new file mode 100644 index 00000000000000..82bf99bca13f79 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Administration/SHOW BROKER.md @@ -0,0 +1,15 @@ +# SHOW BROKER +## description + 该语句用于查看当前存在的 broker + 语法: + SHOW BROKER; + + 说明: + 1. LastStartTime 表示最近一次 BE 启动时间。 + 2. LastHeartbeat 表示最近一次心跳。 + 3. Alive 表示节点是否存活。 + 4. ErrMsg 用于显示心跳失败时的错误信息。 + +## keyword + SHOW, BROKER + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Administration/SHOW FILE.md b/docs/documentation/cn/sql-reference/sql-statements/Administration/SHOW FILE.md new file mode 100644 index 00000000000000..b25b72690da94a --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Administration/SHOW FILE.md @@ -0,0 +1,27 @@ +# SHOW FILE +## description + + 该语句用于展示一个 database 内创建的文件 + + 语法: + + SHOW FILE [FROM database]; + + 说明: + + FileId: 文件ID,全局唯一 + DbName: 所属数据库名称 + Catalog: 自定义分类 + FileName: 文件名 + FileSize: 文件大小,单位字节 + MD5: 文件的 MD5 + +## example + + 1. 查看数据库 my_database 中已上传的文件 + + SHOW FILE FROM my_database; + +## keyword + SHOW,FILE + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Administration/SHOW FRONTENDS.md b/docs/documentation/cn/sql-reference/sql-statements/Administration/SHOW FRONTENDS.md new file mode 100644 index 00000000000000..1b96368b6325e5 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Administration/SHOW FRONTENDS.md @@ -0,0 +1,18 @@ +# SHOW FRONTENDS +## description + 该语句用于查看 FE 节点 + 语法: + SHOW FRONTENDS; + + 说明: + 1. name 表示该 FE 节点在 bdbje 中的名称。 + 2. Join 为 true 表示该节点曾经加入过集群。但不代表当前还在集群内(可能已失联) + 3. Alive 表示节点是否存活。 + 4. ReplayedJournalId 表示该节点当前已经回放的最大元数据日志id。 + 5. LastHeartbeat 是最近一次心跳。 + 6. IsHelper 表示该节点是否是 bdbje 中的 helper 节点。 + 7. ErrMsg 用于显示心跳失败时的错误信息。 + +## keyword + SHOW, FRONTENDS + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Administration/SHOW MIGRATIONS.md b/docs/documentation/cn/sql-reference/sql-statements/Administration/SHOW MIGRATIONS.md new file mode 100644 index 00000000000000..87cb4dc8302f93 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Administration/SHOW MIGRATIONS.md @@ -0,0 +1,12 @@ +# SHOW MIGRATIONS +## description + + 该语句用于查看数据库迁移的进度 + + 语法 + + SHOW MIGRATIONS + +## keyword + SHOW,MIGRATIONS + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Definition/ALTER DATABASE.md b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/ALTER DATABASE.md new file mode 100644 index 00000000000000..a020eceff735f3 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/ALTER DATABASE.md @@ -0,0 +1,29 @@ +# ALTER DATABASE +## description + 该语句用于设置指定数据库的属性。(仅管理员使用) + 语法: + 1) 设置数据库数据量配额,单位为B/K/KB/M/MB/G/GB/T/TB/P/PB + ALTER DATABASE db_name SET DATA QUOTA quota; + + 2) 重命名数据库 + ALTER DATABASE db_name RENAME new_db_name; + + 说明: + 重命名数据库后,如需要,请使用 REVOKE 和 GRANT 命令修改相应的用户权限。 + +## example + 1. 设置指定数据库数据量配额 + ALTER DATABASE example_db SET DATA QUOTA 10995116277760; + 上述单位为字节,等价于 + ALTER DATABASE example_db SET DATA QUOTA 10T; + + ALTER DATABASE example_db SET DATA QUOTA 100G; + + ALTER DATABASE example_db SET DATA QUOTA 200M; + + 2. 将数据库 example_db 重命名为 example_db2 + ALTER DATABASE example_db RENAME example_db2; + +## keyword + ALTER,DATABASE,RENAME + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Definition/ALTER TABLE.md b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/ALTER TABLE.md new file mode 100644 index 00000000000000..e6a8099b18cef0 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/ALTER TABLE.md @@ -0,0 +1,240 @@ +# ALTER TABLE +## description + 该语句用于对已有的 table 进行修改。如果没有指定 rollup index,默认操作 base index。 + 该语句分为三种操作类型: schema change 、rollup 、partition + 这三种操作类型不能同时出现在一条 ALTER TABLE 语句中。 + 其中 schema change 和 rollup 是异步操作,任务提交成功则返回。之后可使用 SHOW ALTER 命令查看进度。 + partition 是同步操作,命令返回表示执行完毕。 + + 语法: + ALTER TABLE [database.]table + alter_clause1[, alter_clause2, ...]; + + alter_clause 分为 partition 、rollup、schema change 和 rename 四种。 + + partition 支持如下几种修改方式 + 1. 增加分区 + 语法: + ADD PARTITION [IF NOT EXISTS] partition_name VALUES LESS THAN [MAXVALUE|("value1")] ["key"="value"] + [DISTRIBUTED BY HASH (k1[,k2 ...]) [BUCKETS num]] + 注意: + 1) 分区为左闭右开区间,用户指定右边界,系统自动确定左边界 + 2) 如果没有指定分桶方式,则自动使用建表使用的分桶方式 + 3) 如指定分桶方式,只能修改分桶数,不可修改分桶方式或分桶列 + 4) ["key"="value"] 部分可以设置分区的一些属性,具体说明见 CREATE TABLE + + 2. 删除分区 + 语法: + DROP PARTITION [IF EXISTS] partition_name + 注意: + 1) 使用分区方式的表至少要保留一个分区。 + 2) 执行 DROP PARTITION 一段时间内,可以通过 RECOVER 语句恢复被删除的 partition。详见 RECOVER 语句 + + 3. 修改分区属性 + 语法: + MODIFY PARTITION partition_name SET ("key" = "value", ...) + 说明: + 1) 当前支持修改分区的 storage_medium、storage_cooldown_time 和 replication_num 三个属性。 + 2) 对于单分区表,partition_name 同表名。 + + rollup 支持如下几种创建方式: + 1. 创建 rollup index + 语法: + ADD ROLLUP rollup_name (column_name1, column_name2, ...) + [FROM from_index_name] + [PROPERTIES ("key"="value", ...)] + 注意: + 1) 如果没有指定 from_index_name,则默认从 base index 创建 + 2) rollup 表中的列必须是 from_index 中已有的列 + 3) 在 properties 中,可以指定存储格式。具体请参阅 CREATE TABLE + + 2. 删除 rollup index + 语法: + DROP ROLLUP rollup_name + [PROPERTIES ("key"="value", ...)] + 注意: + 1) 不能删除 base index + 2) 执行 DROP ROLLUP 一段时间内,可以通过 RECOVER 语句恢复被删除的 rollup index。详见 RECOVER 语句 + + + schema change 支持如下几种修改方式: + 1. 向指定 index 的指定位置添加一列 + 语法: + ADD COLUMN column_name column_type [KEY | agg_type] [DEFAULT "default_value"] + [AFTER column_name|FIRST] + [TO rollup_index_name] + [PROPERTIES ("key"="value", ...)] + 注意: + 1) 聚合模型如果增加 value 列,需要指定 agg_type + 2) 非聚合模型如果增加key列,需要指定KEY关键字 + 3) 不能在 rollup index 中增加 base index 中已经存在的列 + 如有需要,可以重新创建一个 rollup index) + + 2. 向指定 index 添加多列 + 语法: + ADD COLUMN (column_name1 column_type [KEY | agg_type] DEFAULT "default_value", ...) + [TO rollup_index_name] + [PROPERTIES ("key"="value", ...)] + 注意: + 1) 聚合模型如果增加 value 列,需要指定agg_type + 2) 非聚合模型如果增加key列,需要指定KEY关键字 + 3) 不能在 rollup index 中增加 base index 中已经存在的列 + (如有需要,可以重新创建一个 rollup index) + + 3. 从指定 index 中删除一列 + 语法: + DROP COLUMN column_name + [FROM rollup_index_name] + 注意: + 1) 不能删除分区列 + 2) 如果是从 base index 中删除列,则如果 rollup index 中包含该列,也会被删除 + + 4. 修改指定 index 的列类型以及列位置 + 语法: + MODIFY COLUMN column_name column_type [KEY | agg_type] [NULL | NOT NULL] [DEFAULT "default_value"] + [AFTER column_name|FIRST] + [FROM rollup_index_name] + [PROPERTIES ("key"="value", ...)] + 注意: + 1) 聚合模型如果修改 value 列,需要指定 agg_type + 2) 非聚合类型如果修改key列,需要指定KEY关键字 + 3) 只能修改列的类型,列的其他属性维持原样(即其他属性需在语句中按照原属性显式的写出,参见 example 8) + 4) 分区列不能做任何修改 + 5) 目前支持以下类型的转换(精度损失由用户保证) + TINYINT/SMALLINT/INT/BIGINT 转换成 TINYINT/SMALLINT/INT/BIGINT/DOUBLE。 + LARGEINT 转换成 DOUBLE + VARCHAR 支持修改最大长度 + 6) 不支持从NULL转为NOT NULL + + 5. 对指定 index 的列进行重新排序 + 语法: + ORDER BY (column_name1, column_name2, ...) + [FROM rollup_index_name] + [PROPERTIES ("key"="value", ...)] + 注意: + 1) index 中的所有列都要写出来 + 2) value 列在 key 列之后 + + 6. 修改table的属性,目前支持修改bloom filter列和colocate_with 属性 + 语法: + PROPERTIES ("key"="value") + 注意: + 也可以合并到上面的schema change操作中来修改,见下面例子 + + + rename 支持对以下名称进行修改: + 1. 修改表名 + 语法: + RENAME new_table_name; + + 2. 修改 rollup index 名称 + 语法: + RENAME ROLLUP old_rollup_name new_rollup_name; + + 3. 修改 partition 名称 + 语法: + RENAME PARTITION old_partition_name new_partition_name; + +## example + [partition] + 1. 增加分区, 现有分区 [MIN, 2013-01-01),增加分区 [2013-01-01, 2014-01-01),使用默认分桶方式 + ALTER TABLE example_db.my_table + ADD PARTITION p1 VALUES LESS THAN ("2014-01-01"); + + 2. 增加分区,使用新的分桶数 + ALTER TABLE example_db.my_table + ADD PARTITION p1 VALUES LESS THAN ("2015-01-01") + DISTRIBUTED BY HASH(k1) BUCKETS 20; + + 3. 删除分区 + ALTER TABLE example_db.my_table + DROP PARTITION p1; + + [rollup] + 1. 创建 index: example_rollup_index,基于 base index(k1,k2,k3,v1,v2)。列式存储。 + ALTER TABLE example_db.my_table + ADD ROLLUP example_rollup_index(k1, k3, v1, v2) + PROPERTIES("storage_type"="column"); + + 2. 创建 index: example_rollup_index2,基于 example_rollup_index(k1,k3,v1,v2) + ALTER TABLE example_db.my_table + ADD ROLLUP example_rollup_index2 (k1, v1) + FROM example_rollup_index; + + 3. 删除 index: example_rollup_index2 + ALTER TABLE example_db.my_table + DROP ROLLUP example_rollup_index2; + + [schema change] + 1. 向 example_rollup_index 的 col1 后添加一个key列 new_col(非聚合模型) + ALTER TABLE example_db.my_table + ADD COLUMN new_col INT KEY DEFAULT "0" AFTER col1 + TO example_rollup_index; + + 2. 向example_rollup_index的col1后添加一个value列new_col(非聚合模型) + ALTER TABLE example_db.my_table + ADD COLUMN new_col INT DEFAULT "0" AFTER col1 + TO example_rollup_index; + + 3. 向example_rollup_index的col1后添加一个key列new_col(聚合模型) + ALTER TABLE example_db.my_table + ADD COLUMN new_col INT DEFAULT "0" AFTER col1 + TO example_rollup_index; + + 4. 向example_rollup_index的col1后添加一个value列new_col SUM聚合类型(聚合模型) + ALTER TABLE example_db.my_table + ADD COLUMN new_col INT SUM DEFAULT "0" AFTER col1 + TO example_rollup_index; + + 5. 向 example_rollup_index 添加多列(聚合模型) + ALTER TABLE example_db.my_table + ADD COLUMN (col1 INT DEFAULT "1", col2 FLOAT SUM DEFAULT "2.3") + TO example_rollup_index; + + 6. 从 example_rollup_index 删除一列 + ALTER TABLE example_db.my_table + DROP COLUMN col2 + FROM example_rollup_index; + + 7. 修改 base index 的 col1 列的类型为 BIGINT,并移动到 col2 列后面 + ALTER TABLE example_db.my_table + MODIFY COLUMN col1 BIGINT DEFAULT "1" AFTER col2; + + 8. 修改 base index 的 val1 列最大长度。原 val1 为 (val1 VARCHAR(32) REPLACE DEFAULT "abc") + ALTER TABLE example_db.my_table + MODIFY COLUMN val1 VARCHAR(64) REPLACE DEFAULT "abc"; + + 9. 重新排序 example_rollup_index 中的列(设原列顺序为:k1,k2,k3,v1,v2) + ALTER TABLE example_db.my_table + ORDER BY (k3,k1,k2,v2,v1) + FROM example_rollup_index; + + 10. 同时执行两种操作 + ALTER TABLE example_db.my_table + ADD COLUMN v2 INT MAX DEFAULT "0" AFTER k2 TO example_rollup_index, + ORDER BY (k3,k1,k2,v2,v1) FROM example_rollup_index; + + 11. 修改表的 bloom filter 列 + ALTER TABLE example_db.my_table SET ("bloom_filter_columns"="k1,k2,k3"); + + 也可以合并到上面的 schema change 操作中(注意多子句的语法有少许区别) + ALTER TABLE example_db.my_table + DROP COLUMN col2 + PROPERTIES ("bloom_filter_columns"="k1,k2,k3"); + + 12. 修改表的Colocate 属性 + ALTER TABLE example_db.my_table set ("colocate_with"="t1"); + + [rename] + 1. 将名为 table1 的表修改为 table2 + ALTER TABLE table1 RENAME table2; + + 2. 将表 example_table 中名为 rollup1 的 rollup index 修改为 rollup2 + ALTER TABLE example_table RENAME ROLLUP rollup1 rollup2; + + 3. 将表 example_table 中名为 p1 的 partition 修改为 p2 + ALTER TABLE example_table RENAME PARTITION p1 p2; + +## keyword + ALTER,TABLE,ROLLUP,COLUMN,PARTITION,RENAME + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Definition/BACKUP.md b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/BACKUP.md new file mode 100644 index 00000000000000..19c5137ce1b42e --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/BACKUP.md @@ -0,0 +1,39 @@ +# BACKUP +## description + 该语句用于备份指定数据库下的数据。该命令为异步操作。提交成功后,需通过 SHOW BACKUP 命令查看进度。仅支持备份 OLAP 类型的表。 + 语法: + BACKUP SNAPSHOT [db_name].{snapshot_name} + TO `repository_name` + ON ( + `table_name` [PARTITION (`p1`, ...)], + ... + ) + PROPERTIES ("key"="value", ...); + + 说明: + 1. 同一数据库下只能有一个正在执行的 BACKUP 或 RESTORE 任务。 + 2. ON 子句中标识需要备份的表和分区。如果不指定分区,则默认备份该表的所有分区。 + 3. PROPERTIES 目前支持以下属性: + "type" = "full":表示这是一次全量更新(默认)。 + "timeout" = "3600":任务超时时间,默认为一天。单位秒。 + +## example + + 1. 全量备份 example_db 下的表 example_tbl 到仓库 example_repo 中: + BACKUP SNAPSHOT example_db.snapshot_label1 + TO example_repo + ON (example_tbl) + PROPERTIES ("type" = "full"); + + 2. 全量备份 example_db 下,表 example_tbl 的 p1, p2 分区,以及表 example_tbl2 到仓库 example_repo 中: + BACKUP SNAPSHOT example_db.snapshot_label2 + TO example_repo + ON + ( + example_tbl PARTITION (p1,p2), + example_tbl2 + ); + +## keyword + BACKUP + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Definition/CANCEL ALTER.md b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/CANCEL ALTER.md new file mode 100644 index 00000000000000..93c17f921a3379 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/CANCEL ALTER.md @@ -0,0 +1,32 @@ +# CANCEL ALTER +## description + 该语句用于撤销一个 ALTER 操作。 + 1. 撤销 ALTER TABLE COLUMN 操作 + 语法: + CANCEL ALTER TABLE COLUMN + FROM db_name.table_name + + 2. 撤销 ALTER TABLE ROLLUP 操作 + 语法: + CANCEL ALTER TABLE ROLLUP + FROM db_name.table_name + + 2. 撤销 ALTER CLUSTER 操作 + 语法: + (待实现...) + + +## example + [CANCEL ALTER TABLE COLUMN] + 1. 撤销针对 my_table 的 ALTER COLUMN 操作。 + CANCEL ALTER TABLE COLUMN + FROM example_db.my_table; + + [CANCEL ALTER TABLE ROLLUP] + 1. 撤销 my_table 下的 ADD ROLLUP 操作。 + CANCEL ALTER TABLE ROLLUP + FROM example_db.my_table; + +## keyword + CANCEL,ALTER,TABLE,COLUMN,ROLLUP + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Definition/CANCEL BACKUP.md b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/CANCEL BACKUP.md new file mode 100644 index 00000000000000..b6451b99fe7931 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/CANCEL BACKUP.md @@ -0,0 +1,13 @@ +# CANCEL BACKUP +## description + 该语句用于取消一个正在进行的 BACKUP 任务。 + 语法: + CANCEL BACKUP FROM db_name; + +## example + 1. 取消 example_db 下的 BACKUP 任务。 + CANCEL BACKUP FROM example_db; + +## keyword + CANCEL, BACKUP + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Definition/CANCEL RESTORE.md b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/CANCEL RESTORE.md new file mode 100644 index 00000000000000..b2b52bedbc3c7b --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/CANCEL RESTORE.md @@ -0,0 +1,16 @@ +# CANCEL RESTORE +## description + 该语句用于取消一个正在进行的 RESTORE 任务。 + 语法: + CANCEL RESTORE FROM db_name; + + 注意: + 当取消处于 COMMIT 或之后阶段的恢复左右时,可能导致被恢复的表无法访问。此时只能通过再次执行恢复作业进行数据恢复。 + +## example + 1. 取消 example_db 下的 RESTORE 任务。 + CANCEL RESTORE FROM example_db; + +## keyword + CANCEL, RESTORE + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Definition/CREATE DATABASE.md b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/CREATE DATABASE.md new file mode 100644 index 00000000000000..00938857f6ce70 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/CREATE DATABASE.md @@ -0,0 +1,13 @@ +# CREATE DATABASE +## description + 该语句用于新建数据库(database) + 语法: + CREATE DATABASE [IF NOT EXISTS] db_name; + +## example + 1. 新建数据库 db_test + CREATE DATABASE db_test; + +## keyword + CREATE,DATABASE + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Definition/CREATE REPOSITORY.md b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/CREATE REPOSITORY.md new file mode 100644 index 00000000000000..68ca39141597d6 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/CREATE REPOSITORY.md @@ -0,0 +1,50 @@ +# CREATE REPOSITORY +## description + 该语句用于创建仓库。仓库用于属于备份或恢复。仅 root 或 superuser 用户可以创建仓库。 + 语法: + CREATE [READ ONLY] REPOSITORY `repo_name` + WITH BROKER `broker_name` + ON LOCATION `repo_location` + PROPERTIES ("key"="value", ...); + + 说明: + 1. 仓库的创建,依赖于已存在的 broker + 2. 如果是只读仓库,则只能在仓库上进行恢复。如果不是,则可以进行备份和恢复操作。 + 3. 根据 broker 的不同类型,PROPERTIES 有所不同,具体见示例。 + +## example + 1. 创建名为 bos_repo 的仓库,依赖 BOS broker "bos_broker",数据根目录为:bos://palo_backup + CREATE REPOSITORY `bos_repo` + WITH BROKER `bos_broker` + ON LOCATION "bos://palo_backup" + PROPERTIES + ( + "bos_endpoint" = "http://gz.bcebos.com", + "bos_accesskey" = "069fc2786e664e63a5f111111114ddbs22", + "bos_secret_accesskey"="70999999999999de274d59eaa980a" + ); + + 2. 创建和示例 1 相同的仓库,但属性为只读: + CREATE READ ONLY REPOSITORY `bos_repo` + WITH BROKER `bos_broker` + ON LOCATION "bos://palo_backup" + PROPERTIES + ( + "bos_endpoint" = "http://gz.bcebos.com", + "bos_accesskey" = "069fc2786e664e63a5f111111114ddbs22", + "bos_secret_accesskey"="70999999999999de274d59eaa980a" + ); + + 3. 创建名为 hdfs_repo 的仓库,依赖 Baidu hdfs broker "hdfs_broker",数据根目录为:hdfs://hadoop-name-node:54310/path/to/repo/ + CREATE REPOSITORY `hdfs_repo` + WITH BROKER `hdfs_broker` + ON LOCATION "hdfs://hadoop-name-node:54310/path/to/repo/" + PROPERTIES + ( + "username" = "user", + "password" = "password" + ); + +## keyword + CREATE REPOSITORY + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Definition/CREATE TABLE.md b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/CREATE TABLE.md new file mode 100644 index 00000000000000..1af44e7ce4d756 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/CREATE TABLE.md @@ -0,0 +1,334 @@ +# CREATE TABLE +## description + 该语句用于创建 table。 + 语法: + CREATE [EXTERNAL] TABLE [IF NOT EXISTS] [database.]table_name + (column_definition1[, column_definition2, ...]) + [ENGINE = [olap|mysql|broker]] + [key_desc] + [partition_desc] + [distribution_desc] + [PROPERTIES ("key"="value", ...)]; + [BROKER PROPERTIES ("key"="value", ...)]; + + 1. column_definition + 语法: + col_name col_type [agg_type] [NULL | NOT NULL] [DEFAULT "default_value"] + + 说明: + col_name:列名称 + col_type:列类型 + TINYINT(1字节) + 范围:-2^7 + 1 ~ 2^7 - 1 + SMALLINT(2字节) + 范围:-2^15 + 1 ~ 2^15 - 1 + INT(4字节) + 范围:-2^31 + 1 ~ 2^31 - 1 + BIGINT(8字节) + 范围:-2^63 + 1 ~ 2^63 - 1 + LARGEINT(16字节) + 范围:0 ~ 2^127 - 1 + FLOAT(4字节) + 支持科学计数法 + DOUBLE(12字节) + 支持科学计数法 + DECIMAL[(precision, scale)] (40字节) + 保证精度的小数类型。默认是 DECIMAL(10, 0) + precision: 1 ~ 27 + scale: 0 ~ 9 + 其中整数部分为 1 ~ 18 + 不支持科学计数法 + DATE(3字节) + 范围:1900-01-01 ~ 9999-12-31 + DATETIME(8字节) + 范围:1900-01-01 00:00:00 ~ 9999-12-31 23:59:59 + CHAR[(length)] + 定长字符串。长度范围:1 ~ 255。默认为1 + VARCHAR[(length)] + 变长字符串。长度范围:1 ~ 65533 + HLL (1~16385个字节) + hll列类型,不需要指定长度和默认值、长度根据数据的聚合 + 程度系统内控制,并且HLL列只能通过配套的hll_union_agg、Hll_cardinality、hll_hash进行查询或使用 + + agg_type:聚合类型,如果不指定,则该列为 key 列。否则,该列为 value 列 + SUM、MAX、MIN、REPLACE、HLL_UNION(仅用于HLL列,为HLL独有的聚合方式) + 该类型只对聚合模型(key_desc的type为AGGREGATE KEY)有用,其它模型不需要指定这个。 + + 是否允许为NULL: 默认不允许为 NULL。NULL 值在导入数据中用 \N 来表示 + + 2. ENGINE 类型 + 默认为 olap。可选 mysql, broker + 1) 如果是 mysql,则需要在 properties 提供以下信息: + + PROPERTIES ( + "host" = "mysql_server_host", + "port" = "mysql_server_port", + "user" = "your_user_name", + "password" = "your_password", + "database" = "database_name", + "table" = "table_name" + ) + + 注意: + "table" 条目中的 "table_name" 是 mysql 中的真实表名。 + 而 CREATE TABLE 语句中的 table_name 是该 mysql 表在 Palo 中的名字,可以不同。 + + 在 Palo 创建 mysql 表的目的是可以通过 Palo 访问 mysql 数据库。 + 而 Palo 本身并不维护、存储任何 mysql 数据。 + 2) 如果是 broker,表示表的访问需要通过指定的broker, 需要在 properties 提供以下信息: + PROPERTIES ( + "broker_name" = "broker_name", + "paths" = "file_path1[,file_path2]", + "column_separator" = "value_separator" + "line_delimiter" = "value_delimiter" + ) + 另外还需要提供Broker需要的Property信息,通过BROKER PROPERTIES来传递,例如HDFS需要传入 + BROKER PROPERTIES( + "username" = "name", + "password" = "password" + ) + 这个根据不同的Broker类型,需要传入的内容也不相同 + 注意: + "paths" 中如果有多个文件,用逗号[,]分割。如果文件名中包含逗号,那么使用 %2c 来替代。如果文件名中包含 %,使用 %25 代替 + 现在文件内容格式支持CSV,支持GZ,BZ2,LZ4,LZO(LZOP) 压缩格式。 + + 3. key_desc + 语法: + key_type(k1[,k2 ...]) + 说明: + 数据按照指定的key列进行排序,且根据不同的key_type具有不同特性。 + key_type支持一下类型: + AGGREGATE KEY:key列相同的记录,value列按照指定的聚合类型进行聚合, + 适合报表、多维分析等业务场景。 + UNIQUE KEY:key列相同的记录,value列按导入顺序进行覆盖, + 适合按key列进行增删改查的点查询业务。 + DUPLICATE KEY:key列相同的记录,同时存在于Palo中, + 适合存储明细数据或者数据无聚合特性的业务场景。 + 注意: + 除AGGREGATE KEY外,其他key_type在建表时,value列不需要指定聚合类型。 + + 4. partition_desc + 1) Range 分区 + 语法: + PARTITION BY RANGE (k1, k2, ...) + ( + PARTITION partition_name VALUES LESS THAN MAXVALUE|("value1", "value2", ...) + PARTITION partition_name VALUES LESS THAN MAXVALUE|("value1", "value2", ...) + ... + ) + 说明: + 使用指定的 key 列和指定的数值范围进行分区。 + 1) 分区名称仅支持字母开头,字母、数字和下划线组成 + 2) 目前仅支持以下类型的列作为 Range 分区列,且只能指定一个分区列 + TINYINT, SMALLINT, INT, BIGINT, LARGEINT, DATE, DATETIME + 3) 分区为左闭右开区间,首个分区的左边界为做最小值 + 4) NULL 值只会存放在包含最小值的分区中。当包含最小值的分区被删除后,NULL 值将无法导入。 + 5) 可以指定一列或多列作为分区列。如果分区值缺省,则会默认填充最小值。 + + 注意: + 1) 分区一般用于时间维度的数据管理 + 2) 有数据回溯需求的,可以考虑首个分区为空分区,以便后续增加分区 + + 5. distribution_desc + 1) Hash 分桶 + 语法: + DISTRIBUTED BY HASH (k1[,k2 ...]) [BUCKETS num] + 说明: + 使用指定的 key 列进行哈希分桶。默认分区数为10 + + 建议:建议使用Hash分桶方式 + + 6. PROPERTIES + 1) 如果 ENGINE 类型为 olap,则可以在 properties 中指定列存(目前我们仅支持列存) + + PROPERTIES ( + "storage_type" = "[column]", + ) + + 2) 如果 ENGINE 类型为 olap + 可以在 properties 设置该表数据的初始存储介质、存储到期时间和副本数。 + + PROPERTIES ( + "storage_medium" = "[SSD|HDD]", + ["storage_cooldown_time" = "yyyy-MM-dd HH:mm:ss"], + ["replication_num" = "3"] + ) + + storage_medium: 用于指定该分区的初始存储介质,可选择 SSD 或 HDD。默认为 HDD。 + storage_cooldown_time: 当设置存储介质为 SSD 时,指定该分区在 SSD 上的存储到期时间。 + 默认存放 7 天。 + 格式为:"yyyy-MM-dd HH:mm:ss" + replication_num: 指定分区的副本数。默认为 3 + + 当表为单分区表时,这些属性为表的属性。 + 当表为两级分区时,这些属性为附属于每一个分区。 + 如果希望不同分区有不同属性。可以通过 ADD PARTITION 或 MODIFY PARTITION 进行操作 + + 3) 如果 Engine 类型为 olap, 并且 storage_type 为 column, 可以指定某列使用 bloom filter 索引 + bloom filter 索引仅适用于查询条件为 in 和 equal 的情况,该列的值越分散效果越好 + 目前只支持以下情况的列:除了 TINYINT FLOAT DOUBLE 类型以外的 key 列及聚合方法为 REPLACE 的 value 列 + + PROPERTIES ( + "bloom_filter_columns"="k1,k2,k3" + ) + 4) 如果希望使用Colocate Join 特性,需要在 properties 中指定 + + PROPERTIES ( + "colocate_with"="table1" + ) + +## example + 1. 创建一个 olap 表,使用 HASH 分桶,使用列存,相同key的记录进行聚合 + CREATE TABLE example_db.table_hash + ( + k1 TINYINT, + k2 DECIMAL(10, 2) DEFAULT "10.5", + v1 CHAR(10) REPLACE, + v2 INT SUM + ) + ENGINE=olap + AGGREGATE KEY(k1, k2) + DISTRIBUTED BY HASH(k1) BUCKETS 32 + PROPERTIES ("storage_type"="column"); + + 2. 创建一个 olap 表,使用 Hash 分桶,使用列存,相同key的记录进行覆盖, + 设置初始存储介质和冷却时间 + CREATE TABLE example_db.table_hash + ( + k1 BIGINT, + k2 LARGEINT, + v1 VARCHAR(2048) REPLACE, + v2 SMALLINT SUM DEFAULT "10" + ) + ENGINE=olap + UNIQUE KEY(k1, k2) + DISTRIBUTED BY HASH (k1, k2) BUCKETS 32 + PROPERTIES( + "storage_type"="column", + "storage_medium" = "SSD", + "storage_cooldown_time" = "2015-06-04 00:00:00" + ); + + 3. 创建一个 olap 表,使用 Key Range 分区,使用Hash分桶,默认使用列存, + 相同key的记录同时存在,设置初始存储介质和冷却时间 + CREATE TABLE example_db.table_range + ( + k1 DATE, + k2 INT, + k3 SMALLINT, + v1 VARCHAR(2048), + v2 DATETIME DEFAULT "2014-02-04 15:36:00" + ) + ENGINE=olap + DUPLICATE KEY(k1, k2, k3) + PARTITION BY RANGE (k1) + ( + PARTITION p1 VALUES LESS THAN ("2014-01-01"), + PARTITION p2 VALUES LESS THAN ("2014-06-01"), + PARTITION p3 VALUES LESS THAN ("2014-12-01") + ) + DISTRIBUTED BY HASH(k2) BUCKETS 32 + PROPERTIES( + "storage_medium" = "SSD", "storage_cooldown_time" = "2015-06-04 00:00:00" + ); + + 说明: + 这个语句会将数据划分成如下3个分区: + ( { MIN }, {"2014-01-01"} ) + [ {"2014-01-01"}, {"2014-06-01"} ) + [ {"2014-06-01"}, {"2014-12-01"} ) + + 不在这些分区范围内的数据将视为非法数据被过滤 + + 4. 创建一个 mysql 表 + CREATE TABLE example_db.table_mysql + ( + k1 DATE, + k2 INT, + k3 SMALLINT, + k4 VARCHAR(2048), + k5 DATETIME + ) + ENGINE=mysql + PROPERTIES + ( + "host" = "127.0.0.1", + "port" = "8239", + "user" = "mysql_user", + "password" = "mysql_passwd", + "database" = "mysql_db_test", + "table" = "mysql_table_test" + ) + + 5. 创建一个数据文件存储在HDFS上的 broker 外部表, 数据使用 "|" 分割,"\n" 换行 + CREATE EXTERNAL TABLE example_db.table_broker ( + k1 DATE, + k2 INT, + k3 SMALLINT, + k4 VARCHAR(2048), + k5 DATETIME + ) + ENGINE=broker + PROPERTIES ( + "broker_name" = "hdfs", + "path" = "hdfs://hdfs_host:hdfs_port/data1,hdfs://hdfs_host:hdfs_port/data2,hdfs://hdfs_host:hdfs_port/data3%2c4", + "column_separator" = "|", + "line_delimiter" = "\n" + ) + BROKER PROPERTIES ( + "username" = "hdfs_user", + "password" = "hdfs_password" + ) + + 6. 创建一张含有HLL列的表 + CREATE TABLE example_db.example_table + ( + k1 TINYINT, + k2 DECIMAL(10, 2) DEFAULT "10.5", + v1 HLL HLL_UNION, + v2 HLL HLL_UNION + ) + ENGINE=olap + AGGREGATE KEY(k1, k2) + DISTRIBUTED BY HASH(k1) BUCKETS 32 + PROPERTIES ("storage_type"="column"); + + 7. 创建两张支持Colocat Join的表t1 和t2 + CREATE TABLE `t1` ( + `id` int(11) COMMENT "", + `value` varchar(8) COMMENT "" + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS 10 + PROPERTIES ( + "colocate_with" = "t1" + ); + + CREATE TABLE `t2` ( + `id` int(11) COMMENT "", + `value` varchar(8) COMMENT "" + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS 10 + PROPERTIES ( + "colocate_with" = "t1" + ); + + 8. 创建一个数据文件存储在BOS上的 broker 外部表 + CREATE EXTERNAL TABLE example_db.table_broker ( + k1 DATE + ) + ENGINE=broker + PROPERTIES ( + "broker_name" = "bos", + "path" = "bos://my_bucket/input/file", + ) + BROKER PROPERTIES ( + "bos_endpoint" = "http://bj.bcebos.com", + "bos_accesskey" = "xxxxxxxxxxxxxxxxxxxxxxxxxx", + "bos_secret_accesskey"="yyyyyyyyyyyyyyyyyyyy" + ) + +## keyword + CREATE,TABLE + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Definition/CREATE VIEW.md b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/CREATE VIEW.md new file mode 100644 index 00000000000000..a64e611f0b5c25 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/CREATE VIEW.md @@ -0,0 +1,22 @@ +# CREATE VIEW +## description + 该语句用于创建一个逻辑视图 + 语法: + CREATE VIEW [IF NOT EXISTS] + [db_name.]view_name (column1[, column2, ...]) + AS query_stmt + + 说明: + 1. 视图为逻辑视图,没有物理存储。所有在视图上的查询相当于在视图对应的子查询上进行。 + 2. query_stmt 为任意支持的 SQL + +## example + 1. 在 example_db 上创建视图 example_view + CREATE VIEW example_db.example_view (k1, k2, k3, v1) + AS + SELECT c1 as k1, k2, k3, SUM(v1) FROM example_table + WHERE k1 = 20160112 GROUP BY k1,k2,k3; + +## keyword + CREATE,VIEW + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Definition/Colocate Join.md b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/Colocate Join.md new file mode 100644 index 00000000000000..9a8529ffb0ed76 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/Colocate Join.md @@ -0,0 +1,72 @@ +# Colocate Join +## description + Colocate/Local Join 就是指多个节点Join时没有数据移动和网络传输,每个节点只在本地进行Join, + 能够本地进行Join的前提是相同Join Key的数据导入时按照相同规则导入到固定的节点。 + + 1 How To Use: + + 只需要在建表时增加 colocate_with 这个属性即可,colocate_with的值 可以设置成同一组colocate 表中的任意一个, + 不过需要保证colocate_with属性中的表要先建立。 + + 假如需要对table t1 和t2 进行Colocate Join,可以按以下语句建表: + + CREATE TABLE `t1` ( + `id` int(11) COMMENT "", + `value` varchar(8) COMMENT "" + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS 10 + PROPERTIES ( + "colocate_with" = "t1" + ); + + CREATE TABLE `t2` ( + `id` int(11) COMMENT "", + `value` varchar(8) COMMENT "" + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS 10 + PROPERTIES ( + "colocate_with" = "t1" + ); + + 2 Colocate Join 目前的限制: + + 1. Colcoate Table 必须是OLAP类型的表 + 2. 相同colocate_with 属性的表的 BUCKET 数必须一样 + 3. 相同colocate_with 属性的表的 副本数必须一样 + 4. 相同colocate_with 属性的表的 DISTRIBUTED Columns的数据类型必须一样 + + 3 Colocate Join的适用场景: + + Colocate Join 十分适合几张表按照相同字段分桶,并高频根据相同字段Join的场景。 + + 4 FAQ: + + Q: 支持多张表进行Colocate Join 吗? + + A: 支持 + + Q: 支持Colocate 表和正常表 Join 吗? + + A: 支持 + + Q: Colocate 表支持用非分桶的Key进行Join吗? + + A: 支持:不符合Colocate Join条件的Join会使用Shuffle Join或Broadcast Join + + Q: 如何确定Join 是按照Colocate Join 执行的? + + A: explain的结果中Hash Join的孩子节点如果直接是OlapScanNode, 没有Exchange Node,就说明是Colocate Join + + Q: 如何修改colocate_with 属性? + + A: ALTER TABLE example_db.my_table set ("colocate_with"="target_table"); + + Q: 如何禁用colcoate join? + + A: set disable_colocate_join = true; 就可以禁用Colocate Join,查询时就会使用Shuffle Join 和Broadcast Join + +## keyword + + COLOCATE, JOIN, CREATE TABLE diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Definition/DROP DATABASE.md b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/DROP DATABASE.md new file mode 100644 index 00000000000000..7ad5ba06bdddb2 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/DROP DATABASE.md @@ -0,0 +1,16 @@ +# DROP DATABASE +## description + 该语句用于删除数据库(database) + 语法: + DROP DATABASE [IF EXISTS] db_name; + + 说明: + 执行 DROP DATABASE 一段时间内,可以通过 RECOVER 语句恢复被删除的 database。详见 RECOVER 语句 + +## example + 1. 删除数据库 db_test + DROP DATABASE db_test; + +## keyword + DROP,DATABASE + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Definition/DROP REPOSITORY.md b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/DROP REPOSITORY.md new file mode 100644 index 00000000000000..1ecdc6a30a65ce --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/DROP REPOSITORY.md @@ -0,0 +1,16 @@ +# DROP REPOSITORY +## description + 该语句用于删除一个已创建的仓库。仅 root 或 superuser 用户可以删除仓库。 + 语法: + DROP REPOSITORY `repo_name`; + + 说明: + 1. 删除仓库,仅仅是删除该仓库在 Palo 中的映射,不会删除实际的仓库数据。删除后,可以再次通过指定相同的 broker 和 LOCATION 映射到该仓库。 + +## example + 1. 删除名为 bos_repo 的仓库: + DROP REPOSITORY `bos_repo`; + +## keyword + DROP REPOSITORY + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Definition/DROP TABLE.md b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/DROP TABLE.md new file mode 100644 index 00000000000000..9f1473b9a2fef5 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/DROP TABLE.md @@ -0,0 +1,19 @@ +# DROP TABLE +## description + 该语句用于删除 table 。 + 语法: + DROP TABLE [IF EXISTS] [db_name.]table_name; + + 说明: + 执行 DROP TABLE 一段时间内,可以通过 RECOVER 语句恢复被删除的 table。详见 RECOVER 语句 + +## example + 1. 删除一个 table + DROP TABLE my_table; + + 2. 如果存在,删除指定 database 的 table + DROP TABLE IF EXISTS example_db.my_table; + +## keyword + DROP,TABLE + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Definition/DROP VIEW.md b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/DROP VIEW.md new file mode 100644 index 00000000000000..e2feb735b4c5ca --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/DROP VIEW.md @@ -0,0 +1,14 @@ +# DROP VIEW +## description + 该语句用于删除一个逻辑视图 VIEW + 语法: + DROP VIEW [IF EXISTS] + [db_name.]view_name; + +## example + 1. 如果存在,删除 example_db 上的视图 example_view + DROP VIEW IF EXISTS example_db.example_view; + +## keyword + DROP,VIEW + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Definition/HLL.md b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/HLL.md new file mode 100644 index 00000000000000..16b40d7fbb444e --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/HLL.md @@ -0,0 +1,80 @@ +# HLL +## description + HLL是基于HyperLogLog算法的工程实现,用于保存HyperLogLog计算过程的中间结果,它只能作为表的value列类型 + 通过聚合来不断的减少数据量,以此来实现加快查询的目的,基于它到的是一个估算结果,误差大概在1%左右 + hll列是通过其它列或者导入数据里面的数据生成的,导入的时候通过hll_hash函数来指定数据中哪一列用于生成hll列 + 它常用于替代count distinct,通过结合rollup在业务上用于快速计算uv等 + + 相关函数: + + HLL_UNION_AGG(hll) + 此函数为聚合函数,用于计算满足条件的所有数据的基数估算。此函数还可用于分析函数,只支持默认窗口,不支持window从句。 + + HLL_RAW_AGG(hll) + 此函数为聚合函数,用于聚合hll类型字段,并且返回的还是hll类型。 + + HLL_CARDINALITY(hll) + 此函数用于计算单条hll列的基数估算 + + HLL_HASH(column_name) + 生成HLL列类型,用于insert或导入的时候,导入的使用见相关说明 + +## example + 1. 首先创建一张含有hll列的表 + create table test( + dt date, + id int, + name char(10), + province char(10), + os char(1), + set1 hll hll_union, + set2 hll hll_union) + distributed by hash(id) buckets 32; + + 2. 导入数据,导入的方式见相关help curl + + a. 使用表中的列生成hll列 + curl --location-trusted -uname:password -T data http://host/api/test_db/test/_load?label=load_1\&hll=set1,id:set2,name + + b. 使用数据中的某一列生成hll列 + curl --location-trusted -uname:password -T data http://host/api/test_db/test/_load?label=load_1\&hll=set1,cuid:set2,os + \&columns=dt,id,name,province,sex,cuid,os + + 3. 聚合数据,常用方式3种:(如果不聚合直接对base表查询,速度可能跟直接使用ndv速度差不多) + + a. 创建一个rollup,让hll列产生聚合, + alter table test add rollup test_rollup(dt, set1); + + b. 创建另外一张专门计算uv的表,然后insert数据) + + create table test_uv( + dt date, + uv_set hll hll_union) + distributed by hash(id) buckets 32; + + insert into test_uv select dt, set1 from test; + + c. 创建另外一张专门计算uv的表,然后insert并通过hll_hash根据test其它非hll列生成hll列 + + create table test_uv( + dt date, + id_set hll hll_union) + distributed by hash(id) buckets 32; + + insert into test_uv select dt, hll_hash(id) from test; + + 4. 查询,hll列不允许直接查询它的原始值,可以通过配套的函数进行查询 + + a. 求总uv + select HLL_UNION_AGG(uv_set) from test_uv; + + b. 求每一天的uv + select dt, HLL_CARDINALITY(uv_set) from test_uv; + + c. 求test表中set1的聚合值 + select dt, HLL_CARDINALITY(uv) from (select dt, HLL_RAW_AGG(set1) as uv from test group by dt) tmp; + select dt, HLL_UNION_AGG(set1) as uv from test group by dt; + +## keyword + HLL + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Definition/RECOVER.md b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/RECOVER.md new file mode 100644 index 00000000000000..73b52ae2942516 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/RECOVER.md @@ -0,0 +1,28 @@ +# RECOVER +## description + 该语句用于恢复之前删除的 database、table 或者 partition + 语法: + 1) 恢复 database + RECOVER DATABASE db_name; + 2) 恢复 table + RECOVER TABLE [db_name.]table_name; + 3) 恢复 partition + RECOVER PARTITION partition_name FROM [db_name.]table_name; + + 说明: + 1. 该操作仅能恢复之前一段时间内删除的元信息。默认为 3600 秒。 + 2. 如果删除元信息后新建立了同名同类型的元信息,则之前删除的元信息不能被恢复 + +## example + 1. 恢复名为 example_db 的 database + RECOVER DATABASE example_db; + + 2. 恢复名为 example_tbl 的 table + RECOVER TABLE example_db.example_tbl; + + 3. 恢复表 example_tbl 中名为 p1 的 partition + RECOVER PARTITION p1 FROM example_tbl; + +## keyword + RECOVER + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Definition/RESTORE.md b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/RESTORE.md new file mode 100644 index 00000000000000..0f6f71e3a52319 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/RESTORE.md @@ -0,0 +1,52 @@ +# RESTORE +## description + 1. RESTORE + 该语句用于将之前通过 BACKUP 命令备份的数据,恢复到指定数据库下。该命令为异步操作。提交成功后,需通过 SHOW RESTORE 命令查看进度。仅支持恢复 OLAP 类型的表。 + 语法: + RESTORE SNAPSHOT [db_name].{snapshot_name} + FROM `repository_name` + ON ( + `table_name` [PARTITION (`p1`, ...)] [AS `tbl_alias`], + ... + ) + PROPERTIES ("key"="value", ...); + + 说明: + 1. 同一数据库下只能有一个正在执行的 BACKUP 或 RESTORE 任务。 + 2. ON 子句中标识需要恢复的表和分区。如果不指定分区,则默认恢复该表的所有分区。所指定的表和分区必须已存在于仓库备份中。 + 3. 可以通过 AS 语句将仓库中备份的表名恢复为新的表。但新表名不能已存在于数据库中。分区名称不能修改。 + 4. 可以将仓库中备份的表恢复替换数据库中已有的同名表,但须保证两张表的表结构完全一致。表结构包括:表名、列、分区、Rollup等等。 + 5. 可以指定恢复表的部分分区,系统会检查分区 Range 是否能够匹配。 + 6. PROPERTIES 目前支持以下属性: + "backup_timestamp" = "2018-05-04-16-45-08":指定了恢复对应备份的哪个时间版本,必填。该信息可以通过 `SHOW SNAPSHOT ON repo;` 语句获得。 + "replication_num" = "3":指定恢复的表或分区的副本数。默认为3。若恢复已存在的表或分区,则副本数必须和已存在表或分区的副本数相同。同时,必须有足够的 host 容纳多个副本。 + "timeout" = "3600":任务超时时间,默认为一天。单位秒。 + "meta_version" = 40:使用指定的 meta_version 来读取之前备份的元数据。注意,该参数作为临时方案,仅用于恢复老版本 Doris 备份的数据。最新版本的备份数据中已经包含 meta version,无需再指定。 + +## example + 1. 从 example_repo 中恢复备份 snapshot_1 中的表 backup_tbl 到数据库 example_db1,时间版本为 "2018-05-04-16-45-08"。恢复为 1 个副本: + RESTORE SNAPSHOT example_db1.`snapshot_1` + FROM `example_repo` + ON ( `backup_tbl` ) + PROPERTIES + ( + "backup_timestamp"="2018-05-04-16-45-08", + "replication_num" = "1" + ); + + 2. 从 example_repo 中恢复备份 snapshot_2 中的表 backup_tbl 的分区 p1,p2,以及表 backup_tbl2 到数据库 example_db1,并重命名为 new_tbl,时间版本为 "2018-05-04-17-11-01"。默认恢复为 3 个副本: + RESTORE SNAPSHOT example_db1.`snapshot_2` + FROM `example_repo` + ON + ( + `backup_tbl` PARTITION (`p1`, `p2`), + `backup_tbl2` AS `new_tbl` + ) + PROPERTIES + ( + "backup_timestamp"="2018-05-04-17-11-01" + ); + +## keyword + RESTORE + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Definition/TRUNCATE TABLE.md b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/TRUNCATE TABLE.md new file mode 100644 index 00000000000000..28c68c5bc24675 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/TRUNCATE TABLE.md @@ -0,0 +1,27 @@ +# TRUNCATE TABLE +## description + 该语句用于清空指定表和分区的数据 + 语法: + + TRUNCATE TABLE [db.]tbl[ PARTITION(p1, p2, ...)]; + + 说明: + 1. 该语句清空数据,但保留表或分区。 + 2. 不同于 DELETE,该语句只能整体清空指定的表或分区,不能添加过滤条件。 + 3. 不同于 DELETE,使用该方式清空数据不会对查询性能造成影响。 + 4. 该操作删除的数据不可恢复。 + 5. 使用该命令时,表状态需为 NORMAL,即不允许正在进行 SCHEMA CHANGE 等操作。 + +## example + + 1. 清空 example_db 下的表 tbl + + TRUNCATE TABLE example_db.tbl; + + 2. 清空表 tbl 的 p1 和 p2 分区 + + TRUNCATE TABLE tbl PARTITION(p1, p2); + +## keyword + TRUNCATE,TABLE + diff --git a/docs/documentation/cn/sql-reference/sql-statements/create-function.md b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/create-function.md similarity index 97% rename from docs/documentation/cn/sql-reference/sql-statements/create-function.md rename to docs/documentation/cn/sql-reference/sql-statements/Data Definition/create-function.md index b2dbd9954f4b71..00e2d609170f1d 100644 --- a/docs/documentation/cn/sql-reference/sql-statements/create-function.md +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/create-function.md @@ -1,6 +1,6 @@ # CREATE FUNCTION - -## Syntax +## description +### Syntax ``` CREATE [AGGREGATE] FUNCTION function_name @@ -10,13 +10,7 @@ CREATE [AGGREGATE] FUNCTION function_name [PROPERTIES ("key" = "value" [, ...]) ] ``` -## Description - -此语句创建一个自定义函数。执行此命令需要用户拥有 `ADMIN` 权限。 - -如果 `function_name` 中包含了数据库名字,那么这个自定义函数会创建在对应的数据库中,否则这个函数将会创建在当前会话所在的数据库。新函数的名字与参数不能够与当前命名空间中已存在的函数相同,否则会创建失败。但是只有名字相同,参数不同是能够创建成功的。 - -## Parameters +### Parameters > `AGGREGATE`: 如果有此项,表示的是创建的函数是一个聚合函数,否则创建的是一个标量函数。 > @@ -46,7 +40,12 @@ CREATE [AGGREGATE] FUNCTION function_name > > "md5": 函数动态链接库的MD5值,用于校验下载的内容是否正确。此选项是可选项 -## Examples + +此语句创建一个自定义函数。执行此命令需要用户拥有 `ADMIN` 权限。 + +如果 `function_name` 中包含了数据库名字,那么这个自定义函数会创建在对应的数据库中,否则这个函数将会创建在当前会话所在的数据库。新函数的名字与参数不能够与当前命名空间中已存在的函数相同,否则会创建失败。但是只有名字相同,参数不同是能够创建成功的。 + +## example 1. 创建一个自定义标量函数 @@ -68,3 +67,5 @@ CREATE AGGREGATE FUNCTION my_count (BIGINT) RETURNS BIGINT PROPERTIES ( "object_file"="http://host:port/libudasample.so" ); ``` +##keyword +CREATE,FUNCTION diff --git a/docs/documentation/cn/sql-reference/sql-statements/drop-function.md b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/drop-function.md similarity index 81% rename from docs/documentation/cn/sql-reference/sql-statements/drop-function.md rename to docs/documentation/cn/sql-reference/sql-statements/Data Definition/drop-function.md index f228d11a3f444c..419059f68d070d 100644 --- a/docs/documentation/cn/sql-reference/sql-statements/drop-function.md +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/drop-function.md @@ -1,27 +1,28 @@ # DROP FUNCTION - -## Syntax +## description +### Syntax ``` DROP FUNCTION function_name (arg_type [, ...]) ``` -## Description - -删除一个自定义函数。函数的名字、参数类型完全一致才能够被删除 - -## Parameters +### Parameters > `function_name`: 要删除函数的名字 > > `arg_type`: 要删除函数的参数列表 > -## Examples + +删除一个自定义函数。函数的名字、参数类型完全一致才能够被删除 + +## example 1. 删除掉一个函数 ``` DROP FUNCTION my_add(INT, INT) ``` +##keyword +DROP,FUNCTION diff --git a/docs/documentation/cn/sql-reference/sql-statements/show-function.md b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/show-function.md similarity index 94% rename from docs/documentation/cn/sql-reference/sql-statements/show-function.md rename to docs/documentation/cn/sql-reference/sql-statements/Data Definition/show-function.md index ec1818d6de30d9..edd64917304286 100644 --- a/docs/documentation/cn/sql-reference/sql-statements/show-function.md +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Definition/show-function.md @@ -1,22 +1,21 @@ # SHOW FUNCTION - -## Syntax +## description +### Syntax ``` SHOW FUNCTION [FROM db] ``` -## Description +### Parameters -查看数据库下所有的自定义函数。如果用户指定了数据库,那么查看对应数据库的,否则直接查询当前会话所在数据库 +> `db`: 要查询的数据库名字 -需要对这个数据库拥有 `SHOW` 权限 -## Parameters +查看数据库下所有的自定义函数。如果用户指定了数据库,那么查看对应数据库的,否则直接查询当前会话所在数据库 -> `db`: 要查询的数据库名字 +需要对这个数据库拥有 `SHOW` 权限 -## Examples +## example ``` mysql> show function in testDb\G @@ -34,3 +33,5 @@ Intermediate Type: NULL Properties: {"symbol":"_ZN9doris_udf6AddUdfEPNS_15FunctionContextERKNS_6IntValES4_","object_file":"http://host:port/libudfsample.so","md5":"cfe7a362d10f3aaf6c49974ee0f1f878"} 2 rows in set (0.00 sec) ``` +##keyword +SHOW,FUNCTION diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/CANCEL DELETE.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/CANCEL DELETE.md new file mode 100644 index 00000000000000..31f83bf6e793e4 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/CANCEL DELETE.md @@ -0,0 +1,10 @@ +# CANCEL DELETE +## description + + 该语句用于撤销一个 DELETE 操作。(仅管理员使用!)(待实现) + +## example + +## keyword + CANCEL,DELETE + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/CANCEL LABEL.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/CANCEL LABEL.md new file mode 100644 index 00000000000000..c94a0aa232065f --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/CANCEL LABEL.md @@ -0,0 +1,33 @@ +# CANCEL LABEL +## description + NAME: + cancel_label: cancel a transaction with label + + SYNOPSIS + curl -u user:passwd -XPOST http://host:port/api/{db}/{label}/_cancel + + DESCRIPTION + 该命令用于cancel一个指定Label对应的事务,事务在Prepare阶段能够被成功cancel + + RETURN VALUES + 执行完成后,会以Json格式返回这次导入的相关内容。当前包括一下字段 + Status: 是否成功cancel + Success: 成功cancel事务 + 其他: cancel失败 + Message: 具体的失败信息 + + ERRORS + +## example + + 1. cancel testDb, testLabel的作业 + curl -u root -XPOST http://host:port/api/testDb/testLabel/_cancel + +## keyword + CANCEL,LABEL + + + + + + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/CANCEL LOAD.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/CANCEL LOAD.md new file mode 100644 index 00000000000000..02dbb25e66bbe1 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/CANCEL LOAD.md @@ -0,0 +1,20 @@ +# CANCEL LOAD +## description + + 该语句用于撤销指定 load label 的批次的导入作业。 + 这是一个异步操作,任务提交成功则返回。执行后可使用 SHOW LOAD 命令查看进度。 + 语法: + CANCEL LOAD + [FROM db_name] + WHERE LABEL = "load_label"; + +## example + + 1. 撤销数据库 example_db 上, label 为 example_db_test_load_label 的导入作业 + CANCEL LOAD + FROM example_db + WHERE LABEL = "example_db_test_load_label"; + +## keyword + CANCEL,LOAD + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/DELETE.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/DELETE.md new file mode 100644 index 00000000000000..2ed78e10c8e8b9 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/DELETE.md @@ -0,0 +1,36 @@ +# DELETE +## description + + 该语句用于按条件删除指定 table(base index) partition 中的数据。 + 该操作会同时删除和此 base index 相关的 rollup index 的数据。 + 语法: + DELETE FROM table_name [PARTITION partition_name] + WHERE + column_name1 op value[ AND column_name2 op value ...]; + + 说明: + 1) op 的可选类型包括:=, >, <, >=, <=, != + 2) 只能指定 key 列上的条件。 + 2) 当选定的 key 列不存在于某个 rollup 中时,无法进行 delete。 + 3) 条件之间只能是“与”的关系。 + 若希望达成“或”的关系,需要将条件分写在两个 DELETE 语句中。 + 4) 如果为RANGE分区表,则必须指定 PARTITION。如果是单分区表,可以不指定。 + + 注意: + 该语句可能会降低执行后一段时间内的查询效率。 + 影响程度取决于语句中指定的删除条件的数量。 + 指定的条件越多,影响越大。 + +## example + + 1. 删除 my_table partition p1 中 k1 列值为 3 的数据行 + DELETE FROM my_table PARTITION p1 + WHERE k1 = 3; + + 2. 删除 my_table partition p1 中 k1 列值大于等于 3 且 k2 列值为 "abc" 的数据行 + DELETE FROM my_table PARTITION p1 + WHERE k1 >= 3 AND k2 = "abc"; + +## keyword + DELETE + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/EXPORT.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/EXPORT.md new file mode 100644 index 00000000000000..61fc286aef4422 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/EXPORT.md @@ -0,0 +1,58 @@ +# EXPORT +## description + + 该语句用于将指定表的数据导出到指定位置。 + 该功能通过 broker 进程实现。对于不同的目的存储系统,需要部署不同的 broker。可以通过 SHOW BROKER 查看已部署的 broker。 + 这是一个异步操作,任务提交成功则返回。执行后可使用 SHOW EXPORT 命令查看进度。 + + 语法: + EXPORT TABLE table_name + [PARTITION (p1[,p2])] + TO export_path + [opt_properties] + broker; + + 1. table_name + 当前要导出的表的表名,目前支持engine为olap和mysql的表的导出。 + + 2. partition + 可以只导出指定表的某些指定分区 + + 3. export_path + 导出的路径,需为目录。目前不能导出到本地,需要导出到broker。 + + 4. opt_properties + 用于指定一些特殊参数。 + 语法: + [PROPERTIES ("key"="value", ...)] + + 可以指定如下参数: + column_separator: 指定导出的列分隔符,默认为\t。 + line_delimiter: 指定导出的行分隔符,默认为\n。 + exec_mem_limit: 导出在单个 BE 节点的内存使用上限,默认为 2GB,单位为字节。 + timeout:导入作业的超时时间,默认为1天,单位是秒。 + tablet_num_per_task:每个子任务能分配的最大 Tablet 数量。 + + 5. broker + 用于指定导出使用的broker + 语法: + WITH BROKER broker_name ("key"="value"[,...]) + 这里需要指定具体的broker name, 以及所需的broker属性 + + 对于不同存储系统对应的 broker,这里需要输入的参数不同。具体参数可以参阅:`help broker load` 中 broker 所需属性。 + +## example + + 1. 将 testTbl 表中的所有数据导出到 hdfs 上 + EXPORT TABLE testTbl TO "hdfs://hdfs_host:port/a/b/c" WITH BROKER "broker_name" ("username"="xxx", "password"="yyy"); + + 2. 将 testTbl 表中的分区p1,p2导出到 hdfs 上 + + EXPORT TABLE testTbl PARTITION (p1,p2) TO "hdfs://hdfs_host:port/a/b/c" WITH BROKER "broker_name" ("username"="xxx", "password"="yyy"); + 3. 将 testTbl 表中的所有数据导出到 hdfs 上,以","作为列分隔符 + + EXPORT TABLE testTbl TO "hdfs://hdfs_host:port/a/b/c" PROPERTIES ("column_separator"=",") WITH BROKER "broker_name" ("username"="xxx", "password"="yyy"); + +## keyword + EXPORT + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/GET LABEL STATE.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/GET LABEL STATE.md new file mode 100644 index 00000000000000..069a86dc7bde5e --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/GET LABEL STATE.md @@ -0,0 +1,33 @@ +# GET LABEL STATE +## description + NAME: + get_label_state: get label's state + + SYNOPSIS + curl -u user:passwd http://host:port/api/{db}/{label}/_state + + DESCRIPTION + 该命令用于查看一个Label对应的事务状态 + + RETURN VALUES + 执行完毕后,会以Json格式返回这次导入的相关内容。当前包括一下字段 + Label:本次导入的 label,如果没有指定,则为一个 uuid。 + Status:此命令是否成功执行,Success表示成功执行 + Message: 具体的执行信息 + State: 只有在Status为Success时才有意义 + UNKNOWN: 没有找到对应的Label + PREPARE: 对应的事务已经prepare,但尚未提交 + COMMITTED: 事务已经提交,不能被cancel + VISIBLE: 事务提交,并且数据可见,不能被cancel + ABORTED: 事务已经被ROLLBACK,导入已经失败。 + + ERRORS + +## example + + 1. 获得testDb, testLabel的状态 + curl -u root http://host:port/api/testDb/testLabel/_state + +## keyword + GET, LABEL, STATE + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/LOAD.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/LOAD.md new file mode 100644 index 00000000000000..34a7c8c8ad5690 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/LOAD.md @@ -0,0 +1,284 @@ +# LOAD +## description + + Palo 目前支持以下4种导入方式: + + 1. Hadoop Load:基于 MR 进行 ETL 的导入。 + 2. Broker Load:使用 broker 进行进行数据导入。 + 3. Mini Load:通过 http 协议上传文件进行批量数据导入。 + 4. Stream Load:通过 http 协议进行流式数据导入。 + + 本帮助主要描述第一种导入方式,即 Hadoop Load 相关帮助信息。其余导入方式可以使用以下命令查看帮助: + + !!!该导入方式可能在后续某个版本即不再支持,建议使用其他导入方式进行数据导入。!!! + + 1. help broker load; + 2. help mini load; + 3. help stream load; + + Hadoop Load 仅适用于百度内部环境。公有云、私有云以及开源环境无法使用这种导入方式。 + 该导入方式必须设置用于 ETL 的 Hadoop 计算队列,设置方式可以通过 help set property 命令查看帮助。 + + Stream load 暂时只支持百度内部用户使用。开源社区和公有云用户将在后续版本更新中支持。 + +语法: + + LOAD LABEL load_label + ( + data_desc1[, data_desc2, ...] + ) + [opt_properties]; + + 1. load_label + + 当前导入批次的标签。在一个 database 内唯一。 + 语法: + [database_name.]your_label + + 2. data_desc + + 用于描述一批导入数据。 + 语法: + DATA INFILE + ( + "file_path1"[, file_path2, ...] + ) + [NEGATIVE] + INTO TABLE `table_name` + [PARTITION (p1, p2)] + [COLUMNS TERMINATED BY "column_separator"] + [FORMAT AS "file_type"] + [(column_list)] + [SET (k1 = func(k2))] + + 说明: + file_path: + + 文件路径,可以指定到一个文件,也可以用 * 通配符指定某个目录下的所有文件。通配符必须匹配到文件,而不能是目录。 + + PARTITION: + + 如果指定此参数,则只会导入指定的分区,导入分区以外的数据会被过滤掉。 + 如果不指定,默认导入table的所有分区。 + + NEGATIVE: + 如果指定此参数,则相当于导入一批“负”数据。用于抵消之前导入的同一批数据。 + 该参数仅适用于存在 value 列,并且 value 列的聚合类型仅为 SUM 的情况。 + + column_separator: + + 用于指定导入文件中的列分隔符。默认为 \t + 如果是不可见字符,则需要加\\x作为前缀,使用十六进制来表示分隔符。 + 如hive文件的分隔符\x01,指定为"\\x01" + + file_type: + + 用于指定导入文件的类型,例如:parquet、csv。默认值通过文件后缀名判断。 + + column_list: + + 用于指定导入文件中的列和 table 中的列的对应关系。 + 当需要跳过导入文件中的某一列时,将该列指定为 table 中不存在的列名即可。 + 语法: + (col_name1, col_name2, ...) + + SET: + + 如果指定此参数,可以将源文件某一列按照函数进行转化,然后将转化后的结果导入到table中。 + 目前支持的函数有: + + strftime(fmt, column) 日期转换函数 + fmt: 日期格式,形如%Y%m%d%H%M%S (年月日时分秒) + column: column_list中的列,即输入文件中的列。存储内容应为数字型的时间戳。 + 如果没有column_list,则按照palo表的列顺序默认输入文件的列。 + + time_format(output_fmt, input_fmt, column) 日期格式转化 + output_fmt: 转化后的日期格式,形如%Y%m%d%H%M%S (年月日时分秒) + input_fmt: 转化前column列的日期格式,形如%Y%m%d%H%M%S (年月日时分秒) + column: column_list中的列,即输入文件中的列。存储内容应为input_fmt格式的日期字符串。 + 如果没有column_list,则按照palo表的列顺序默认输入文件的列。 + + alignment_timestamp(precision, column) 将时间戳对齐到指定精度 + precision: year|month|day|hour + column: column_list中的列,即输入文件中的列。存储内容应为数字型的时间戳。 + 如果没有column_list,则按照palo表的列顺序默认输入文件的列。 + 注意:对齐精度为year、month的时候,只支持20050101~20191231范围内的时间戳。 + + default_value(value) 设置某一列导入的默认值 + 不指定则使用建表时列的默认值 + + md5sum(column1, column2, ...) 将指定的导入列的值求md5sum,返回32位16进制字符串 + + replace_value(old_value[, new_value]) 将导入文件中指定的old_value替换为new_value + new_value如不指定则使用建表时列的默认值 + + hll_hash(column) 用于将表或数据里面的某一列转化成HLL列的数据结构 + + 3. opt_properties + + 用于指定一些特殊参数。 + 语法: + [PROPERTIES ("key"="value", ...)] + + 可以指定如下参数: + cluster: 导入所使用的 Hadoop 计算队列。 + timeout: 指定导入操作的超时时间。默认超时为3天。单位秒。 + max_filter_ratio:最大容忍可过滤(数据不规范等原因)的数据比例。默认零容忍。 + load_delete_flag:指定该导入是否通过导入key列的方式删除数据,仅适用于UNIQUE KEY, + 导入时可不指定value列。默认为false。 + + 5. 导入数据格式样例 + + 整型类(TINYINT/SMALLINT/INT/BIGINT/LARGEINT):1, 1000, 1234 + 浮点类(FLOAT/DOUBLE/DECIMAL):1.1, 0.23, .356 + 日期类(DATE/DATETIME):2017-10-03, 2017-06-13 12:34:03。 + (注:如果是其他日期格式,可以在导入命令中,使用 strftime 或者 time_format 函数进行转换) + 字符串类(CHAR/VARCHAR):"I am a student", "a" + NULL值:\N + +## example + + 1. 导入一批数据,指定超时时间和过滤比例。指定导入队列为 my_cluster。 + + LOAD LABEL example_db.label1 + ( + DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/file") + INTO TABLE `my_table` + ) + PROPERTIES + ( + "cluster" = "my_cluster", + "timeout" = "3600", + "max_filter_ratio" = "0.1" + ); + + 其中 hdfs_host 为 namenode 的 host,hdfs_port 为 fs.defaultFS 端口(默认9000) + + 2. 导入一批数据,包含多个文件。导入不同的 table,指定分隔符,指定列对应关系 + + LOAD LABEL example_db.label2 + ( + DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/file1") + INTO TABLE `my_table_1` + COLUMNS TERMINATED BY "," + (k1, k3, k2, v1, v2), + DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/file2") + INTO TABLE `my_table_2` + COLUMNS TERMINATED BY "\t" + (k1, k2, k3, v2, v1) + ); + + 3. 导入一批数据,指定hive的默认分隔符\x01,并使用通配符*指定目录下的所有文件 + + LOAD LABEL example_db.label3 + ( + DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/*") + NEGATIVE + INTO TABLE `my_table` + COLUMNS TERMINATED BY "\\x01" + ); + + 4. 导入一批“负”数据 + + LOAD LABEL example_db.label4 + ( + DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/old_file) + NEGATIVE + INTO TABLE `my_table` + COLUMNS TERMINATED BY "\t" + ); + + 5. 导入一批数据,指定分区 + + LOAD LABEL example_db.label5 + ( + DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/file") + INTO TABLE `my_table` + PARTITION (p1, p2) + COLUMNS TERMINATED BY "," + (k1, k3, k2, v1, v2) + ); + + 6. 导入一批数据,指定分区, 并对导入文件的列做一些转化,如下: + 表结构为: + k1 datetime + k2 date + k3 bigint + k4 varchar(20) + k5 varchar(64) + k6 int + + 假设数据文件只有一行数据,5列,逗号分隔: + + 1537002087,2018-08-09 11:12:13,1537002087,-,1 + + 数据文件中各列,对应导入语句中指定的各列: + tmp_k1, tmp_k2, tmp_k3, k6, v1 + + 转换如下: + + 1) k1:将 tmp_k1 时间戳列转化为 datetime 类型的数据 + 2) k2:将 tmp_k2 datetime 类型的数据转化为 date 的数据 + 3) k3:将 tmp_k3 时间戳列转化为天级别时间戳 + 4) k4:指定导入默认值为1 + 5) k5:将 tmp_k1、tmp_k2、tmp_k3 列计算 md5 值 + 6) k6:将导入文件中的 - 值替换为 10 + + LOAD LABEL example_db.label6 + ( + DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/file") + INTO TABLE `my_table` + PARTITION (p1, p2) + COLUMNS TERMINATED BY "," + (tmp_k1, tmp_k2, tmp_k3, k6, v1) + SET ( + k1 = strftime("%Y-%m-%d %H:%M:%S", tmp_k1), + k2 = time_format("%Y-%m-%d %H:%M:%S", "%Y-%m-%d", tmp_k2), + k3 = alignment_timestamp("day", tmp_k3), + k4 = default_value("1"), + k5 = md5sum(tmp_k1, tmp_k2, tmp_k3), + k6 = replace_value("-", "10") + ) + ); + + 7. 导入数据到含有HLL列的表,可以是表中的列或者数据里面的列 + + LOAD LABEL example_db.label7 + ( + DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/file") + INTO TABLE `my_table` + PARTITION (p1, p2) + COLUMNS TERMINATED BY "," + SET ( + v1 = hll_hash(k1), + v2 = hll_hash(k2) + ) + ); + + LOAD LABEL example_db.label8 + ( + DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/file") + INTO TABLE `my_table` + PARTITION (p1, p2) + COLUMNS TERMINATED BY "," + (k1, k2, tmp_k3, tmp_k4, v1, v2) + SET ( + v1 = hll_hash(tmp_k3), + v2 = hll_hash(tmp_k4) + ) + ) + WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); + + 8. 导入Parquet文件中数据 指定FORMAT 为parquet, 默认是通过文件后缀判断 + LOAD LABEL example_db.label9 + ( + DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/file") + INTO TABLE `my_table` + FORMAT AS "parquet" + (k1, k2, k3) + ) + WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); + +## keyword + LOAD + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/MINI LOAD.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/MINI LOAD.md new file mode 100644 index 00000000000000..81a63f73bdbf93 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/MINI LOAD.md @@ -0,0 +1,104 @@ +# MINI LOAD +## description + + MINI LOAD 和 STREAM LOAD 的导入实现方式完全一致。在导入功能支持上,MINI LOAD 的功能是 STREAM LOAD 的子集。 + 后续的导入新功能只会在 STREAM LOAD 中支持,MINI LOAD 将不再新增功能。建议改用 STREAM LOAD,具体使用方式请 HELP STREAM LOAD。 + + MINI LOAD 是 通过 http 协议完成的导入方式。用户可以不依赖 Hadoop,也无需通过 Mysql 客户端,即可完成导入。 + 用户通过 http 协议描述导入,数据在接受 http 请求的过程中被流式的导入 Doris , **导入作业完成后** 返回给用户导入的结果。 + + * 注:为兼容旧版本 mini load 使用习惯,用户依旧可以通过 'SHOW LOAD' 命令来查看导入结果。 + + 语法: + 导入: + + curl --location-trusted -u user:passwd -T data.file http://host:port/api/{db}/{table}/_load?label=xxx + + 查看导入信息 + + curl -u user:passwd http://host:port/api/{db}/_load_info?label=xxx + + HTTP协议相关说明 + + 权限认证 当前 Doris 使用 http 的 Basic 方式权限认证。所以在导入的时候需要指定用户名密码 + 这种方式是明文传递密码的,暂不支持加密传输。 + + Expect Doris 需要发送过来的 http 请求带有 'Expect' 头部信息,内容为 '100-continue'。 + 为什么呢?因为我们需要将请求进行 redirect,那么必须在传输数据内容之前, + 这样可以避免造成数据的多次传输,从而提高效率。 + + Content-Length Doris 需要在发送请求时带有 'Content-Length' 这个头部信息。如果发送的内容比 + 'Content-Length' 要少,那么 Doris 认为传输出现问题,则提交此次任务失败。 + NOTE: 如果,发送的数据比 'Content-Length' 要多,那么 Doris 只读取 'Content-Length' + 长度的内容,并进行导入 + + + 参数说明: + + user: 用户如果是在default_cluster中的,user即为user_name。否则为user_name@cluster_name。 + + label: 用于指定这一批次导入的 label,用于后期进行作业查询等。 + 这个参数是必须传入的。 + + columns: 用于描述导入文件中对应的列名字。 + 如果不传入,那么认为文件中的列顺序与建表的顺序一致, + 指定的方式为逗号分隔,例如:columns=k1,k2,k3,k4 + + column_separator: 用于指定列与列之间的分隔符,默认的为'\t' + NOTE: 需要进行url编码,譬如 + 需要指定'\t'为分隔符,那么应该传入'column_separator=%09' + 需要指定'\x01'为分隔符,那么应该传入'column_separator=%01' + 需要指定','为分隔符,那么应该传入'column_separator=%2c' + + + max_filter_ratio: 用于指定允许过滤不规范数据的最大比例,默认是0,不允许过滤 + 自定义指定应该如下:'max_filter_ratio=0.2',含义是允许20%的错误率 + + timeout: 指定 load 作业的超时时间,单位是秒。当load执行时间超过该阈值时,会自动取消。默认超时时间是 86400 秒。 + 建议指定 timeout 时间小于 86400 秒。 + + hll: 用于指定数据里面和表里面的HLL列的对应关系,表中的列和数据里面指定的列 + (如果不指定columns,则数据列面的列也可以是表里面的其它非HLL列)通过","分割 + 指定多个hll列使用“:”分割,例如: 'hll1,cuid:hll2,device' + + NOTE: + 1. 此种导入方式当前是在一台机器上完成导入工作,因而不宜进行数据量较大的导入工作。 + 建议导入数据量不要超过 1 GB + + 2. 当前无法使用 `curl -T "{file1, file2}"` 这样的方式提交多个文件,因为curl是将其拆成多个 + 请求发送的,多个请求不能共用一个label号,所以无法使用 + + 3. mini load 的导入方式和 streaming 完全一致,都是在流式的完成导入后,同步的返回结果给用户。 + 后续查询虽可以查到 mini load 的信息,但不能对其进行操作,查询只为兼容旧的使用方式。 + + 4. 当使用 curl 命令行导入时,需要在 & 前加入 \ 转义,否则参数信息会丢失。 + +## example + + 1. 将本地文件'testData'中的数据导入到数据库'testDb'中'testTbl'的表(用户是defalut_cluster中的) + curl --location-trusted -u root -T testData http://host:port/api/testDb/testTbl/_load?label=123 + + 2. 将本地文件'testData'中的数据导入到数据库'testDb'中'testTbl'的表(用户是test_cluster中的)。超时时间是 3600 秒 + curl --location-trusted -u root@test_cluster:root -T testData http://fe.host:port/api/testDb/testTbl/_load?label=123&timeout=3600 + + 3. 将本地文件'testData'中的数据导入到数据库'testDb'中'testTbl'的表, 允许20%的错误率(用户是defalut_cluster中的) + curl --location-trusted -u root -T testData http://host:port/api/testDb/testTbl/_load?label=123\&max_filter_ratio=0.2 + + 4. 将本地文件'testData'中的数据导入到数据库'testDb'中'testTbl'的表, 允许20%的错误率,并且指定文件的列名(用户是defalut_cluster中的) + curl --location-trusted -u root -T testData http://host:port/api/testDb/testTbl/_load?label=123\&max_filter_ratio=0.2\&columns=k1,k2,k3 + + 5. 使用streaming方式导入(用户是defalut_cluster中的) + seq 1 10 | awk '{OFS="\t"}{print $1, $1 * 10}' | curl --location-trusted -u root -T - http://host:port/api/testDb/testTbl/_load?label=123 + + 6. 导入含有HLL列的表,可以是表中的列或者数据中的列用于生成HLL列(用户是defalut_cluster中的 + + curl --location-trusted -u root -T testData http://host:port/api/testDb/testTbl/_load?label=123\&max_filter_ratio=0.2 + \&hll=hll_column1,tmp_k4:hll_column2,tmp_k5\&columns=k1,k2,k3,tmp_k4,tmp_k5 + + 7. 查看提交后的导入情况 + + curl -u root http://host:port/api/testDb/_load_info?label=123 + +## keyword + MINI, LOAD + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/MULTI LOAD.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/MULTI LOAD.md new file mode 100644 index 00000000000000..dc58fb39d75992 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/MULTI LOAD.md @@ -0,0 +1,82 @@ +# MULTI LOAD +## description + + Syntax: + curl --location-trusted -u user:passwd -XPOST http://host:port/api/{db}/_multi_start?label=xxx + curl --location-trusted -u user:passwd -T data.file http://host:port/api/{db}/{table1}/_load?label=xxx\&sub_label=yyy + curl --location-trusted -u user:passwd -T data.file http://host:port/api/{db}/{table2}/_load?label=xxx\&sub_label=zzz + curl --location-trusted -u user:passwd -XPOST http://host:port/api/{db}/_multi_commit?label=xxx + curl --location-trusted -u user:passwd -XPOST http://host:port/api/{db}/_multi_desc?label=xxx + + 'MULTI LOAD'在'MINI LOAD'的基础上,可以支持用户同时向多个表进行导入,具体的命令如上面所示 + '/api/{db}/_multi_start' 开始一个多表导入任务 + '/api/{db}/{table}/_load' 向一个导入任务添加一个要导入的表,与'MINI LOAD'的主要区别是,需要传入'sub_label'参数 + '/api/{db}/_multi_commit' 提交整个多表导入任务,后台开始进行处理 + '/api/{db}/_multi_abort' 放弃一个多表导入任务 + '/api/{db}/_multi_desc' 可以展示某个多表导入任务已经提交的作业数 + + HTTP协议相关说明 + 权限认证 当前 Doris 使用http的Basic方式权限认证。所以在导入的时候需要指定用户名密码 + 这种方式是明文传递密码的,鉴于我们当前都是内网环境。。。 + + Expect Doris 需要发送过来的http请求,需要有'Expect'头部信息,内容为'100-continue' + 为什么呢?因为我们需要将请求进行redirect,那么必须在传输数据内容之前, + 这样可以避免造成数据的多次传输,从而提高效率。 + + Content-Length Doris 需要在发送请求是带有'Content-Length'这个头部信息。如果发送的内容比 + 'Content-Length'要少,那么Palo认为传输出现问题,则提交此次任务失败。 + NOTE: 如果,发送的数据比'Content-Length'要多,那么 Doris 只读取'Content-Length' + 长度的内容,并进行导入 + + 参数说明: + user: 用户如果是在default_cluster中的,user即为user_name。否则为user_name@cluster_name。 + + label: 用于指定这一批次导入的label号,用于后期进行作业状态查询等。 + 这个参数是必须传入的。 + + sub_label: 用于指定一个多表导入任务内部的子版本号。对于多表导入的load, 这个参数是必须传入的。 + + columns: 用于描述导入文件中对应的列名字。 + 如果不传入,那么认为文件中的列顺序与建表的顺序一致, + 指定的方式为逗号分隔,例如:columns=k1,k2,k3,k4 + + column_separator: 用于指定列与列之间的分隔符,默认的为'\t' + NOTE: 需要进行url编码,譬如需要指定'\t'为分隔符, + 那么应该传入'column_separator=%09' + + max_filter_ratio: 用于指定允许过滤不规范数据的最大比例,默认是0,不允许过滤 + 自定义指定应该如下:'max_filter_ratio=0.2',含义是允许20%的错误率 + 在'_multi_start'时传入有效果 + + NOTE: + 1. 此种导入方式当前是在一台机器上完成导入工作,因而不宜进行数据量较大的导入工作。 + 建议导入数据量不要超过1GB + + 2. 当前无法使用`curl -T "{file1, file2}"`这样的方式提交多个文件,因为curl是将其拆成多个 + 请求发送的,多个请求不能共用一个label号,所以无法使用 + + 3. 支持类似streaming的方式使用curl来向 Doris 中导入数据,但是,只有等这个streaming结束后 Doris + 才会发生真实的导入行为,这中方式数据量也不能过大。 + +## example + + 1. 将本地文件'testData1'中的数据导入到数据库'testDb'中'testTbl1'的表,并且 + 把'testData2'的数据导入到'testDb'中的表'testTbl2'(用户是defalut_cluster中的) + curl --location-trusted -u root -XPOST http://host:port/api/testDb/_multi_start?label=123 + curl --location-trusted -u root -T testData1 http://host:port/api/testDb/testTbl1/_load?label=123\&sub_label=1 + curl --location-trusted -u root -T testData2 http://host:port/api/testDb/testTbl2/_load?label=123\&sub_label=2 + curl --location-trusted -u root -XPOST http://host:port/api/testDb/_multi_commit?label=123 + + 2. 多表导入中途放弃(用户是defalut_cluster中的) + curl --location-trusted -u root -XPOST http://host:port/api/testDb/_multi_start?label=123 + curl --location-trusted -u root -T testData1 http://host:port/api/testDb/testTbl1/_load?label=123\&sub_label=1 + curl --location-trusted -u root -XPOST http://host:port/api/testDb/_multi_abort?label=123 + + 3. 多表导入查看已经提交多少内容(用户是defalut_cluster中的) + curl --location-trusted -u root -XPOST http://host:port/api/testDb/_multi_start?label=123 + curl --location-trusted -u root -T testData1 http://host:port/api/testDb/testTbl1/_load?label=123\&sub_label=1 + curl --location-trusted -u root -XPOST http://host:port/api/testDb/_multi_desc?label=123 + +## keyword + MULTI, MINI, LOAD + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/PAUSE ROUTINE LOAD.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/PAUSE ROUTINE LOAD.md new file mode 100644 index 00000000000000..13a371d05291b3 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/PAUSE ROUTINE LOAD.md @@ -0,0 +1,10 @@ +# PAUSE ROUTINE LOAD +## example + +1. 暂停名称为 test1 的例行导入作业。 + + PAUSE ROUTINE LOAD FOR test1; + +## keyword + PAUSE,ROUTINE,LOAD + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/RESTORE TABLET.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/RESTORE TABLET.md new file mode 100644 index 00000000000000..1693bbc71ad6f8 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/RESTORE TABLET.md @@ -0,0 +1,15 @@ +# RESTORE TABLET +## description + + 该功能用于恢复trash目录中被误删的tablet数据。 + + 说明:这个功能暂时只在be服务中提供一个http接口。如果要使用, + 需要向要进行数据恢复的那台be机器的http端口发送restore tablet api请求。api格式如下: + METHOD: POST + URI: http://be_host:be_http_port/api/restore_tablet?tablet_id=xxx&schema_hash=xxx + +## example + + curl -X POST "http://hostname:8088/api/restore_tablet?tablet_id=123456&schema_hash=1111111" +##keyword +RESTORE,TABLET,RESTORE,TABLET diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/RESUME ROUTINE LOAD.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/RESUME ROUTINE LOAD.md new file mode 100644 index 00000000000000..2401b627ef6e10 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/RESUME ROUTINE LOAD.md @@ -0,0 +1,10 @@ +# RESUME ROUTINE LOAD +## example + +1. 恢复名称为 test1 的例行导入作业。 + + RESUME ROUTINE LOAD FOR test1; + +## keyword + RESUME,ROUTINE,LOAD + diff --git a/docs/help/Contents/Data Manipulation/routine_load.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/ROUTINE LOAD.md similarity index 62% rename from docs/help/Contents/Data Manipulation/routine_load.md rename to docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/ROUTINE LOAD.md index 9dcd7d687cc651..66cd410b1eb78d 100644 --- a/docs/help/Contents/Data Manipulation/routine_load.md +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/ROUTINE LOAD.md @@ -247,197 +247,3 @@ ## keyword CREATE,ROUTINE,LOAD -# PAUSE ROUTINE LOAD - - 该语句用于暂停一个指定的例行导入作业。 - -语法: - - PAUSE ROUTINE LOAD FOR [db.]name; - -## example - -1. 暂停名称为 test1 的例行导入作业。 - - PAUSE ROUTINE LOAD FOR test1; - -## keyword - PAUSE,ROUTINE,LOAD - -# RESUME ROUTINE LOAD - - 该语句用于恢复一个被暂停的例行导入作业。 - -语法: - - RESUME ROUTINE LOAD FOR [db.]name; - -## example - -1. 恢复名称为 test1 的例行导入作业。 - - RESUME ROUTINE LOAD FOR test1; - -## keyword - RESUME,ROUTINE,LOAD - -# STOP ROUTINE LOAD - - 该语句用于停止一个被暂停的例行导入作业。 - -语法: - - STOP ROUTINE LOAD FOR [db.]name; - - 被停止的作业无法再恢复运行。 - -## example - -1. 停止名称为 test1 的例行导入作业。 - - STOP ROUTINE LOAD FOR test1; - -## keyword - STOP,ROUTINE,LOAD - -# SHOW ROUTINE LOAD - - 该语句用于展示指定名称的例行导入作业的详细信息。 - -语法: - - SHOW [ALL] ROUTINE LOAD FOR [[db.]name] - -展示结果包括如下信息: - - 1. Id:作业id。 - 2. Name:作业的名称。 - 3. CreateTime:作业创建时间。 - 4. PauseTime:作业暂停时间。 - 5. EndTime:作业结束时间。 - 6. DdName:数据库名称。 - 7. TableName:目的表名称。 - 8. State:作业状态。 - - NEED_SCHEDULE:等待被调度。 - RUNNING:运行中。 - PAUSE:暂停中。 - STOPPED:作业由用户停止。 - CANCELLED:作业因失败停止。 - - 9. DataSourceType:数据源类型。 - - KAFKA - - 10. CurrentTaskNum:当前正在运行的子任务的个数 - 11. JobProperties:作业相关配置信息,对应创建语句中的 load_properties 和 job_properties。以 json 格式表示。 - - { - "partitions": "*", // 目的表的分区,星号表示没有指定。 - "columnToColumnExpr": "k1,yyy,v1,v2,v3,v4,v5,v6,k2=`k1` + 1", - "maxBatchIntervalS": "10", - "whereExpr": "`k1` > 100", - "maxBatchSizeBytes": "104857600", - "columnSeparator": "\t", - "maxErrorNum": "0", - "currentTaskConcurrentNum": "3", // 当前例行作业的子任务并发数 - "maxBatchRows": "200000" - } - - 12. Statistic:作业运行状态的统计信息。以 json 格式表示。 - - { - "errorRows": 0, // 总的错误行数 - "loadedRows": 6584959, // 总导入的行数 - "unselectedRows": 2392, // 被 where 条件过滤的行数 - "totalRows": 6587351, // 总消费的行数,totalRows = errorRows + loadedRows + unselectedRows - "loadRowsRate": 91000, // 导入速率(rows/s) - "receivedBytes": 861626324, // 总消费的字节数 - "receivedBytesRate": 11915000, // 消费速率 (Bytes/s) - "committedTaskNum": 33, // 提交成功的子任务数 - "abortedTaskNum": 2, // 失败的子任务数 - "taskExecuteTaskMs": 72312 // 子任务执行时间,单位毫秒 - } - - 13. Progress:作业进度。以 json 格式表示。 - - 如果数据源是 Kafka,则显示每个 kafka partition,当前已经被消费的 offset。 - - { - "0": 2199288, - "1": 2194329, - "2": 2193731 - } - - OFFSET_BEGINNING: 表示用户指定了从头开始消费,并且还未开始消费。 - OFFSET_END: 表示用户指定了从末尾开始消费,并且还未开始消费。 - OFFSET_ZERO: 表示用户指定了从 0 开始消费,并且还未开始消费。 - - 14. CustomProperties: 自定义参数。 - -## example - -1. 展示名称为 test1 的所有例行导入作业(包括已停止或取消的作业)。结果为一行或多行。 - - SHOW ALL ROUTINE LOAD FOR test1; - -2. 展示名称为 test1 的当前正在运行的例行导入作业 - - SHOW ROUTINE LOAD FOR test1; - -3. 显示 example_db 下,所有的例行导入作业(包括已停止或取消的作业)。结果为一行或多行。 - - use example_db; - SHOW ALL ROUTINE LOAD; - -4. 显示 example_db 下,所有正在运行的例行导入作业 - - use example_db; - SHOW ROUTINE LOAD; - -5. 显示 example_db 下,名称为 test1 的当前正在运行的例行导入作业 - - SHOW ROUTINE LOAD FOR example_db.test1; - -6. 显示 example_db 下,名称为 test1 的所有例行导入作业(包括已停止或取消的作业)。结果为一行或多行。 - - SHOW ALL ROUTINE LOAD FOR example_db.test1; - -## keyword - SHOW,ROUTINE,LOAD - -# SHOW ROUTINE LOAD TASK - - 该语句用于展示指定例行导入作业,当前正在运行的子任务信息。 - -语法: - - SHOW ROUTINE LOAD TASK [FROM db] WHERE JobName = "name"; - -展示结果包括如下信息: - - TaskId:task id。 - TxnId:task 对应的事务id。 - JobId:作业id。 - CreateTime:任务创建时间。 - ExecuteStartTime:任务开始执行的时间。 - BeId:任务所在的 Backend id。 - DataSourceProperties: - - 任务的参数,以 json 格式展示。 - 当数据源为 Kafka 时,显示如下: - - { - "2":2193732 - } - - 表示该任务准备消费的 kafka partition 和起始 offset。 - -## example - -1. 展示名为 test1 的例行导入任务的子任务信息。 - - SHOW ROUTINE LOAD TASK WHERE JobName = "test1"; - -# keyword - SHOW,ROUTINE,LOAD,TASK diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW ALTER.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW ALTER.md new file mode 100644 index 00000000000000..54d1d51fe619f7 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW ALTER.md @@ -0,0 +1,25 @@ +# SHOW ALTER +## description + 该语句用于展示当前正在进行的各类修改任务的执行情况 + 语法: + SHOW ALTER [CLUSTER | TABLE [COLUMN | ROLLUP] [FROM db_name]]; + + 说明: + TABLE COLUMN:展示修改列的 ALTER 任务 + TABLE ROLLUP:展示创建或删除 ROLLUP index 的任务 + 如果不指定 db_name,使用当前默认 db + CLUSTER: 展示集群操作相关任务情况(仅管理员使用!待实现...) + +## example + 1. 展示默认 db 的所有修改列的任务执行情况 + SHOW ALTER TABLE COLUMN; + + 2. 展示指定 db 的创建或删除 ROLLUP index 的任务执行情况 + SHOW ALTER TABLE ROLLUP FROM example_db; + + 3. 展示集群操作相关任务(仅管理员使用!待实现...) + SHOW ALTER CLUSTER; + +## keyword + SHOW,ALTER + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW BACKUP.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW BACKUP.md new file mode 100644 index 00000000000000..97ad8c193fc7ae --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW BACKUP.md @@ -0,0 +1,37 @@ +# SHOW BACKUP +## description + 该语句用于查看 BACKUP 任务 + 语法: + SHOW BACKUP [FROM db_name] + + 说明: + 1. Palo 中仅保存最近一次 BACKUP 任务。 + 2. 各列含义如下: + JobId: 唯一作业id + SnapshotName: 备份的名称 + DbName: 所属数据库 + State: 当前阶段 + PENDING: 提交作业后的初始状态 + SNAPSHOTING: 执行快照中 + UPLOAD_SNAPSHOT:快照完成,准备上传 + UPLOADING: 快照上传中 + SAVE_META: 将作业元信息保存为本地文件 + UPLOAD_INFO: 上传作业元信息 + FINISHED: 作业成功 + CANCELLED: 作业失败 + BackupObjs: 备份的表和分区 + CreateTime: 任务提交时间 + SnapshotFinishedTime: 快照完成时间 + UploadFinishedTime: 快照上传完成时间 + FinishedTime: 作业结束时间 + UnfinishedTasks: 在 SNAPSHOTING 和 UPLOADING 阶段会显示还未完成的子任务id + Status: 如果作业失败,显示失败信息 + Timeout: 作业超时时间,单位秒 + +## example + 1. 查看 example_db 下最后一次 BACKUP 任务。 + SHOW BACKUP FROM example_db; + +## keyword + SHOW, BACKUP + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW DATA.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW DATA.md new file mode 100644 index 00000000000000..961758677ed0cc --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW DATA.md @@ -0,0 +1,21 @@ +# SHOW DATA +## description + 该语句用于展示数据量 + 语法: + SHOW DATA [FROM db_name[.table_name]]; + + 说明: + 1. 如果不指定 FROM 子句,使用展示当前 db 下细分到各个 table 的数据量 + 2. 如果指定 FROM 子句,则展示 table 下细分到各个 index 的数据量 + 3. 如果想查看各个 Partition 的大小,请参阅 help show partitions + +## example + 1. 展示默认 db 的各个 table 的数据量及汇总数据量 + SHOW DATA; + + 2. 展示指定 db 的下指定表的细分数据量 + SHOW DATA FROM example_db.table_name; + +## keyword + SHOW,DATA + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW DATABASES.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW DATABASES.md new file mode 100644 index 00000000000000..bc46fd04ba83c2 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW DATABASES.md @@ -0,0 +1,9 @@ +# SHOW DATABASES +## description + 该语句用于展示当前可见的 db + 语法: + SHOW DATABASES; + +## keyword + SHOW,DATABASES + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW DELETE.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW DELETE.md new file mode 100644 index 00000000000000..89af796c75bc63 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW DELETE.md @@ -0,0 +1,13 @@ +# SHOW DELETE +## description + 该语句用于展示已执行成功的历史 delete 任务 + 语法: + SHOW DELETE [FROM db_name] + +## example + 1. 展示数据库 database 的所有历史 delete 任务 + SHOW DELETE FROM database; + +## keyword + SHOW,DELETE + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW EXPORT.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW EXPORT.md new file mode 100644 index 00000000000000..54de1c9b64fc06 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW EXPORT.md @@ -0,0 +1,36 @@ +# SHOW EXPORT +## description + 该语句用于展示指定的导出任务的执行情况 + 语法: + SHOW EXPORT + [FROM db_name] + [ + WHERE + [EXPORT_JOB_ID = your_job_id] + [STATE = ["PENDING"|"EXPORTING"|"FINISHED"|"CANCELLED"]] + ] + [ORDER BY ...] + [LIMIT limit]; + + 说明: + 1) 如果不指定 db_name,使用当前默认db + 2) 如果指定了 STATE,则匹配 EXPORT 状态 + 3) 可以使用 ORDER BY 对任意列组合进行排序 + 4) 如果指定了 LIMIT,则显示 limit 条匹配记录。否则全部显示 + +## example + 1. 展示默认 db 的所有导出任务 + SHOW EXPORT; + + 2. 展示指定 db 的导出任务,按 StartTime 降序排序 + SHOW EXPORT FROM example_db ORDER BY StartTime DESC; + + 3. 展示指定 db 的导出任务,state 为 "exporting", 并按 StartTime 降序排序 + SHOW EXPORT FROM example_db WHERE STATE = "exporting" ORDER BY StartTime DESC; + + 4. 展示指定db,指定job_id的导出任务 + SHOW EXPORT FROM example_db WHERE EXPORT_JOB_ID = job_id; + +## keyword + SHOW,EXPORT + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW LOAD.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW LOAD.md new file mode 100644 index 00000000000000..c4b56bd2c0340b --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW LOAD.md @@ -0,0 +1,49 @@ +# SHOW LOAD +## description + 该语句用于展示指定的导入任务的执行情况 + 语法: + SHOW LOAD + [FROM db_name] + [ + WHERE + [LABEL [ = "your_label" | LIKE "label_matcher"]] + [STATE = ["PENDING"|"ETL"|"LOADING"|"FINISHED"|"CANCELLED"|]] + ] + [ORDER BY ...] + [LIMIT limit][OFFSET offset]; + + 说明: + 1) 如果不指定 db_name,使用当前默认db + 2) 如果使用 LABEL LIKE,则会匹配导入任务的 label 包含 label_matcher 的导入任务 + 3) 如果使用 LABEL = ,则精确匹配指定的 label + 4) 如果指定了 STATE,则匹配 LOAD 状态 + 5) 可以使用 ORDER BY 对任意列组合进行排序 + 6) 如果指定了 LIMIT,则显示 limit 条匹配记录。否则全部显示 + 7) 如果指定了 OFFSET,则从偏移量offset开始显示查询结果。默认情况下偏移量为0。 + 8) 如果是使用 broker/mini load,则 URL 列中的连接可以使用以下命令查看: + + SHOW LOAD WARNINGS ON 'url' + +## example + 1. 展示默认 db 的所有导入任务 + SHOW LOAD; + + 2. 展示指定 db 的导入任务,label 中包含字符串 "2014_01_02",展示最老的10个 + SHOW LOAD FROM example_db WHERE LABEL LIKE "2014_01_02" LIMIT 10; + + 3. 展示指定 db 的导入任务,指定 label 为 "load_example_db_20140102" 并按 LoadStartTime 降序排序 + SHOW LOAD FROM example_db WHERE LABEL = "load_example_db_20140102" ORDER BY LoadStartTime DESC; + + 4. 展示指定 db 的导入任务,指定 label 为 "load_example_db_20140102" ,state 为 "loading", 并按 LoadStartTime 降序排序 + SHOW LOAD FROM example_db WHERE LABEL = "load_example_db_20140102" AND STATE = "loading" ORDER BY LoadStartTime DESC; + + 5. 展示指定 db 的导入任务 并按 LoadStartTime 降序排序,并从偏移量5开始显示10条查询结果 + SHOW LOAD FROM example_db ORDER BY LoadStartTime DESC limit 5,10; + SHOW LOAD FROM example_db ORDER BY LoadStartTime DESC limit 10 offset 5; + + 6. 小批量导入是查看导入状态的命令 + curl --location-trusted -u {user}:{passwd} http://{hostname}:{port}/api/{database}/_load_info?label={labelname} + +## keyword + SHOW,LOAD + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW PARTITIONS.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW PARTITIONS.md new file mode 100644 index 00000000000000..44e959b55bbed5 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW PARTITIONS.md @@ -0,0 +1,16 @@ +# SHOW PARTITIONS +## description + 该语句用于展示分区信息 + 语法: + SHOW PARTITIONS FROM [db_name.]table_name [PARTITION partition_name]; + +## example + 1. 展示指定 db 的下指定表的分区信息 + SHOW PARTITIONS FROM example_db.table_name; + + 1. 展示指定 db 的下指定表的指定分区的信息 + SHOW PARTITIONS FROM example_db.table_name PARTITION p1; + +## keyword + SHOW,PARTITIONS + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW PROPERTY.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW PROPERTY.md new file mode 100644 index 00000000000000..17ca6eea5bd1e4 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW PROPERTY.md @@ -0,0 +1,16 @@ +# SHOW PROPERTY +## description + 该语句用于查看用户的属性 + 语法: + SHOW PROPERTY [FOR user] [LIKE key] + +## example + 1. 查看 jack 用户的属性 + SHOW PROPERTY FOR 'jack' + + 2. 查看 jack 用户导入cluster相关属性 + SHOW PROPERTY FOR 'jack' LIKE '%load_cluster%' + +## keyword + SHOW, PROPERTY + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW REPOSITORIES.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW REPOSITORIES.md new file mode 100644 index 00000000000000..9fb7f159938d25 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW REPOSITORIES.md @@ -0,0 +1,23 @@ +# SHOW REPOSITORIES +## description + 该语句用于查看当前已创建的仓库。 + 语法: + SHOW REPOSITORIES; + + 说明: + 1. 各列含义如下: + RepoId: 唯一的仓库ID + RepoName: 仓库名称 + CreateTime: 第一次创建该仓库的时间 + IsReadOnly: 是否为只读仓库 + Location: 仓库中用于备份数据的根目录 + Broker: 依赖的 Broker + ErrMsg: Palo 会定期检查仓库的连通性,如果出现问题,这里会显示错误信息 + +## example + 1. 查看已创建的仓库: + SHOW REPOSITORIES; + +## keyword + SHOW, REPOSITORY, REPOSITORIES + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW RESTORE.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW RESTORE.md new file mode 100644 index 00000000000000..1139a627977e45 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW RESTORE.md @@ -0,0 +1,41 @@ +# SHOW RESTORE +## description + 该语句用于查看 RESTORE 任务 + 语法: + SHOW RESTORE [FROM db_name] + + 说明: + 1. Palo 中仅保存最近一次 RESTORE 任务。 + 2. 各列含义如下: + JobId: 唯一作业id + Label: 要恢复的备份的名称 + Timestamp: 要恢复的备份的时间版本 + DbName: 所属数据库 + State: 当前阶段 + PENDING: 提交作业后的初始状态 + SNAPSHOTING: 执行快照中 + DOWNLOAD: 快照完成,准备下载仓库中的快照 + DOWNLOADING: 快照下载中 + COMMIT: 快照下载完成,准备生效 + COMMITING: 生效中 + FINISHED: 作业成功 + CANCELLED: 作业失败 + AllowLoad: 恢复时是否允许导入(当前不支持) + ReplicationNum: 指定恢复的副本数 + RestoreJobs: 要恢复的表和分区 + CreateTime: 任务提交时间 + MetaPreparedTime: 元数据准备完成时间 + SnapshotFinishedTime: 快照完成时间 + DownloadFinishedTime: 快照下载完成时间 + FinishedTime: 作业结束时间 + UnfinishedTasks: 在 SNAPSHOTING、DOWNLOADING 和 COMMITING 阶段会显示还未完成的子任务id + Status: 如果作业失败,显示失败信息 + Timeout: 作业超时时间,单位秒 + +## example + 1. 查看 example_db 下最近一次 RESTORE 任务。 + SHOW RESTORE FROM example_db; + +## keyword + SHOW, RESTORE + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW ROUTINE LOAD TASK.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW ROUTINE LOAD TASK.md new file mode 100644 index 00000000000000..d01e3bb3c5b1af --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW ROUTINE LOAD TASK.md @@ -0,0 +1,9 @@ +# SHOW ROUTINE LOAD TASK +## example + +1. 展示名为 test1 的例行导入任务的子任务信息。 + + SHOW ROUTINE LOAD TASK WHERE JobName = "test1"; + +## keyword + SHOW,ROUTINE,LOAD,TASK diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW ROUTINE LOAD.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW ROUTINE LOAD.md new file mode 100644 index 00000000000000..f453e6c2ca5b96 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW ROUTINE LOAD.md @@ -0,0 +1,32 @@ +# SHOW ROUTINE LOAD +## example + +1. 展示名称为 test1 的所有例行导入作业(包括已停止或取消的作业)。结果为一行或多行。 + + SHOW ALL ROUTINE LOAD FOR test1; + +2. 展示名称为 test1 的当前正在运行的例行导入作业 + + SHOW ROUTINE LOAD FOR test1; + +3. 显示 example_db 下,所有的例行导入作业(包括已停止或取消的作业)。结果为一行或多行。 + + use example_db; + SHOW ALL ROUTINE LOAD; + +4. 显示 example_db 下,所有正在运行的例行导入作业 + + use example_db; + SHOW ROUTINE LOAD; + +5. 显示 example_db 下,名称为 test1 的当前正在运行的例行导入作业 + + SHOW ROUTINE LOAD FOR example_db.test1; + +6. 显示 example_db 下,名称为 test1 的所有例行导入作业(包括已停止或取消的作业)。结果为一行或多行。 + + SHOW ALL ROUTINE LOAD FOR example_db.test1; + +## keyword + SHOW,ROUTINE,LOAD + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW SNAPSHOT.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW SNAPSHOT.md new file mode 100644 index 00000000000000..6758e865c32839 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW SNAPSHOT.md @@ -0,0 +1,31 @@ +# SHOW SNAPSHOT +## description + 该语句用于查看仓库中已存在的备份。 + 语法: + SHOW SNAPSHOT ON `repo_name` + [WHERE SNAPSHOT = "snapshot" [AND TIMESTAMP = "backup_timestamp"]]; + + 说明: + 1. 各列含义如下: + Snapshot: 备份的名称 + Timestamp: 对应备份的时间版本 + Status: 如果备份正常,则显示 OK,否则显示错误信息 + + 2. 如果指定了 TIMESTAMP,则会额外显示如下信息: + Database: 备份数据原属的数据库名称 + Details: 以 Json 的形式,展示整个备份的数据目录及文件结构 + +## example + 1. 查看仓库 example_repo 中已有的备份: + SHOW SNAPSHOT ON example_repo; + + 2. 仅查看仓库 example_repo 中名称为 backup1 的备份: + SHOW SNAPSHOT ON example_repo WHERE SNAPSHOT = "backup1"; + + 2. 查看仓库 example_repo 中名称为 backup1 的备份,时间版本为 "2018-05-05-15-34-26" 的详细信息: + SHOW SNAPSHOT ON example_repo + WHERE SNAPSHOT = "backup1" AND TIMESTAMP = "2018-05-05-15-34-26"; + +## keyword + SHOW, SNAPSHOT + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW TABLES.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW TABLES.md new file mode 100644 index 00000000000000..b2670efd724338 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW TABLES.md @@ -0,0 +1,9 @@ +# SHOW TABLES +## description + 该语句用于展示当前 db 下所有的 table + 语法: + SHOW TABLES; + +## keyword + SHOW,TABLES + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW TABLET.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW TABLET.md new file mode 100644 index 00000000000000..29452c5e34d901 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/SHOW TABLET.md @@ -0,0 +1,17 @@ +# SHOW TABLET +## description + 该语句用于显示 tablet 相关的信息(仅管理员使用) + 语法: + SHOW TABLET + [FROM [db_name.]table_name | tablet_id] + +## example + 1. 显示指定 db 的下指定表所有 tablet 信息 + SHOW TABLET FROM example_db.table_name; + + 2. 显示指定 tablet id 为 10000 的 tablet 的父层级 id 信息 + SHOW TABLET 10000; + +## keyword + SHOW,TABLET + diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/STOP ROUTINE LOAD.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/STOP ROUTINE LOAD.md new file mode 100644 index 00000000000000..3e7cd6f7ca0f17 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/STOP ROUTINE LOAD.md @@ -0,0 +1,10 @@ +# STOP ROUTINE LOAD +## example + +1. 停止名称为 test1 的例行导入作业。 + + STOP ROUTINE LOAD FOR test1; + +## keyword + STOP,ROUTINE,LOAD + diff --git a/docs/help/Contents/Data Manipulation/streaming.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/STREAM LOAD.md similarity index 78% rename from docs/help/Contents/Data Manipulation/streaming.md rename to docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/STREAM LOAD.md index 3a279e2d0c94ac..e8298dae3c063f 100644 --- a/docs/help/Contents/Data Manipulation/streaming.md +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/STREAM LOAD.md @@ -90,69 +90,3 @@ ## keyword STREAM,LOAD -# GET LABEL STATE -## description - NAME: - get_label_state: get label's state - - SYNOPSIS - curl -u user:passwd http://host:port/api/{db}/{label}/_state - - DESCRIPTION - 该命令用于查看一个Label对应的事务状态 - - RETURN VALUES - 执行完毕后,会以Json格式返回这次导入的相关内容。当前包括一下字段 - Label:本次导入的 label,如果没有指定,则为一个 uuid。 - Status:此命令是否成功执行,Success表示成功执行 - Message: 具体的执行信息 - State: 只有在Status为Success时才有意义 - UNKNOWN: 没有找到对应的Label - PREPARE: 对应的事务已经prepare,但尚未提交 - COMMITTED: 事务已经提交,不能被cancel - VISIBLE: 事务提交,并且数据可见,不能被cancel - ABORTED: 事务已经被ROLLBACK,导入已经失败。 - - ERRORS - -## example - - 1. 获得testDb, testLabel的状态 - curl -u root http://host:port/api/testDb/testLabel/_state - -## keyword - GET, LABEL, STATE - -# CANCEL LABEL -## description - NAME: - cancel_label: cancel a transaction with label - - SYNOPSIS - curl -u user:passwd -XPOST http://host:port/api/{db}/{label}/_cancel - - DESCRIPTION - 该命令用于cancel一个指定Label对应的事务,事务在Prepare阶段能够被成功cancel - - RETURN VALUES - 执行完成后,会以Json格式返回这次导入的相关内容。当前包括一下字段 - Status: 是否成功cancel - Success: 成功cancel事务 - 其他: cancel失败 - Message: 具体的失败信息 - - ERRORS - -## example - - 1. cancel testDb, testLabel的作业 - curl -u root -XPOST http://host:port/api/testDb/testLabel/_cancel - -## keyword - CANCEL,LABEL - - - - - - diff --git a/docs/help/Contents/Data Manipulation/broker_load.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/broker_load.md similarity index 99% rename from docs/help/Contents/Data Manipulation/broker_load.md rename to docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/broker_load.md index 3f83280c16da4d..b6b445a1b3ceeb 100644 --- a/docs/help/Contents/Data Manipulation/broker_load.md +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/broker_load.md @@ -363,4 +363,4 @@ WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); ## keyword - BROKER LOAD + BROKER,LOAD diff --git a/docs/documentation/cn/sql-reference/sql-statements/insert.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/insert.md similarity index 97% rename from docs/documentation/cn/sql-reference/sql-statements/insert.md rename to docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/insert.md index 6f4cf4b24d4f20..14e3cfe9f7751c 100644 --- a/docs/documentation/cn/sql-reference/sql-statements/insert.md +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/insert.md @@ -1,6 +1,6 @@ # INSERT - -## Syntax +## description +### Syntax ``` INSERT INTO table_name @@ -10,7 +10,7 @@ INSERT INTO table_name { VALUES ( { expression | DEFAULT } [, ...] ) [, ...] | query } ``` -## Parameters +### Parameters > tablet_name: 导入数据的目的表。可以是 `db_name.table_name` 形式 > @@ -27,11 +27,11 @@ INSERT INTO table_name > hint: 用于指示 `INSERT` 执行行为的一些指示符。`streaming` 和 默认的非 `streaming` 方式均会使用同步方式完成 `INSERT` 语句执行 > 非 `streaming` 方式在执行完成后会返回一个 label 方便用户通过 `SHOW LOAD` 查询导入的状态 -## Note +### Note 当前执行 `INSERT` 语句时,对于有不符合目标表格式的数据,默认的行为是过滤,比如字符串超长等。但是对于有要求数据不能够被过滤的业务场景,可以通过设置会话变量 `enable_insert_strict` 为 `true` 来确保当有数据被过滤掉的时候,`INSERT` 不会被执行成功。 -## Examples +## example `test` 表包含两个列`c1`, `c2`。 @@ -76,3 +76,5 @@ INSERT INTO test (c1, c2) SELECT * from test2 异步的导入其实是,一个同步的导入封装成了异步。填写 streaming 和不填写的*执行效率是一样*的。 由于Doris之前的导入方式都是异步导入方式,为了兼容旧有的使用习惯,不加 streaming 的 `INSERT` 语句依旧会返回一个 label,用户需要通过`SHOW LOAD`命令查看此`label`导入作业的状态。 +##keyword +INSERT diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Types/BIGINT.md b/docs/documentation/cn/sql-reference/sql-statements/Data Types/BIGINT.md new file mode 100644 index 00000000000000..af2e6bb8296e3b --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Types/BIGINT.md @@ -0,0 +1,7 @@ +# BIGINT +## description + BIGINT + 8字节有符号整数,范围[-9223372036854775808, 9223372036854775807] + +##keyword +BIGINT diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Types/BOOLEAN.md b/docs/documentation/cn/sql-reference/sql-statements/Data Types/BOOLEAN.md new file mode 100644 index 00000000000000..caa44dc17fa96e --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Types/BOOLEAN.md @@ -0,0 +1,7 @@ +# BOOLEAN +## description + BOOL, BOOLEN + 与TINYINT一样,0代表false,1代表true + +##keyword +BOOLEAN diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Types/CHAR.md b/docs/documentation/cn/sql-reference/sql-statements/Data Types/CHAR.md new file mode 100644 index 00000000000000..df645ae99eec48 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Types/CHAR.md @@ -0,0 +1,7 @@ +# CHAR +## description + CHAR(M) + 定长字符串,M代表的是定长字符串的长度。M的范围是1-255 + +##keyword +CHAR diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Types/DATE.md b/docs/documentation/cn/sql-reference/sql-statements/Data Types/DATE.md new file mode 100644 index 00000000000000..d0082d7f674d07 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Types/DATE.md @@ -0,0 +1,15 @@ +# DATE +## description + DATE函数 + Syntax: + DATE(expr) + 将输入的类型转化为DATE类型 + DATE类型 + 日期类型,目前的取值范围是['1900-01-01', '9999-12-31'], 默认的打印形式是'YYYY-MM-DD' + +## example + mysql> SELECT DATE('2003-12-31 01:02:03'); + -> '2003-12-31' + +##keyword +DATE diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Types/DATETIME.md b/docs/documentation/cn/sql-reference/sql-statements/Data Types/DATETIME.md new file mode 100644 index 00000000000000..858e838c633d10 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Types/DATETIME.md @@ -0,0 +1,8 @@ +# DATETIME +## description + DATETIME + 日期时间类型,取值范围是['1000-01-01 00:00:00', '9999-12-31 23:59:59']. + 打印的形式是'YYYY-MM-DD HH:MM:SS' + +##keyword +DATETIME diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Types/DECIMAL.md b/docs/documentation/cn/sql-reference/sql-statements/Data Types/DECIMAL.md new file mode 100644 index 00000000000000..26e91fe0c42bc0 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Types/DECIMAL.md @@ -0,0 +1,8 @@ +# DECIMAL +## description + DECIMAL(M[,D]) + 高精度定点数,M代表一共有多少个有效数字(precision),D代表小数点后最多有多少数字(scale) + M的范围是[1,27], D的范围[1, 9], 另外,M必须要大于等于D的取值。默认的D取值为0 + +##keyword +DECIMAL diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Types/DOUBLE.md b/docs/documentation/cn/sql-reference/sql-statements/Data Types/DOUBLE.md new file mode 100644 index 00000000000000..6334fe5223b8ae --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Types/DOUBLE.md @@ -0,0 +1,7 @@ +# DOUBLE +## description + DOUBLE + 8字节浮点数 + +##keyword +DOUBLE diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Types/FLOAT.md b/docs/documentation/cn/sql-reference/sql-statements/Data Types/FLOAT.md new file mode 100644 index 00000000000000..fe72e134a44697 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Types/FLOAT.md @@ -0,0 +1,7 @@ +# FLOAT +## description + FLOAT + 4字节浮点数 + +##keyword +FLOAT diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Types/HLL(HyperLogLog).md b/docs/documentation/cn/sql-reference/sql-statements/Data Types/HLL(HyperLogLog).md new file mode 100644 index 00000000000000..0d6ded08d88b41 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Types/HLL(HyperLogLog).md @@ -0,0 +1,9 @@ +# HLL(HyperLogLog) +## description + VARCHAR(M) + 变长字符串,M代表的是变长字符串的长度。M的范围是1-16385 + 用户不需要指定长度和默认值。长度根据数据的聚合程度系统内控制 + 并且HLL列只能通过配套的hll_union_agg、hll_raw_agg、hll_cardinality、hll_hash进行查询或使用 + +##keyword +HLL(HYPERLOGLOG),HLL,HYPERLOGLOG diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Types/INT.md b/docs/documentation/cn/sql-reference/sql-statements/Data Types/INT.md new file mode 100644 index 00000000000000..3632659b9158f4 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Types/INT.md @@ -0,0 +1,7 @@ +# INT +## description + INT + 4字节有符号整数,范围[-2147483648, 2147483647] + +##keyword +INT diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Types/SMALLINT.md b/docs/documentation/cn/sql-reference/sql-statements/Data Types/SMALLINT.md new file mode 100644 index 00000000000000..6fc2e83235af36 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Types/SMALLINT.md @@ -0,0 +1,7 @@ +# SMALLINT +## description + SMALLINT + 2字节有符号整数,范围[-32768, 32767] + +##keyword +SMALLINT diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Types/TINYINT.md b/docs/documentation/cn/sql-reference/sql-statements/Data Types/TINYINT.md new file mode 100644 index 00000000000000..c9ba5ed88d734c --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Types/TINYINT.md @@ -0,0 +1,7 @@ +# TINYINT +## description + TINYINT + 1字节有符号整数,范围[-128, 127] + +##keyword +TINYINT diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Types/VARCHAR.md b/docs/documentation/cn/sql-reference/sql-statements/Data Types/VARCHAR.md new file mode 100644 index 00000000000000..dc3dc5adce2e6a --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Types/VARCHAR.md @@ -0,0 +1,7 @@ +# VARCHAR +## description + VARCHAR(M) + 变长字符串,M代表的是变长字符串的长度。M的范围是1-65535 + +##keyword +VARCHAR diff --git a/docs/help/Contents/Utility/util_stmt.md b/docs/documentation/cn/sql-reference/sql-statements/Utility/util_stmt.md similarity index 81% rename from docs/help/Contents/Utility/util_stmt.md rename to docs/documentation/cn/sql-reference/sql-statements/Utility/util_stmt.md index 2fbafe8295f9b0..146a4b68974806 100644 --- a/docs/help/Contents/Utility/util_stmt.md +++ b/docs/documentation/cn/sql-reference/sql-statements/Utility/util_stmt.md @@ -1,13 +1,13 @@ -# DESCRIBE -## description - 该语句用于展示指定 table 的 schema 信息 - 语法: - DESC[RIBE] [db_name.]table_name [ALL]; - - 说明: - 如果指定 ALL,则显示该 table 的所有 index 的 schema - -## example - -## keyword - DESCRIBE,DESC \ No newline at end of file +# DESCRIBE +## description + 该语句用于展示指定 table 的 schema 信息 + 语法: + DESC[RIBE] [db_name.]table_name [ALL]; + + 说明: + 如果指定 ALL,则显示该 table 的所有 index 的 schema + +## example + +## keyword + DESCRIBE,DESC diff --git a/docs/help/Contents/Account Management/help.md b/docs/help/Contents/Account Management/help.md deleted file mode 100644 index c3b81a6ee40d6f..00000000000000 --- a/docs/help/Contents/Account Management/help.md +++ /dev/null @@ -1,335 +0,0 @@ -# CREATE USER -## description - -Syntax: - - CREATE USER user_identity [IDENTIFIED BY 'password'] [DEFAULT ROLE 'role_name'] - - user_identity: - 'user_name'@'host' - -CREATE USER 命令用于创建一个 Doris 用户。在 Doris 中,一个 user_identity 唯一标识一个用户。user_identity 由两部分组成,user_name 和 host,其中 username 为用户名。host 标识用户端连接所在的主机地址。host 部分可以使用 % 进行模糊匹配。如果不指定 host,默认为 '%',即表示该用户可以从任意 host 连接到 Doris。 - -host 部分也可指定为 domain,语法为:'user_name'@['domain'],即使用中括号包围,则 Doris 会认为这个是一个 domain,并尝试解析其 ip 地址。目前仅支持百度内部的 BNS 解析。 - -如果指定了角色(ROLE),则会自动将该角色所拥有的权限赋予新创建的这个用户。如果不指定,则该用户默认没有任何权限。指定的 ROLE 必须已经存在。 - -## example - -1. 创建一个无密码用户(不指定 host,则等价于 jack@'%') - - CREATE USER 'jack'; - -2. 创建一个有密码用户,允许从 '172.10.1.10' 登陆 - - CREATE USER jack@'172.10.1.10' IDENTIFIED BY '123456'; - -3. 为了避免传递明文,用例2也可以使用下面的方式来创建 - - CREATE USER jack@'172.10.1.10' IDENTIFIED BY PASSWORD '*6BB4837EB74329105EE4568DDA7DC67ED2CA2AD9'; - - 后面加密的内容可以通过PASSWORD()获得到,例如: - - SELECT PASSWORD('123456'); - -4. 创建一个允许从 '192.168' 子网登陆的用户,同时指定其角色为 example_role - - CREATE USER 'jack'@'192.168.%' DEFAULT ROLE 'example_role'; - -5. 创建一个允许从域名 'example_domain' 登陆的用户 - - CREATE USER 'jack'@['example_domain'] IDENTIFIED BY '12345'; - -6. 创建一个用户,并指定一个角色 - - CREATE USER 'jack'@'%' IDENTIFIED BY '12345' DEFAULT ROLE 'my_role'; - -## keyword - CREATE, USER - -# DROP USER -## description - -Syntax: - - DROP USER 'user_name' - - DROP USER 命令会删除一个 palo 用户。这里 Doris 不支持删除指定的 user_identity。当删除一个指定用户后,该用户所对应的所有 user_identity 都会被删除。比如之前通过 CREATE USER 语句创建了 jack@'192.%' 以及 jack@['domain'] 两个用户,则在执行 DROP USER 'jack' 后,jack@'192.%' 以及 jack@['domain'] 都将被删除。 - -## example - -1. 删除用户 jack - - DROP USER 'jack' - -## keyword - DROP, USER - -# SET PASSWORD -## description - -Syntax: - - SET PASSWORD [FOR user_identity] = - [PASSWORD('plain password')]|['hashed password'] - - SET PASSWORD 命令可以用于修改一个用户的登录密码。如果 [FOR user_identity] 字段不存在,那么修改当前用户的密码。 - - 注意这里的 user_identity 必须完全匹配在使用 CREATE USER 创建用户时指定的 user_identity,否则会报错用户不存在。如果不指定 user_identity,则当前用户为 'username'@'ip',这个当前用户,可能无法匹配任何 user_identity。可以通过 SHOW GRANTS 查看当前用户。 - - PASSWORD() 方式输入的是明文密码; 而直接使用字符串,需要传递的是已加密的密码。 - 如果修改其他用户的密码,需要具有管理员权限。 - -## example - -1. 修改当前用户的密码 - - SET PASSWORD = PASSWORD('123456') - SET PASSWORD = '*6BB4837EB74329105EE4568DDA7DC67ED2CA2AD9' - -2. 修改指定用户密码 - - SET PASSWORD FOR 'jack'@'192.%' = PASSWORD('123456') - SET PASSWORD FOR 'jack'@['domain'] = '*6BB4837EB74329105EE4568DDA7DC67ED2CA2AD9' - -## keyword - SET, PASSWORD - -# GRANT -## description - -GRANT 命令用于赋予指定用户或角色指定的权限。 - -Syntax: - - GRANT privilege_list ON db_name[.tbl_name] TO user_identity [ROLE role_name] - - -privilege_list 是需要赋予的权限列表,以逗号分隔。当前 Doris 支持如下权限: - - NODE_PRIV:集群节点操作权限,包括节点上下线等操作,只有 root 用户有该权限,不可赋予其他用户。 - ADMIN_PRIV:除 NODE_PRIV 以外的所有权限。 - GRANT_PRIV: 操作权限的权限。包括创建删除用户、角色,授权和撤权,设置密码等。 - SELECT_PRIV:对指定的库或表的读取权限 - LOAD_PRIV:对指定的库或表的导入权限 - ALTER_PRIV:对指定的库或表的schema变更权限 - CREATE_PRIV:对指定的库或表的创建权限 - DROP_PRIV:对指定的库或表的删除权限 - - 旧版权限中的 ALL 和 READ_WRITE 会被转换成:SELECT_PRIV,LOAD_PRIV,ALTER_PRIV,CREATE_PRIV,DROP_PRIV; - READ_ONLY 会被转换为 SELECT_PRIV。 - -db_name[.tbl_name] 支持以下三种形式: - - 1. *.* 权限可以应用于所有库及其中所有表 - 2. db.* 权限可以应用于指定库下的所有表 - 3. db.tbl 权限可以应用于指定库下的指定表 - - 这里指定的库或表可以是不存在的库和表。 - -user_identity: - - 这里的 user_identity 语法同 CREATE USER。且必须为使用 CREATE USER 创建过的 user_identity。user_identity 中的host可以是域名,如果是域名的话,权限的生效时间可能会有1分钟左右的延迟。 - - 也可以将权限赋予指定的 ROLE,如果指定的 ROLE 不存在,则会自动创建。 - -## example - - 1. 授予所有库和表的权限给用户 - - GRANT SELECT_PRIV ON *.* TO 'jack'@'%'; - - 2. 授予指定库表的权限给用户 - - GRANT SELECT_PRIV,ALTER_PRIVS,LOAD_PRIV ON db1.tbl1 TO 'jack'@'192.8.%'; - - 3. 授予指定库表的权限给角色 - - GRANT LOAD_PRIV ON db1.* TO ROLE 'my_role'; - -## keyword - GRANT - -# REVOKE -## description - - REVOKE 命令用于撤销指定用户或角色指定的权限。 - Syntax: - REVOKE privilege_list ON db_name[.tbl_name] FROM user_identity [ROLE role_name] - - user_identity: - - 这里的 user_identity 语法同 CREATE USER。且必须为使用 CREATE USER 创建过的 user_identity。user_identity 中的host可以是域名,如果是域名的话,权限的撤销时间可能会有1分钟左右的延迟。 - - 也可以撤销指定的 ROLE 的权限,执行的 ROLE 必须存在。 - -## example - - 1. 撤销用户 jack 数据库 testDb 的权限 - - REVOKE SELECT_PRIV ON db1.* FROM 'jack'@'192.%'; - -## keyword - - REVOKE - -# SET PROPERTY -## description - - Syntax: - - SET PROPERTY [FOR 'user'] 'key' = 'value' [, 'key' = 'value'] - - 设置用户的属性,包括分配给用户的资源、导入cluster等。这里设置的用户属性,是针对 user 的,而不是 user_identity。即假设通过 CREATE USER 语句创建了两个用户 'jack'@'%' 和 'jack'@'192.%',则使用 SET PROPERTY 语句,只能针对 jack 这个用户,而不是 'jack'@'%' 或 'jack'@'192.%' - - 导入 cluster 仅适用于百度内部用户。 - - key: - - 超级用户权限: - max_user_connections: 最大连接数。 - resource.cpu_share: cpu资源分配。 - load_cluster.{cluster_name}.priority: 为指定的cluster分配优先级,可以为 HIGH 或 NORMAL - - 普通用户权限: - quota.normal: normal级别的资源分配。 - quota.high: high级别的资源分配。 - quota.low: low级别的资源分配。 - - load_cluster.{cluster_name}.hadoop_palo_path: palo使用的hadoop目录,需要存放etl程序及etl生成的中间数据供palo导入。导入完成后会自动清理中间数据,etl程序自动保留下次使用。 - load_cluster.{cluster_name}.hadoop_configs: hadoop的配置,其中fs.default.name、mapred.job.tracker、hadoop.job.ugi必须填写。 - load_cluster.{cluster_name}.hadoop_http_port: hadoop hdfs name node http端口。其中 hdfs 默认为8070,afs 默认 8010。 - default_load_cluster: 默认的导入cluster。 - -## example - - 1. 修改用户 jack 最大连接数为1000 - SET PROPERTY FOR 'jack' 'max_user_connections' = '1000'; - - 2. 修改用户 jack 的cpu_share为1000 - SET PROPERTY FOR 'jack' 'resource.cpu_share' = '1000'; - - 3. 修改 jack 用户的normal组的权重 - SET PROPERTY FOR 'jack' 'quota.normal' = '400'; - - 4. 为用户 jack 添加导入cluster - SET PROPERTY FOR 'jack' - 'load_cluster.{cluster_name}.hadoop_palo_path' = '/user/palo/palo_path', - 'load_cluster.{cluster_name}.hadoop_configs' = 'fs.default.name=hdfs://dpp.cluster.com:port;mapred.job.tracker=dpp.cluster.com:port;hadoop.job.ugi=user,password;mapred.job.queue.name=job_queue_name_in_hadoop;mapred.job.priority=HIGH;'; - - 5. 删除用户 jack 下的导入cluster。 - SET PROPERTY FOR 'jack' 'load_cluster.{cluster_name}' = ''; - - 6. 修改用户 jack 默认的导入cluster - SET PROPERTY FOR 'jack' 'default_load_cluster' = '{cluster_name}'; - - 7. 修改用户 jack 的集群优先级为 HIGH - SET PROPERTY FOR 'jack' 'load_cluster.{cluster_name}.priority' = 'HIGH'; - -## keyword - SET, PROPERTY - -# CREATE ROLE - -## description - 该语句用户创建一个角色 - - 语法: - CREATE ROLE role1; - - 该语句创建一个无权限的角色,可以后续通过 GRANT 命令赋予该角色权限。 - -## example - - 1. 创建一个角色 - - CREATE ROLE role1; - -## keyword - CREATE, ROLE - - -# DROP ROLE - -## description - 该语句用户删除一个角色 - - 语法: - DROP ROLE role1; - - 删除一个角色,不会影响之前属于该角色的用户的权限。仅相当于将该角色与用户解耦。用户已经从该角色中获取到的权限,不会改变。 - -## example - - 1. 删除一个角色 - - DROP ROLE role1; - -## keyword - DROP, ROLE - -# SHOW ROLES - -## description - 该语句用于展示所有已创建的角色信息,包括角色名称,包含的用户以及权限。 - - 语法: - SHOW ROLES; - -## example - - 1. 查看已创建的角色: - - SHOW ROLES; - -## keyword - SHOW,ROLES - -## description - 该语句用户删除一个角色 - - 语法: - DROP ROLE role1; - - 删除一个角色,不会影响之前属于该角色的用户的权限。仅相当于将该角色与用户解耦。用户已经从该角色中获取到的权限,不会改变。 - -## example - - 1. 删除一个角色 - - DROP ROLE role1; - -## keyword - DROP, ROLE - -# SHOW GRANTS - -## description - - 该语句用于查看用户权限。 - - 语法: - SHOW [ALL] GRANTS [FOR user_identity]; - - 说明: - 1. SHOW ALL GRANTS 可以查看所有用户的权限。 - 2. 如果指定 user_identity,则查看该指定用户的权限。且该 user_identity 必须为通过 CREATE USER 命令创建的。 - 3. 如果不指定 user_identity,则查看当前用户的权限。 - - -## example - - 1. 查看所有用户权限信息 - - SHOW ALL GRANTS; - - 2. 查看指定 user 的权限 - - SHOW GRANTS FOR jack@'%'; - - 3. 查看当前用户的权限 - - SHOW GRANTS; - -## keyword - SHOW, GRANTS diff --git a/docs/help/Contents/Administration/admin_show_stmt.md b/docs/help/Contents/Administration/admin_show_stmt.md deleted file mode 100644 index 1ee64b1e5b6808..00000000000000 --- a/docs/help/Contents/Administration/admin_show_stmt.md +++ /dev/null @@ -1,92 +0,0 @@ -# ADMIN SHOW REPLICA STATUS -## description - - 该语句用于展示一个表或分区的副本状态信息 - - 语法: - - ADMIN SHOW REPLICA STATUS FROM [db_name.]tbl_name [PARTITION (p1, ...)] - [where_clause]; - - where_clause: - WHERE STATUS [!]= "replica_status" - - replica_status: - OK: replica 处于健康状态 - DEAD: replica 所在 Backend 不可用 - VERSION_ERROR: replica 数据版本有缺失 - SCHEMA_ERROR: replica 的 schema hash 不正确 - MISSING: replica 不存在 - -## example - - 1. 查看表全部的副本状态 - - ADMIN SHOW REPLICA STATUS FROM db1.tbl1; - - 2. 查看表某个分区状态为 VERSION_ERROR 的副本 - - ADMIN SHOW REPLICA STATUS FROM tbl1 PARTITION (p1, p2) - WHERE STATUS = "VERSION_ERROR"; - - 3. 查看表所有状态不健康的副本 - - ADMIN SHOW REPLICA STATUS FROM tbl1 - WHERE STATUS != "OK"; - -## keyword - ADMIN,SHOW,REPLICA,STATUS - -# ADMIN SHOW REPLICA DISTRIBUTION -## description - - 该语句用于展示一个表或分区副本分布状态 - - 语法: - - ADMIN SHOW REPLICA DISTRIBUTION FROM [db_name.]tbl_name [PARTITION (p1, ...)]; - - 说明: - - 结果中的 Graph 列以图形的形式展示副本分布比例 - -## example - - 1. 查看表的副本分布 - - ADMIN SHOW REPLICA DISTRIBUTION FROM tbl1; - - 2. 查看表的分区的副本分布 - - ADMIN SHOW REPLICA DISTRIBUTION FROM db1.tbl1 PARTITION(p1, p2); - -## keyword - ADMIN,SHOW,REPLICA,DISTRIBUTION - -# ADMIN SHOW CONFIG -## description - - 该语句用于展示当前集群的配置(当前仅支持展示 FE 的配置项) - - 语法: - - ADMIN SHOW FRONTEND CONFIG; - - 说明: - - 结果中的各列含义如下: - 1. Key: 配置项名称 - 2. Value: 配置项值 - 3. Type: 配置项类型 - 4. IsMutable: 是否可以通过 ADMIN SET CONFIG 命令设置 - 5. MasterOnly: 是否仅适用于 Master FE - 6. Comment: 配置项说明 - -## example - - 1. 查看当前FE节点的配置 - - ADMIN SHOW FRONTEND CONFIG; - -## keyword - ADMIN,SHOW,CONFIG diff --git a/docs/help/Contents/Administration/admin_stmt.md b/docs/help/Contents/Administration/admin_stmt.md deleted file mode 100644 index 97ae32fdd5ea75..00000000000000 --- a/docs/help/Contents/Administration/admin_stmt.md +++ /dev/null @@ -1,338 +0,0 @@ -# ALTER SYSTEM -## description - - 该语句用于操作一个系统内的节点。(仅管理员使用!) - 语法: - 1) 增加节点(不使用多租户功能则按照此方法添加) - ALTER SYSTEM ADD BACKEND "host:heartbeat_port"[,"host:heartbeat_port"...]; - 2) 增加空闲节点(即添加不属于任何cluster的BACKEND) - ALTER SYSTEM ADD FREE BACKEND "host:heartbeat_port"[,"host:heartbeat_port"...]; - 3) 增加节点到某个cluster - ALTER SYSTEM ADD BACKEND TO cluster_name "host:heartbeat_port"[,"host:heartbeat_port"...]; - 4) 删除节点 - ALTER SYSTEM DROP BACKEND "host:heartbeat_port"[,"host:heartbeat_port"...]; - 5) 节点下线 - ALTER SYSTEM DECOMMISSION BACKEND "host:heartbeat_port"[,"host:heartbeat_port"...]; - 6) 增加Broker - ALTER SYSTEM ADD BROKER broker_name "host:port"[,"host:port"...]; - 7) 减少Broker - ALTER SYSTEM DROP BROKER broker_name "host:port"[,"host:port"...]; - 8) 删除所有Broker - ALTER SYSTEM DROP ALL BROKER broker_name - 9) 设置一个 Load error hub,用于集中展示导入时的错误信息 - ALTER SYSTEM SET LOAD ERRORS HUB PROPERTIES ("key" = "value"[, ...]); - - 说明: - 1) host 可以是主机名或者ip地址 - 2) heartbeat_port 为该节点的心跳端口 - 3) 增加和删除节点为同步操作。这两种操作不考虑节点上已有的数据,节点直接从元数据中删除,请谨慎使用。 - 4) 节点下线操作用于安全下线节点。该操作为异步操作。如果成功,节点最终会从元数据中删除。如果失败,则不会完成下线。 - 5) 可以手动取消节点下线操作。详见 CANCEL DECOMMISSION - 6) Load error hub: - 当前支持两种类型的 Hub:Mysql 和 Broker。需在 PROPERTIES 中指定 "type" = "mysql" 或 "type" = "broker"。 - 如果需要删除当前的 load error hub,可以将 type 设为 null。 - 1) 当使用 Mysql 类型时,导入时产生的错误信息将会插入到指定的 mysql 库表中,之后可以通过 show load warnings 语句直接查看错误信息。 - - Mysql 类型的 Hub 需指定以下参数: - host:mysql host - port:mysql port - user:mysql user - password:mysql password - database:mysql database - table:mysql table - - 2) 当使用 Broker 类型时,导入时产生的错误信息会形成一个文件,通过 broker,写入到指定的远端存储系统中。须确保已经部署对应的 broker - Broker 类型的 Hub 需指定以下参数: - broker: broker 的名称 - path: 远端存储路径 - other properties: 其他访问远端存储所必须的信息,比如认证信息等。 - -## example - - 1. 增加一个节点 - ALTER SYSTEM ADD BACKEND "host:port"; - - 2. 增加一个空闲节点 - ALTER SYSTEM ADD FREE BACKEND "host:port"; - - 3. 删除两个节点 - ALTER SYSTEM DROP BACKEND "host1:port", "host2:port"; - - 4. 下线两个节点 - ALTER SYSTEM DECOMMISSION BACKEND "host1:port", "host2:port"; - - 5. 增加两个Hdfs Broker - ALTER SYSTEM ADD BROKER hdfs "host1:port", "host2:port"; - - 6. 添加一个 Mysql 类型的 load error hub - ALTER SYSTEM SET LOAD ERRORS HUB PROPERTIES - ("type"= "mysql", - "host" = "192.168.1.17" - "port" = "3306", - "user" = "my_name", - "password" = "my_passwd", - "database" = "doris_load", - "table" = "load_errors" - ); - - 7. 添加一个 Broker 类型的 load error hub - ALTER SYSTEM SET LOAD ERRORS HUB PROPERTIES - ("type"= "broker", - "name" = "bos", - "path" = "bos://backup-cmy/logs", - "bos_endpoint" = "http://gz.bcebos.com", - "bos_accesskey" = "069fc278xxxxxx24ddb522", - "bos_secret_accesskey"="700adb0c6xxxxxx74d59eaa980a" - ); - - 8. 删除当前的 load error hub - ALTER SYSTEM SET LOAD ERRORS HUB PROPERTIES - ("type"= "null"); - -## keyword - ALTER,SYSTEM,BACKEND,BROKER,FREE - -# CANCEL DECOMMISSION -## description - - 该语句用于撤销一个节点下线操作。(仅管理员使用!) - 语法: - CANCEL DECOMMISSION BACKEND "host:heartbeat_port"[,"host:heartbeat_port"...]; - -## example - - 1. 取消两个节点的下线操作: - CANCEL DECOMMISSION BACKEND "host1:port", "host2:port"; - -## keyword - CANCEL,DECOMMISSION,BACKEND - -# CREATE CLUSTER -## description - - 该语句用于新建逻辑集群 (cluster), 需要管理员权限。如果不使用多租户,直接创建一个名称为default_cluster的cluster。否则创建一个自定义名称的cluster。 - - 语法 - - CREATE CLUSTER [IF NOT EXISTS] cluster_name - - PROPERTIES ("key"="value", ...) - - IDENTIFIED BY 'password' - - 1. PROPERTIES - - 指定逻辑集群的属性 - - PROPERTIES ("instance_num" = "3") - - instance_num 逻辑集群节点树 - - 2. identified by ‘password' 每个逻辑集群含有一个superuser,创建逻辑集群时必须指定其密码 - -## example - - 1. 新建一个含有3个be节点逻辑集群 test_cluster, 并指定其superuser用户密码 - - CREATE CLUSTER test_cluster PROPERTIES("instance_num"="3") IDENTIFIED BY 'test'; - - 2. 新建一个含有3个be节点逻辑集群 default_cluster(不使用多租户), 并指定其superuser用户密码 - - CREATE CLUSTER default_cluster PROPERTIES("instance_num"="3") IDENTIFIED BY 'test'; - -## keyword - CREATE,CLUSTER - -# ALTER CLUSTER -## description - - 该语句用于更新逻辑集群。需要有管理员权限 - - 语法 - - ALTER CLUSTER cluster_name PROPERTIES ("key"="value", ...); - - 1. 缩容,扩容 (根据集群现有的be数目,大则为扩容,小则为缩容), 扩容为同步操作,缩容为异步操作,通过backend的状态可以得知是否缩容完成 - - PROERTIES ("instance_num" = "3") - - instance_num 逻辑集群节点树 - -## example - - 1. 缩容,减少含有3个be的逻辑集群test_cluster的be数为2 - - ALTER CLUSTER test_cluster PROPERTIES ("instance_num"="2"); - - 2. 扩容,增加含有3个be的逻辑集群test_cluster的be数为4 - - ALTER CLUSTER test_cluster PROPERTIES ("instance_num"="4"); - -## keyword - ALTER,CLUSTER - -# DROP CLUSTER -## description - - 该语句用于删除逻辑集群,成功删除逻辑集群需要首先删除集群内的db,需要管理员权限 - - 语法 - - DROP CLUSTER [IF EXISTS] cluster_name - -## example - - 删除逻辑集群 test_cluster - - DROP CLUSTER test_cluster; - -## keyword - DROP,CLUSTER - -# LINK DATABASE -## description - - 该语句用户链接一个逻辑集群的数据库到另外一个逻辑集群, 一个数据库只允许同时被链接一次,删除链接的数据库 - - 并不会删除数据,并且被链接的数据库不能被删除, 需要管理员权限 - - 语法 - - LINK DATABASE src_cluster_name.src_db_name des_cluster_name.des_db_name - -## example - - 1. 链接test_clusterA中的test_db到test_clusterB,并命名为link_test_db - - LINK DATABASE test_clusterA.test_db test_clusterB.link_test_db; - - 2. 删除链接的数据库link_test_db - - DROP DATABASE link_test_db; - -## keyword - LINK,DATABASE - -# MIGRATE DATABASE -## description - - 该语句用于迁移一个逻辑集群的数据库到另外一个逻辑集群,执行此操作前数据库必须已经处于链接状态, 需要管理 - - 员权限 - - 语法 - - MIGRATE DATABASE src_cluster_name.src_db_name des_cluster_name.des_db_name - -## example - - 1. 迁移test_clusterA中的test_db到test_clusterB - - MIGRATE DATABASE test_clusterA.test_db test_clusterB.link_test_db; - -## keyword - MIGRATE,DATABASE - -# SHOW MIGRATIONS -## description - - 该语句用于查看数据库迁移的进度 - - 语法 - - SHOW MIGRATIONS - -## keyword - SHOW,MIGRATIONS - -# ENTER -## description - - 该语句用于进入一个逻辑集群, 所有创建用户、创建数据库都需要在一个逻辑集群内执行,创建后并且隶属于这个逻 - - 辑集群,需要管理员权限 - - ENTER cluster_name - -## example - - 1. 进入逻辑集群test_cluster - - ENTER test_cluster; - -## keyword - ENTER - -# SHOW BACKENDS -## description - 该语句用于查看 cluster 内的 BE 节点 - 语法: - SHOW BACKENDS; - - 说明: - 1. LastStartTime 表示最近一次 BE 启动时间。 - 2. LastHeartbeat 表示最近一次心跳。 - 3. Alive 表示节点是否存活。 - 4. SystemDecommissioned 为 true 表示节点正在安全下线中。 - 5. ClusterDecommissioned 为 true 表示节点正在冲当前cluster中下线。 - 6. TabletNum 表示该节点上分片数量。 - 7. DataUsedCapacity 表示实际用户数据所占用的空间。 - 8. AvailCapacity 表示磁盘的可使用空间。 - 9. TotalCapacity 表示总磁盘空间。TotalCapacity = AvailCapacity + DataUsedCapacity + 其他非用户数据文件占用空间。 - 10. UsedPct 表示磁盘已使用量百分比。 - 11. ErrMsg 用于显示心跳失败时的错误信息。 - -## keyword - SHOW, BACKENDS - -# SHOW FRONTENDS -## description - 该语句用于查看 FE 节点 - 语法: - SHOW FRONTENDS; - - 说明: - 1. name 表示该 FE 节点在 bdbje 中的名称。 - 2. Join 为 true 表示该节点曾经加入过集群。但不代表当前还在集群内(可能已失联) - 3. Alive 表示节点是否存活。 - 4. ReplayedJournalId 表示该节点当前已经回放的最大元数据日志id。 - 5. LastHeartbeat 是最近一次心跳。 - 6. IsHelper 表示该节点是否是 bdbje 中的 helper 节点。 - 7. ErrMsg 用于显示心跳失败时的错误信息。 - -## keyword - SHOW, FRONTENDS - -# SHOW BROKER -## description - 该语句用于查看当前存在的 broker - 语法: - SHOW BROKER; - - 说明: - 1. LastStartTime 表示最近一次 BE 启动时间。 - 2. LastHeartbeat 表示最近一次心跳。 - 3. Alive 表示节点是否存活。 - 4. ErrMsg 用于显示心跳失败时的错误信息。 - -## keyword - SHOW, BROKER - -# ADMIN SET CONFIG -## description - - 该语句用于设置集群的配置项(当前仅支持设置FE的配置项)。 - 可设置的配置项,可以通过 AMDIN SHOW FRONTEND CONFIG; 命令查看。 - - 语法: - - ADMIN SET FRONTEND CONFIG ("key" = "value"); - -## example - - 1. 设置 'disable_balance' 为 true - - ADMIN SET FRONTEND CONFIG ("disable_balance" = "true"); - -## keyword - ADMIN,SET,CONFIG diff --git a/docs/help/Contents/Data Definition/ddl_stmt.md b/docs/help/Contents/Data Definition/ddl_stmt.md deleted file mode 100644 index 147511f1e329f1..00000000000000 --- a/docs/help/Contents/Data Definition/ddl_stmt.md +++ /dev/null @@ -1,1153 +0,0 @@ -# CREATE DATABASE -## description - 该语句用于新建数据库(database) - 语法: - CREATE DATABASE [IF NOT EXISTS] db_name; - -## example - 1. 新建数据库 db_test - CREATE DATABASE db_test; - -## keyword - CREATE,DATABASE - -# DROP DATABASE -## description - 该语句用于删除数据库(database) - 语法: - DROP DATABASE [IF EXISTS] db_name; - - 说明: - 执行 DROP DATABASE 一段时间内,可以通过 RECOVER 语句恢复被删除的 database。详见 RECOVER 语句 - -## example - 1. 删除数据库 db_test - DROP DATABASE db_test; - -## keyword - DROP,DATABASE - -# CREATE TABLE -## description - 该语句用于创建 table。 - 语法: - CREATE [EXTERNAL] TABLE [IF NOT EXISTS] [database.]table_name - (column_definition1[, column_definition2, ...]) - [ENGINE = [olap|mysql|broker]] - [key_desc] - [partition_desc] - [distribution_desc] - [PROPERTIES ("key"="value", ...)]; - [BROKER PROPERTIES ("key"="value", ...)]; - - 1. column_definition - 语法: - col_name col_type [agg_type] [NULL | NOT NULL] [DEFAULT "default_value"] - - 说明: - col_name:列名称 - col_type:列类型 - TINYINT(1字节) - 范围:-2^7 + 1 ~ 2^7 - 1 - SMALLINT(2字节) - 范围:-2^15 + 1 ~ 2^15 - 1 - INT(4字节) - 范围:-2^31 + 1 ~ 2^31 - 1 - BIGINT(8字节) - 范围:-2^63 + 1 ~ 2^63 - 1 - LARGEINT(16字节) - 范围:0 ~ 2^127 - 1 - FLOAT(4字节) - 支持科学计数法 - DOUBLE(12字节) - 支持科学计数法 - DECIMAL[(precision, scale)] (40字节) - 保证精度的小数类型。默认是 DECIMAL(10, 0) - precision: 1 ~ 27 - scale: 0 ~ 9 - 其中整数部分为 1 ~ 18 - 不支持科学计数法 - DATE(3字节) - 范围:1900-01-01 ~ 9999-12-31 - DATETIME(8字节) - 范围:1900-01-01 00:00:00 ~ 9999-12-31 23:59:59 - CHAR[(length)] - 定长字符串。长度范围:1 ~ 255。默认为1 - VARCHAR[(length)] - 变长字符串。长度范围:1 ~ 65533 - HLL (1~16385个字节) - hll列类型,不需要指定长度和默认值、长度根据数据的聚合 - 程度系统内控制,并且HLL列只能通过配套的hll_union_agg、Hll_cardinality、hll_hash进行查询或使用 - - agg_type:聚合类型,如果不指定,则该列为 key 列。否则,该列为 value 列 - SUM、MAX、MIN、REPLACE、HLL_UNION(仅用于HLL列,为HLL独有的聚合方式) - 该类型只对聚合模型(key_desc的type为AGGREGATE KEY)有用,其它模型不需要指定这个。 - - 是否允许为NULL: 默认不允许为 NULL。NULL 值在导入数据中用 \N 来表示 - - 2. ENGINE 类型 - 默认为 olap。可选 mysql, broker - 1) 如果是 mysql,则需要在 properties 提供以下信息: - - PROPERTIES ( - "host" = "mysql_server_host", - "port" = "mysql_server_port", - "user" = "your_user_name", - "password" = "your_password", - "database" = "database_name", - "table" = "table_name" - ) - - 注意: - "table" 条目中的 "table_name" 是 mysql 中的真实表名。 - 而 CREATE TABLE 语句中的 table_name 是该 mysql 表在 Palo 中的名字,可以不同。 - - 在 Palo 创建 mysql 表的目的是可以通过 Palo 访问 mysql 数据库。 - 而 Palo 本身并不维护、存储任何 mysql 数据。 - 2) 如果是 broker,表示表的访问需要通过指定的broker, 需要在 properties 提供以下信息: - PROPERTIES ( - "broker_name" = "broker_name", - "paths" = "file_path1[,file_path2]", - "column_separator" = "value_separator" - "line_delimiter" = "value_delimiter" - ) - 另外还需要提供Broker需要的Property信息,通过BROKER PROPERTIES来传递,例如HDFS需要传入 - BROKER PROPERTIES( - "username" = "name", - "password" = "password" - ) - 这个根据不同的Broker类型,需要传入的内容也不相同 - 注意: - "paths" 中如果有多个文件,用逗号[,]分割。如果文件名中包含逗号,那么使用 %2c 来替代。如果文件名中包含 %,使用 %25 代替 - 现在文件内容格式支持CSV,支持GZ,BZ2,LZ4,LZO(LZOP) 压缩格式。 - - 3. key_desc - 语法: - key_type(k1[,k2 ...]) - 说明: - 数据按照指定的key列进行排序,且根据不同的key_type具有不同特性。 - key_type支持一下类型: - AGGREGATE KEY:key列相同的记录,value列按照指定的聚合类型进行聚合, - 适合报表、多维分析等业务场景。 - UNIQUE KEY:key列相同的记录,value列按导入顺序进行覆盖, - 适合按key列进行增删改查的点查询业务。 - DUPLICATE KEY:key列相同的记录,同时存在于Palo中, - 适合存储明细数据或者数据无聚合特性的业务场景。 - 注意: - 除AGGREGATE KEY外,其他key_type在建表时,value列不需要指定聚合类型。 - - 4. partition_desc - 1) Range 分区 - 语法: - PARTITION BY RANGE (k1, k2, ...) - ( - PARTITION partition_name VALUES LESS THAN MAXVALUE|("value1", "value2", ...) - PARTITION partition_name VALUES LESS THAN MAXVALUE|("value1", "value2", ...) - ... - ) - 说明: - 使用指定的 key 列和指定的数值范围进行分区。 - 1) 分区名称仅支持字母开头,字母、数字和下划线组成 - 2) 目前仅支持以下类型的列作为 Range 分区列,且只能指定一个分区列 - TINYINT, SMALLINT, INT, BIGINT, LARGEINT, DATE, DATETIME - 3) 分区为左闭右开区间,首个分区的左边界为做最小值 - 4) NULL 值只会存放在包含最小值的分区中。当包含最小值的分区被删除后,NULL 值将无法导入。 - 5) 可以指定一列或多列作为分区列。如果分区值缺省,则会默认填充最小值。 - - 注意: - 1) 分区一般用于时间维度的数据管理 - 2) 有数据回溯需求的,可以考虑首个分区为空分区,以便后续增加分区 - - 5. distribution_desc - 1) Hash 分桶 - 语法: - DISTRIBUTED BY HASH (k1[,k2 ...]) [BUCKETS num] - 说明: - 使用指定的 key 列进行哈希分桶。默认分区数为10 - - 建议:建议使用Hash分桶方式 - - 6. PROPERTIES - 1) 如果 ENGINE 类型为 olap,则可以在 properties 中指定列存(目前我们仅支持列存) - - PROPERTIES ( - "storage_type" = "[column]", - ) - - 2) 如果 ENGINE 类型为 olap - 可以在 properties 设置该表数据的初始存储介质、存储到期时间和副本数。 - - PROPERTIES ( - "storage_medium" = "[SSD|HDD]", - ["storage_cooldown_time" = "yyyy-MM-dd HH:mm:ss"], - ["replication_num" = "3"] - ) - - storage_medium: 用于指定该分区的初始存储介质,可选择 SSD 或 HDD。默认为 HDD。 - storage_cooldown_time: 当设置存储介质为 SSD 时,指定该分区在 SSD 上的存储到期时间。 - 默认存放 7 天。 - 格式为:"yyyy-MM-dd HH:mm:ss" - replication_num: 指定分区的副本数。默认为 3 - - 当表为单分区表时,这些属性为表的属性。 - 当表为两级分区时,这些属性为附属于每一个分区。 - 如果希望不同分区有不同属性。可以通过 ADD PARTITION 或 MODIFY PARTITION 进行操作 - - 3) 如果 Engine 类型为 olap, 并且 storage_type 为 column, 可以指定某列使用 bloom filter 索引 - bloom filter 索引仅适用于查询条件为 in 和 equal 的情况,该列的值越分散效果越好 - 目前只支持以下情况的列:除了 TINYINT FLOAT DOUBLE 类型以外的 key 列及聚合方法为 REPLACE 的 value 列 - - PROPERTIES ( - "bloom_filter_columns"="k1,k2,k3" - ) - 4) 如果希望使用Colocate Join 特性,需要在 properties 中指定 - - PROPERTIES ( - "colocate_with"="table1" - ) - -## example - 1. 创建一个 olap 表,使用 HASH 分桶,使用列存,相同key的记录进行聚合 - CREATE TABLE example_db.table_hash - ( - k1 TINYINT, - k2 DECIMAL(10, 2) DEFAULT "10.5", - v1 CHAR(10) REPLACE, - v2 INT SUM - ) - ENGINE=olap - AGGREGATE KEY(k1, k2) - DISTRIBUTED BY HASH(k1) BUCKETS 32 - PROPERTIES ("storage_type"="column"); - - 2. 创建一个 olap 表,使用 Hash 分桶,使用列存,相同key的记录进行覆盖, - 设置初始存储介质和冷却时间 - CREATE TABLE example_db.table_hash - ( - k1 BIGINT, - k2 LARGEINT, - v1 VARCHAR(2048) REPLACE, - v2 SMALLINT SUM DEFAULT "10" - ) - ENGINE=olap - UNIQUE KEY(k1, k2) - DISTRIBUTED BY HASH (k1, k2) BUCKETS 32 - PROPERTIES( - "storage_type"="column", - "storage_medium" = "SSD", - "storage_cooldown_time" = "2015-06-04 00:00:00" - ); - - 3. 创建一个 olap 表,使用 Key Range 分区,使用Hash分桶,默认使用列存, - 相同key的记录同时存在,设置初始存储介质和冷却时间 - CREATE TABLE example_db.table_range - ( - k1 DATE, - k2 INT, - k3 SMALLINT, - v1 VARCHAR(2048), - v2 DATETIME DEFAULT "2014-02-04 15:36:00" - ) - ENGINE=olap - DUPLICATE KEY(k1, k2, k3) - PARTITION BY RANGE (k1) - ( - PARTITION p1 VALUES LESS THAN ("2014-01-01"), - PARTITION p2 VALUES LESS THAN ("2014-06-01"), - PARTITION p3 VALUES LESS THAN ("2014-12-01") - ) - DISTRIBUTED BY HASH(k2) BUCKETS 32 - PROPERTIES( - "storage_medium" = "SSD", "storage_cooldown_time" = "2015-06-04 00:00:00" - ); - - 说明: - 这个语句会将数据划分成如下3个分区: - ( { MIN }, {"2014-01-01"} ) - [ {"2014-01-01"}, {"2014-06-01"} ) - [ {"2014-06-01"}, {"2014-12-01"} ) - - 不在这些分区范围内的数据将视为非法数据被过滤 - - 4. 创建一个 mysql 表 - CREATE TABLE example_db.table_mysql - ( - k1 DATE, - k2 INT, - k3 SMALLINT, - k4 VARCHAR(2048), - k5 DATETIME - ) - ENGINE=mysql - PROPERTIES - ( - "host" = "127.0.0.1", - "port" = "8239", - "user" = "mysql_user", - "password" = "mysql_passwd", - "database" = "mysql_db_test", - "table" = "mysql_table_test" - ) - - 5. 创建一个数据文件存储在HDFS上的 broker 外部表, 数据使用 "|" 分割,"\n" 换行 - CREATE EXTERNAL TABLE example_db.table_broker ( - k1 DATE, - k2 INT, - k3 SMALLINT, - k4 VARCHAR(2048), - k5 DATETIME - ) - ENGINE=broker - PROPERTIES ( - "broker_name" = "hdfs", - "path" = "hdfs://hdfs_host:hdfs_port/data1,hdfs://hdfs_host:hdfs_port/data2,hdfs://hdfs_host:hdfs_port/data3%2c4", - "column_separator" = "|", - "line_delimiter" = "\n" - ) - BROKER PROPERTIES ( - "username" = "hdfs_user", - "password" = "hdfs_password" - ) - - 6. 创建一张含有HLL列的表 - CREATE TABLE example_db.example_table - ( - k1 TINYINT, - k2 DECIMAL(10, 2) DEFAULT "10.5", - v1 HLL HLL_UNION, - v2 HLL HLL_UNION - ) - ENGINE=olap - AGGREGATE KEY(k1, k2) - DISTRIBUTED BY HASH(k1) BUCKETS 32 - PROPERTIES ("storage_type"="column"); - - 7. 创建两张支持Colocat Join的表t1 和t2 - CREATE TABLE `t1` ( - `id` int(11) COMMENT "", - `value` varchar(8) COMMENT "" - ) ENGINE=OLAP - DUPLICATE KEY(`id`) - DISTRIBUTED BY HASH(`id`) BUCKETS 10 - PROPERTIES ( - "colocate_with" = "t1" - ); - - CREATE TABLE `t2` ( - `id` int(11) COMMENT "", - `value` varchar(8) COMMENT "" - ) ENGINE=OLAP - DUPLICATE KEY(`id`) - DISTRIBUTED BY HASH(`id`) BUCKETS 10 - PROPERTIES ( - "colocate_with" = "t1" - ); - - 8. 创建一个数据文件存储在BOS上的 broker 外部表 - CREATE EXTERNAL TABLE example_db.table_broker ( - k1 DATE - ) - ENGINE=broker - PROPERTIES ( - "broker_name" = "bos", - "path" = "bos://my_bucket/input/file", - ) - BROKER PROPERTIES ( - "bos_endpoint" = "http://bj.bcebos.com", - "bos_accesskey" = "xxxxxxxxxxxxxxxxxxxxxxxxxx", - "bos_secret_accesskey"="yyyyyyyyyyyyyyyyyyyy" - ) - -## keyword - CREATE,TABLE - -# DROP TABLE -## description - 该语句用于删除 table 。 - 语法: - DROP TABLE [IF EXISTS] [db_name.]table_name; - - 说明: - 执行 DROP TABLE 一段时间内,可以通过 RECOVER 语句恢复被删除的 table。详见 RECOVER 语句 - -## example - 1. 删除一个 table - DROP TABLE my_table; - - 2. 如果存在,删除指定 database 的 table - DROP TABLE IF EXISTS example_db.my_table; - -## keyword - DROP,TABLE - -# ALTER TABLE -## description - 该语句用于对已有的 table 进行修改。如果没有指定 rollup index,默认操作 base index。 - 该语句分为三种操作类型: schema change 、rollup 、partition - 这三种操作类型不能同时出现在一条 ALTER TABLE 语句中。 - 其中 schema change 和 rollup 是异步操作,任务提交成功则返回。之后可使用 SHOW ALTER 命令查看进度。 - partition 是同步操作,命令返回表示执行完毕。 - - 语法: - ALTER TABLE [database.]table - alter_clause1[, alter_clause2, ...]; - - alter_clause 分为 partition 、rollup、schema change 和 rename 四种。 - - partition 支持如下几种修改方式 - 1. 增加分区 - 语法: - ADD PARTITION [IF NOT EXISTS] partition_name VALUES LESS THAN [MAXVALUE|("value1")] ["key"="value"] - [DISTRIBUTED BY HASH (k1[,k2 ...]) [BUCKETS num]] - 注意: - 1) 分区为左闭右开区间,用户指定右边界,系统自动确定左边界 - 2) 如果没有指定分桶方式,则自动使用建表使用的分桶方式 - 3) 如指定分桶方式,只能修改分桶数,不可修改分桶方式或分桶列 - 4) ["key"="value"] 部分可以设置分区的一些属性,具体说明见 CREATE TABLE - - 2. 删除分区 - 语法: - DROP PARTITION [IF EXISTS] partition_name - 注意: - 1) 使用分区方式的表至少要保留一个分区。 - 2) 执行 DROP PARTITION 一段时间内,可以通过 RECOVER 语句恢复被删除的 partition。详见 RECOVER 语句 - - 3. 修改分区属性 - 语法: - MODIFY PARTITION partition_name SET ("key" = "value", ...) - 说明: - 1) 当前支持修改分区的 storage_medium、storage_cooldown_time 和 replication_num 三个属性。 - 2) 对于单分区表,partition_name 同表名。 - - rollup 支持如下几种创建方式: - 1. 创建 rollup index - 语法: - ADD ROLLUP rollup_name (column_name1, column_name2, ...) - [FROM from_index_name] - [PROPERTIES ("key"="value", ...)] - 注意: - 1) 如果没有指定 from_index_name,则默认从 base index 创建 - 2) rollup 表中的列必须是 from_index 中已有的列 - 3) 在 properties 中,可以指定存储格式。具体请参阅 CREATE TABLE - - 2. 删除 rollup index - 语法: - DROP ROLLUP rollup_name - [PROPERTIES ("key"="value", ...)] - 注意: - 1) 不能删除 base index - 2) 执行 DROP ROLLUP 一段时间内,可以通过 RECOVER 语句恢复被删除的 rollup index。详见 RECOVER 语句 - - - schema change 支持如下几种修改方式: - 1. 向指定 index 的指定位置添加一列 - 语法: - ADD COLUMN column_name column_type [KEY | agg_type] [DEFAULT "default_value"] - [AFTER column_name|FIRST] - [TO rollup_index_name] - [PROPERTIES ("key"="value", ...)] - 注意: - 1) 聚合模型如果增加 value 列,需要指定 agg_type - 2) 非聚合模型如果增加key列,需要指定KEY关键字 - 3) 不能在 rollup index 中增加 base index 中已经存在的列 - 如有需要,可以重新创建一个 rollup index) - - 2. 向指定 index 添加多列 - 语法: - ADD COLUMN (column_name1 column_type [KEY | agg_type] DEFAULT "default_value", ...) - [TO rollup_index_name] - [PROPERTIES ("key"="value", ...)] - 注意: - 1) 聚合模型如果增加 value 列,需要指定agg_type - 2) 非聚合模型如果增加key列,需要指定KEY关键字 - 3) 不能在 rollup index 中增加 base index 中已经存在的列 - (如有需要,可以重新创建一个 rollup index) - - 3. 从指定 index 中删除一列 - 语法: - DROP COLUMN column_name - [FROM rollup_index_name] - 注意: - 1) 不能删除分区列 - 2) 如果是从 base index 中删除列,则如果 rollup index 中包含该列,也会被删除 - - 4. 修改指定 index 的列类型以及列位置 - 语法: - MODIFY COLUMN column_name column_type [KEY | agg_type] [NULL | NOT NULL] [DEFAULT "default_value"] - [AFTER column_name|FIRST] - [FROM rollup_index_name] - [PROPERTIES ("key"="value", ...)] - 注意: - 1) 聚合模型如果修改 value 列,需要指定 agg_type - 2) 非聚合类型如果修改key列,需要指定KEY关键字 - 3) 只能修改列的类型,列的其他属性维持原样(即其他属性需在语句中按照原属性显式的写出,参见 example 8) - 4) 分区列不能做任何修改 - 5) 目前支持以下类型的转换(精度损失由用户保证) - TINYINT/SMALLINT/INT/BIGINT 转换成 TINYINT/SMALLINT/INT/BIGINT/DOUBLE。 - LARGEINT 转换成 DOUBLE - VARCHAR 支持修改最大长度 - 6) 不支持从NULL转为NOT NULL - - 5. 对指定 index 的列进行重新排序 - 语法: - ORDER BY (column_name1, column_name2, ...) - [FROM rollup_index_name] - [PROPERTIES ("key"="value", ...)] - 注意: - 1) index 中的所有列都要写出来 - 2) value 列在 key 列之后 - - 6. 修改table的属性,目前支持修改bloom filter列和colocate_with 属性 - 语法: - PROPERTIES ("key"="value") - 注意: - 也可以合并到上面的schema change操作中来修改,见下面例子 - - - rename 支持对以下名称进行修改: - 1. 修改表名 - 语法: - RENAME new_table_name; - - 2. 修改 rollup index 名称 - 语法: - RENAME ROLLUP old_rollup_name new_rollup_name; - - 3. 修改 partition 名称 - 语法: - RENAME PARTITION old_partition_name new_partition_name; - -## example - [partition] - 1. 增加分区, 现有分区 [MIN, 2013-01-01),增加分区 [2013-01-01, 2014-01-01),使用默认分桶方式 - ALTER TABLE example_db.my_table - ADD PARTITION p1 VALUES LESS THAN ("2014-01-01"); - - 2. 增加分区,使用新的分桶数 - ALTER TABLE example_db.my_table - ADD PARTITION p1 VALUES LESS THAN ("2015-01-01") - DISTRIBUTED BY HASH(k1) BUCKETS 20; - - 3. 删除分区 - ALTER TABLE example_db.my_table - DROP PARTITION p1; - - [rollup] - 1. 创建 index: example_rollup_index,基于 base index(k1,k2,k3,v1,v2)。列式存储。 - ALTER TABLE example_db.my_table - ADD ROLLUP example_rollup_index(k1, k3, v1, v2) - PROPERTIES("storage_type"="column"); - - 2. 创建 index: example_rollup_index2,基于 example_rollup_index(k1,k3,v1,v2) - ALTER TABLE example_db.my_table - ADD ROLLUP example_rollup_index2 (k1, v1) - FROM example_rollup_index; - - 3. 删除 index: example_rollup_index2 - ALTER TABLE example_db.my_table - DROP ROLLUP example_rollup_index2; - - [schema change] - 1. 向 example_rollup_index 的 col1 后添加一个key列 new_col(非聚合模型) - ALTER TABLE example_db.my_table - ADD COLUMN new_col INT KEY DEFAULT "0" AFTER col1 - TO example_rollup_index; - - 2. 向example_rollup_index的col1后添加一个value列new_col(非聚合模型) - ALTER TABLE example_db.my_table - ADD COLUMN new_col INT DEFAULT "0" AFTER col1 - TO example_rollup_index; - - 3. 向example_rollup_index的col1后添加一个key列new_col(聚合模型) - ALTER TABLE example_db.my_table - ADD COLUMN new_col INT DEFAULT "0" AFTER col1 - TO example_rollup_index; - - 4. 向example_rollup_index的col1后添加一个value列new_col SUM聚合类型(聚合模型) - ALTER TABLE example_db.my_table - ADD COLUMN new_col INT SUM DEFAULT "0" AFTER col1 - TO example_rollup_index; - - 5. 向 example_rollup_index 添加多列(聚合模型) - ALTER TABLE example_db.my_table - ADD COLUMN (col1 INT DEFAULT "1", col2 FLOAT SUM DEFAULT "2.3") - TO example_rollup_index; - - 6. 从 example_rollup_index 删除一列 - ALTER TABLE example_db.my_table - DROP COLUMN col2 - FROM example_rollup_index; - - 7. 修改 base index 的 col1 列的类型为 BIGINT,并移动到 col2 列后面 - ALTER TABLE example_db.my_table - MODIFY COLUMN col1 BIGINT DEFAULT "1" AFTER col2; - - 8. 修改 base index 的 val1 列最大长度。原 val1 为 (val1 VARCHAR(32) REPLACE DEFAULT "abc") - ALTER TABLE example_db.my_table - MODIFY COLUMN val1 VARCHAR(64) REPLACE DEFAULT "abc"; - - 9. 重新排序 example_rollup_index 中的列(设原列顺序为:k1,k2,k3,v1,v2) - ALTER TABLE example_db.my_table - ORDER BY (k3,k1,k2,v2,v1) - FROM example_rollup_index; - - 10. 同时执行两种操作 - ALTER TABLE example_db.my_table - ADD COLUMN v2 INT MAX DEFAULT "0" AFTER k2 TO example_rollup_index, - ORDER BY (k3,k1,k2,v2,v1) FROM example_rollup_index; - - 11. 修改表的 bloom filter 列 - ALTER TABLE example_db.my_table SET ("bloom_filter_columns"="k1,k2,k3"); - - 也可以合并到上面的 schema change 操作中(注意多子句的语法有少许区别) - ALTER TABLE example_db.my_table - DROP COLUMN col2 - PROPERTIES ("bloom_filter_columns"="k1,k2,k3"); - - 12. 修改表的Colocate 属性 - ALTER TABLE example_db.my_table set ("colocate_with"="t1"); - - [rename] - 1. 将名为 table1 的表修改为 table2 - ALTER TABLE table1 RENAME table2; - - 2. 将表 example_table 中名为 rollup1 的 rollup index 修改为 rollup2 - ALTER TABLE example_table RENAME ROLLUP rollup1 rollup2; - - 3. 将表 example_table 中名为 p1 的 partition 修改为 p2 - ALTER TABLE example_table RENAME PARTITION p1 p2; - -## keyword - ALTER,TABLE,ROLLUP,COLUMN,PARTITION,RENAME - -# CANCEL ALTER -## description - 该语句用于撤销一个 ALTER 操作。 - 1. 撤销 ALTER TABLE COLUMN 操作 - 语法: - CANCEL ALTER TABLE COLUMN - FROM db_name.table_name - - 2. 撤销 ALTER TABLE ROLLUP 操作 - 语法: - CANCEL ALTER TABLE ROLLUP - FROM db_name.table_name - - 2. 撤销 ALTER CLUSTER 操作 - 语法: - (待实现...) - - -## example - [CANCEL ALTER TABLE COLUMN] - 1. 撤销针对 my_table 的 ALTER COLUMN 操作。 - CANCEL ALTER TABLE COLUMN - FROM example_db.my_table; - - [CANCEL ALTER TABLE ROLLUP] - 1. 撤销 my_table 下的 ADD ROLLUP 操作。 - CANCEL ALTER TABLE ROLLUP - FROM example_db.my_table; - -## keyword - CANCEL,ALTER,TABLE,COLUMN,ROLLUP - -# CREATE VIEW -## description - 该语句用于创建一个逻辑视图 - 语法: - CREATE VIEW [IF NOT EXISTS] - [db_name.]view_name (column1[, column2, ...]) - AS query_stmt - - 说明: - 1. 视图为逻辑视图,没有物理存储。所有在视图上的查询相当于在视图对应的子查询上进行。 - 2. query_stmt 为任意支持的 SQL - -## example - 1. 在 example_db 上创建视图 example_view - CREATE VIEW example_db.example_view (k1, k2, k3, v1) - AS - SELECT c1 as k1, k2, k3, SUM(v1) FROM example_table - WHERE k1 = 20160112 GROUP BY k1,k2,k3; - -## keyword - CREATE,VIEW - -# DROP VIEW -## description - 该语句用于删除一个逻辑视图 VIEW - 语法: - DROP VIEW [IF EXISTS] - [db_name.]view_name; - -## example - 1. 如果存在,删除 example_db 上的视图 example_view - DROP VIEW IF EXISTS example_db.example_view; - -## keyword - DROP,VIEW - -# RECOVER -## description - 该语句用于恢复之前删除的 database、table 或者 partition - 语法: - 1) 恢复 database - RECOVER DATABASE db_name; - 2) 恢复 table - RECOVER TABLE [db_name.]table_name; - 3) 恢复 partition - RECOVER PARTITION partition_name FROM [db_name.]table_name; - - 说明: - 1. 该操作仅能恢复之前一段时间内删除的元信息。默认为 3600 秒。 - 2. 如果删除元信息后新建立了同名同类型的元信息,则之前删除的元信息不能被恢复 - -## example - 1. 恢复名为 example_db 的 database - RECOVER DATABASE example_db; - - 2. 恢复名为 example_tbl 的 table - RECOVER TABLE example_db.example_tbl; - - 3. 恢复表 example_tbl 中名为 p1 的 partition - RECOVER PARTITION p1 FROM example_tbl; - -## keyword - RECOVER - -# ALTER DATABASE -## description - 该语句用于设置指定数据库的属性。(仅管理员使用) - 语法: - 1) 设置数据库数据量配额,单位为B/K/KB/M/MB/G/GB/T/TB/P/PB - ALTER DATABASE db_name SET DATA QUOTA quota; - - 2) 重命名数据库 - ALTER DATABASE db_name RENAME new_db_name; - - 说明: - 重命名数据库后,如需要,请使用 REVOKE 和 GRANT 命令修改相应的用户权限。 - -## example - 1. 设置指定数据库数据量配额 - ALTER DATABASE example_db SET DATA QUOTA 10995116277760; - 上述单位为字节,等价于 - ALTER DATABASE example_db SET DATA QUOTA 10T; - - ALTER DATABASE example_db SET DATA QUOTA 100G; - - ALTER DATABASE example_db SET DATA QUOTA 200M; - - 2. 将数据库 example_db 重命名为 example_db2 - ALTER DATABASE example_db RENAME example_db2; - -## keyword - ALTER,DATABASE,RENAME - -# CREATE REPOSITORY -## description - 该语句用于创建仓库。仓库用于属于备份或恢复。仅 root 或 superuser 用户可以创建仓库。 - 语法: - CREATE [READ ONLY] REPOSITORY `repo_name` - WITH BROKER `broker_name` - ON LOCATION `repo_location` - PROPERTIES ("key"="value", ...); - - 说明: - 1. 仓库的创建,依赖于已存在的 broker - 2. 如果是只读仓库,则只能在仓库上进行恢复。如果不是,则可以进行备份和恢复操作。 - 3. 根据 broker 的不同类型,PROPERTIES 有所不同,具体见示例。 - -## example - 1. 创建名为 bos_repo 的仓库,依赖 BOS broker "bos_broker",数据根目录为:bos://palo_backup - CREATE REPOSITORY `bos_repo` - WITH BROKER `bos_broker` - ON LOCATION "bos://palo_backup" - PROPERTIES - ( - "bos_endpoint" = "http://gz.bcebos.com", - "bos_accesskey" = "069fc2786e664e63a5f111111114ddbs22", - "bos_secret_accesskey"="70999999999999de274d59eaa980a" - ); - - 2. 创建和示例 1 相同的仓库,但属性为只读: - CREATE READ ONLY REPOSITORY `bos_repo` - WITH BROKER `bos_broker` - ON LOCATION "bos://palo_backup" - PROPERTIES - ( - "bos_endpoint" = "http://gz.bcebos.com", - "bos_accesskey" = "069fc2786e664e63a5f111111114ddbs22", - "bos_secret_accesskey"="70999999999999de274d59eaa980a" - ); - - 3. 创建名为 hdfs_repo 的仓库,依赖 Baidu hdfs broker "hdfs_broker",数据根目录为:hdfs://hadoop-name-node:54310/path/to/repo/ - CREATE REPOSITORY `hdfs_repo` - WITH BROKER `hdfs_broker` - ON LOCATION "hdfs://hadoop-name-node:54310/path/to/repo/" - PROPERTIES - ( - "username" = "user", - "password" = "password" - ); - -## keyword - CREATE REPOSITORY - -# DROP REPOSITORY -## description - 该语句用于删除一个已创建的仓库。仅 root 或 superuser 用户可以删除仓库。 - 语法: - DROP REPOSITORY `repo_name`; - - 说明: - 1. 删除仓库,仅仅是删除该仓库在 Palo 中的映射,不会删除实际的仓库数据。删除后,可以再次通过指定相同的 broker 和 LOCATION 映射到该仓库。 - -## example - 1. 删除名为 bos_repo 的仓库: - DROP REPOSITORY `bos_repo`; - -## keyword - DROP REPOSITORY - -# BACKUP -## description - 该语句用于备份指定数据库下的数据。该命令为异步操作。提交成功后,需通过 SHOW BACKUP 命令查看进度。仅支持备份 OLAP 类型的表。 - 语法: - BACKUP SNAPSHOT [db_name].{snapshot_name} - TO `repository_name` - ON ( - `table_name` [PARTITION (`p1`, ...)], - ... - ) - PROPERTIES ("key"="value", ...); - - 说明: - 1. 同一数据库下只能有一个正在执行的 BACKUP 或 RESTORE 任务。 - 2. ON 子句中标识需要备份的表和分区。如果不指定分区,则默认备份该表的所有分区。 - 3. PROPERTIES 目前支持以下属性: - "type" = "full":表示这是一次全量更新(默认)。 - "timeout" = "3600":任务超时时间,默认为一天。单位秒。 - -## example - - 1. 全量备份 example_db 下的表 example_tbl 到仓库 example_repo 中: - BACKUP SNAPSHOT example_db.snapshot_label1 - TO example_repo - ON (example_tbl) - PROPERTIES ("type" = "full"); - - 2. 全量备份 example_db 下,表 example_tbl 的 p1, p2 分区,以及表 example_tbl2 到仓库 example_repo 中: - BACKUP SNAPSHOT example_db.snapshot_label2 - TO example_repo - ON - ( - example_tbl PARTITION (p1,p2), - example_tbl2 - ); - -## keyword - BACKUP - -# RESTORE -## description - 1. RESTORE - 该语句用于将之前通过 BACKUP 命令备份的数据,恢复到指定数据库下。该命令为异步操作。提交成功后,需通过 SHOW RESTORE 命令查看进度。仅支持恢复 OLAP 类型的表。 - 语法: - RESTORE SNAPSHOT [db_name].{snapshot_name} - FROM `repository_name` - ON ( - `table_name` [PARTITION (`p1`, ...)] [AS `tbl_alias`], - ... - ) - PROPERTIES ("key"="value", ...); - - 说明: - 1. 同一数据库下只能有一个正在执行的 BACKUP 或 RESTORE 任务。 - 2. ON 子句中标识需要恢复的表和分区。如果不指定分区,则默认恢复该表的所有分区。所指定的表和分区必须已存在于仓库备份中。 - 3. 可以通过 AS 语句将仓库中备份的表名恢复为新的表。但新表名不能已存在于数据库中。分区名称不能修改。 - 4. 可以将仓库中备份的表恢复替换数据库中已有的同名表,但须保证两张表的表结构完全一致。表结构包括:表名、列、分区、Rollup等等。 - 5. 可以指定恢复表的部分分区,系统会检查分区 Range 是否能够匹配。 - 6. PROPERTIES 目前支持以下属性: - "backup_timestamp" = "2018-05-04-16-45-08":指定了恢复对应备份的哪个时间版本,必填。该信息可以通过 `SHOW SNAPSHOT ON repo;` 语句获得。 - "replication_num" = "3":指定恢复的表或分区的副本数。默认为3。若恢复已存在的表或分区,则副本数必须和已存在表或分区的副本数相同。同时,必须有足够的 host 容纳多个副本。 - "timeout" = "3600":任务超时时间,默认为一天。单位秒。 - "meta_version" = 40:使用指定的 meta_version 来读取之前备份的元数据。注意,该参数作为临时方案,仅用于恢复老版本 Doris 备份的数据。最新版本的备份数据中已经包含 meta version,无需再指定。 - -## example - 1. 从 example_repo 中恢复备份 snapshot_1 中的表 backup_tbl 到数据库 example_db1,时间版本为 "2018-05-04-16-45-08"。恢复为 1 个副本: - RESTORE SNAPSHOT example_db1.`snapshot_1` - FROM `example_repo` - ON ( `backup_tbl` ) - PROPERTIES - ( - "backup_timestamp"="2018-05-04-16-45-08", - "replication_num" = "1" - ); - - 2. 从 example_repo 中恢复备份 snapshot_2 中的表 backup_tbl 的分区 p1,p2,以及表 backup_tbl2 到数据库 example_db1,并重命名为 new_tbl,时间版本为 "2018-05-04-17-11-01"。默认恢复为 3 个副本: - RESTORE SNAPSHOT example_db1.`snapshot_2` - FROM `example_repo` - ON - ( - `backup_tbl` PARTITION (`p1`, `p2`), - `backup_tbl2` AS `new_tbl` - ) - PROPERTIES - ( - "backup_timestamp"="2018-05-04-17-11-01" - ); - -## keyword - RESTORE - -# CANCEL BACKUP -## description - 该语句用于取消一个正在进行的 BACKUP 任务。 - 语法: - CANCEL BACKUP FROM db_name; - -## example - 1. 取消 example_db 下的 BACKUP 任务。 - CANCEL BACKUP FROM example_db; - -## keyword - CANCEL, BACKUP - -# CANCEL RESTORE -## description - 该语句用于取消一个正在进行的 RESTORE 任务。 - 语法: - CANCEL RESTORE FROM db_name; - - 注意: - 当取消处于 COMMIT 或之后阶段的恢复左右时,可能导致被恢复的表无法访问。此时只能通过再次执行恢复作业进行数据恢复。 - -## example - 1. 取消 example_db 下的 RESTORE 任务。 - CANCEL RESTORE FROM example_db; - -## keyword - CANCEL, RESTORE - -# HLL -## description - HLL是基于HyperLogLog算法的工程实现,用于保存HyperLogLog计算过程的中间结果,它只能作为表的value列类型 - 通过聚合来不断的减少数据量,以此来实现加快查询的目的,基于它到的是一个估算结果,误差大概在1%左右 - hll列是通过其它列或者导入数据里面的数据生成的,导入的时候通过hll_hash函数来指定数据中哪一列用于生成hll列 - 它常用于替代count distinct,通过结合rollup在业务上用于快速计算uv等 - - 相关函数: - - HLL_UNION_AGG(hll) - 此函数为聚合函数,用于计算满足条件的所有数据的基数估算。此函数还可用于分析函数,只支持默认窗口,不支持window从句。 - - HLL_RAW_AGG(hll) - 此函数为聚合函数,用于聚合hll类型字段,并且返回的还是hll类型。 - - HLL_CARDINALITY(hll) - 此函数用于计算单条hll列的基数估算 - - HLL_HASH(column_name) - 生成HLL列类型,用于insert或导入的时候,导入的使用见相关说明 - -## example - 1. 首先创建一张含有hll列的表 - create table test( - dt date, - id int, - name char(10), - province char(10), - os char(1), - set1 hll hll_union, - set2 hll hll_union) - distributed by hash(id) buckets 32; - - 2. 导入数据,导入的方式见相关help curl - - a. 使用表中的列生成hll列 - curl --location-trusted -uname:password -T data http://host/api/test_db/test/_load?label=load_1\&hll=set1,id:set2,name - - b. 使用数据中的某一列生成hll列 - curl --location-trusted -uname:password -T data http://host/api/test_db/test/_load?label=load_1\&hll=set1,cuid:set2,os - \&columns=dt,id,name,province,sex,cuid,os - - 3. 聚合数据,常用方式3种:(如果不聚合直接对base表查询,速度可能跟直接使用ndv速度差不多) - - a. 创建一个rollup,让hll列产生聚合, - alter table test add rollup test_rollup(dt, set1); - - b. 创建另外一张专门计算uv的表,然后insert数据) - - create table test_uv( - dt date, - uv_set hll hll_union) - distributed by hash(id) buckets 32; - - insert into test_uv select dt, set1 from test; - - c. 创建另外一张专门计算uv的表,然后insert并通过hll_hash根据test其它非hll列生成hll列 - - create table test_uv( - dt date, - id_set hll hll_union) - distributed by hash(id) buckets 32; - - insert into test_uv select dt, hll_hash(id) from test; - - 4. 查询,hll列不允许直接查询它的原始值,可以通过配套的函数进行查询 - - a. 求总uv - select HLL_UNION_AGG(uv_set) from test_uv; - - b. 求每一天的uv - select dt, HLL_CARDINALITY(uv_set) from test_uv; - - c. 求test表中set1的聚合值 - select dt, HLL_CARDINALITY(uv) from (select dt, HLL_RAW_AGG(set1) as uv from test group by dt) tmp; - select dt, HLL_UNION_AGG(set1) as uv from test group by dt; - -## keyword - HLL - -# TRUNCATE TABLE -## description - 该语句用于清空指定表和分区的数据 - 语法: - - TRUNCATE TABLE [db.]tbl[ PARTITION(p1, p2, ...)]; - - 说明: - 1. 该语句清空数据,但保留表或分区。 - 2. 不同于 DELETE,该语句只能整体清空指定的表或分区,不能添加过滤条件。 - 3. 不同于 DELETE,使用该方式清空数据不会对查询性能造成影响。 - 4. 该操作删除的数据不可恢复。 - 5. 使用该命令时,表状态需为 NORMAL,即不允许正在进行 SCHEMA CHANGE 等操作。 - -## example - - 1. 清空 example_db 下的表 tbl - - TRUNCATE TABLE example_db.tbl; - - 2. 清空表 tbl 的 p1 和 p2 分区 - - TRUNCATE TABLE tbl PARTITION(p1, p2); - -## keyword - TRUNCATE,TABLE - -# Colocate Join -## description - Colocate/Local Join 就是指多个节点Join时没有数据移动和网络传输,每个节点只在本地进行Join, - 能够本地进行Join的前提是相同Join Key的数据导入时按照相同规则导入到固定的节点。 - - 1 How To Use: - - 只需要在建表时增加 colocate_with 这个属性即可,colocate_with的值 可以设置成同一组colocate 表中的任意一个, - 不过需要保证colocate_with属性中的表要先建立。 - - 假如需要对table t1 和t2 进行Colocate Join,可以按以下语句建表: - - CREATE TABLE `t1` ( - `id` int(11) COMMENT "", - `value` varchar(8) COMMENT "" - ) ENGINE=OLAP - DUPLICATE KEY(`id`) - DISTRIBUTED BY HASH(`id`) BUCKETS 10 - PROPERTIES ( - "colocate_with" = "t1" - ); - - CREATE TABLE `t2` ( - `id` int(11) COMMENT "", - `value` varchar(8) COMMENT "" - ) ENGINE=OLAP - DUPLICATE KEY(`id`) - DISTRIBUTED BY HASH(`id`) BUCKETS 10 - PROPERTIES ( - "colocate_with" = "t1" - ); - - 2 Colocate Join 目前的限制: - - 1. Colcoate Table 必须是OLAP类型的表 - 2. 相同colocate_with 属性的表的 BUCKET 数必须一样 - 3. 相同colocate_with 属性的表的 副本数必须一样 - 4. 相同colocate_with 属性的表的 DISTRIBUTED Columns的数据类型必须一样 - - 3 Colocate Join的适用场景: - - Colocate Join 十分适合几张表按照相同字段分桶,并高频根据相同字段Join的场景。 - - 4 FAQ: - - Q: 支持多张表进行Colocate Join 吗? - - A: 支持 - - Q: 支持Colocate 表和正常表 Join 吗? - - A: 支持 - - Q: Colocate 表支持用非分桶的Key进行Join吗? - - A: 支持:不符合Colocate Join条件的Join会使用Shuffle Join或Broadcast Join - - Q: 如何确定Join 是按照Colocate Join 执行的? - - A: explain的结果中Hash Join的孩子节点如果直接是OlapScanNode, 没有Exchange Node,就说明是Colocate Join - - Q: 如何修改colocate_with 属性? - - A: ALTER TABLE example_db.my_table set ("colocate_with"="target_table"); - - Q: 如何禁用colcoate join? - - A: set disable_colocate_join = true; 就可以禁用Colocate Join,查询时就会使用Shuffle Join 和Broadcast Join - -## keyword - - COLOCATE, JOIN, CREATE TABLE - -# CREATE FUNCTION -## description - Used to create a UDF/UDAF/UDTF - Syntax: - CREATE [AGGREGATE] FUNCTION funcName (argType [, ...]) - RETURNS retType - PROPERTIES ( - k1=v1 [, k2=v2] - ) - - valid PROPERTIES: - "symbol": UDF's symbol, which Doris call this symbol's function to execute. MUST BE SET - "object_file": UDF library's URL, Doris use it to download library. MUST BE SET - "md5": when this property is set, Doris will check library's md5um against this value. This is a option - -## example - 1. create a function "my_func", receive two int and return one int - CREATE FUNCTION my_func (int, int) RETURNS int - PROPERTIES ("symbol"="my_func_symbol", "object_file"="http://127.0.0.1/my_func.so") - 2. create a variadic function "my_func" - CREATE FUNCTION my_func (int, ...) RETURNS int - PROPERTIES ("symbol"="my_func_symbol", "object_file"="http://127.0.0.1/my_func.so") - -## keyword - CREATE, FUNCTION - -# DROP FUNCTION -## description - Used to drop a UDF/UDAF/UDTF - Syntax: - DROP FUNCTION funcName (argType [, ...]) - -## example - 1. drop a UDF whose name is my_func - DROP FUNCTION my_func (int, int) - 2. drop a variadic function - DROP FUNCTION my_func (int, ...) - -## keyword - DROP, FUNCTION diff --git a/docs/help/Contents/Data Manipulation/manipulation_stmt.md b/docs/help/Contents/Data Manipulation/manipulation_stmt.md deleted file mode 100644 index cc55e5f25602a9..00000000000000 --- a/docs/help/Contents/Data Manipulation/manipulation_stmt.md +++ /dev/null @@ -1,950 +0,0 @@ -# LOAD -## description - - Palo 目前支持以下4种导入方式: - - 1. Hadoop Load:基于 MR 进行 ETL 的导入。 - 2. Broker Load:使用 broker 进行进行数据导入。 - 3. Mini Load:通过 http 协议上传文件进行批量数据导入。 - 4. Stream Load:通过 http 协议进行流式数据导入。 - - 本帮助主要描述第一种导入方式,即 Hadoop Load 相关帮助信息。其余导入方式可以使用以下命令查看帮助: - - !!!该导入方式可能在后续某个版本即不再支持,建议使用其他导入方式进行数据导入。!!! - - 1. help broker load; - 2. help mini load; - 3. help stream load; - - Hadoop Load 仅适用于百度内部环境。公有云、私有云以及开源环境无法使用这种导入方式。 - 该导入方式必须设置用于 ETL 的 Hadoop 计算队列,设置方式可以通过 help set property 命令查看帮助。 - - Stream load 暂时只支持百度内部用户使用。开源社区和公有云用户将在后续版本更新中支持。 - -语法: - - LOAD LABEL load_label - ( - data_desc1[, data_desc2, ...] - ) - [opt_properties]; - - 1. load_label - - 当前导入批次的标签。在一个 database 内唯一。 - 语法: - [database_name.]your_label - - 2. data_desc - - 用于描述一批导入数据。 - 语法: - DATA INFILE - ( - "file_path1"[, file_path2, ...] - ) - [NEGATIVE] - INTO TABLE `table_name` - [PARTITION (p1, p2)] - [COLUMNS TERMINATED BY "column_separator"] - [FORMAT AS "file_type"] - [(column_list)] - [SET (k1 = func(k2))] - - 说明: - file_path: - - 文件路径,可以指定到一个文件,也可以用 * 通配符指定某个目录下的所有文件。通配符必须匹配到文件,而不能是目录。 - - PARTITION: - - 如果指定此参数,则只会导入指定的分区,导入分区以外的数据会被过滤掉。 - 如果不指定,默认导入table的所有分区。 - - NEGATIVE: - 如果指定此参数,则相当于导入一批“负”数据。用于抵消之前导入的同一批数据。 - 该参数仅适用于存在 value 列,并且 value 列的聚合类型仅为 SUM 的情况。 - - column_separator: - - 用于指定导入文件中的列分隔符。默认为 \t - 如果是不可见字符,则需要加\\x作为前缀,使用十六进制来表示分隔符。 - 如hive文件的分隔符\x01,指定为"\\x01" - - file_type: - - 用于指定导入文件的类型,例如:parquet、csv。默认值通过文件后缀名判断。 - - column_list: - - 用于指定导入文件中的列和 table 中的列的对应关系。 - 当需要跳过导入文件中的某一列时,将该列指定为 table 中不存在的列名即可。 - 语法: - (col_name1, col_name2, ...) - - SET: - - 如果指定此参数,可以将源文件某一列按照函数进行转化,然后将转化后的结果导入到table中。 - 目前支持的函数有: - - strftime(fmt, column) 日期转换函数 - fmt: 日期格式,形如%Y%m%d%H%M%S (年月日时分秒) - column: column_list中的列,即输入文件中的列。存储内容应为数字型的时间戳。 - 如果没有column_list,则按照palo表的列顺序默认输入文件的列。 - - time_format(output_fmt, input_fmt, column) 日期格式转化 - output_fmt: 转化后的日期格式,形如%Y%m%d%H%M%S (年月日时分秒) - input_fmt: 转化前column列的日期格式,形如%Y%m%d%H%M%S (年月日时分秒) - column: column_list中的列,即输入文件中的列。存储内容应为input_fmt格式的日期字符串。 - 如果没有column_list,则按照palo表的列顺序默认输入文件的列。 - - alignment_timestamp(precision, column) 将时间戳对齐到指定精度 - precision: year|month|day|hour - column: column_list中的列,即输入文件中的列。存储内容应为数字型的时间戳。 - 如果没有column_list,则按照palo表的列顺序默认输入文件的列。 - 注意:对齐精度为year、month的时候,只支持20050101~20191231范围内的时间戳。 - - default_value(value) 设置某一列导入的默认值 - 不指定则使用建表时列的默认值 - - md5sum(column1, column2, ...) 将指定的导入列的值求md5sum,返回32位16进制字符串 - - replace_value(old_value[, new_value]) 将导入文件中指定的old_value替换为new_value - new_value如不指定则使用建表时列的默认值 - - hll_hash(column) 用于将表或数据里面的某一列转化成HLL列的数据结构 - - 3. opt_properties - - 用于指定一些特殊参数。 - 语法: - [PROPERTIES ("key"="value", ...)] - - 可以指定如下参数: - cluster: 导入所使用的 Hadoop 计算队列。 - timeout: 指定导入操作的超时时间。默认超时为3天。单位秒。 - max_filter_ratio:最大容忍可过滤(数据不规范等原因)的数据比例。默认零容忍。 - load_delete_flag:指定该导入是否通过导入key列的方式删除数据,仅适用于UNIQUE KEY, - 导入时可不指定value列。默认为false。 - - 5. 导入数据格式样例 - - 整型类(TINYINT/SMALLINT/INT/BIGINT/LARGEINT):1, 1000, 1234 - 浮点类(FLOAT/DOUBLE/DECIMAL):1.1, 0.23, .356 - 日期类(DATE/DATETIME):2017-10-03, 2017-06-13 12:34:03。 - (注:如果是其他日期格式,可以在导入命令中,使用 strftime 或者 time_format 函数进行转换) - 字符串类(CHAR/VARCHAR):"I am a student", "a" - NULL值:\N - -## example - - 1. 导入一批数据,指定超时时间和过滤比例。指定导入队列为 my_cluster。 - - LOAD LABEL example_db.label1 - ( - DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/file") - INTO TABLE `my_table` - ) - PROPERTIES - ( - "cluster" = "my_cluster", - "timeout" = "3600", - "max_filter_ratio" = "0.1" - ); - - 其中 hdfs_host 为 namenode 的 host,hdfs_port 为 fs.defaultFS 端口(默认9000) - - 2. 导入一批数据,包含多个文件。导入不同的 table,指定分隔符,指定列对应关系 - - LOAD LABEL example_db.label2 - ( - DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/file1") - INTO TABLE `my_table_1` - COLUMNS TERMINATED BY "," - (k1, k3, k2, v1, v2), - DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/file2") - INTO TABLE `my_table_2` - COLUMNS TERMINATED BY "\t" - (k1, k2, k3, v2, v1) - ); - - 3. 导入一批数据,指定hive的默认分隔符\x01,并使用通配符*指定目录下的所有文件 - - LOAD LABEL example_db.label3 - ( - DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/*") - NEGATIVE - INTO TABLE `my_table` - COLUMNS TERMINATED BY "\\x01" - ); - - 4. 导入一批“负”数据 - - LOAD LABEL example_db.label4 - ( - DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/old_file) - NEGATIVE - INTO TABLE `my_table` - COLUMNS TERMINATED BY "\t" - ); - - 5. 导入一批数据,指定分区 - - LOAD LABEL example_db.label5 - ( - DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/file") - INTO TABLE `my_table` - PARTITION (p1, p2) - COLUMNS TERMINATED BY "," - (k1, k3, k2, v1, v2) - ); - - 6. 导入一批数据,指定分区, 并对导入文件的列做一些转化,如下: - 表结构为: - k1 datetime - k2 date - k3 bigint - k4 varchar(20) - k5 varchar(64) - k6 int - - 假设数据文件只有一行数据,5列,逗号分隔: - - 1537002087,2018-08-09 11:12:13,1537002087,-,1 - - 数据文件中各列,对应导入语句中指定的各列: - tmp_k1, tmp_k2, tmp_k3, k6, v1 - - 转换如下: - - 1) k1:将 tmp_k1 时间戳列转化为 datetime 类型的数据 - 2) k2:将 tmp_k2 datetime 类型的数据转化为 date 的数据 - 3) k3:将 tmp_k3 时间戳列转化为天级别时间戳 - 4) k4:指定导入默认值为1 - 5) k5:将 tmp_k1、tmp_k2、tmp_k3 列计算 md5 值 - 6) k6:将导入文件中的 - 值替换为 10 - - LOAD LABEL example_db.label6 - ( - DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/file") - INTO TABLE `my_table` - PARTITION (p1, p2) - COLUMNS TERMINATED BY "," - (tmp_k1, tmp_k2, tmp_k3, k6, v1) - SET ( - k1 = strftime("%Y-%m-%d %H:%M:%S", tmp_k1), - k2 = time_format("%Y-%m-%d %H:%M:%S", "%Y-%m-%d", tmp_k2), - k3 = alignment_timestamp("day", tmp_k3), - k4 = default_value("1"), - k5 = md5sum(tmp_k1, tmp_k2, tmp_k3), - k6 = replace_value("-", "10") - ) - ); - - 7. 导入数据到含有HLL列的表,可以是表中的列或者数据里面的列 - - LOAD LABEL example_db.label7 - ( - DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/file") - INTO TABLE `my_table` - PARTITION (p1, p2) - COLUMNS TERMINATED BY "," - SET ( - v1 = hll_hash(k1), - v2 = hll_hash(k2) - ) - ); - - LOAD LABEL example_db.label8 - ( - DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/file") - INTO TABLE `my_table` - PARTITION (p1, p2) - COLUMNS TERMINATED BY "," - (k1, k2, tmp_k3, tmp_k4, v1, v2) - SET ( - v1 = hll_hash(tmp_k3), - v2 = hll_hash(tmp_k4) - ) - ) - WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); - - 8. 导入Parquet文件中数据 指定FORMAT 为parquet, 默认是通过文件后缀判断 - LOAD LABEL example_db.label9 - ( - DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/file") - INTO TABLE `my_table` - FORMAT AS "parquet" - (k1, k2, k3) - ) - WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); - -## keyword - LOAD - -# CANCEL LOAD -## description - - 该语句用于撤销指定 load label 的批次的导入作业。 - 这是一个异步操作,任务提交成功则返回。执行后可使用 SHOW LOAD 命令查看进度。 - 语法: - CANCEL LOAD - [FROM db_name] - WHERE LABEL = "load_label"; - -## example - - 1. 撤销数据库 example_db 上, label 为 example_db_test_load_label 的导入作业 - CANCEL LOAD - FROM example_db - WHERE LABEL = "example_db_test_load_label"; - -## keyword - CANCEL,LOAD - -# DELETE -## description - - 该语句用于按条件删除指定 table(base index) partition 中的数据。 - 该操作会同时删除和此 base index 相关的 rollup index 的数据。 - 语法: - DELETE FROM table_name [PARTITION partition_name] - WHERE - column_name1 op value[ AND column_name2 op value ...]; - - 说明: - 1) op 的可选类型包括:=, >, <, >=, <=, != - 2) 只能指定 key 列上的条件。 - 2) 当选定的 key 列不存在于某个 rollup 中时,无法进行 delete。 - 3) 条件之间只能是“与”的关系。 - 若希望达成“或”的关系,需要将条件分写在两个 DELETE 语句中。 - 4) 如果为RANGE分区表,则必须指定 PARTITION。如果是单分区表,可以不指定。 - - 注意: - 该语句可能会降低执行后一段时间内的查询效率。 - 影响程度取决于语句中指定的删除条件的数量。 - 指定的条件越多,影响越大。 - -## example - - 1. 删除 my_table partition p1 中 k1 列值为 3 的数据行 - DELETE FROM my_table PARTITION p1 - WHERE k1 = 3; - - 2. 删除 my_table partition p1 中 k1 列值大于等于 3 且 k2 列值为 "abc" 的数据行 - DELETE FROM my_table PARTITION p1 - WHERE k1 >= 3 AND k2 = "abc"; - -## keyword - DELETE - -# CANCEL DELETE -## description - - 该语句用于撤销一个 DELETE 操作。(仅管理员使用!)(待实现) - -## example - -## keyword - CANCEL,DELETE - -# MINI LOAD -## description - - MINI LOAD 和 STREAM LOAD 的导入实现方式完全一致。在导入功能支持上,MINI LOAD 的功能是 STREAM LOAD 的子集。 - 后续的导入新功能只会在 STREAM LOAD 中支持,MINI LOAD 将不再新增功能。建议改用 STREAM LOAD,具体使用方式请 HELP STREAM LOAD。 - - MINI LOAD 是 通过 http 协议完成的导入方式。用户可以不依赖 Hadoop,也无需通过 Mysql 客户端,即可完成导入。 - 用户通过 http 协议描述导入,数据在接受 http 请求的过程中被流式的导入 Doris , **导入作业完成后** 返回给用户导入的结果。 - - * 注:为兼容旧版本 mini load 使用习惯,用户依旧可以通过 'SHOW LOAD' 命令来查看导入结果。 - - 语法: - 导入: - - curl --location-trusted -u user:passwd -T data.file http://host:port/api/{db}/{table}/_load?label=xxx - - 查看导入信息 - - curl -u user:passwd http://host:port/api/{db}/_load_info?label=xxx - - HTTP协议相关说明 - - 权限认证 当前 Doris 使用 http 的 Basic 方式权限认证。所以在导入的时候需要指定用户名密码 - 这种方式是明文传递密码的,暂不支持加密传输。 - - Expect Doris 需要发送过来的 http 请求带有 'Expect' 头部信息,内容为 '100-continue'。 - 为什么呢?因为我们需要将请求进行 redirect,那么必须在传输数据内容之前, - 这样可以避免造成数据的多次传输,从而提高效率。 - - Content-Length Doris 需要在发送请求时带有 'Content-Length' 这个头部信息。如果发送的内容比 - 'Content-Length' 要少,那么 Doris 认为传输出现问题,则提交此次任务失败。 - NOTE: 如果,发送的数据比 'Content-Length' 要多,那么 Doris 只读取 'Content-Length' - 长度的内容,并进行导入 - - - 参数说明: - - user: 用户如果是在default_cluster中的,user即为user_name。否则为user_name@cluster_name。 - - label: 用于指定这一批次导入的 label,用于后期进行作业查询等。 - 这个参数是必须传入的。 - - columns: 用于描述导入文件中对应的列名字。 - 如果不传入,那么认为文件中的列顺序与建表的顺序一致, - 指定的方式为逗号分隔,例如:columns=k1,k2,k3,k4 - - column_separator: 用于指定列与列之间的分隔符,默认的为'\t' - NOTE: 需要进行url编码,譬如 - 需要指定'\t'为分隔符,那么应该传入'column_separator=%09' - 需要指定'\x01'为分隔符,那么应该传入'column_separator=%01' - 需要指定','为分隔符,那么应该传入'column_separator=%2c' - - - max_filter_ratio: 用于指定允许过滤不规范数据的最大比例,默认是0,不允许过滤 - 自定义指定应该如下:'max_filter_ratio=0.2',含义是允许20%的错误率 - - timeout: 指定 load 作业的超时时间,单位是秒。当load执行时间超过该阈值时,会自动取消。默认超时时间是 86400 秒。 - 建议指定 timeout 时间小于 86400 秒。 - - hll: 用于指定数据里面和表里面的HLL列的对应关系,表中的列和数据里面指定的列 - (如果不指定columns,则数据列面的列也可以是表里面的其它非HLL列)通过","分割 - 指定多个hll列使用“:”分割,例如: 'hll1,cuid:hll2,device' - - NOTE: - 1. 此种导入方式当前是在一台机器上完成导入工作,因而不宜进行数据量较大的导入工作。 - 建议导入数据量不要超过 1 GB - - 2. 当前无法使用 `curl -T "{file1, file2}"` 这样的方式提交多个文件,因为curl是将其拆成多个 - 请求发送的,多个请求不能共用一个label号,所以无法使用 - - 3. mini load 的导入方式和 streaming 完全一致,都是在流式的完成导入后,同步的返回结果给用户。 - 后续查询虽可以查到 mini load 的信息,但不能对其进行操作,查询只为兼容旧的使用方式。 - - 4. 当使用 curl 命令行导入时,需要在 & 前加入 \ 转义,否则参数信息会丢失。 - -## example - - 1. 将本地文件'testData'中的数据导入到数据库'testDb'中'testTbl'的表(用户是defalut_cluster中的) - curl --location-trusted -u root -T testData http://host:port/api/testDb/testTbl/_load?label=123 - - 2. 将本地文件'testData'中的数据导入到数据库'testDb'中'testTbl'的表(用户是test_cluster中的)。超时时间是 3600 秒 - curl --location-trusted -u root@test_cluster:root -T testData http://fe.host:port/api/testDb/testTbl/_load?label=123&timeout=3600 - - 3. 将本地文件'testData'中的数据导入到数据库'testDb'中'testTbl'的表, 允许20%的错误率(用户是defalut_cluster中的) - curl --location-trusted -u root -T testData http://host:port/api/testDb/testTbl/_load?label=123\&max_filter_ratio=0.2 - - 4. 将本地文件'testData'中的数据导入到数据库'testDb'中'testTbl'的表, 允许20%的错误率,并且指定文件的列名(用户是defalut_cluster中的) - curl --location-trusted -u root -T testData http://host:port/api/testDb/testTbl/_load?label=123\&max_filter_ratio=0.2\&columns=k1,k2,k3 - - 5. 使用streaming方式导入(用户是defalut_cluster中的) - seq 1 10 | awk '{OFS="\t"}{print $1, $1 * 10}' | curl --location-trusted -u root -T - http://host:port/api/testDb/testTbl/_load?label=123 - - 6. 导入含有HLL列的表,可以是表中的列或者数据中的列用于生成HLL列(用户是defalut_cluster中的 - - curl --location-trusted -u root -T testData http://host:port/api/testDb/testTbl/_load?label=123\&max_filter_ratio=0.2 - \&hll=hll_column1,tmp_k4:hll_column2,tmp_k5\&columns=k1,k2,k3,tmp_k4,tmp_k5 - - 7. 查看提交后的导入情况 - - curl -u root http://host:port/api/testDb/_load_info?label=123 - -## keyword - MINI, LOAD - -# MULTI LOAD -## description - - Syntax: - curl --location-trusted -u user:passwd -XPOST http://host:port/api/{db}/_multi_start?label=xxx - curl --location-trusted -u user:passwd -T data.file http://host:port/api/{db}/{table1}/_load?label=xxx\&sub_label=yyy - curl --location-trusted -u user:passwd -T data.file http://host:port/api/{db}/{table2}/_load?label=xxx\&sub_label=zzz - curl --location-trusted -u user:passwd -XPOST http://host:port/api/{db}/_multi_commit?label=xxx - curl --location-trusted -u user:passwd -XPOST http://host:port/api/{db}/_multi_desc?label=xxx - - 'MULTI LOAD'在'MINI LOAD'的基础上,可以支持用户同时向多个表进行导入,具体的命令如上面所示 - '/api/{db}/_multi_start' 开始一个多表导入任务 - '/api/{db}/{table}/_load' 向一个导入任务添加一个要导入的表,与'MINI LOAD'的主要区别是,需要传入'sub_label'参数 - '/api/{db}/_multi_commit' 提交整个多表导入任务,后台开始进行处理 - '/api/{db}/_multi_abort' 放弃一个多表导入任务 - '/api/{db}/_multi_desc' 可以展示某个多表导入任务已经提交的作业数 - - HTTP协议相关说明 - 权限认证 当前 Doris 使用http的Basic方式权限认证。所以在导入的时候需要指定用户名密码 - 这种方式是明文传递密码的,鉴于我们当前都是内网环境。。。 - - Expect Doris 需要发送过来的http请求,需要有'Expect'头部信息,内容为'100-continue' - 为什么呢?因为我们需要将请求进行redirect,那么必须在传输数据内容之前, - 这样可以避免造成数据的多次传输,从而提高效率。 - - Content-Length Doris 需要在发送请求是带有'Content-Length'这个头部信息。如果发送的内容比 - 'Content-Length'要少,那么Palo认为传输出现问题,则提交此次任务失败。 - NOTE: 如果,发送的数据比'Content-Length'要多,那么 Doris 只读取'Content-Length' - 长度的内容,并进行导入 - - 参数说明: - user: 用户如果是在default_cluster中的,user即为user_name。否则为user_name@cluster_name。 - - label: 用于指定这一批次导入的label号,用于后期进行作业状态查询等。 - 这个参数是必须传入的。 - - sub_label: 用于指定一个多表导入任务内部的子版本号。对于多表导入的load, 这个参数是必须传入的。 - - columns: 用于描述导入文件中对应的列名字。 - 如果不传入,那么认为文件中的列顺序与建表的顺序一致, - 指定的方式为逗号分隔,例如:columns=k1,k2,k3,k4 - - column_separator: 用于指定列与列之间的分隔符,默认的为'\t' - NOTE: 需要进行url编码,譬如需要指定'\t'为分隔符, - 那么应该传入'column_separator=%09' - - max_filter_ratio: 用于指定允许过滤不规范数据的最大比例,默认是0,不允许过滤 - 自定义指定应该如下:'max_filter_ratio=0.2',含义是允许20%的错误率 - 在'_multi_start'时传入有效果 - - NOTE: - 1. 此种导入方式当前是在一台机器上完成导入工作,因而不宜进行数据量较大的导入工作。 - 建议导入数据量不要超过1GB - - 2. 当前无法使用`curl -T "{file1, file2}"`这样的方式提交多个文件,因为curl是将其拆成多个 - 请求发送的,多个请求不能共用一个label号,所以无法使用 - - 3. 支持类似streaming的方式使用curl来向 Doris 中导入数据,但是,只有等这个streaming结束后 Doris - 才会发生真实的导入行为,这中方式数据量也不能过大。 - -## example - - 1. 将本地文件'testData1'中的数据导入到数据库'testDb'中'testTbl1'的表,并且 - 把'testData2'的数据导入到'testDb'中的表'testTbl2'(用户是defalut_cluster中的) - curl --location-trusted -u root -XPOST http://host:port/api/testDb/_multi_start?label=123 - curl --location-trusted -u root -T testData1 http://host:port/api/testDb/testTbl1/_load?label=123\&sub_label=1 - curl --location-trusted -u root -T testData2 http://host:port/api/testDb/testTbl2/_load?label=123\&sub_label=2 - curl --location-trusted -u root -XPOST http://host:port/api/testDb/_multi_commit?label=123 - - 2. 多表导入中途放弃(用户是defalut_cluster中的) - curl --location-trusted -u root -XPOST http://host:port/api/testDb/_multi_start?label=123 - curl --location-trusted -u root -T testData1 http://host:port/api/testDb/testTbl1/_load?label=123\&sub_label=1 - curl --location-trusted -u root -XPOST http://host:port/api/testDb/_multi_abort?label=123 - - 3. 多表导入查看已经提交多少内容(用户是defalut_cluster中的) - curl --location-trusted -u root -XPOST http://host:port/api/testDb/_multi_start?label=123 - curl --location-trusted -u root -T testData1 http://host:port/api/testDb/testTbl1/_load?label=123\&sub_label=1 - curl --location-trusted -u root -XPOST http://host:port/api/testDb/_multi_desc?label=123 - -## keyword - MULTI, MINI, LOAD - -# EXPORT -## description - - 该语句用于将指定表的数据导出到指定位置。 - 该功能通过 broker 进程实现。对于不同的目的存储系统,需要部署不同的 broker。可以通过 SHOW BROKER 查看已部署的 broker。 - 这是一个异步操作,任务提交成功则返回。执行后可使用 SHOW EXPORT 命令查看进度。 - - 语法: - EXPORT TABLE table_name - [PARTITION (p1[,p2])] - TO export_path - [opt_properties] - broker; - - 1. table_name - 当前要导出的表的表名,目前支持engine为olap和mysql的表的导出。 - - 2. partition - 可以只导出指定表的某些指定分区 - - 3. export_path - 导出的路径,需为目录。目前不能导出到本地,需要导出到broker。 - - 4. opt_properties - 用于指定一些特殊参数。 - 语法: - [PROPERTIES ("key"="value", ...)] - - 可以指定如下参数: - column_separator: 指定导出的列分隔符,默认为\t。 - line_delimiter: 指定导出的行分隔符,默认为\n。 - exec_mem_limit: 导出在单个 BE 节点的内存使用上限,默认为 2GB,单位为字节。 - timeout:导入作业的超时时间,默认为1天,单位是秒。 - tablet_num_per_task:每个子任务能分配的最大 Tablet 数量。 - - 5. broker - 用于指定导出使用的broker - 语法: - WITH BROKER broker_name ("key"="value"[,...]) - 这里需要指定具体的broker name, 以及所需的broker属性 - - 对于不同存储系统对应的 broker,这里需要输入的参数不同。具体参数可以参阅:`help broker load` 中 broker 所需属性。 - -## example - - 1. 将 testTbl 表中的所有数据导出到 hdfs 上 - EXPORT TABLE testTbl TO "hdfs://hdfs_host:port/a/b/c" WITH BROKER "broker_name" ("username"="xxx", "password"="yyy"); - - 2. 将 testTbl 表中的分区p1,p2导出到 hdfs 上 - - EXPORT TABLE testTbl PARTITION (p1,p2) TO "hdfs://hdfs_host:port/a/b/c" WITH BROKER "broker_name" ("username"="xxx", "password"="yyy"); - 3. 将 testTbl 表中的所有数据导出到 hdfs 上,以","作为列分隔符 - - EXPORT TABLE testTbl TO "hdfs://hdfs_host:port/a/b/c" PROPERTIES ("column_separator"=",") WITH BROKER "broker_name" ("username"="xxx", "password"="yyy"); - -## keyword - EXPORT - -# SHOW DATABASES -## description - 该语句用于展示当前可见的 db - 语法: - SHOW DATABASES; - -## keyword - SHOW,DATABASES - -# SHOW TABLES -## description - 该语句用于展示当前 db 下所有的 table - 语法: - SHOW TABLES; - -## keyword - SHOW,TABLES - -# SHOW LOAD -## description - 该语句用于展示指定的导入任务的执行情况 - 语法: - SHOW LOAD - [FROM db_name] - [ - WHERE - [LABEL [ = "your_label" | LIKE "label_matcher"]] - [STATE = ["PENDING"|"ETL"|"LOADING"|"FINISHED"|"CANCELLED"|]] - ] - [ORDER BY ...] - [LIMIT limit][OFFSET offset]; - - 说明: - 1) 如果不指定 db_name,使用当前默认db - 2) 如果使用 LABEL LIKE,则会匹配导入任务的 label 包含 label_matcher 的导入任务 - 3) 如果使用 LABEL = ,则精确匹配指定的 label - 4) 如果指定了 STATE,则匹配 LOAD 状态 - 5) 可以使用 ORDER BY 对任意列组合进行排序 - 6) 如果指定了 LIMIT,则显示 limit 条匹配记录。否则全部显示 - 7) 如果指定了 OFFSET,则从偏移量offset开始显示查询结果。默认情况下偏移量为0。 - 8) 如果是使用 broker/mini load,则 URL 列中的连接可以使用以下命令查看: - - SHOW LOAD WARNINGS ON 'url' - -## example - 1. 展示默认 db 的所有导入任务 - SHOW LOAD; - - 2. 展示指定 db 的导入任务,label 中包含字符串 "2014_01_02",展示最老的10个 - SHOW LOAD FROM example_db WHERE LABEL LIKE "2014_01_02" LIMIT 10; - - 3. 展示指定 db 的导入任务,指定 label 为 "load_example_db_20140102" 并按 LoadStartTime 降序排序 - SHOW LOAD FROM example_db WHERE LABEL = "load_example_db_20140102" ORDER BY LoadStartTime DESC; - - 4. 展示指定 db 的导入任务,指定 label 为 "load_example_db_20140102" ,state 为 "loading", 并按 LoadStartTime 降序排序 - SHOW LOAD FROM example_db WHERE LABEL = "load_example_db_20140102" AND STATE = "loading" ORDER BY LoadStartTime DESC; - - 5. 展示指定 db 的导入任务 并按 LoadStartTime 降序排序,并从偏移量5开始显示10条查询结果 - SHOW LOAD FROM example_db ORDER BY LoadStartTime DESC limit 5,10; - SHOW LOAD FROM example_db ORDER BY LoadStartTime DESC limit 10 offset 5; - - 6. 小批量导入是查看导入状态的命令 - curl --location-trusted -u {user}:{passwd} http://{hostname}:{port}/api/{database}/_load_info?label={labelname} - -## keyword - SHOW,LOAD - -# SHOW EXPORT -## description - 该语句用于展示指定的导出任务的执行情况 - 语法: - SHOW EXPORT - [FROM db_name] - [ - WHERE - [EXPORT_JOB_ID = your_job_id] - [STATE = ["PENDING"|"EXPORTING"|"FINISHED"|"CANCELLED"]] - ] - [ORDER BY ...] - [LIMIT limit]; - - 说明: - 1) 如果不指定 db_name,使用当前默认db - 2) 如果指定了 STATE,则匹配 EXPORT 状态 - 3) 可以使用 ORDER BY 对任意列组合进行排序 - 4) 如果指定了 LIMIT,则显示 limit 条匹配记录。否则全部显示 - -## example - 1. 展示默认 db 的所有导出任务 - SHOW EXPORT; - - 2. 展示指定 db 的导出任务,按 StartTime 降序排序 - SHOW EXPORT FROM example_db ORDER BY StartTime DESC; - - 3. 展示指定 db 的导出任务,state 为 "exporting", 并按 StartTime 降序排序 - SHOW EXPORT FROM example_db WHERE STATE = "exporting" ORDER BY StartTime DESC; - - 4. 展示指定db,指定job_id的导出任务 - SHOW EXPORT FROM example_db WHERE EXPORT_JOB_ID = job_id; - -## keyword - SHOW,EXPORT - -# SHOW DELETE -## description - 该语句用于展示已执行成功的历史 delete 任务 - 语法: - SHOW DELETE [FROM db_name] - -## example - 1. 展示数据库 database 的所有历史 delete 任务 - SHOW DELETE FROM database; - -## keyword - SHOW,DELETE - -# SHOW ALTER -## description - 该语句用于展示当前正在进行的各类修改任务的执行情况 - 语法: - SHOW ALTER [CLUSTER | TABLE [COLUMN | ROLLUP] [FROM db_name]]; - - 说明: - TABLE COLUMN:展示修改列的 ALTER 任务 - TABLE ROLLUP:展示创建或删除 ROLLUP index 的任务 - 如果不指定 db_name,使用当前默认 db - CLUSTER: 展示集群操作相关任务情况(仅管理员使用!待实现...) - -## example - 1. 展示默认 db 的所有修改列的任务执行情况 - SHOW ALTER TABLE COLUMN; - - 2. 展示指定 db 的创建或删除 ROLLUP index 的任务执行情况 - SHOW ALTER TABLE ROLLUP FROM example_db; - - 3. 展示集群操作相关任务(仅管理员使用!待实现...) - SHOW ALTER CLUSTER; - -## keyword - SHOW,ALTER - -# SHOW DATA -## description - 该语句用于展示数据量 - 语法: - SHOW DATA [FROM db_name[.table_name]]; - - 说明: - 1. 如果不指定 FROM 子句,使用展示当前 db 下细分到各个 table 的数据量 - 2. 如果指定 FROM 子句,则展示 table 下细分到各个 index 的数据量 - 3. 如果想查看各个 Partition 的大小,请参阅 help show partitions - -## example - 1. 展示默认 db 的各个 table 的数据量及汇总数据量 - SHOW DATA; - - 2. 展示指定 db 的下指定表的细分数据量 - SHOW DATA FROM example_db.table_name; - -## keyword - SHOW,DATA - -# SHOW PARTITIONS -## description - 该语句用于展示分区信息 - 语法: - SHOW PARTITIONS FROM [db_name.]table_name [PARTITION partition_name]; - -## example - 1. 展示指定 db 的下指定表的分区信息 - SHOW PARTITIONS FROM example_db.table_name; - - 1. 展示指定 db 的下指定表的指定分区的信息 - SHOW PARTITIONS FROM example_db.table_name PARTITION p1; - -## keyword - SHOW,PARTITIONS - -# SHOW TABLET -## description - 该语句用于显示 tablet 相关的信息(仅管理员使用) - 语法: - SHOW TABLET - [FROM [db_name.]table_name | tablet_id] - -## example - 1. 显示指定 db 的下指定表所有 tablet 信息 - SHOW TABLET FROM example_db.table_name; - - 2. 显示指定 tablet id 为 10000 的 tablet 的父层级 id 信息 - SHOW TABLET 10000; - -## keyword - SHOW,TABLET - -# SHOW PROPERTY -## description - 该语句用于查看用户的属性 - 语法: - SHOW PROPERTY [FOR user] [LIKE key] - -## example - 1. 查看 jack 用户的属性 - SHOW PROPERTY FOR 'jack' - - 2. 查看 jack 用户导入cluster相关属性 - SHOW PROPERTY FOR 'jack' LIKE '%load_cluster%' - -## keyword - SHOW, PROPERTY - -# SHOW BACKUP -## description - 该语句用于查看 BACKUP 任务 - 语法: - SHOW BACKUP [FROM db_name] - - 说明: - 1. Palo 中仅保存最近一次 BACKUP 任务。 - 2. 各列含义如下: - JobId: 唯一作业id - SnapshotName: 备份的名称 - DbName: 所属数据库 - State: 当前阶段 - PENDING: 提交作业后的初始状态 - SNAPSHOTING: 执行快照中 - UPLOAD_SNAPSHOT:快照完成,准备上传 - UPLOADING: 快照上传中 - SAVE_META: 将作业元信息保存为本地文件 - UPLOAD_INFO: 上传作业元信息 - FINISHED: 作业成功 - CANCELLED: 作业失败 - BackupObjs: 备份的表和分区 - CreateTime: 任务提交时间 - SnapshotFinishedTime: 快照完成时间 - UploadFinishedTime: 快照上传完成时间 - FinishedTime: 作业结束时间 - UnfinishedTasks: 在 SNAPSHOTING 和 UPLOADING 阶段会显示还未完成的子任务id - Status: 如果作业失败,显示失败信息 - Timeout: 作业超时时间,单位秒 - -## example - 1. 查看 example_db 下最后一次 BACKUP 任务。 - SHOW BACKUP FROM example_db; - -## keyword - SHOW, BACKUP - -# SHOW RESTORE -## description - 该语句用于查看 RESTORE 任务 - 语法: - SHOW RESTORE [FROM db_name] - - 说明: - 1. Palo 中仅保存最近一次 RESTORE 任务。 - 2. 各列含义如下: - JobId: 唯一作业id - Label: 要恢复的备份的名称 - Timestamp: 要恢复的备份的时间版本 - DbName: 所属数据库 - State: 当前阶段 - PENDING: 提交作业后的初始状态 - SNAPSHOTING: 执行快照中 - DOWNLOAD: 快照完成,准备下载仓库中的快照 - DOWNLOADING: 快照下载中 - COMMIT: 快照下载完成,准备生效 - COMMITING: 生效中 - FINISHED: 作业成功 - CANCELLED: 作业失败 - AllowLoad: 恢复时是否允许导入(当前不支持) - ReplicationNum: 指定恢复的副本数 - RestoreJobs: 要恢复的表和分区 - CreateTime: 任务提交时间 - MetaPreparedTime: 元数据准备完成时间 - SnapshotFinishedTime: 快照完成时间 - DownloadFinishedTime: 快照下载完成时间 - FinishedTime: 作业结束时间 - UnfinishedTasks: 在 SNAPSHOTING、DOWNLOADING 和 COMMITING 阶段会显示还未完成的子任务id - Status: 如果作业失败,显示失败信息 - Timeout: 作业超时时间,单位秒 - -## example - 1. 查看 example_db 下最近一次 RESTORE 任务。 - SHOW RESTORE FROM example_db; - -## keyword - SHOW, RESTORE - -# SHOW REPOSITORIES -## description - 该语句用于查看当前已创建的仓库。 - 语法: - SHOW REPOSITORIES; - - 说明: - 1. 各列含义如下: - RepoId: 唯一的仓库ID - RepoName: 仓库名称 - CreateTime: 第一次创建该仓库的时间 - IsReadOnly: 是否为只读仓库 - Location: 仓库中用于备份数据的根目录 - Broker: 依赖的 Broker - ErrMsg: Palo 会定期检查仓库的连通性,如果出现问题,这里会显示错误信息 - -## example - 1. 查看已创建的仓库: - SHOW REPOSITORIES; - -## keyword - SHOW, REPOSITORY, REPOSITORIES - -# SHOW SNAPSHOT -## description - 该语句用于查看仓库中已存在的备份。 - 语法: - SHOW SNAPSHOT ON `repo_name` - [WHERE SNAPSHOT = "snapshot" [AND TIMESTAMP = "backup_timestamp"]]; - - 说明: - 1. 各列含义如下: - Snapshot: 备份的名称 - Timestamp: 对应备份的时间版本 - Status: 如果备份正常,则显示 OK,否则显示错误信息 - - 2. 如果指定了 TIMESTAMP,则会额外显示如下信息: - Database: 备份数据原属的数据库名称 - Details: 以 Json 的形式,展示整个备份的数据目录及文件结构 - -## example - 1. 查看仓库 example_repo 中已有的备份: - SHOW SNAPSHOT ON example_repo; - - 2. 仅查看仓库 example_repo 中名称为 backup1 的备份: - SHOW SNAPSHOT ON example_repo WHERE SNAPSHOT = "backup1"; - - 2. 查看仓库 example_repo 中名称为 backup1 的备份,时间版本为 "2018-05-05-15-34-26" 的详细信息: - SHOW SNAPSHOT ON example_repo - WHERE SNAPSHOT = "backup1" AND TIMESTAMP = "2018-05-05-15-34-26"; - -## keyword - SHOW, SNAPSHOT - -# RESTORE TABLET -## description - - 该功能用于恢复trash目录中被误删的tablet数据。 - - 说明:这个功能暂时只在be服务中提供一个http接口。如果要使用, - 需要向要进行数据恢复的那台be机器的http端口发送restore tablet api请求。api格式如下: - METHOD: POST - URI: http://be_host:be_http_port/api/restore_tablet?tablet_id=xxx&schema_hash=xxx - -## example - - curl -X POST "http://hostname:8088/api/restore_tablet?tablet_id=123456&schema_hash=1111111" diff --git a/docs/help/Contents/Data Types/help.md b/docs/help/Contents/Data Types/help.md deleted file mode 100644 index b9f9351d021db0..00000000000000 --- a/docs/help/Contents/Data Types/help.md +++ /dev/null @@ -1,77 +0,0 @@ -# BOOLEAN -## description - BOOL, BOOLEN - 与TINYINT一样,0代表false,1代表true - -# TINYINT -## description - TINYINT - 1字节有符号整数,范围[-128, 127] - -# SMALLINT -## description - SMALLINT - 2字节有符号整数,范围[-32768, 32767] - -# INT -## description - INT - 4字节有符号整数,范围[-2147483648, 2147483647] - -# BIGINT -## description - BIGINT - 8字节有符号整数,范围[-9223372036854775808, 9223372036854775807] - -# FLOAT -## description - FLOAT - 4字节浮点数 - -# DOUBLE -## description - DOUBLE - 8字节浮点数 - -# DECIMAL -## description - DECIMAL(M[,D]) - 高精度定点数,M代表一共有多少个有效数字(precision),D代表小数点后最多有多少数字(scale) - M的范围是[1,27], D的范围[1, 9], 另外,M必须要大于等于D的取值。默认的D取值为0 - -# CHAR -## description - CHAR(M) - 定长字符串,M代表的是定长字符串的长度。M的范围是1-255 - -# VARCHAR -## description - VARCHAR(M) - 变长字符串,M代表的是变长字符串的长度。M的范围是1-65535 - -# DATE -## description - DATE函数 - Syntax: - DATE(expr) - 将输入的类型转化为DATE类型 - DATE类型 - 日期类型,目前的取值范围是['1900-01-01', '9999-12-31'], 默认的打印形式是'YYYY-MM-DD' - -## example - mysql> SELECT DATE('2003-12-31 01:02:03'); - -> '2003-12-31' - -# DATETIME -## description - DATETIME - 日期时间类型,取值范围是['1000-01-01 00:00:00', '9999-12-31 23:59:59']. - 打印的形式是'YYYY-MM-DD HH:MM:SS' - -# HLL(HyperLogLog) -## description - VARCHAR(M) - 变长字符串,M代表的是变长字符串的长度。M的范围是1-16385 - 用户不需要指定长度和默认值。长度根据数据的聚合程度系统内控制 - 并且HLL列只能通过配套的hll_union_agg、hll_raw_agg、hll_cardinality、hll_hash进行查询或使用 - diff --git a/docs/help/Contents/Functions/Date and Time Functions/help.md b/docs/help/Contents/Functions/Date and Time Functions/help.md deleted file mode 100644 index dedb8ca2616e91..00000000000000 --- a/docs/help/Contents/Functions/Date and Time Functions/help.md +++ /dev/null @@ -1,310 +0,0 @@ -# unix_timestamp -## description -Syntax: -UNIX_TIMESTAMP(), UNIX_TIMESTAMP(date) - -将Date或者Datetime类型转化为unix时间戳 -如果没有参数,则是将当前的时间转化为时间戳 -参数需要是Date或者Datetime类型 - -## example -mysql> SELECT UNIX_TIMESTAMP(); - -> 1196440210 -mysql> SELECT UNIX_TIMESTAMP('2007-11-30 10:30:19'); - -> 1196418619 - -# from_unixtime -## description -Syntax: - FROM_UNIXTIME(int unix_timestamp[, string string_format]) - -将unix时间戳转化位对应的time格式,返回的格式由string_format指定 -默认为yyyy-MM-dd HH:mm:ss -传入的是整形,返回的是字符串类型 -目前string_format只支持两种类型的格式:yyyy-MM-dd,yyyy-MM-dd HH:mm:ss -其余string_format格式是非法的,返回NULL - -## example -mysql> SELECT FROM_UNIXTIME(1196440219); - -> '2007-12-01 00:30:19' - -mysql> SELECT FROM_UNIXTIME(1196440219, 'yyyy-MM-dd'); - -> '2007-12-01' - -mysql> SELECT FROM_UNIXTIME(1196440219, 'yyyy-MM-dd HH:mm:ss'); - -> '2007-12-01 00:30:19' - -# year -## description -Syntax: -YEAR(date) - -返回date类型的year部分,范围从1000-9999 -参数为Date或者Datetime类型 -## example -mysql> SELECT YEAR('1987-01-01'); - -> 1987 - -# month -## description -Syntax: -MONTH(date) - -返回时间类型中的月份信息,范围是1, 12 -参数为Date或者Datetime类型 - -## example -mysql> SELECT MONTH('1987-01-02'); - -> 01 - -# day -## description -Syntax: -DAY(date) - -与DAYOFMONTH是同义词,请`help dayofmonth` - -# dayofmonth -## description -Syntax: -DAYOFMONTH(date) - -获得日期中的天信息,返回值范围从1-31。 -需要传入date类型 - -## example -mysql> SELECT DAYOFMONTH('1987-01-02'); - -> 2 - -# dayofyear -## description -Syntax: -DAYOFYEAR(date) - -获得日期中对应当年中的哪一天。 -输入值为date类型 - -## example -mysql> SELECT DAYOFYEAR('2007-02-03'); - -> 34 - -# weekofyear -## description -Syntax: -WEEKOFYEAR(date) - -获得一年中的第几周 -输入值为date类型 - -## example -mysql> SELECT WEEKOFYEAR('2008-02-20'); - -> 8 - -# hour -## description -Syntax: -HOUR(date) - -获得时间中对应的小时信息 -这里Palo跟MySQL不太一样,因为MySQL是支持Time类型的 -Palo没有Time类型,所以输入的内容是Date或者Datetime。 - -## example -mysql> select hour("2000-01-02 12:34:56"); - -> 12 - -# minute -## description -Syntax: -MINUTE(date) - -获得日期中的分钟信息 -这里Palo跟MySQL不太一样,因为MySQL是支持Time类型的 -Palo没有Time类型,所以输入的内容是Date或者Datetime。 - -## example -mysql> SELECT MINUTE("2000-01-02 12:34:56"); - -> 34 - -# second -## description -Syntax: -SECOND(date) - -获得时间中的秒信息 -这里Palo跟MySQL不太一样,因为MySQL是支持Time类型的 -Palo没有Time类型,所以输入的内容是Date或者Datetime。 -## example -mysql> SELECT SECOND("2000-01-02 12:34:56"); - -> 56 - -# now -## description -Syntax: -NOW() - -获得当前的时间,以Datetime类型返回 - -## example -mysql> SELECT NOW(); - -> '2007-12-15 23:50:26' - -# current_timestamp -## description -Syntax: -CURRENT_TIMESTAMP() - -与NOW()是同义词 - -# datediff -## description -Syntax: -DATEDIFF(expr1,expr2) - -计算expr1 - expr2,结果精确到天。 -要求传入的两个值需要是datetime类型。 - -## example -mysql> SELECT DATEDIFF(CAST ('2007-12-31 23:59:59' AS DATETIME),CAST ('2007-12-30' AS DATETIME)); - -> 1 -mysql> SELECT DATEDIFF(CAST('2010-11-30 23:59:59' AS DATETIME), CAST('2010-12-31' AS DATETIME)); - -> -31 - -# date_add -## description -Syntax: -DATE_ADD(date,INTERVAL expr unit) - -对时间类型进行加法运算 -支持的time unit包括 -YEAR, MONTH, DAY, HOUR, MINUTE, SECOND - -## example -mysql> DATE_ADD(date,INTERVAL expr unit) - -> 1987-01-02 00:00:00 - -# date_sub -## description -Syntax: -DATE_SUB(date,INTERVAL expr unit) - -与DATE_ADD相反,对时间类型进行减法运算 -支持的time unit包括 -YEAR, MONTH, DAY, HOUR, MINUTE, SECOND - -## example -mysql> DATE_SUB(date,INTERVAL expr unit) - -> 1986-12-31 00:00:00 - -# date_format -## description -Syntax: -DATE_FORMAT(date, format) - -将日期类型按照format的类型转化位字符串, -当前支持最大128字节的字符串,如果返回值长度超过128,则返回NULL -format的含义如下: -%a Abbreviated weekday name (Sun..Sat) -%b Abbreviated month name (Jan..Dec) -%c Month, numeric (0..12) -%D Day of the month with English suffix (0th, 1st, 2nd, 3rd, …) -%d Day of the month, numeric (00..31) -%e Day of the month, numeric (0..31) -%f Microseconds (000000..999999) -%H Hour (00..23) -%h Hour (01..12) -%I Hour (01..12) -%i Minutes, numeric (00..59) -%j Day of year (001..366) -%k Hour (0..23) -%l Hour (1..12) -%M Month name (January..December) -%m Month, numeric (00..12) -%p AM or PM -%r Time, 12-hour (hh:mm:ss followed by AM or PM) -%S Seconds (00..59) -%s Seconds (00..59) -%T Time, 24-hour (hh:mm:ss) -%U Week (00..53), where Sunday is the first day of the week; WEEK() mode 0 -%u Week (00..53), where Monday is the first day of the week; WEEK() mode 1 -%V Week (01..53), where Sunday is the first day of the week; WEEK() mode 2; used with %X -%v Week (01..53), where Monday is the first day of the week; WEEK() mode 3; used with %x -%W Weekday name (Sunday..Saturday) -%w Day of the week (0=Sunday..6=Saturday) -%X Year for the week where Sunday is the first day of the week, numeric, four digits; used with %V -%x Year for the week, where Monday is the first day of the week, numeric, four digits; used with %v -%Y Year, numeric, four digits -%y Year, numeric (two digits) -%% A literal “%” character -%x x, for any “x” not listed above - -## example -mysql> SELECT DATE_FORMAT('2009-10-04 22:23:00', '%W %M %Y'); - -> 'Sunday October 2009' -mysql> SELECT DATE_FORMAT('2007-10-04 22:23:00', '%H:%i:%s'); - -> '22:23:00' -mysql> SELECT DATE_FORMAT('1900-10-04 22:23:00', '%D %y %a %d %m %b %j'); - -> '4th 00 Thu 04 10 Oct 277' -mysql> SELECT DATE_FORMAT('1997-10-04 22:23:00', '%H %k %I %r %T %S %w'); - -> '22 22 10 10:23:00 PM 22:23:00 00 6' -mysql> SELECT DATE_FORMAT('1999-01-01', '%X %V'); - -> '1998 52' -mysql> SELECT DATE_FORMAT('2006-06-01', '%d'); - -> '01' - -# from_days -## description -Syntax: -FROM_DAYS(N) -通过距离0000-01-01日的天数计算出哪一天 - -## example -mysql> SELECT FROM_DAYS(730669); - -> '2007-07-03' - -# to_days -## description -Syntax: -TO_DAYS(date) -返回date距离0000-01-01的天数 - -## example -mysql> SELECT TO_DAYS(950501); - -> 728779 -mysql> SELECT TO_DAYS('2007-10-07'); - -> 733321 - -# str_to_date -## description -Syntax: -STR_TO_DATE(str, format) -通过format指定的方式将str转化为DATE类型,如果转化结果不对返回NULL -支持的format格式与date_format一致 - -## example -mysql> SELECT STR_TO_DATE('2014-12-21 12:34:56', '%Y-%m-%d %H:%i:%s'); - -> 2014-12-21 12:34:56 -mysql> SELECT STR_TO_DATE('200442 Monday', '%X%V %W'); - -> 2004-10-18 - -# monthname -## description -Syntax: -MONTHNAME(DATE) - -返回日期对应的月份名字 - -## example -mysql> SELECT MONTHNAME('2008-02-03'); - -> 'February' - -# monthname -## description -Syntax: -MONTHNAME(DATE) - -返回日期对应的日期名字 - -## example -mysql> SELECT DAYNAME('2007-02-03'); - -> 'Saturday' diff --git a/fe/src/main/java/org/apache/doris/common/MarkDownParser.java b/fe/src/main/java/org/apache/doris/common/MarkDownParser.java old mode 100644 new mode 100755 index 573f2333822230..0ad093ff91b40c --- a/fe/src/main/java/org/apache/doris/common/MarkDownParser.java +++ b/fe/src/main/java/org/apache/doris/common/MarkDownParser.java @@ -99,8 +99,9 @@ public Map> parse() throws UserException { } else if (headLevel == 2) { keyValues.put(keyValue.getKey(), keyValue.getValue()); } else { - // State error - throw new UserException("Unknown head level when parsing head level(2)"); + //Ignore headlevel greater than 2 instead of throwing a exception + //State error + //throw new UserException("Unknown head level when parsing head level(2)"); } break; default: @@ -133,7 +134,13 @@ private Map.Entry parseOneItem() { if (!lines.get(nextToRead).startsWith("#")) { sb.append(lines.get(nextToRead)).append('\n'); nextToRead++; - } else { + } + // Ignore headlevel greater than 2 + else if (lines.get(nextToRead).startsWith("###")) { + sb.append(lines.get(nextToRead).replaceAll("#","")).append('\n'); + nextToRead++; + } + else { break; } } diff --git a/fe/src/test/java/org/apache/doris/common/MarkDownParserTest.java b/fe/src/test/java/org/apache/doris/common/MarkDownParserTest.java old mode 100644 new mode 100755 index 3cd40e74844398..57fa3d1d287a3b --- a/fe/src/test/java/org/apache/doris/common/MarkDownParserTest.java +++ b/fe/src/test/java/org/apache/doris/common/MarkDownParserTest.java @@ -117,17 +117,48 @@ public void testNoFirst() throws UserException { Assert.fail("No exception throws."); } - @Test(expected = UserException.class) - public void testErrorState() throws UserException { +// When encounter a headlevel at 3 or greater, we ignore it rather than throw exception +// @Test(expected = UserException.class) +// public void testErrorState() throws UserException { +// List lines = Lists.newArrayList(); +// lines.add("# SHOW TABLES"); +// lines.add("## name"); +// lines.add("### name"); +// MarkDownParser parser = new MarkDownParser(lines); +// Map> map = parser.parse(); +// Assert.fail("No exception throws."); +// } + + @Test + public void testMultiHeadLevel() throws UserException { List lines = Lists.newArrayList(); lines.add("# SHOW TABLES"); lines.add("## name"); - lines.add("### name"); + lines.add(" SHOW TABLES"); + lines.add("## description"); + lines.add("###Syntax"); + lines.add("SYNTAX:\n\tSHOW TABLES [FROM] database"); + lines.add("####Parameter"); + lines.add(">table_name"); + lines.add("## example"); + lines.add("show tables;"); + lines.add("### Exam1"); + lines.add("exam1"); + lines.add("## keyword"); + lines.add("SHOW, TABLES"); + lines.add("## url"); + lines.add("http://www.baidu.com"); MarkDownParser parser = new MarkDownParser(lines); Map> map = parser.parse(); - Assert.fail("No exception throws."); + Assert.assertNotNull(map.get("SHOW TABLES")); + Assert.assertEquals(" SHOW TABLES\n", map.get("SHOW TABLES").get("name")); + Assert.assertEquals("Syntax\nSYNTAX:\n\tSHOW TABLES [FROM] database\nParameter\n>table_name\n", map.get("SHOW TABLES").get("description")); + Assert.assertEquals("show tables;\n Exam1\nexam1\n", map.get("SHOW TABLES").get("example")); + Assert.assertEquals("SHOW, TABLES\n", map.get("SHOW TABLES").get("keyword")); + Assert.assertEquals("http://www.baidu.com\n", map.get("SHOW TABLES").get("url")); } + @Test public void testEmptyTitle() throws UserException { List lines = Lists.newArrayList(); From 60d997fe67dc3a550335c5ea398c85e9b2e4034c Mon Sep 17 00:00:00 2001 From: "Yunfeng,Wu" Date: Thu, 8 Aug 2019 09:29:23 +0800 Subject: [PATCH 19/26] Fix errors when ES username and passwd is empty (#1601) --- .../java/org/apache/doris/external/EsRestClient.java | 12 ++++++++---- .../java/org/apache/doris/external/EsStateStore.java | 2 +- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/fe/src/main/java/org/apache/doris/external/EsRestClient.java b/fe/src/main/java/org/apache/doris/external/EsRestClient.java index 1159cadc646f30..88cd61c0dbb4e1 100644 --- a/fe/src/main/java/org/apache/doris/external/EsRestClient.java +++ b/fe/src/main/java/org/apache/doris/external/EsRestClient.java @@ -95,6 +95,7 @@ public String getIndexMetaData(String indexName) { /** * execute request for specific path + * * @param path the path must not leading with '/' * @return */ @@ -102,11 +103,14 @@ private String execute(String path) { selectNextNode(); boolean nextNode; do { - Request request = new Request.Builder() - .get() - .addHeader("Authorization", basicAuth) + Request.Builder builder = new Request.Builder(); + if (!Strings.isEmpty(basicAuth)) { + builder.addHeader("Authorization", basicAuth); + } + Request request = builder.get() .url(currentNode + "/" + path) .build(); + LOG.trace("es rest client request URL: {}", currentNode + "/" + path); try { Response response = networkClient.newCall(request).execute(); if (response.isSuccessful()) { @@ -117,7 +121,7 @@ private String execute(String path) { } nextNode = selectNextNode(); if (!nextNode) { - LOG.error("try all nodes [{}],no other nodes left", nodes); + LOG.warn("try all nodes [{}],no other nodes left", nodes); } } while (nextNode); return null; diff --git a/fe/src/main/java/org/apache/doris/external/EsStateStore.java b/fe/src/main/java/org/apache/doris/external/EsStateStore.java index 8bab6180fc6bce..c641bfbb911ba0 100644 --- a/fe/src/main/java/org/apache/doris/external/EsStateStore.java +++ b/fe/src/main/java/org/apache/doris/external/EsStateStore.java @@ -103,7 +103,7 @@ protected void runOneCycle() { } esTable.setEsTableState(esTableState); } catch (Throwable e) { - LOG.error("errors while load table {} state from es", esTable.getName()); + LOG.warn("Exception happens when fetch index [{}] meta data from remote es cluster", esTable.getName(), e); } } } From b9378871331518b9729e5b93b17fbe6f1db5b2ed Mon Sep 17 00:00:00 2001 From: lenmom Date: Thu, 8 Aug 2019 09:30:21 +0800 Subject: [PATCH 20/26] =?UTF-8?q?Include=20header=20file=20for=20=E2=80=98?= =?UTF-8?q?preadv'=20=20which=20caused=20break=20build=20on=20ubuntu=2018.?= =?UTF-8?q?04=20(#1602)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- be/src/env/env_posix.cpp | 1 + thirdparty/build-thirdparty.sh | 1 - 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/be/src/env/env_posix.cpp b/be/src/env/env_posix.cpp index 3bc6c08fafbb2b..f86fc5457ce178 100644 --- a/be/src/env/env_posix.cpp +++ b/be/src/env/env_posix.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include "common/logging.h" #include "gutil/macros.h" diff --git a/thirdparty/build-thirdparty.sh b/thirdparty/build-thirdparty.sh index 5fe0450eaab7e5..92513ae224e01e 100755 --- a/thirdparty/build-thirdparty.sh +++ b/thirdparty/build-thirdparty.sh @@ -661,4 +661,3 @@ build_bitshuffle build_croaringbitmap echo "Finihsed to build all thirdparties" - From 133e90e531526602f24deb2a9897c54cc9ad530a Mon Sep 17 00:00:00 2001 From: yuanlihan Date: Thu, 1 Aug 2019 00:58:26 +0800 Subject: [PATCH 21/26] Enable partition discovery when loading data from parquet file --- be/src/exec/parquet_reader.cpp | 34 +++++++++---- be/src/exec/parquet_reader.h | 3 +- be/src/exec/parquet_scanner.cpp | 2 +- be/test/exec/parquet_scanner_test.cpp | 11 ++-- .../Data Manipulation/broker_load.md | 15 ++++++ .../apache/doris/common/util/BrokerUtil.java | 50 +++++++++++-------- .../apache/doris/planner/BrokerScanNode.java | 38 ++++++++++++++ gensrc/thrift/PlanNodes.thrift | 2 + 8 files changed, 119 insertions(+), 36 deletions(-) diff --git a/be/src/exec/parquet_reader.cpp b/be/src/exec/parquet_reader.cpp index b91d14713364fd..b7942d4b4e2cfd 100644 --- a/be/src/exec/parquet_reader.cpp +++ b/be/src/exec/parquet_reader.cpp @@ -34,8 +34,8 @@ namespace doris { // Broker -ParquetReaderWrap::ParquetReaderWrap(FileReader *file_reader) : - _total_groups(0), _current_group(0), _rows_of_group(0), _current_line_of_group(0) { +ParquetReaderWrap::ParquetReaderWrap(FileReader *file_reader, const std::map& partition_columns) : + _partition_columns(partition_columns), _total_groups(0), _current_group(0), _rows_of_group(0), _current_line_of_group(0) { _parquet = std::shared_ptr(new ParquetFile(file_reader)); _properties = parquet::ReaderProperties(); _properties.enable_buffered_stream(); @@ -125,13 +125,17 @@ Status ParquetReaderWrap::column_indices(const std::vector& tup for (auto slot_desc : tuple_slot_descs) { // Get the Column Reader for the boolean column auto iter = _map_column.find(slot_desc->col_name()); - if (iter == _map_column.end()) { - std::stringstream str_error; - str_error << "Invalid Column Name:" << slot_desc->col_name(); - LOG(WARNING) << str_error.str(); - return Status::InvalidArgument(str_error.str()); + if (iter != _map_column.end()) { + _parquet_column_ids.emplace_back(iter->second); + } else { + auto iter_1 = _partition_columns.find(slot_desc->col_name()); + if (iter_1 == _partition_columns.end()) { + std::stringstream str_error; + str_error << "Invalid Column Name:" << slot_desc->col_name(); + LOG(WARNING) << str_error.str(); + return Status::InvalidArgument(str_error.str()); + } } - _parquet_column_ids.emplace_back(iter->second); } return Status::OK(); } @@ -204,13 +208,23 @@ Status ParquetReaderWrap::read(Tuple* tuple, const std::vector& uint8_t tmp_buf[128] = {0}; int32_t wbytes = 0; const uint8_t *value = nullptr; + int index = 0; int column_index = 0; try { size_t slots = tuple_slot_descs.size(); for (size_t i = 0; i < slots; ++i) { auto slot_desc = tuple_slot_descs[i]; - column_index = i;// column index in batch record - switch (_parquet_column_type[i]) { + auto iter = _partition_columns.find(slot_desc->col_name()); + if (iter != _partition_columns.end()) { + std::string partitioned_field = iter->second; + value = reinterpret_cast(partitioned_field.c_str()); + wbtyes = partitioned_field.size(); + fill_slot(tuple, slot_desc, mem_pool, value, wbtyes); + continue; + } else { + column_index = index++; // column index in batch record + } + switch (_parquet_column_type[column_index]) { case arrow::Type::type::STRING: { auto str_array = std::dynamic_pointer_cast(_batch->column(column_index)); if (str_array->IsNull(_current_line_of_group)) { diff --git a/be/src/exec/parquet_reader.h b/be/src/exec/parquet_reader.h index defe3d9ebf09fd..107588d79d9aa8 100644 --- a/be/src/exec/parquet_reader.h +++ b/be/src/exec/parquet_reader.h @@ -68,7 +68,7 @@ class ParquetFile : public arrow::io::RandomAccessFile { // Reader of broker parquet file class ParquetReaderWrap { public: - ParquetReaderWrap(FileReader *file_reader); + ParquetReaderWrap(FileReader *file_reader, const std::map& partition_columns); virtual ~ParquetReaderWrap(); // Read @@ -85,6 +85,7 @@ class ParquetReaderWrap { Status handle_timestamp(const std::shared_ptr& ts_array, uint8_t *buf, int32_t *wbtyes); private: + const std::map& _partition_columns; parquet::ReaderProperties _properties; std::shared_ptr _parquet; diff --git a/be/src/exec/parquet_scanner.cpp b/be/src/exec/parquet_scanner.cpp index 2ce43a121956a1..9d8550fabe51f7 100644 --- a/be/src/exec/parquet_scanner.cpp +++ b/be/src/exec/parquet_scanner.cpp @@ -141,7 +141,7 @@ Status ParquetScanner::open_next_reader() { file_reader->close(); continue; } - _cur_file_reader = new ParquetReaderWrap(file_reader.release()); + _cur_file_reader = new ParquetReaderWrap(file_reader.release(), range.partition_columns); Status status = _cur_file_reader->init_parquet_reader(_src_slot_descs); if (status.is_end_of_file()) { continue; diff --git a/be/test/exec/parquet_scanner_test.cpp b/be/test/exec/parquet_scanner_test.cpp index 4bea0130c33672..e95472e8b947f8 100644 --- a/be/test/exec/parquet_scanner_test.cpp +++ b/be/test/exec/parquet_scanner_test.cpp @@ -68,14 +68,14 @@ class ParquetSannerTest : public testing::Test { #define TUPLE_ID_DST 0 #define TUPLE_ID_SRC 1 -#define CLOMN_NUMBERS 19 +#define CLOMN_NUMBERS 20 #define DST_TUPLE_SLOT_ID_START 1 -#define SRC_TUPLE_SLOT_ID_START 20 +#define SRC_TUPLE_SLOT_ID_START 21 int ParquetSannerTest::create_src_tuple(TDescriptorTable& t_desc_table, int next_slot_id) { const char *clomnNames[] = {"log_version", "log_time", "log_time_stamp", "js_version", "vst_cookie", "vst_ip", "vst_user_id", "vst_user_agent", "device_resolution", "page_url", "page_refer_url", "page_yyid", "page_type", "pos_type", "content_id", "media_id", - "spm_cnt", "spm_pre", "scm_cnt"}; + "spm_cnt", "spm_pre", "scm_cnt", "partition_column"}; for (int i = 0; i < CLOMN_NUMBERS; i++) { TSlotDescriptor slot_desc; @@ -201,7 +201,7 @@ int ParquetSannerTest::create_dst_tuple(TDescriptorTable& t_desc_table, int next const char *clomnNames[] = {"log_version", "log_time", "log_time_stamp", "js_version", "vst_cookie", "vst_ip", "vst_user_id", "vst_user_agent", "device_resolution", "page_url", "page_refer_url", "page_yyid", "page_type", "pos_type", "content_id", "media_id", - "spm_cnt", "spm_pre", "scm_cnt"}; + "spm_cnt", "spm_pre", "scm_cnt", "partition_column"}; for (int i = 3; i < CLOMN_NUMBERS; i++, byteOffset+=16) { TSlotDescriptor slot_desc; @@ -435,6 +435,9 @@ TEST_F(ParquetSannerTest, normal) { range.size = -1; range.format_type = TFileFormatType::FORMAT_PARQUET; range.splittable = true; + + std::map partition_columns = {{"partition_column", "value"}}; + range.__set_partition_columns(partition_columns); #if 1 range.path = "./be/test/exec/test_data/parquet_scanner/localfile.parquet"; range.file_type = TFileType::FILE_LOCAL; diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/broker_load.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/broker_load.md index b6b445a1b3ceeb..2601872553ebbe 100644 --- a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/broker_load.md +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/broker_load.md @@ -353,6 +353,9 @@ WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); 8. 导入Parquet文件中数据 指定FORMAT 为parquet, 默认是通过文件后缀判断 + 如果导入路径为目录(base_path),则递归地列出该目录下的所有parquet文件 + 如果需要,则会根据表中定义的字段类型解析base_path目录下的partitioned fields,实现类似Spark中读parquet文件 + LOAD LABEL example_db.label9 ( DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/file") @@ -361,6 +364,18 @@ (k1, k2, k3) ) WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); + + LOAD LABEL example_db.label10 + ( + DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir") + INTO TABLE `my_table` + FORMAT AS "parquet" + (k1, k2, k3) + ) + WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); + + hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir目录下包括如下文件:[hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/k1=key1/xxx.parquet, hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/k1=key2/xxx.parquet, ...] + 如果my_table中定义了列k1,则会从文件path中提取k1对应的partitioned field的值,并完成数据导入 ## keyword BROKER,LOAD diff --git a/fe/src/main/java/org/apache/doris/common/util/BrokerUtil.java b/fe/src/main/java/org/apache/doris/common/util/BrokerUtil.java index a6cb66832c5369..725d8160a2444c 100644 --- a/fe/src/main/java/org/apache/doris/common/util/BrokerUtil.java +++ b/fe/src/main/java/org/apache/doris/common/util/BrokerUtil.java @@ -61,28 +61,10 @@ public static void parseBrokerFile(String path, BrokerDesc brokerDesc, List parsePartitionedFields(String filePath) { + for (BrokerFileGroup fileGroup : fileGroups) { + String fileFormat = fileGroup.getFileFormat(); + if ((fileFormat == null || !fileFormat.toLowerCase().equals("parquet")) || !filePath.endsWith(".parquet")) { + continue; + } + for (String base : fileGroup.getFilePaths()) { + if (base.endsWith("/*")) { + base = base.substring(0, base.indexOf("/*")); + } + if (!filePath.startsWith(base)) { + continue; + } + String subPath = filePath.substring(base.length()); + String[] strings = subPath.split("/"); + Map partitionedFields = new HashMap<>(); + for (String str : strings) { + if (str == null || str.isEmpty() || !str.contains("=")) { + continue; + } + String[] pair = str.split("="); + if (pair.length != 2) { + continue; + } + Column column = targetTable.getColumn(pair[0]); + if (column == null) { + continue; + } + partitionedFields.put(pair[0], pair[1]); + } + return partitionedFields; + } + } + return Collections.emptyMap(); + } + // If fileFormat is not null, we use fileFormat instead of check file's suffix private void processFileGroup( String fileFormat, @@ -650,6 +687,7 @@ private TBrokerRangeDesc createBrokerRangeDesc(long curFileOffset, TBrokerFileSt rangeDesc.setStart_offset(curFileOffset); rangeDesc.setSize(rangeBytes); rangeDesc.setFile_size(fileStatus.size); + rangeDesc.setPartition_columns(parsePartitionedFields(fileStatus.path)); return rangeDesc; } diff --git a/gensrc/thrift/PlanNodes.thrift b/gensrc/thrift/PlanNodes.thrift index d587e34b58b8c9..fd3529fd3a3d9c 100644 --- a/gensrc/thrift/PlanNodes.thrift +++ b/gensrc/thrift/PlanNodes.thrift @@ -114,6 +114,8 @@ struct TBrokerRangeDesc { 7: optional Types.TUniqueId load_id // total size of the file 8: optional i64 file_size + // partition columns of this file(parquet only) + 9: optional map partition_columns } struct TBrokerScanRangeParams { From 067c0dfe1e5eb9b9ba241d89dd75edb2296d6cc9 Mon Sep 17 00:00:00 2001 From: yuanlihan Date: Sun, 4 Aug 2019 23:28:24 +0800 Subject: [PATCH 22/26] Read date type column from parquet --- be/src/exec/parquet_reader.cpp | 29 +++++++++++++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/be/src/exec/parquet_reader.cpp b/be/src/exec/parquet_reader.cpp index b7942d4b4e2cfd..88bb51b4e4574f 100644 --- a/be/src/exec/parquet_reader.cpp +++ b/be/src/exec/parquet_reader.cpp @@ -410,6 +410,35 @@ Status ParquetReaderWrap::read(Tuple* tuple, const std::vector& } break; } + case arrow::Type::type::DATE32: { + auto ts_array = std::dynamic_pointer_cast(_batch->column(column_index)); + if (ts_array->IsNull(_current_line_of_group)) { + RETURN_IF_ERROR(set_field_null(tuple, slot_desc)); + } else { + time_t timestamp = (time_t)((int64_t)ts_array->Value(_current_line_of_group) * 24 * 60 * 60); + tm* local; + local = localtime(×tamp); + char* to = reinterpret_cast(&tmp_buf); + wbtyes = (uint32_t)strftime(to, 64, "%Y-%m-%d", local); + fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbtyes); + } + break; + } + case arrow::Type::type::DATE64: { + auto ts_array = std::dynamic_pointer_cast(_batch->column(column_index)); + if (ts_array->IsNull(_current_line_of_group)) { + RETURN_IF_ERROR(set_field_null(tuple, slot_desc)); + } else { + // convert milliseconds to seconds + time_t timestamp = (time_t)((int64_t)ts_array->Value(_current_line_of_group) / 1000); + tm* local; + local = localtime(×tamp); + char* to = reinterpret_cast(&tmp_buf); + wbtyes = (uint32_t)strftime(to, 64, "%Y-%m-%d %H:%M:%S", local); + fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbtyes); + } + break; + } default: { // other type not support. std::stringstream str_error; From 7da20bc32d0cca1dd73273b4cfbc3c5413e7dbc1 Mon Sep 17 00:00:00 2001 From: yuanlihan Date: Sun, 4 Aug 2019 23:31:27 +0800 Subject: [PATCH 23/26] Enable partition discovery for broker loading --- be/src/exec/broker_scanner.cpp | 39 +++++++--- be/src/exec/broker_scanner.h | 2 + be/test/exec/broker_scan_node_test.cpp | 71 ++++++++++++++++-- .../Data Manipulation/broker_load.md | 52 +++++++++---- fe/src/main/cup/sql_parser.cup | 16 +++- .../doris/analysis/DataDescription.java | 19 +++++ .../org/apache/doris/common/FeConstants.java | 2 +- .../apache/doris/common/FeMetaVersion.java | 2 + .../apache/doris/load/BrokerFileGroup.java | 20 +++++ .../apache/doris/planner/BrokerScanNode.java | 74 ++++++++++--------- fe/src/main/jflex/sql_scanner.flex | 1 + 11 files changed, 225 insertions(+), 73 deletions(-) diff --git a/be/src/exec/broker_scanner.cpp b/be/src/exec/broker_scanner.cpp index 19cd829f6826e6..062a476ab537ea 100644 --- a/be/src/exec/broker_scanner.cpp +++ b/be/src/exec/broker_scanner.cpp @@ -54,6 +54,7 @@ BrokerScanner::BrokerScanner(RuntimeState* state, _cur_decompressor(nullptr), _next_range(0), _cur_line_reader_eof(false), + _partition_columns(), _scanner_eof(false), _skip_next_line(false) { } @@ -236,6 +237,9 @@ Status BrokerScanner::open_line_reader() { // _decompressor may be NULL if this is not a compressed file RETURN_IF_ERROR(create_decompressor(range.format_type)); + // set partitioned columns + _partition_columns = range.partition_columns; + // open line reader switch (range.format_type) { case TFileFormatType::FORMAT_CSV_PLAIN: @@ -452,6 +456,17 @@ bool BrokerScanner::convert_one_row( return fill_dest_tuple(line, tuple, tuple_pool); } +inline void BrokerScanner::fill_slot(SlotDescriptor* slot_desc, const Slice& value) { + if (slot_desc->is_nullable() && is_null(value)) { + _src_tuple->set_null(slot_desc->null_indicator_offset()); + } + _src_tuple->set_not_null(slot_desc->null_indicator_offset()); + void* slot = _src_tuple->get_slot(slot_desc->tuple_offset()); + StringValue* str_slot = reinterpret_cast(slot); + str_slot->ptr = value.data; + str_slot->len = value.size; +} + // Convert one row to this tuple bool BrokerScanner::line_to_src_tuple(const Slice& line) { @@ -469,7 +484,7 @@ bool BrokerScanner::line_to_src_tuple(const Slice& line) { split_line(line, &values); } - if (values.size() < _src_slot_descs.size()) { + if (values.size() + _partition_columns.size() < _src_slot_descs.size()) { std::stringstream error_msg; error_msg << "actual column number is less than schema column number. " << "actual number: " << values.size() << " sep: " << _value_separator << ", " @@ -478,7 +493,7 @@ bool BrokerScanner::line_to_src_tuple(const Slice& line) { error_msg.str()); _counter->num_rows_filtered++; return false; - } else if (values.size() > _src_slot_descs.size()) { + } else if (values.size() + _partition_columns.size() > _src_slot_descs.size()) { std::stringstream error_msg; error_msg << "actual column number is more than schema column number. " << "actual number: " << values.size() << " sep: " << _value_separator << ", " @@ -489,18 +504,18 @@ bool BrokerScanner::line_to_src_tuple(const Slice& line) { return false; } - for (int i = 0; i < values.size(); ++i) { + int file_column_index = 0; + for (int i = 0; i < _src_slot_descs.size(); ++i) { auto slot_desc = _src_slot_descs[i]; - const Slice& value = values[i]; - if (slot_desc->is_nullable() && is_null(value)) { - _src_tuple->set_null(slot_desc->null_indicator_offset()); - continue; + auto iter = _partition_columns.find(slot_desc->col_name()); + if (iter != _partition_columns.end()) { + std::string partitioned_field = iter->second; + const Slice value = Slice(partitioned_field.c_str(), partitioned_field.size()); + fill_slot(slot_desc, value); + } else { + const Slice& value = values[file_column_index++]; + fill_slot(slot_desc, value); } - _src_tuple->set_not_null(slot_desc->null_indicator_offset()); - void* slot = _src_tuple->get_slot(slot_desc->tuple_offset()); - StringValue* str_slot = reinterpret_cast(slot); - str_slot->ptr = value.data; - str_slot->len = value.size; } return true; diff --git a/be/src/exec/broker_scanner.h b/be/src/exec/broker_scanner.h index b9a798694dc8d9..ea464dfc48f6d3 100644 --- a/be/src/exec/broker_scanner.h +++ b/be/src/exec/broker_scanner.h @@ -104,6 +104,7 @@ class BrokerScanner : public BaseScanner { //Status init_expr_ctxes(); Status line_to_src_tuple(); + void fill_slot(SlotDescriptor* slot_desc, const Slice& value); bool line_to_src_tuple(const Slice& line); private:; const std::vector& _ranges; @@ -120,6 +121,7 @@ private:; Decompressor* _cur_decompressor; int _next_range; bool _cur_line_reader_eof; + std::map _partition_columns; bool _scanner_eof; diff --git a/be/test/exec/broker_scan_node_test.cpp b/be/test/exec/broker_scan_node_test.cpp index 066f468dfa99cb..bc4cc533d2455d 100644 --- a/be/test/exec/broker_scan_node_test.cpp +++ b/be/test/exec/broker_scan_node_test.cpp @@ -153,7 +153,33 @@ void BrokerScanNodeTest::init_desc_table() { slot_desc.nullIndicatorByte = 0; slot_desc.nullIndicatorBit = -1; slot_desc.colName = "k3"; - slot_desc.slotIdx = 2; + slot_desc.slotIdx = 3; + slot_desc.isMaterialized = true; + + t_desc_table.slotDescriptors.push_back(slot_desc); + } + // k4(partitioned column) + { + TSlotDescriptor slot_desc; + + slot_desc.id = next_slot_id++; + slot_desc.parent = 0; + TTypeDesc type; + { + TTypeNode node; + node.__set_type(TTypeNodeType::SCALAR); + TScalarType scalar_type; + scalar_type.__set_type(TPrimitiveType::INT); + node.__set_scalar_type(scalar_type); + type.types.push_back(node); + } + slot_desc.slotType = type; + slot_desc.columnPos = 1; + slot_desc.byteOffset = 12; + slot_desc.nullIndicatorByte = 0; + slot_desc.nullIndicatorBit = -1; + slot_desc.colName = "k4"; + slot_desc.slotIdx = 4; slot_desc.isMaterialized = true; t_desc_table.slotDescriptors.push_back(slot_desc); @@ -164,7 +190,7 @@ void BrokerScanNodeTest::init_desc_table() { // TTupleDescriptor dest TTupleDescriptor t_tuple_desc; t_tuple_desc.id = 0; - t_tuple_desc.byteSize = 12; + t_tuple_desc.byteSize = 16; t_tuple_desc.numNullBytes = 0; t_tuple_desc.tableId = 0; t_tuple_desc.__isset.tableId = true; @@ -251,7 +277,34 @@ void BrokerScanNodeTest::init_desc_table() { slot_desc.nullIndicatorByte = 0; slot_desc.nullIndicatorBit = -1; slot_desc.colName = "k3"; - slot_desc.slotIdx = 2; + slot_desc.slotIdx = 3; + slot_desc.isMaterialized = true; + + t_desc_table.slotDescriptors.push_back(slot_desc); + } + // k4(partitioned column) + { + TSlotDescriptor slot_desc; + + slot_desc.id = next_slot_id++; + slot_desc.parent = 1; + TTypeDesc type; + { + TTypeNode node; + node.__set_type(TTypeNodeType::SCALAR); + TScalarType scalar_type; + scalar_type.__set_type(TPrimitiveType::VARCHAR); + scalar_type.__set_len(65535); + node.__set_scalar_type(scalar_type); + type.types.push_back(node); + } + slot_desc.slotType = type; + slot_desc.columnPos = 1; + slot_desc.byteOffset = 48; + slot_desc.nullIndicatorByte = 0; + slot_desc.nullIndicatorBit = -1; + slot_desc.colName = "k4"; + slot_desc.slotIdx = 4; slot_desc.isMaterialized = true; t_desc_table.slotDescriptors.push_back(slot_desc); @@ -261,7 +314,7 @@ void BrokerScanNodeTest::init_desc_table() { // TTupleDescriptor source TTupleDescriptor t_tuple_desc; t_tuple_desc.id = 1; - t_tuple_desc.byteSize = 48; + t_tuple_desc.byteSize = 60; t_tuple_desc.numNullBytes = 0; t_tuple_desc.tableId = 0; t_tuple_desc.__isset.tableId = true; @@ -297,7 +350,7 @@ void BrokerScanNodeTest::init() { varchar_type.types.push_back(node); } - for (int i = 0; i < 3; ++i) { + for (int i = 0; i < 4; ++i) { TExprNode cast_expr; cast_expr.node_type = TExprNodeType::CAST_EXPR; cast_expr.type = int_type; @@ -319,7 +372,7 @@ void BrokerScanNodeTest::init() { slot_ref.type = varchar_type; slot_ref.num_children = 0; slot_ref.__isset.slot_ref = true; - slot_ref.slot_ref.slot_id = 4 + i; + slot_ref.slot_ref.slot_id = 5 + i; slot_ref.slot_ref.tuple_id = 1; TExpr expr; @@ -327,7 +380,7 @@ void BrokerScanNodeTest::init() { expr.nodes.push_back(slot_ref); _params.expr_of_dest_slot.emplace(i + 1, expr); - _params.src_slot_ids.push_back(4 + i); + _params.src_slot_ids.push_back(5 + i); } // _params.__isset.expr_of_dest_slot = true; _params.__set_dest_tuple_id(0); @@ -367,6 +420,8 @@ TEST_F(BrokerScanNodeTest, normal) { range.file_type = TFileType::FILE_LOCAL; range.format_type = TFileFormatType::FORMAT_CSV_PLAIN; range.splittable = true; + std::map partition_columns = {{"k4", "1"}}; + range.__set_partition_columns(partition_columns); broker_scan_range.ranges.push_back(range); scan_range_params.scan_range.__set_broker_scan_range(broker_scan_range); @@ -386,6 +441,8 @@ TEST_F(BrokerScanNodeTest, normal) { range.file_type = TFileType::FILE_LOCAL; range.format_type = TFileFormatType::FORMAT_CSV_PLAIN; range.splittable = true; + std::map partition_columns = {{"k4", "2"}}; + range.__set_partition_columns(partition_columns); broker_scan_range.ranges.push_back(range); scan_range_params.scan_range.__set_broker_scan_range(broker_scan_range); diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/broker_load.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/broker_load.md index 2601872553ebbe..503201548665a4 100644 --- a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/broker_load.md +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/broker_load.md @@ -39,6 +39,7 @@ [PARTITION (p1, p2)] [COLUMNS TERMINATED BY "column_separator"] [FORMAT AS "file_type"] + [BASE_PATH AS "base_path"] [(column_list)] [SET (k1 = func(k2))] @@ -65,6 +66,10 @@ file_type: 用于指定导入文件的类型,例如:parquet、csv。默认值通过文件后缀名判断。 + + base_path: + + 用于指定作为Partition Discovery的基础路径。 column_list: @@ -352,10 +357,7 @@ ) WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); - 8. 导入Parquet文件中数据 指定FORMAT 为parquet, 默认是通过文件后缀判断 - 如果导入路径为目录(base_path),则递归地列出该目录下的所有parquet文件 - 如果需要,则会根据表中定义的字段类型解析base_path目录下的partitioned fields,实现类似Spark中读parquet文件 - + 8. 导入Parquet文件中数据 指定FORMAT 为parquet, 默认是通过文件后缀判断 LOAD LABEL example_db.label9 ( DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/file") @@ -363,19 +365,39 @@ FORMAT AS "parquet" (k1, k2, k3) ) - WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); + WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); - LOAD LABEL example_db.label10 - ( - DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir") - INTO TABLE `my_table` - FORMAT AS "parquet" - (k1, k2, k3) - ) - WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); + 9. 通过Partition Discovery提取文件路径中的压缩字段 + 如果导入路径为目录,则递归地列出该目录下的所有parquet文件 + 如果需要,则会根据表中定义的字段类型解析文件路径中的partitioned fields,实现类似Spark中读parquet文件 + 1. 不指定Partition Discovery的基础路径(BASE_PATH) + LOAD LABEL example_db.label10 + ( + DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir") + INTO TABLE `my_table` + FORMAT AS "parquet" + (k1, k2, k3) + ) + WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); + + hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir目录下包括如下文件:[hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/k1=key1/xxx.parquet, hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/k1=key2/xxx.parquet, ...] + 则会从文件path中提取k1对应的partitioned field的值,并完成数据导入 + + 2. 指定Partition Discovery的基础路径(BASE_PATH) + LOAD LABEL example_db.label11 + ( + DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/city=beijing/utc_date=2019-06-26") + INTO TABLE `my_table` + FORMAT AS "csv" + BASE_PATH AS "hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/" + (k1, k2, k3, utc_date,city) + SET (uniq_id = md5sum(k1, city)) + ) + WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); + + hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/city=beijing/utc_date=2019-06-26目录下包括如下文件:[hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/city=beijing/utc_date=2019-06-26/0000.csv, hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/city=beijing/utc_date=2019-06-26/0001.csv, ...] + 假设CSV文件中仅包括3列(k1, k2和k3),则会根据base_path(hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/)提取文件路径的中的city和utc_date字段 - hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir目录下包括如下文件:[hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/k1=key1/xxx.parquet, hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/k1=key2/xxx.parquet, ...] - 如果my_table中定义了列k1,则会从文件path中提取k1对应的partitioned field的值,并完成数据导入 ## keyword BROKER,LOAD diff --git a/fe/src/main/cup/sql_parser.cup b/fe/src/main/cup/sql_parser.cup index 5dbcfe99d66977..bc4ee491d6c303 100644 --- a/fe/src/main/cup/sql_parser.cup +++ b/fe/src/main/cup/sql_parser.cup @@ -191,7 +191,7 @@ parser code {: // Total keywords of doris terminal String KW_ADD, KW_ADMIN, KW_AFTER, KW_AGGREGATE, KW_ALL, KW_ALTER, KW_AND, KW_ANTI, KW_AS, KW_ASC, KW_AUTHORS, - KW_BACKEND, KW_BACKUP, KW_BETWEEN, KW_BEGIN, KW_BIGINT, KW_BOOLEAN, KW_BOTH, KW_BROKER, KW_BACKENDS, KW_BY, + KW_BACKEND, KW_BACKUP, KW_BASE_PATH, KW_BETWEEN, KW_BEGIN, KW_BIGINT, KW_BOOLEAN, KW_BOTH, KW_BROKER, KW_BACKENDS, KW_BY, KW_CANCEL, KW_CASE, KW_CAST, KW_CHAIN, KW_CHAR, KW_CHARSET, KW_CLUSTER, KW_CLUSTERS, KW_COLLATE, KW_COLLATION, KW_COLUMN, KW_COLUMNS, KW_COMMENT, KW_COMMIT, KW_COMMITTED, KW_CONFIG, KW_CONNECTION, KW_CONNECTION_ID, KW_CONSISTENT, KW_COUNT, KW_CREATE, KW_CROSS, KW_CURRENT, KW_CURRENT_USER, @@ -403,7 +403,7 @@ nonterminal List alter_table_clause_list; // nonterminal String keyword, ident, ident_or_text, variable_name, text_or_password, charset_name_or_default, old_or_new_charset_name_or_default, opt_collate, - collation_name_or_default, type_func_name_keyword, type_function_name, opt_file_format; + collation_name_or_default, type_func_name_keyword, type_function_name, opt_file_format, opt_base_path; nonterminal String opt_db, opt_partition_name, procedure_or_function, opt_comment, opt_engine; nonterminal ColumnDef.DefaultValue opt_default_value; @@ -1050,10 +1050,11 @@ data_desc ::= opt_partitions:partitionNames opt_field_term:colSep opt_file_format:fileFormat + opt_base_path:basePath opt_col_list:colList opt_col_mapping_list:colMappingList {: - RESULT = new DataDescription(tableName, partitionNames, files, colList, colSep, fileFormat, isNeg, colMappingList); + RESULT = new DataDescription(tableName, partitionNames, files, colList, colSep, fileFormat, basePath, isNeg, colMappingList); :} ; @@ -1110,6 +1111,13 @@ opt_file_format ::= {: RESULT = format; :} ; +opt_base_path ::= + /* Empty */ + {: RESULT = null; :} + | KW_BASE_PATH KW_AS ident_or_text:basePath + {: RESULT = basePath; :} + ; + opt_col_list ::= {: RESULT = null; @@ -3924,6 +3932,8 @@ keyword ::= {: RESULT = id; :} | KW_FORMAT:id {: RESULT = id; :} + | KW_BASE_PATH:id + {: RESULT = id; :} | KW_FUNCTION:id {: RESULT = id; :} | KW_END:id diff --git a/fe/src/main/java/org/apache/doris/analysis/DataDescription.java b/fe/src/main/java/org/apache/doris/analysis/DataDescription.java index 981ba82567e785..2e200fde2b1df4 100644 --- a/fe/src/main/java/org/apache/doris/analysis/DataDescription.java +++ b/fe/src/main/java/org/apache/doris/analysis/DataDescription.java @@ -60,6 +60,7 @@ public class DataDescription { private final List columnNames; private final ColumnSeparator columnSeparator; private final String fileFormat; + private final String basePath; private final boolean isNegative; private final List columnMappingList; @@ -85,6 +86,7 @@ public DataDescription(String tableName, this.columnNames = columnNames; this.columnSeparator = columnSeparator; this.fileFormat = null; + this.basePath = null; this.isNegative = isNegative; this.columnMappingList = columnMappingList; } @@ -95,6 +97,7 @@ public DataDescription(String tableName, List columnNames, ColumnSeparator columnSeparator, String fileFormat, + String basePath, boolean isNegative, List columnMappingList) { this.tableName = tableName; @@ -103,10 +106,22 @@ public DataDescription(String tableName, this.columnNames = columnNames; this.columnSeparator = columnSeparator; this.fileFormat = fileFormat; + this.basePath = basePath; this.isNegative = isNegative; this.columnMappingList = columnMappingList; } + public DataDescription(String tableName, + List partitionNames, + List filePaths, + List columnNames, + ColumnSeparator columnSeparator, + String fileFormat, + boolean isNegative, + List columnMappingList) { + this(tableName, partitionNames, filePaths, columnNames, columnSeparator, fileFormat, null, isNegative, columnMappingList); + } + public String getTableName() { return tableName; } @@ -127,6 +142,10 @@ public String getFileFormat() { return fileFormat; } + public String getBasePath() { + return basePath; + } + public String getColumnSeparator() { if (columnSeparator == null) { return null; diff --git a/fe/src/main/java/org/apache/doris/common/FeConstants.java b/fe/src/main/java/org/apache/doris/common/FeConstants.java index 084a5b642cc9ab..95683e20ac3693 100644 --- a/fe/src/main/java/org/apache/doris/common/FeConstants.java +++ b/fe/src/main/java/org/apache/doris/common/FeConstants.java @@ -35,5 +35,5 @@ public class FeConstants { // general model // Current meta data version. Use this version to write journals and image - public static int meta_version = FeMetaVersion.VERSION_57; + public static int meta_version = FeMetaVersion.VERSION_58; } diff --git a/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java b/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java index 62acd73006cdd5..05721da5a989b4 100644 --- a/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java +++ b/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java @@ -124,4 +124,6 @@ public final class FeMetaVersion { public static final int VERSION_56 = 56; // for base index using different id public static final int VERSION_57 = 57; + // basePath for partition discovery when loading data using broker scan + public static final int VERSION_58 = 58; } diff --git a/fe/src/main/java/org/apache/doris/load/BrokerFileGroup.java b/fe/src/main/java/org/apache/doris/load/BrokerFileGroup.java index 075134e3cd1f77..3af4a38615434f 100644 --- a/fe/src/main/java/org/apache/doris/load/BrokerFileGroup.java +++ b/fe/src/main/java/org/apache/doris/load/BrokerFileGroup.java @@ -58,6 +58,7 @@ public class BrokerFileGroup implements Writable { private String lineDelimiter; // fileFormat may be null, which means format will be decided by file's suffix private String fileFormat; + private String basePath; private boolean isNegative; private List partitionIds; private List fileFieldNames; @@ -82,6 +83,7 @@ public BrokerFileGroup(BrokerTable table) throws AnalysisException { public BrokerFileGroup(DataDescription dataDescription) { this.dataDescription = dataDescription; + this.basePath = dataDescription.getBasePath(); exprColumnMap = dataDescription.getParsedExprMap(); } @@ -159,6 +161,10 @@ public String getFileFormat() { return fileFormat; } + public String getBasePath() { + return basePath; + } + public boolean isNegative() { return isNegative; } @@ -208,6 +214,7 @@ public String toString() { sb.append(",valueSeparator=").append(valueSeparator) .append(",lineDelimiter=").append(lineDelimiter) .append(",fileFormat=").append(fileFormat) + .append(",basePath=").append(basePath) .append(",isNegative=").append(isNegative); sb.append(",fileInfos=["); int idx = 0; @@ -275,6 +282,13 @@ public void write(DataOutput out) throws IOException { out.writeBoolean(true); Text.writeString(out, fileFormat); } + // basePath + if (basePath == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + Text.writeString(out, basePath); + } } @Override @@ -328,6 +342,12 @@ public void readFields(DataInput in) throws IOException { fileFormat = Text.readString(in); } } + // basePath + if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_58) { + if (in.readBoolean()) { + basePath = Text.readString(in); + } + } } public static BrokerFileGroup read(DataInput in) throws IOException { diff --git a/fe/src/main/java/org/apache/doris/planner/BrokerScanNode.java b/fe/src/main/java/org/apache/doris/planner/BrokerScanNode.java index 1a103dfaf7243c..84d8e02357e244 100644 --- a/fe/src/main/java/org/apache/doris/planner/BrokerScanNode.java +++ b/fe/src/main/java/org/apache/doris/planner/BrokerScanNode.java @@ -589,51 +589,55 @@ private TFileFormatType formatType(String fileFormat, String path) { } } - private Map parsePartitionedFields(String filePath) { - for (BrokerFileGroup fileGroup : fileGroups) { - String fileFormat = fileGroup.getFileFormat(); - if ((fileFormat == null || !fileFormat.toLowerCase().equals("parquet")) || !filePath.endsWith(".parquet")) { - continue; - } + private Map parsePartitionedFields(String filePath, BrokerFileGroup fileGroup) { + String basePath = fileGroup.getBasePath(); + if (basePath == null) { + // find the default base path for (String base : fileGroup.getFilePaths()) { if (base.endsWith("/*")) { base = base.substring(0, base.indexOf("/*")); } - if (!filePath.startsWith(base)) { - continue; + if (filePath.startsWith(base)) { + basePath = base; + break; } - String subPath = filePath.substring(base.length()); - String[] strings = subPath.split("/"); - Map partitionedFields = new HashMap<>(); - for (String str : strings) { - if (str == null || str.isEmpty() || !str.contains("=")) { - continue; - } - String[] pair = str.split("="); - if (pair.length != 2) { - continue; - } - Column column = targetTable.getColumn(pair[0]); - if (column == null) { - continue; - } - partitionedFields.put(pair[0], pair[1]); - } - return partitionedFields; } } - return Collections.emptyMap(); + + if (basePath == null || !filePath.startsWith(basePath)) { + return Collections.emptyMap(); + } + List fileFieldNames = fileGroup.getFileFieldNames(); + String subPath = filePath.substring(basePath.length()); + String[] strings = subPath.split("/"); + Map partitionedFields = new HashMap<>(); + for (String str : strings) { + if (str == null || str.isEmpty() || !str.contains("=")) { + continue; + } + String[] pair = str.split("="); + if (pair.length != 2) { + continue; + } + if (!fileFieldNames.contains(pair[0])) { + continue; + } + partitionedFields.put(pair[0], pair[1]); + } + return partitionedFields; } // If fileFormat is not null, we use fileFormat instead of check file's suffix private void processFileGroup( - String fileFormat, - TBrokerScanRangeParams params, + ParamCreateContext context, List fileStatuses) throws UserException { if (fileStatuses == null || fileStatuses.isEmpty()) { return; } + String fileFormat = context.fileGroup.getFileFormat(); + TBrokerScanRangeParams params = context.params; + BrokerFileGroup fileGroup = context.fileGroup; TScanRangeLocations curLocations = newLocations(params, brokerDesc.getName()); long curInstanceBytes = 0; @@ -647,11 +651,11 @@ private void processFileGroup( // Now only support split plain text if (formatType == TFileFormatType.FORMAT_CSV_PLAIN && fileStatus.isSplitable) { long rangeBytes = bytesPerInstance - curInstanceBytes; - TBrokerRangeDesc rangeDesc = createBrokerRangeDesc(curFileOffset, fileStatus, formatType, rangeBytes); + TBrokerRangeDesc rangeDesc = createBrokerRangeDesc(curFileOffset, fileStatus, formatType, rangeBytes, fileGroup); brokerScanRange(curLocations).addToRanges(rangeDesc); curFileOffset += rangeBytes; } else { - TBrokerRangeDesc rangeDesc = createBrokerRangeDesc(curFileOffset, fileStatus, formatType, leftBytes); + TBrokerRangeDesc rangeDesc = createBrokerRangeDesc(curFileOffset, fileStatus, formatType, leftBytes, fileGroup); brokerScanRange(curLocations).addToRanges(rangeDesc); curFileOffset = 0; i++; @@ -663,7 +667,7 @@ private void processFileGroup( curInstanceBytes = 0; } else { - TBrokerRangeDesc rangeDesc = createBrokerRangeDesc(curFileOffset, fileStatus, formatType, leftBytes); + TBrokerRangeDesc rangeDesc = createBrokerRangeDesc(curFileOffset, fileStatus, formatType, leftBytes, fileGroup); brokerScanRange(curLocations).addToRanges(rangeDesc); curFileOffset = 0; curInstanceBytes += leftBytes; @@ -678,7 +682,7 @@ private void processFileGroup( } private TBrokerRangeDesc createBrokerRangeDesc(long curFileOffset, TBrokerFileStatus fileStatus, - TFileFormatType formatType, long rangeBytes) { + TFileFormatType formatType, long rangeBytes, BrokerFileGroup fileGroup) { TBrokerRangeDesc rangeDesc = new TBrokerRangeDesc(); rangeDesc.setFile_type(TFileType.FILE_BROKER); rangeDesc.setFormat_type(formatType); @@ -687,7 +691,7 @@ private TBrokerRangeDesc createBrokerRangeDesc(long curFileOffset, TBrokerFileSt rangeDesc.setStart_offset(curFileOffset); rangeDesc.setSize(rangeBytes); rangeDesc.setFile_size(fileStatus.size); - rangeDesc.setPartition_columns(parsePartitionedFields(fileStatus.path)); + rangeDesc.setPartition_columns(parsePartitionedFields(fileStatus.path, fileGroup)); return rangeDesc; } @@ -706,7 +710,7 @@ public void finalize(Analyzer analyzer) throws UserException { } catch (AnalysisException e) { throw new UserException(e.getMessage()); } - processFileGroup(context.fileGroup.getFileFormat(), context.params, fileStatuses); + processFileGroup(context, fileStatuses); } if (LOG.isDebugEnabled()) { for (TScanRangeLocations locations : locationsList) { diff --git a/fe/src/main/jflex/sql_scanner.flex b/fe/src/main/jflex/sql_scanner.flex index 07d1fb61999444..feaeb12e0091ac 100644 --- a/fe/src/main/jflex/sql_scanner.flex +++ b/fe/src/main/jflex/sql_scanner.flex @@ -155,6 +155,7 @@ import org.apache.doris.common.util.SqlUtils; keywordMap.put("following", new Integer(SqlParserSymbols.KW_FOLLOWING)); keywordMap.put("for", new Integer(SqlParserSymbols.KW_FOR)); keywordMap.put("format", new Integer(SqlParserSymbols.KW_FORMAT)); + keywordMap.put("base_path", new Integer(SqlParserSymbols.KW_BASE_PATH)); keywordMap.put("from", new Integer(SqlParserSymbols.KW_FROM)); keywordMap.put("frontend", new Integer(SqlParserSymbols.KW_FRONTEND)); keywordMap.put("frontends", new Integer(SqlParserSymbols.KW_FRONTENDS)); From 66b9ffd1b952a53efd0ded03f48acdca47afe450 Mon Sep 17 00:00:00 2001 From: yuanlihan Date: Mon, 5 Aug 2019 00:14:27 +0800 Subject: [PATCH 24/26] Update comment --- gensrc/thrift/PlanNodes.thrift | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gensrc/thrift/PlanNodes.thrift b/gensrc/thrift/PlanNodes.thrift index fd3529fd3a3d9c..bbbfe2a7d9325b 100644 --- a/gensrc/thrift/PlanNodes.thrift +++ b/gensrc/thrift/PlanNodes.thrift @@ -114,7 +114,7 @@ struct TBrokerRangeDesc { 7: optional Types.TUniqueId load_id // total size of the file 8: optional i64 file_size - // partition columns of this file(parquet only) + // partition columns of this file 9: optional map partition_columns } From 2f11132a8abb5ab4658091f1a40a047764ed24fe Mon Sep 17 00:00:00 2001 From: yuanlihan Date: Mon, 12 Aug 2019 01:13:45 +0800 Subject: [PATCH 25/26] Enable parsing columns from file path for Broker Load --- be/src/exec/broker_scanner.cpp | 12 ++-- be/src/exec/broker_scanner.h | 2 +- be/src/exec/parquet_reader.cpp | 24 ++++---- be/src/exec/parquet_reader.h | 4 +- be/src/exec/parquet_scanner.cpp | 2 +- be/test/exec/broker_scan_node_test.cpp | 8 +-- be/test/exec/parquet_scanner_test.cpp | 4 +- fe/src/main/cup/sql_parser.cup | 20 +++--- .../doris/analysis/DataDescription.java | 24 +++++--- .../apache/doris/common/FeMetaVersion.java | 2 +- .../apache/doris/common/util/BrokerUtil.java | 50 ++++++--------- .../apache/doris/load/BrokerFileGroup.java | 52 ++++++++++------ .../main/java/org/apache/doris/load/Load.java | 10 ++- .../apache/doris/planner/BrokerScanNode.java | 61 +++++++++---------- fe/src/main/jflex/sql_scanner.flex | 2 +- gensrc/thrift/PlanNodes.thrift | 4 +- 16 files changed, 149 insertions(+), 132 deletions(-) diff --git a/be/src/exec/broker_scanner.cpp b/be/src/exec/broker_scanner.cpp index 062a476ab537ea..da7db208a14d2b 100644 --- a/be/src/exec/broker_scanner.cpp +++ b/be/src/exec/broker_scanner.cpp @@ -54,7 +54,7 @@ BrokerScanner::BrokerScanner(RuntimeState* state, _cur_decompressor(nullptr), _next_range(0), _cur_line_reader_eof(false), - _partition_columns(), + _columns_from_path(), _scanner_eof(false), _skip_next_line(false) { } @@ -238,7 +238,7 @@ Status BrokerScanner::open_line_reader() { RETURN_IF_ERROR(create_decompressor(range.format_type)); // set partitioned columns - _partition_columns = range.partition_columns; + _columns_from_path = range.columns_from_path; // open line reader switch (range.format_type) { @@ -484,7 +484,7 @@ bool BrokerScanner::line_to_src_tuple(const Slice& line) { split_line(line, &values); } - if (values.size() + _partition_columns.size() < _src_slot_descs.size()) { + if (values.size() + _columns_from_path.size() < _src_slot_descs.size()) { std::stringstream error_msg; error_msg << "actual column number is less than schema column number. " << "actual number: " << values.size() << " sep: " << _value_separator << ", " @@ -493,7 +493,7 @@ bool BrokerScanner::line_to_src_tuple(const Slice& line) { error_msg.str()); _counter->num_rows_filtered++; return false; - } else if (values.size() + _partition_columns.size() > _src_slot_descs.size()) { + } else if (values.size() + _columns_from_path.size() > _src_slot_descs.size()) { std::stringstream error_msg; error_msg << "actual column number is more than schema column number. " << "actual number: " << values.size() << " sep: " << _value_separator << ", " @@ -507,8 +507,8 @@ bool BrokerScanner::line_to_src_tuple(const Slice& line) { int file_column_index = 0; for (int i = 0; i < _src_slot_descs.size(); ++i) { auto slot_desc = _src_slot_descs[i]; - auto iter = _partition_columns.find(slot_desc->col_name()); - if (iter != _partition_columns.end()) { + auto iter = _columns_from_path.find(slot_desc->col_name()); + if (iter != _columns_from_path.end()) { std::string partitioned_field = iter->second; const Slice value = Slice(partitioned_field.c_str(), partitioned_field.size()); fill_slot(slot_desc, value); diff --git a/be/src/exec/broker_scanner.h b/be/src/exec/broker_scanner.h index ea464dfc48f6d3..e21db9263554eb 100644 --- a/be/src/exec/broker_scanner.h +++ b/be/src/exec/broker_scanner.h @@ -121,7 +121,7 @@ private:; Decompressor* _cur_decompressor; int _next_range; bool _cur_line_reader_eof; - std::map _partition_columns; + std::map _columns_from_path; bool _scanner_eof; diff --git a/be/src/exec/parquet_reader.cpp b/be/src/exec/parquet_reader.cpp index 88bb51b4e4574f..95d5e9a4390b9d 100644 --- a/be/src/exec/parquet_reader.cpp +++ b/be/src/exec/parquet_reader.cpp @@ -34,8 +34,8 @@ namespace doris { // Broker -ParquetReaderWrap::ParquetReaderWrap(FileReader *file_reader, const std::map& partition_columns) : - _partition_columns(partition_columns), _total_groups(0), _current_group(0), _rows_of_group(0), _current_line_of_group(0) { +ParquetReaderWrap::ParquetReaderWrap(FileReader *file_reader, const std::map& columns_from_path) : + _columns_from_path(columns_from_path), _total_groups(0), _current_group(0), _rows_of_group(0), _current_line_of_group(0) { _parquet = std::shared_ptr(new ParquetFile(file_reader)); _properties = parquet::ReaderProperties(); _properties.enable_buffered_stream(); @@ -128,8 +128,8 @@ Status ParquetReaderWrap::column_indices(const std::vector& tup if (iter != _map_column.end()) { _parquet_column_ids.emplace_back(iter->second); } else { - auto iter_1 = _partition_columns.find(slot_desc->col_name()); - if (iter_1 == _partition_columns.end()) { + auto iter_1 = _columns_from_path.find(slot_desc->col_name()); + if (iter_1 == _columns_from_path.end()) { std::stringstream str_error; str_error << "Invalid Column Name:" << slot_desc->col_name(); LOG(WARNING) << str_error.str(); @@ -214,12 +214,12 @@ Status ParquetReaderWrap::read(Tuple* tuple, const std::vector& size_t slots = tuple_slot_descs.size(); for (size_t i = 0; i < slots; ++i) { auto slot_desc = tuple_slot_descs[i]; - auto iter = _partition_columns.find(slot_desc->col_name()); - if (iter != _partition_columns.end()) { + auto iter = _columns_from_path.find(slot_desc->col_name()); + if (iter != _columns_from_path.end()) { std::string partitioned_field = iter->second; value = reinterpret_cast(partitioned_field.c_str()); - wbtyes = partitioned_field.size(); - fill_slot(tuple, slot_desc, mem_pool, value, wbtyes); + wbytes = partitioned_field.size(); + fill_slot(tuple, slot_desc, mem_pool, value, wbytes); continue; } else { column_index = index++; // column index in batch record @@ -419,8 +419,8 @@ Status ParquetReaderWrap::read(Tuple* tuple, const std::vector& tm* local; local = localtime(×tamp); char* to = reinterpret_cast(&tmp_buf); - wbtyes = (uint32_t)strftime(to, 64, "%Y-%m-%d", local); - fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbtyes); + wbytes = (uint32_t)strftime(to, 64, "%Y-%m-%d", local); + fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbytes); } break; } @@ -434,8 +434,8 @@ Status ParquetReaderWrap::read(Tuple* tuple, const std::vector& tm* local; local = localtime(×tamp); char* to = reinterpret_cast(&tmp_buf); - wbtyes = (uint32_t)strftime(to, 64, "%Y-%m-%d %H:%M:%S", local); - fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbtyes); + wbytes = (uint32_t)strftime(to, 64, "%Y-%m-%d %H:%M:%S", local); + fill_slot(tuple, slot_desc, mem_pool, tmp_buf, wbytes); } break; } diff --git a/be/src/exec/parquet_reader.h b/be/src/exec/parquet_reader.h index 107588d79d9aa8..d1445e63870706 100644 --- a/be/src/exec/parquet_reader.h +++ b/be/src/exec/parquet_reader.h @@ -68,7 +68,7 @@ class ParquetFile : public arrow::io::RandomAccessFile { // Reader of broker parquet file class ParquetReaderWrap { public: - ParquetReaderWrap(FileReader *file_reader, const std::map& partition_columns); + ParquetReaderWrap(FileReader *file_reader, const std::map& columns_from_path); virtual ~ParquetReaderWrap(); // Read @@ -85,7 +85,7 @@ class ParquetReaderWrap { Status handle_timestamp(const std::shared_ptr& ts_array, uint8_t *buf, int32_t *wbtyes); private: - const std::map& _partition_columns; + const std::map& _columns_from_path; parquet::ReaderProperties _properties; std::shared_ptr _parquet; diff --git a/be/src/exec/parquet_scanner.cpp b/be/src/exec/parquet_scanner.cpp index 9d8550fabe51f7..0d4bcfdfeffcd3 100644 --- a/be/src/exec/parquet_scanner.cpp +++ b/be/src/exec/parquet_scanner.cpp @@ -141,7 +141,7 @@ Status ParquetScanner::open_next_reader() { file_reader->close(); continue; } - _cur_file_reader = new ParquetReaderWrap(file_reader.release(), range.partition_columns); + _cur_file_reader = new ParquetReaderWrap(file_reader.release(), range.columns_from_path); Status status = _cur_file_reader->init_parquet_reader(_src_slot_descs); if (status.is_end_of_file()) { continue; diff --git a/be/test/exec/broker_scan_node_test.cpp b/be/test/exec/broker_scan_node_test.cpp index bc4cc533d2455d..2805a8d1beda0d 100644 --- a/be/test/exec/broker_scan_node_test.cpp +++ b/be/test/exec/broker_scan_node_test.cpp @@ -420,8 +420,8 @@ TEST_F(BrokerScanNodeTest, normal) { range.file_type = TFileType::FILE_LOCAL; range.format_type = TFileFormatType::FORMAT_CSV_PLAIN; range.splittable = true; - std::map partition_columns = {{"k4", "1"}}; - range.__set_partition_columns(partition_columns); + std::map columns_from_path = {{"k4", "1"}}; + range.__set_columns_from_path(columns_from_path); broker_scan_range.ranges.push_back(range); scan_range_params.scan_range.__set_broker_scan_range(broker_scan_range); @@ -441,8 +441,8 @@ TEST_F(BrokerScanNodeTest, normal) { range.file_type = TFileType::FILE_LOCAL; range.format_type = TFileFormatType::FORMAT_CSV_PLAIN; range.splittable = true; - std::map partition_columns = {{"k4", "2"}}; - range.__set_partition_columns(partition_columns); + std::map columns_from_path = {{"k4", "2"}}; + range.__set_columns_from_path(columns_from_path); broker_scan_range.ranges.push_back(range); scan_range_params.scan_range.__set_broker_scan_range(broker_scan_range); diff --git a/be/test/exec/parquet_scanner_test.cpp b/be/test/exec/parquet_scanner_test.cpp index e95472e8b947f8..7d36c6fba34d68 100644 --- a/be/test/exec/parquet_scanner_test.cpp +++ b/be/test/exec/parquet_scanner_test.cpp @@ -436,8 +436,8 @@ TEST_F(ParquetSannerTest, normal) { range.format_type = TFileFormatType::FORMAT_PARQUET; range.splittable = true; - std::map partition_columns = {{"partition_column", "value"}}; - range.__set_partition_columns(partition_columns); + std::map columns_from_path = {{"partition_column", "value"}}; + range.__set_columns_from_path(columns_from_path); #if 1 range.path = "./be/test/exec/test_data/parquet_scanner/localfile.parquet"; range.file_type = TFileType::FILE_LOCAL; diff --git a/fe/src/main/cup/sql_parser.cup b/fe/src/main/cup/sql_parser.cup index bc4ee491d6c303..735bca9a857dc8 100644 --- a/fe/src/main/cup/sql_parser.cup +++ b/fe/src/main/cup/sql_parser.cup @@ -191,7 +191,7 @@ parser code {: // Total keywords of doris terminal String KW_ADD, KW_ADMIN, KW_AFTER, KW_AGGREGATE, KW_ALL, KW_ALTER, KW_AND, KW_ANTI, KW_AS, KW_ASC, KW_AUTHORS, - KW_BACKEND, KW_BACKUP, KW_BASE_PATH, KW_BETWEEN, KW_BEGIN, KW_BIGINT, KW_BOOLEAN, KW_BOTH, KW_BROKER, KW_BACKENDS, KW_BY, + KW_BACKEND, KW_BACKUP, KW_BETWEEN, KW_BEGIN, KW_BIGINT, KW_BOOLEAN, KW_BOTH, KW_BROKER, KW_BACKENDS, KW_BY, KW_CANCEL, KW_CASE, KW_CAST, KW_CHAIN, KW_CHAR, KW_CHARSET, KW_CLUSTER, KW_CLUSTERS, KW_COLLATE, KW_COLLATION, KW_COLUMN, KW_COLUMNS, KW_COMMENT, KW_COMMIT, KW_COMMITTED, KW_CONFIG, KW_CONNECTION, KW_CONNECTION_ID, KW_CONSISTENT, KW_COUNT, KW_CREATE, KW_CROSS, KW_CURRENT, KW_CURRENT_USER, @@ -211,7 +211,7 @@ terminal String KW_ADD, KW_ADMIN, KW_AFTER, KW_AGGREGATE, KW_ALL, KW_ALTER, KW_A KW_MAX, KW_MAX_VALUE, KW_MERGE, KW_MIN, KW_MIGRATE, KW_MIGRATIONS, KW_MODIFY, KW_NAME, KW_NAMES, KW_NEGATIVE, KW_NO, KW_NOT, KW_NULL, KW_NULLS, KW_OBSERVER, KW_OFFSET, KW_ON, KW_ONLY, KW_OPEN, KW_OR, KW_ORDER, KW_OUTER, KW_OVER, - KW_PARTITION, KW_PARTITIONS, KW_PRECEDING, + KW_PARTITION, KW_PARTITIONS, KW_PATH, KW_PRECEDING, KW_PASSWORD, KW_PLUGIN, KW_PLUGINS, KW_PRIMARY, KW_PROC, KW_PROCEDURE, KW_PROCESSLIST, KW_PROPERTIES, KW_PROPERTY, @@ -379,7 +379,7 @@ nonterminal LabelName job_label; nonterminal String opt_system; nonterminal String opt_cluster; nonterminal BrokerDesc opt_broker; -nonterminal List opt_col_list, col_list, opt_dup_keys; +nonterminal List opt_col_list, col_list, opt_dup_keys, opt_columns_from_path; nonterminal List opt_partitions, partitions; nonterminal List opt_col_mapping_list; nonterminal ColumnSeparator opt_field_term, column_separator; @@ -403,7 +403,7 @@ nonterminal List alter_table_clause_list; // nonterminal String keyword, ident, ident_or_text, variable_name, text_or_password, charset_name_or_default, old_or_new_charset_name_or_default, opt_collate, - collation_name_or_default, type_func_name_keyword, type_function_name, opt_file_format, opt_base_path; + collation_name_or_default, type_func_name_keyword, type_function_name, opt_file_format; nonterminal String opt_db, opt_partition_name, procedure_or_function, opt_comment, opt_engine; nonterminal ColumnDef.DefaultValue opt_default_value; @@ -1050,11 +1050,11 @@ data_desc ::= opt_partitions:partitionNames opt_field_term:colSep opt_file_format:fileFormat - opt_base_path:basePath + opt_columns_from_path:columnsFromPath opt_col_list:colList opt_col_mapping_list:colMappingList {: - RESULT = new DataDescription(tableName, partitionNames, files, colList, colSep, fileFormat, basePath, isNeg, colMappingList); + RESULT = new DataDescription(tableName, partitionNames, files, colList, colSep, fileFormat, columnsFromPath, isNeg, colMappingList); :} ; @@ -1111,11 +1111,11 @@ opt_file_format ::= {: RESULT = format; :} ; -opt_base_path ::= +opt_columns_from_path ::= /* Empty */ {: RESULT = null; :} - | KW_BASE_PATH KW_AS ident_or_text:basePath - {: RESULT = basePath; :} + | KW_COLUMNS KW_FROM KW_PATH KW_AS LPAREN ident_list:columnsFromPath RPAREN + {: RESULT = columnsFromPath; :} ; opt_col_list ::= @@ -3932,7 +3932,7 @@ keyword ::= {: RESULT = id; :} | KW_FORMAT:id {: RESULT = id; :} - | KW_BASE_PATH:id + | KW_PATH:id {: RESULT = id; :} | KW_FUNCTION:id {: RESULT = id; :} diff --git a/fe/src/main/java/org/apache/doris/analysis/DataDescription.java b/fe/src/main/java/org/apache/doris/analysis/DataDescription.java index 2e200fde2b1df4..d58ed054e554fb 100644 --- a/fe/src/main/java/org/apache/doris/analysis/DataDescription.java +++ b/fe/src/main/java/org/apache/doris/analysis/DataDescription.java @@ -49,6 +49,7 @@ // [PARTITION (p1, p2)] // [COLUMNS TERMINATED BY separator] // [FORMAT AS format] +// [COLUMNS FROM PATH AS (col1, ...)] // [(col1, ...)] // [SET (k1=f1(xx), k2=f2(xx))] public class DataDescription { @@ -60,7 +61,7 @@ public class DataDescription { private final List columnNames; private final ColumnSeparator columnSeparator; private final String fileFormat; - private final String basePath; + private final List columnsFromPath; private final boolean isNegative; private final List columnMappingList; @@ -86,7 +87,7 @@ public DataDescription(String tableName, this.columnNames = columnNames; this.columnSeparator = columnSeparator; this.fileFormat = null; - this.basePath = null; + this.columnsFromPath = null; this.isNegative = isNegative; this.columnMappingList = columnMappingList; } @@ -97,7 +98,7 @@ public DataDescription(String tableName, List columnNames, ColumnSeparator columnSeparator, String fileFormat, - String basePath, + List columnsFromPath, boolean isNegative, List columnMappingList) { this.tableName = tableName; @@ -106,7 +107,7 @@ public DataDescription(String tableName, this.columnNames = columnNames; this.columnSeparator = columnSeparator; this.fileFormat = fileFormat; - this.basePath = basePath; + this.columnsFromPath = columnsFromPath; this.isNegative = isNegative; this.columnMappingList = columnMappingList; } @@ -142,8 +143,8 @@ public String getFileFormat() { return fileFormat; } - public String getBasePath() { - return basePath; + public List getColumnsFromPath() { + return columnsFromPath; } public String getColumnSeparator() { @@ -209,11 +210,18 @@ public boolean isPullLoad() { } private void checkColumnInfo() throws AnalysisException { - if (columnNames == null || columnNames.isEmpty()) { + List columns = Lists.newArrayList(); + if (columnNames != null) { + columns.addAll(columnNames); + } + if (columnsFromPath != null) { + columns.addAll(columnsFromPath); + } + if (columns.isEmpty()) { return; } Set columnSet = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); - for (String col : columnNames) { + for (String col : columns) { if (!columnSet.add(col)) { ErrorReport.reportAnalysisException(ErrorCode.ERR_DUP_FIELDNAME, col); } diff --git a/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java b/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java index 05721da5a989b4..42875470db5bd4 100644 --- a/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java +++ b/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java @@ -124,6 +124,6 @@ public final class FeMetaVersion { public static final int VERSION_56 = 56; // for base index using different id public static final int VERSION_57 = 57; - // basePath for partition discovery when loading data using broker scan + // columns parsed from file path when loading data using broker scan public static final int VERSION_58 = 58; } diff --git a/fe/src/main/java/org/apache/doris/common/util/BrokerUtil.java b/fe/src/main/java/org/apache/doris/common/util/BrokerUtil.java index 725d8160a2444c..a6cb66832c5369 100644 --- a/fe/src/main/java/org/apache/doris/common/util/BrokerUtil.java +++ b/fe/src/main/java/org/apache/doris/common/util/BrokerUtil.java @@ -61,10 +61,28 @@ public static void parseBrokerFile(String path, BrokerDesc brokerDesc, List partitionIds; private List fileFieldNames; + private List columnsFromPath; private List filePaths; // This column need expression to get column @@ -83,7 +83,7 @@ public BrokerFileGroup(BrokerTable table) throws AnalysisException { public BrokerFileGroup(DataDescription dataDescription) { this.dataDescription = dataDescription; - this.basePath = dataDescription.getBasePath(); + this.columnsFromPath = dataDescription.getColumnsFromPath(); exprColumnMap = dataDescription.getParsedExprMap(); } @@ -161,8 +161,8 @@ public String getFileFormat() { return fileFormat; } - public String getBasePath() { - return basePath; + public List getColumnsFromPath() { + return columnsFromPath; } public boolean isNegative() { @@ -211,10 +211,20 @@ public String toString() { } sb.append("]"); } + if (columnsFromPath != null) { + sb.append(",columnsFromPath=["); + int idx = 0; + for (String name : columnsFromPath) { + if (idx++ != 0) { + sb.append(","); + } + sb.append(name); + } + sb.append("]"); + } sb.append(",valueSeparator=").append(valueSeparator) .append(",lineDelimiter=").append(lineDelimiter) .append(",fileFormat=").append(fileFormat) - .append(",basePath=").append(basePath) .append(",isNegative=").append(isNegative); sb.append(",fileInfos=["); int idx = 0; @@ -259,6 +269,15 @@ public void write(DataOutput out) throws IOException { Text.writeString(out, name); } } + // columnsFromPath + if (columnsFromPath == null) { + out.writeInt(0); + } else { + out.writeInt(columnsFromPath.size()); + for (String name : columnsFromPath) { + Text.writeString(out, name); + } + } // filePaths out.writeInt(filePaths.size()); for (String path : filePaths) { @@ -282,13 +301,6 @@ public void write(DataOutput out) throws IOException { out.writeBoolean(true); Text.writeString(out, fileFormat); } - // basePath - if (basePath == null) { - out.writeBoolean(false); - } else { - out.writeBoolean(true); - Text.writeString(out, basePath); - } } @Override @@ -317,6 +329,16 @@ public void readFields(DataInput in) throws IOException { } } } + // columnsFromPath + if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_58) { + int fileFieldNameSize = in.readInt(); + if (fileFieldNameSize > 0) { + columnsFromPath = Lists.newArrayList(); + for (int i = 0; i < fileFieldNameSize; ++i) { + columnsFromPath.add(Text.readString(in)); + } + } + } // fileInfos { int size = in.readInt(); @@ -342,12 +364,6 @@ public void readFields(DataInput in) throws IOException { fileFormat = Text.readString(in); } } - // basePath - if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_58) { - if (in.readBoolean()) { - basePath = Text.readString(in); - } - } } public static BrokerFileGroup read(DataInput in) throws IOException { diff --git a/fe/src/main/java/org/apache/doris/load/Load.java b/fe/src/main/java/org/apache/doris/load/Load.java index 536e8dd14a4974..1004730315ad44 100644 --- a/fe/src/main/java/org/apache/doris/load/Load.java +++ b/fe/src/main/java/org/apache/doris/load/Load.java @@ -703,8 +703,14 @@ public static void checkAndCreateSource(Database db, DataDescription dataDescrip // source columns List columnNames = Lists.newArrayList(); - List assignColumnNames = dataDescription.getColumnNames(); - if (assignColumnNames == null) { + List assignColumnNames = Lists.newArrayList(); + if (dataDescription.getColumnNames() != null) { + assignColumnNames.addAll(dataDescription.getColumnNames()); + } + if (dataDescription.getColumnsFromPath() != null) { + assignColumnNames.addAll(dataDescription.getColumnsFromPath()); + } + if (assignColumnNames.isEmpty()) { // use table columns for (Column column : tableSchema) { columnNames.add(column.getName()); diff --git a/fe/src/main/java/org/apache/doris/planner/BrokerScanNode.java b/fe/src/main/java/org/apache/doris/planner/BrokerScanNode.java index 84d8e02357e244..af4a84a6e13ded 100644 --- a/fe/src/main/java/org/apache/doris/planner/BrokerScanNode.java +++ b/fe/src/main/java/org/apache/doris/planner/BrokerScanNode.java @@ -353,15 +353,20 @@ private void initParams(ParamCreateContext context) throws AnalysisException, Us parseExprMap(context.exprMap); // Generate expr - List fileFieldNames = fileGroup.getFileFieldNames(); - if (fileFieldNames == null) { - fileFieldNames = Lists.newArrayList(); + List fieldNames = Lists.newArrayList(); + if (fileGroup.getFileFieldNames() != null) { + fieldNames.addAll(fileGroup.getFileFieldNames()); + } + if (fileGroup.getColumnsFromPath() != null) { + fieldNames.addAll(fileGroup.getColumnsFromPath()); + } + if (fieldNames.isEmpty()) { for (Column column : targetTable.getBaseSchema()) { - fileFieldNames.add(column.getName()); + fieldNames.add(column.getName()); } } else { // change fileFiledName to real column name(case match) - fileFieldNames = fileFieldNames.stream().map( + fieldNames = fieldNames.stream().map( f -> targetTable.getColumn(f) == null ? f : targetTable.getColumn(f).getName()).collect( Collectors.toList()); } @@ -372,7 +377,7 @@ private void initParams(ParamCreateContext context) throws AnalysisException, Us Map slotDescByName = Maps.newHashMap(); context.slotDescByName = slotDescByName; - for (String fieldName : fileFieldNames) { + for (String fieldName : fieldNames) { SlotDescriptor slotDesc = analyzer.getDescTbl().addSlotDescriptor(srcTupleDesc); slotDesc.setType(ScalarType.createType(PrimitiveType.VARCHAR)); slotDesc.setIsMaterialized(true); @@ -589,29 +594,15 @@ private TFileFormatType formatType(String fileFormat, String path) { } } - private Map parsePartitionedFields(String filePath, BrokerFileGroup fileGroup) { - String basePath = fileGroup.getBasePath(); - if (basePath == null) { - // find the default base path - for (String base : fileGroup.getFilePaths()) { - if (base.endsWith("/*")) { - base = base.substring(0, base.indexOf("/*")); - } - if (filePath.startsWith(base)) { - basePath = base; - break; - } - } - } - - if (basePath == null || !filePath.startsWith(basePath)) { + private Map parseColumnsFromPath(String filePath, BrokerFileGroup fileGroup) throws UserException { + List columnsFromPath = fileGroup.getColumnsFromPath(); + if (columnsFromPath == null || columnsFromPath.isEmpty()) { return Collections.emptyMap(); } - List fileFieldNames = fileGroup.getFileFieldNames(); - String subPath = filePath.substring(basePath.length()); - String[] strings = subPath.split("/"); - Map partitionedFields = new HashMap<>(); - for (String str : strings) { + String[] strings = filePath.split("/"); + Map columns = new HashMap<>(); + for (int i = strings.length - 1; i >= 0; i--) { + String str = strings[i]; if (str == null || str.isEmpty() || !str.contains("=")) { continue; } @@ -619,12 +610,18 @@ private Map parsePartitionedFields(String filePath, BrokerFileGr if (pair.length != 2) { continue; } - if (!fileFieldNames.contains(pair[0])) { + if (!columnsFromPath.contains(pair[0])) { continue; } - partitionedFields.put(pair[0], pair[1]); + columns.put(pair[0], pair[1]); + if (columns.size() > columnsFromPath.size()) { + break; + } + } + if (columns.size() != columnsFromPath.size()) { + throw new UserException("Fail to parse columnsFromPath, expected: " + columnsFromPath + ", filePath: " + filePath); } - return partitionedFields; + return columns; } // If fileFormat is not null, we use fileFormat instead of check file's suffix @@ -682,7 +679,7 @@ private void processFileGroup( } private TBrokerRangeDesc createBrokerRangeDesc(long curFileOffset, TBrokerFileStatus fileStatus, - TFileFormatType formatType, long rangeBytes, BrokerFileGroup fileGroup) { + TFileFormatType formatType, long rangeBytes, BrokerFileGroup fileGroup) throws UserException { TBrokerRangeDesc rangeDesc = new TBrokerRangeDesc(); rangeDesc.setFile_type(TFileType.FILE_BROKER); rangeDesc.setFormat_type(formatType); @@ -691,7 +688,7 @@ private TBrokerRangeDesc createBrokerRangeDesc(long curFileOffset, TBrokerFileSt rangeDesc.setStart_offset(curFileOffset); rangeDesc.setSize(rangeBytes); rangeDesc.setFile_size(fileStatus.size); - rangeDesc.setPartition_columns(parsePartitionedFields(fileStatus.path, fileGroup)); + rangeDesc.setColumns_from_path(parseColumnsFromPath(fileStatus.path, fileGroup)); return rangeDesc; } diff --git a/fe/src/main/jflex/sql_scanner.flex b/fe/src/main/jflex/sql_scanner.flex index feaeb12e0091ac..724849f4b8e26b 100644 --- a/fe/src/main/jflex/sql_scanner.flex +++ b/fe/src/main/jflex/sql_scanner.flex @@ -155,7 +155,6 @@ import org.apache.doris.common.util.SqlUtils; keywordMap.put("following", new Integer(SqlParserSymbols.KW_FOLLOWING)); keywordMap.put("for", new Integer(SqlParserSymbols.KW_FOR)); keywordMap.put("format", new Integer(SqlParserSymbols.KW_FORMAT)); - keywordMap.put("base_path", new Integer(SqlParserSymbols.KW_BASE_PATH)); keywordMap.put("from", new Integer(SqlParserSymbols.KW_FROM)); keywordMap.put("frontend", new Integer(SqlParserSymbols.KW_FRONTEND)); keywordMap.put("frontends", new Integer(SqlParserSymbols.KW_FRONTENDS)); @@ -228,6 +227,7 @@ import org.apache.doris.common.util.SqlUtils; keywordMap.put("over", new Integer(SqlParserSymbols.KW_OVER)); keywordMap.put("partition", new Integer(SqlParserSymbols.KW_PARTITION)); keywordMap.put("partitions", new Integer(SqlParserSymbols.KW_PARTITIONS)); + keywordMap.put("path", new Integer(SqlParserSymbols.KW_PATH)); keywordMap.put("preceding", new Integer(SqlParserSymbols.KW_PRECEDING)); keywordMap.put("range", new Integer(SqlParserSymbols.KW_RANGE)); keywordMap.put("password", new Integer(SqlParserSymbols.KW_PASSWORD)); diff --git a/gensrc/thrift/PlanNodes.thrift b/gensrc/thrift/PlanNodes.thrift index bbbfe2a7d9325b..1ad19717a75acd 100644 --- a/gensrc/thrift/PlanNodes.thrift +++ b/gensrc/thrift/PlanNodes.thrift @@ -114,8 +114,8 @@ struct TBrokerRangeDesc { 7: optional Types.TUniqueId load_id // total size of the file 8: optional i64 file_size - // partition columns of this file - 9: optional map partition_columns + // columns parsed from file path + 9: optional map columns_from_path } struct TBrokerScanRangeParams { From 3ed92488877b18225657f291c4ce0c283b8d3fba Mon Sep 17 00:00:00 2001 From: yuanlihan Date: Mon, 12 Aug 2019 01:17:14 +0800 Subject: [PATCH 26/26] Update docs --- .../Data Manipulation/broker_load.md | 51 +++++++------------ 1 file changed, 19 insertions(+), 32 deletions(-) diff --git a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/broker_load.md b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/broker_load.md index 503201548665a4..ba8ac9f27c1b4b 100644 --- a/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/broker_load.md +++ b/docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/broker_load.md @@ -39,7 +39,7 @@ [PARTITION (p1, p2)] [COLUMNS TERMINATED BY "column_separator"] [FORMAT AS "file_type"] - [BASE_PATH AS "base_path"] + [COLUMNS FROM PATH AS (columns_from_path)] [(column_list)] [SET (k1 = func(k2))] @@ -67,9 +67,11 @@ 用于指定导入文件的类型,例如:parquet、csv。默认值通过文件后缀名判断。 - base_path: + columns_from_path: - 用于指定作为Partition Discovery的基础路径。 + 用于指定需要从文件路径中解析的字段。 + 语法: + (col_from_path_name1, col_from_path_name2, ...) column_list: @@ -367,36 +369,21 @@ ) WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); - 9. 通过Partition Discovery提取文件路径中的压缩字段 - 如果导入路径为目录,则递归地列出该目录下的所有parquet文件 - 如果需要,则会根据表中定义的字段类型解析文件路径中的partitioned fields,实现类似Spark中读parquet文件 - 1. 不指定Partition Discovery的基础路径(BASE_PATH) - LOAD LABEL example_db.label10 - ( - DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir") - INTO TABLE `my_table` - FORMAT AS "parquet" - (k1, k2, k3) - ) - WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); - - hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir目录下包括如下文件:[hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/k1=key1/xxx.parquet, hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/k1=key2/xxx.parquet, ...] - 则会从文件path中提取k1对应的partitioned field的值,并完成数据导入 - - 2. 指定Partition Discovery的基础路径(BASE_PATH) - LOAD LABEL example_db.label11 - ( - DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/city=beijing/utc_date=2019-06-26") - INTO TABLE `my_table` - FORMAT AS "csv" - BASE_PATH AS "hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/" - (k1, k2, k3, utc_date,city) - SET (uniq_id = md5sum(k1, city)) - ) - WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); + 9. 提取文件路径中的压缩字段 + 如果需要,则会根据表中定义的字段类型解析文件路径中的压缩字段(partitioned fields),类似Spark中Partition Discovery的功能 + LOAD LABEL example_db.label10 + ( + DATA INFILE("hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/city=beijing/*/*") + INTO TABLE `my_table` + FORMAT AS "csv" + COLUMNS FROM PATH AS (city, utc_date) + (k1, k2, k3) + SET (uniq_id = md5sum(k1, city)) + ) + WITH BROKER hdfs ("username"="hdfs_user", "password"="hdfs_password"); - hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/city=beijing/utc_date=2019-06-26目录下包括如下文件:[hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/city=beijing/utc_date=2019-06-26/0000.csv, hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/city=beijing/utc_date=2019-06-26/0001.csv, ...] - 假设CSV文件中仅包括3列(k1, k2和k3),则会根据base_path(hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/)提取文件路径的中的city和utc_date字段 + hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/city=beijing目录下包括如下文件:[hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/city=beijing/utc_date=2019-06-26/0000.csv, hdfs://hdfs_host:hdfs_port/user/palo/data/input/dir/city=beijing/utc_date=2019-06-26/0001.csv, ...] + 则提取文件路径的中的city和utc_date字段 ## keyword