KYLIN-1513 Time partitioning doesn't work across multiple days Project: http://git-wip-us.apache.org/repos/asf/kylin/repo Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/b37c5bb2 Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/b37c5bb2 Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/b37c5bb2
Branch: refs/heads/1.5.x-HBase1.1.3 Commit: b37c5bb293fe2025bf7e7b0c4901d13c02fc86eb Parents: be6e206 Author: shaofengshi <shaofeng...@apache.org> Authored: Wed Apr 6 15:49:07 2016 +0800 Committer: shaofengshi <shaofeng...@apache.org> Committed: Wed Apr 6 15:49:29 2016 +0800 ---------------------------------------------------------------------- .../kylin/metadata/model/PartitionDesc.java | 37 +++++++++++++++++--- 1 file changed, 32 insertions(+), 5 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/kylin/blob/b37c5bb2/core-metadata/src/main/java/org/apache/kylin/metadata/model/PartitionDesc.java ---------------------------------------------------------------------- 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 1738c27..f1339cd 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 @@ -166,14 +166,14 @@ public class PartitionDesc { String partitionDateColumnName = partDesc.getPartitionDateColumn(); String partitionTimeColumnName = partDesc.getPartitionTimeColumn(); - if (partitionDateColumnName != null) { + if (partitionDateColumnName != null && partitionTimeColumnName == null) { buildSingleColumnRangeCondition(builder, partitionDateColumnName, startInclusive, endExclusive, partDesc.getPartitionDateFormat(), tableAlias); - } - if (partitionTimeColumnName != null) { - if (partitionDateColumnName != null) - builder.append(" AND "); + } else if (partitionDateColumnName == null && partitionTimeColumnName != null) { buildSingleColumnRangeCondition(builder, partitionTimeColumnName, startInclusive, endExclusive, partDesc.getPartitionTimeFormat(), tableAlias); + } else if (partitionDateColumnName != null && partitionTimeColumnName != null) { + buildMultipleColumnRangeCondition(builder, partitionDateColumnName, partitionTimeColumnName, startInclusive, endExclusive, partDesc.getPartitionDateFormat(), partDesc.getPartitionTimeFormat(), tableAlias); } + return builder.toString(); } @@ -198,6 +198,33 @@ public class PartitionDesc { } builder.append(partitionColumnName + " < '" + DateFormat.formatToDateStr(endExclusive, partitionColumnDateFormat) + "'"); } + + private static void buildMultipleColumnRangeCondition(StringBuilder builder, String partitionDateColumnName, String partitionTimeColumnName, long startInclusive, long endExclusive, String partitionColumnDateFormat, String partitionColumnTimeFormat, Map<String, String> tableAlias) { + partitionDateColumnName = replaceColumnNameWithAlias(partitionDateColumnName, tableAlias); + partitionTimeColumnName = replaceColumnNameWithAlias(partitionTimeColumnName, tableAlias); + if (startInclusive > 0) { + builder.append("("); + builder.append("("); + builder.append(partitionDateColumnName + " = '" + DateFormat.formatToDateStr(startInclusive, partitionColumnDateFormat) + "'").append(" AND ").append(partitionTimeColumnName + " >= '" + DateFormat.formatToDateStr(startInclusive, partitionColumnTimeFormat) + "'"); + builder.append(")"); + builder.append(" OR "); + builder.append("("); + builder.append(partitionDateColumnName + " > '" + DateFormat.formatToDateStr(startInclusive, partitionColumnDateFormat) + "'"); + builder.append(")"); + builder.append(")"); + builder.append(" AND "); + } + + builder.append("("); + builder.append("("); + builder.append(partitionDateColumnName + " = '" + DateFormat.formatToDateStr(endExclusive, partitionColumnDateFormat) + "'").append(" AND ").append(partitionTimeColumnName + " < '" + DateFormat.formatToDateStr(endExclusive, partitionColumnTimeFormat) + "'"); + builder.append(")"); + builder.append(" OR "); + builder.append("("); + builder.append(partitionDateColumnName + " < '" + DateFormat.formatToDateStr(endExclusive, partitionColumnDateFormat) + "'"); + builder.append(")"); + builder.append(")"); + } } /**