You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by ni...@apache.org on 2019/12/17 08:01:49 UTC

[kylin] 01/03: KYLIN-4211 PartitionDesc support custom year、month、day partitions name

This is an automated email from the ASF dual-hosted git repository.

nic pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kylin.git

commit 2b830a3e6d6e17d488af307d030e0d73d094aa74
Author: bigxiaochu <cr...@163.com>
AuthorDate: Tue Oct 29 11:59:26 2019 +0800

    KYLIN-4211 PartitionDesc support custom year、month、day partitions name
---
 .../apache/kylin/metadata/model/PartitionDesc.java | 63 +++++++++++++++++++---
 1 file changed, 57 insertions(+), 6 deletions(-)

diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/PartitionDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/PartitionDesc.java
index f93996e..a4c4cc7 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/PartitionDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/PartitionDesc.java
@@ -71,13 +71,19 @@ public class PartitionDesc implements Serializable {
         if (StringUtils.isEmpty(partitionDateColumn))
             return;
 
-        partitionDateColumnRef = model.findColumn(partitionDateColumn);
-        partitionDateColumn = partitionDateColumnRef.getIdentity();
-        if (StringUtils.isBlank(partitionTimeColumn) == false) {
-            partitionTimeColumnRef = model.findColumn(partitionTimeColumn);
-            partitionTimeColumn = partitionTimeColumnRef.getIdentity();
-        }
+        //Support CustomYearMonthDayFieldPartitionConditionBuilder, partitionDateColumn split by ","
         partitionConditionBuilder = (IPartitionConditionBuilder) ClassUtil.newInstance(partitionConditionBuilderClz);
+        if (partitionConditionBuilder instanceof CustomYearMonthDayFieldPartitionConditionBuilder) {
+            ((CustomYearMonthDayFieldPartitionConditionBuilder)partitionConditionBuilder).init(this, model);
+        } else {
+            partitionDateColumnRef = model.findColumn(partitionDateColumn);
+            partitionDateColumn = partitionDateColumnRef.getIdentity();
+            if (StringUtils.isBlank(partitionTimeColumn) == false) {
+                partitionTimeColumnRef = model.findColumn(partitionTimeColumn);
+                partitionTimeColumn = partitionTimeColumnRef.getIdentity();
+            }
+        }
+
     }
 
     public boolean partitionColumnIsYmdInt() {
@@ -340,6 +346,51 @@ public class PartitionDesc implements Serializable {
         }
     }
 
+    /**
+     * Another implementation of IPartitionConditionBuilder, for the fact tables which have three custom partition columns like "Y", "M", "D" means "YEAR", "MONTH", and "DAY";
+     * This class will conicat the three columns into yyyy-MM-dd format for query hive;
+     * implements Serializable for spark build
+     */
+    public static class CustomYearMonthDayFieldPartitionConditionBuilder implements IPartitionConditionBuilder, Serializable {
+        private String yearPartitionDateColumn;
+        private String monthPartitionDateColumn;
+        private String dayPartitionDateColumn;
+        @Override
+        public String buildDateRangeCondition(PartitionDesc partDesc, ISegment seg, SegmentRange segRange, Function<TblColRef, String> func) {
+            long startInclusive = (Long) segRange.start.v;
+            long endExclusive = (Long) segRange.end.v;
+
+            TblColRef partitionColumn = partDesc.getPartitionDateColumnRef();
+            if (partitionColumn != null) {
+                partitionColumn.setQuotedFunc(func);
+            }
+            String concatField = String.format(Locale.ROOT, "CONCAT(%s,'-',%s,'-',%s)", yearPartitionDateColumn,
+                    monthPartitionDateColumn, dayPartitionDateColumn);
+            StringBuilder builder = new StringBuilder();
+
+            if (startInclusive > 0) {
+                builder.append(concatField + " >= '" + DateFormat.formatToDateStr(startInclusive) + "' ");
+                builder.append("AND ");
+            }
+            builder.append(concatField + " < '" + DateFormat.formatToDateStr(endExclusive) + "'");
+
+            return builder.toString();
+        }
+
+        public void init(PartitionDesc partitionDesc, DataModelDesc model) {
+            String[] yearMonthDayColumns = partitionDesc.getPartitionDateColumn().split(",");
+            if (yearMonthDayColumns.length != 3) {
+                throw new IllegalArgumentException(partitionDesc.getPartitionDateColumn() + " is not year, month, and day columns");
+            }
+            TblColRef yearRef = model.findColumn(yearMonthDayColumns[0]);
+            yearPartitionDateColumn = yearRef.getIdentity();
+            monthPartitionDateColumn = model.findColumn(yearMonthDayColumns[1]).getIdentity();
+            dayPartitionDateColumn = model.findColumn(yearMonthDayColumns[2]).getIdentity();
+            //for partition desc isPartitioned() true
+            partitionDesc.setPartitionDateColumnRef(yearRef);
+        }
+    }
+
     public static PartitionDesc getCopyOf(PartitionDesc orig) {
         PartitionDesc ret = new PartitionDesc();
         ret.partitionDateColumn = orig.partitionDateColumn;