http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java ---------------------------------------------------------------------- diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java index 278b59f..7c26e8f 100644 --- a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java +++ b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java @@ -18,11 +18,15 @@ package org.apache.kylin.cube.model; +import org.apache.commons.lang.StringUtils; +import org.apache.kylin.cube.kv.RowConstants; import org.apache.kylin.metadata.model.TblColRef; import com.fasterxml.jackson.annotation.JsonAutoDetect; import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Objects; +import com.google.common.base.Preconditions; /** * @author yangli9 @@ -31,37 +35,74 @@ import com.fasterxml.jackson.annotation.JsonProperty; @JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE) public class RowKeyColDesc { + public enum ColEncodingType { + DICT, FIXED_LEN + } + + public class ColEncoding { + public ColEncodingType type; + public Object param; + + public ColEncoding(ColEncodingType type, Object param) { + this.type = type; + this.param = param; + } + } + @JsonProperty("column") private String column; - @JsonProperty("length") - private int length; - @JsonProperty("dictionary") - private String dictionary; - @JsonProperty("mandatory") - private boolean mandatory = false; + @JsonProperty("encoding") + private String encoding; // computed + private ColEncoding colEncoding; private int bitIndex; private TblColRef colRef; - public String getDictionary() { - return dictionary; + public void init() { + + //dict or fix length? + Preconditions.checkState(StringUtils.isNotEmpty(this.encoding)); + if (this.encoding.equalsIgnoreCase("dict")) { + this.colEncoding = new ColEncoding(ColEncodingType.DICT, null); + } else if (this.encoding.startsWith("fixed_length")) { + int length = RowConstants.ROWKEY_COL_DEFAULT_LENGTH; + if (this.encoding.indexOf(":") > 0) { + length = Integer.parseInt(this.encoding.substring(this.encoding.indexOf(":") + 1)); + } + this.colEncoding = new ColEncoding(ColEncodingType.FIXED_LEN, length); + } else { + throw new IllegalArgumentException("Not supported row key col encoding:" + this.encoding); + } + } + + public String getEncoding() { + return encoding; + } + + public void setEncoding(String encoding) { + this.encoding = encoding; } public String getColumn() { return column; } - void setColumn(String column) { + public void setColumn(String column) { this.column = column; } - public int getLength() { - return length; + public boolean isUsingDictionary() { + return this.colEncoding.type == ColEncodingType.DICT; + } - public boolean isMandatory() { - return mandatory; + public int getLength() { + if (this.colEncoding.type == ColEncodingType.FIXED_LEN) { + return (Integer) this.colEncoding.param; + } else { + return 0; + } } public int getBitIndex() { @@ -80,13 +121,9 @@ public class RowKeyColDesc { this.colRef = colRef; } - public void setDictionary(String dictionary) { - this.dictionary = dictionary; - } - @Override public String toString() { - return "RowKeyColDesc [column=" + column + ", length=" + length + ", dictionary=" + dictionary + ", mandatory=" + mandatory + "]"; + return Objects.toStringHelper(this).add("column", column).add("encoding", encoding).toString(); } }
http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyDesc.java ---------------------------------------------------------------------- diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyDesc.java index abec436..7254df0 100644 --- a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyDesc.java +++ b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyDesc.java @@ -18,139 +18,65 @@ package org.apache.kylin.cube.model; -import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; -import java.util.HashSet; -import java.util.List; import java.util.Map; -import java.util.Set; -import org.apache.commons.lang.StringUtils; -import org.apache.kylin.common.util.StringUtil; import org.apache.kylin.metadata.model.TblColRef; import com.fasterxml.jackson.annotation.JsonAutoDetect; import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Objects; /** */ @JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE) public class RowKeyDesc { - public static class HierarchyMask { - public long fullMask; - public long[] allMasks; - } - - public static class AggrGroupMask { - public AggrGroupMask(int size) { - groupOneBitMasks = new long[size]; - } - - public long groupMask; - public long groupOneBitMasks[]; - public long uniqueMask; - public long leftoverMask; - } - @JsonProperty("rowkey_columns") private RowKeyColDesc[] rowkeyColumns; - @JsonProperty("aggregation_groups") - private String[][] aggregationGroups; // computed content + private long fullMask; private CubeDesc cubeDesc; private Map<TblColRef, RowKeyColDesc> columnMap; - private long fullMask; - private long mandatoryColumnMask; - private AggrGroupMask[] aggrGroupMasks; - private long aggrGroupFullMask; - private long tailMask; - - private List<HierarchyMask> hierarchyMasks; - public RowKeyColDesc[] getRowKeyColumns() { return rowkeyColumns; } - // search a specific row key col - public int getRowKeyIndexByColumnName(String columnName) { - if (this.rowkeyColumns == null) - return -1; - - for (int i = 0; i < this.rowkeyColumns.length; ++i) { - RowKeyColDesc desc = this.rowkeyColumns[i]; - if (desc.getColumn().equalsIgnoreCase(columnName)) { - return i; - } - } - return -1; - } - - public int getNCuboidBuildLevels() { - // N aggregation columns requires N levels of cuboid build - // - N columns requires N-1 levels build - // - zero tail cuboid needs one more additional level - Set<String> aggDims = new HashSet<String>(); - for (String[] aggrGroup : aggregationGroups) { - for (String dim : aggrGroup) { - aggDims.add(dim); - } - } - return aggDims.size(); - } - - public String[][] getAggregationGroups() { - return aggregationGroups; - } - - public CubeDesc getCubeRef() { - return cubeDesc; - } - - public void setCubeRef(CubeDesc cubeRef) { + // public int getNCuboidBuildLevels() { + // // N aggregation columns requires N levels of cuboid build + // // - N columns requires N-1 levels build + // // - zero tail cuboid needs one more additional level + // Set<String> aggDims = new HashSet<String>(); + // for (String[] aggrGroup : aggregationGroups) { + // for (String dim : aggrGroup) { + // aggDims.add(dim); + // } + // } + // return aggDims.size(); + // } + + public void setCubeDesc(CubeDesc cubeRef) { this.cubeDesc = cubeRef; } - public long getFullMask() { - return fullMask; - } - - public long getMandatoryColumnMask() { - return mandatoryColumnMask; - } - - public long getAggrGroupFullMask() { - return aggrGroupFullMask; - } - - public AggrGroupMask[] getAggrGroupMasks() { - return aggrGroupMasks; - } - - public List<HierarchyMask> getHierarchyMasks() { - return hierarchyMasks; - } - - public long getTailMask() { - return tailMask; - } - public int getColumnBitIndex(TblColRef col) { return getColDesc(col).getBitIndex(); } + /** + * caller of this method must make sure that col is NOT using dictionary + * otherwise this will always return 0 for dict columns + * @param col + * @return + */ public int getColumnLength(TblColRef col) { return getColDesc(col).getLength(); } - public String getDictionary(TblColRef col) { - return getColDesc(col).getDictionary(); - } - private RowKeyColDesc getColDesc(TblColRef col) { RowKeyColDesc desc = columnMap.get(col); if (desc == null) @@ -158,40 +84,33 @@ public class RowKeyDesc { return desc; } - public boolean isUseDictionary(int index) { - String useDictionary = rowkeyColumns[index].getDictionary(); - return useDictionary(useDictionary); - } - public boolean isUseDictionary(TblColRef col) { - String useDictionary = getDictionary(col); - return useDictionary(useDictionary); + return getColDesc(col).isUsingDictionary(); } - private boolean useDictionary(String useDictionary) { - return !StringUtils.isBlank(useDictionary) && !"false".equals(useDictionary); - } + public void init(CubeDesc cubeDesc) { - public void init(CubeDesc cube) { - setCubeRef(cube); - Map<String, TblColRef> colNameAbbr = cube.buildColumnNameAbbreviation(); + setCubeDesc(cubeDesc); + Map<String, TblColRef> colNameAbbr = cubeDesc.buildColumnNameAbbreviation(); buildRowKey(colNameAbbr); - buildAggregationGroups(colNameAbbr); - buildHierarchyMasks(); + } + + public void setRowkeyColumns(RowKeyColDesc[] rowkeyColumns) { + this.rowkeyColumns = rowkeyColumns; } @Override public String toString() { - return "RowKeyDesc [rowkeyColumns=" + Arrays.toString(rowkeyColumns) + ", aggregationGroups=" + Arrays.toString(aggregationGroups) + "]"; + return Objects.toStringHelper(this).add("RowKeyColumns", Arrays.toString(rowkeyColumns)).toString(); } private void buildRowKey(Map<String, TblColRef> colNameAbbr) { columnMap = new HashMap<TblColRef, RowKeyColDesc>(); - mandatoryColumnMask = 0; for (int i = 0; i < rowkeyColumns.length; i++) { RowKeyColDesc rowKeyColDesc = rowkeyColumns[i]; + rowKeyColDesc.init(); String column = rowKeyColDesc.getColumn(); rowKeyColDesc.setColumn(column.toUpperCase()); rowKeyColDesc.setBitIndex(rowkeyColumns.length - i - 1); @@ -201,95 +120,17 @@ public class RowKeyDesc { } columnMap.put(rowKeyColDesc.getColRef(), rowKeyColDesc); - - if (rowKeyColDesc.isMandatory()) { - mandatoryColumnMask |= 1L << rowKeyColDesc.getBitIndex(); - } - } - } - - private void buildAggregationGroups(Map<String, TblColRef> colNameAbbr) { - if (aggregationGroups == null) { - aggregationGroups = new String[0][]; - } - - for (int i = 0; i < aggregationGroups.length; i++) { - StringUtil.toUpperCaseArray(aggregationGroups[i], this.aggregationGroups[i]); } + this.fullMask = 0L; for (int i = 0; i < this.rowkeyColumns.length; i++) { int index = rowkeyColumns[i].getBitIndex(); this.fullMask |= 1L << index; } - - this.aggrGroupMasks = new AggrGroupMask[aggregationGroups.length]; - for (int i = 0; i < this.aggregationGroups.length; i++) { - String[] aggGrp = this.aggregationGroups[i]; - AggrGroupMask mask = new AggrGroupMask(aggGrp.length); - - for (int j = 0; j < aggGrp.length; j++) { - TblColRef aggCol = colNameAbbr.get(aggGrp[j].toUpperCase()); - if (aggCol == null) { - throw new IllegalArgumentException("Can't find aggregation column " + aggGrp[j] + " in cube " + this.cubeDesc.getName()); - } - Integer index = getColumnBitIndex(aggCol); - mask.groupMask |= 1L << index; - mask.groupOneBitMasks[j] = 1L << index; - this.aggrGroupFullMask |= 1L << index; - } - this.aggrGroupMasks[i] = mask; - } - - this.tailMask = fullMask ^ mandatoryColumnMask ^ aggrGroupFullMask; - - // unique mask = (bits in this group) - (bits in following groups) - // leftover mask = (tail bits) + (bits in following groups) - (bits in - // this group) - for (int i = 0; i < aggrGroupMasks.length; i++) { - AggrGroupMask mask = aggrGroupMasks[i]; - - mask.uniqueMask = mask.groupMask; - for (int j = i + 1; j < aggrGroupMasks.length; j++) { - mask.uniqueMask &= ~aggrGroupMasks[j].groupMask; - } - - mask.leftoverMask = tailMask; - for (int j = i + 1; j < aggrGroupMasks.length; j++) { - mask.leftoverMask |= aggrGroupMasks[j].groupMask; - } - mask.leftoverMask &= ~mask.groupMask; - } } - private void buildHierarchyMasks() { - this.hierarchyMasks = new ArrayList<HierarchyMask>(); - - for (DimensionDesc dimension : this.cubeDesc.getDimensions()) { - HierarchyDesc[] hierarchies = dimension.getHierarchy(); - if (hierarchies == null || hierarchies.length == 0) - continue; - - HierarchyMask mask = new HierarchyMask(); - ArrayList<Long> allMaskList = new ArrayList<Long>(); - for (int i = 0; i < hierarchies.length; i++) { - TblColRef hColumn = hierarchies[i].getColumnRef(); - Integer index = getColumnBitIndex(hColumn); - long bit = 1L << index; - - if ((tailMask & bit) > 0) - continue; // ignore levels in tail, they don't participate - // aggregation group combination anyway - - mask.fullMask |= bit; - allMaskList.add(mask.fullMask); - } - - mask.allMasks = new long[allMaskList.size()]; - for (int i = 0; i < allMaskList.size(); i++) - mask.allMasks[i] = allMaskList.get(i); - - this.hierarchyMasks.add(mask); - } + public long getFullMask() { + return this.fullMask; } } http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/main/java/org/apache/kylin/cube/model/SelectRule.java ---------------------------------------------------------------------- diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/SelectRule.java b/core-cube/src/main/java/org/apache/kylin/cube/model/SelectRule.java new file mode 100644 index 0000000..63b0fc4 --- /dev/null +++ b/core-cube/src/main/java/org/apache/kylin/cube/model/SelectRule.java @@ -0,0 +1,30 @@ +/* + * 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. + */ + +package org.apache.kylin.cube.model; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class SelectRule { + @JsonProperty("hierarchy_dims") + public String[][] hierarchy_dims; + @JsonProperty("mandatory_dims") + public String[] mandatory_dims; + @JsonProperty("joint_dims") + public String[][] joint_dims; +} http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/CubeBuildTypeEnum.java ---------------------------------------------------------------------- diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/CubeBuildTypeEnum.java b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/CubeBuildTypeEnum.java new file mode 100644 index 0000000..2544913 --- /dev/null +++ b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/CubeBuildTypeEnum.java @@ -0,0 +1,39 @@ +/* + * 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. + */ + +package org.apache.kylin.cube.model.v2; + +/** + * @author xduo + * + */ +public enum CubeBuildTypeEnum { + /** + * rebuild a segment or incremental build + */ + BUILD, + /** + * merge segments + */ + MERGE, + + /** + * refresh segments + */ + REFRESH +} http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/CubeDesc.java ---------------------------------------------------------------------- diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/CubeDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/CubeDesc.java new file mode 100644 index 0000000..b37e5a1 --- /dev/null +++ b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/CubeDesc.java @@ -0,0 +1,826 @@ +/* + * 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. + */ + +package org.apache.kylin.cube.model.v2; + +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import org.apache.commons.codec.binary.Base64; +import org.apache.commons.lang.ArrayUtils; +import org.apache.commons.lang.StringUtils; +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.common.persistence.ResourceStore; +import org.apache.kylin.common.persistence.RootPersistentEntity; +import org.apache.kylin.common.util.Array; +import org.apache.kylin.common.util.CaseInsensitiveStringMap; +import org.apache.kylin.common.util.JsonUtil; +import org.apache.kylin.metadata.MetadataConstants; +import org.apache.kylin.metadata.MetadataManager; +import org.apache.kylin.metadata.model.ColumnDesc; +import org.apache.kylin.metadata.model.DataModelDesc; +import org.apache.kylin.metadata.model.FunctionDesc; +import org.apache.kylin.metadata.model.IEngineAware; +import org.apache.kylin.metadata.model.IStorageAware; +import org.apache.kylin.metadata.model.JoinDesc; +import org.apache.kylin.metadata.model.MeasureDesc; +import org.apache.kylin.metadata.model.TableDesc; +import org.apache.kylin.metadata.model.TblColRef; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +/** + */ +@SuppressWarnings("serial") +@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE) +public class CubeDesc extends RootPersistentEntity { + + public static enum DeriveType { + LOOKUP, PK_FK + } + + public static class DeriveInfo { + public DeriveType type; + public DimensionDesc dimension; + public TblColRef[] columns; + public boolean isOneToOne; // only used when ref from derived to host + + DeriveInfo(DeriveType type, DimensionDesc dimension, TblColRef[] columns, boolean isOneToOne) { + this.type = type; + this.dimension = dimension; + this.columns = columns; + this.isOneToOne = isOneToOne; + } + + @Override + public String toString() { + return "DeriveInfo [type=" + type + ", dimension=" + dimension + ", columns=" + Arrays.toString(columns) + ", isOneToOne=" + isOneToOne + "]"; + } + + } + + private KylinConfig config; + private DataModelDesc model; + + @JsonProperty("name") + private String name; + @JsonProperty("model_name") + private String modelName; + @JsonProperty("description") + private String description; + @JsonProperty("null_string") + private String[] nullStrings; + @JsonProperty("dimensions") + private List<DimensionDesc> dimensions; + @JsonProperty("measures") + private List<MeasureDesc> measures; + @JsonProperty("rowkey") + private RowKeyDesc rowkey; + @JsonProperty("hbase_mapping") + private HBaseMappingDesc hbaseMapping; + @JsonProperty("signature") + private String signature; + @JsonProperty("notify_list") + private List<String> notifyList; + @JsonProperty("status_need_notify") + private List<String> statusNeedNotify = Collections.emptyList(); + @JsonProperty("auto_merge_time_ranges") + private long[] autoMergeTimeRanges; + @JsonProperty("retention_range") + private long retentionRange = 0; + + @JsonProperty("engine_type") + private int engineType = IEngineAware.ID_MR_V1; + @JsonProperty("storage_type") + private int storageType = IStorageAware.ID_HBASE; + + private Map<String, Map<String, TblColRef>> columnMap = new HashMap<String, Map<String, TblColRef>>(); + private LinkedHashSet<TblColRef> allColumns = new LinkedHashSet<TblColRef>(); + private LinkedHashSet<TblColRef> dimensionColumns = new LinkedHashSet<TblColRef>(); + + private Map<TblColRef, DeriveInfo> derivedToHostMap = Maps.newHashMap(); + private Map<Array<TblColRef>, List<DeriveInfo>> hostToDerivedMap = Maps.newHashMap(); + + public boolean isEnableSharding() { + //in the future may extend to other storage that is shard-able + return storageType == IStorageAware.ID_SHARDED_HBASE; + } + + /** + * Error messages during resolving json metadata + */ + private List<String> errors = new ArrayList<String>(); + + /** + * @return all columns this cube can support, including derived + */ + public Set<TblColRef> listAllColumns() { + return allColumns; + } + + /** + * @return dimension columns including derived, BUT NOT measures + */ + public Set<TblColRef> listDimensionColumnsIncludingDerived() { + return dimensionColumns; + } + + /** + * @return dimension columns excluding derived and measures + */ + public List<TblColRef> listDimensionColumnsExcludingDerived() { + List<TblColRef> result = new ArrayList<TblColRef>(); + for (TblColRef col : dimensionColumns) { + if (isDerived(col) == false) + result.add(col); + } + return result; + } + + /** + * Find FunctionDesc by Full Expression. + * + * @return + */ + public FunctionDesc findFunctionOnCube(FunctionDesc manualFunc) { + for (MeasureDesc m : measures) { + if (m.getFunction().equals(manualFunc)) + return m.getFunction(); + } + return null; + } + + public TblColRef findColumnRef(String table, String column) { + Map<String, TblColRef> cols = columnMap.get(table); + if (cols == null) + return null; + else + return cols.get(column); + } + + public DimensionDesc findDimensionByColumn(TblColRef col) { + for (DimensionDesc dim : dimensions) { + if (ArrayUtils.contains(dim.getColumnRefs(), col)) + return dim; + } + return null; + } + + public DimensionDesc findDimensionByTable(String lookupTableName) { + lookupTableName = lookupTableName.toUpperCase(); + for (DimensionDesc dim : dimensions) + if (dim.getTable() != null && dim.getTable().equals(lookupTableName)) + return dim; + return null; + } + + public DimensionDesc findDimensionByName(String dimName) { + dimName = dimName.toUpperCase(); + for (DimensionDesc dim : dimensions) { + if (dimName.equals(dim.getName())) + return dim; + } + return null; + } + + /** + * Get all functions from each measure. + * + * @return + */ + public List<FunctionDesc> listAllFunctions() { + List<FunctionDesc> functions = new ArrayList<FunctionDesc>(); + for (MeasureDesc m : measures) { + functions.add(m.getFunction()); + } + return functions; + } + + public boolean isDerived(TblColRef col) { + return derivedToHostMap.containsKey(col); + } + + public DeriveInfo getHostInfo(TblColRef derived) { + return derivedToHostMap.get(derived); + } + + public Map<Array<TblColRef>, List<DeriveInfo>> getHostToDerivedInfo(List<TblColRef> rowCols, Collection<TblColRef> wantedCols) { + Map<Array<TblColRef>, List<DeriveInfo>> result = new HashMap<Array<TblColRef>, List<DeriveInfo>>(); + for (Entry<Array<TblColRef>, List<DeriveInfo>> entry : hostToDerivedMap.entrySet()) { + Array<TblColRef> hostCols = entry.getKey(); + boolean hostOnRow = rowCols.containsAll(Arrays.asList(hostCols.data)); + if (!hostOnRow) + continue; + + List<DeriveInfo> wantedInfo = new ArrayList<DeriveInfo>(); + for (DeriveInfo info : entry.getValue()) { + if (wantedCols == null || Collections.disjoint(wantedCols, Arrays.asList(info.columns)) == false) // has + // any + // wanted + // columns? + wantedInfo.add(info); + } + + if (wantedInfo.size() > 0) + result.put(hostCols, wantedInfo); + } + return result; + } + + public String getResourcePath() { + return getCubeDescResourcePath(name); + } + + public static String getCubeDescResourcePath(String descName) { + return ResourceStore.CUBE_DESC_RESOURCE_ROOT + "/" + descName + MetadataConstants.FILE_SURFIX; + } + + // ============================================================================ + + public HBaseMappingDesc getHBaseMapping() { + return hbaseMapping; + } + + public void setHBaseMapping(HBaseMappingDesc hbaseMapping) { + this.hbaseMapping = hbaseMapping; + } + + public KylinConfig getConfig() { + return config; + } + + public void setConfig(KylinConfig config) { + this.config = config; + } + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public String getModelName() { + return modelName; + } + + public void setModelName(String modelName) { + this.modelName = modelName; + } + + public DataModelDesc getModel() { + return model; + } + + public void setModel(DataModelDesc model) { + this.model = model; + } + + public String getDescription() { + return description; + } + + public void setDescription(String description) { + this.description = description; + } + + public String getFactTable() { + return model.getFactTable(); + } + + public TableDesc getFactTableDesc() { + return model.getFactTableDesc(); + } + + public String[] getNullStrings() { + return nullStrings; + } + + public List<DimensionDesc> getDimensions() { + return dimensions; + } + + public void setDimensions(List<DimensionDesc> dimensions) { + this.dimensions = dimensions; + } + + public List<MeasureDesc> getMeasures() { + return measures; + } + + public void setMeasures(List<MeasureDesc> measures) { + this.measures = measures; + } + + public RowKeyDesc getRowkey() { + return rowkey; + } + + public void setRowkey(RowKeyDesc rowkey) { + this.rowkey = rowkey; + } + + public String getSignature() { + return signature; + } + + public void setSignature(String signature) { + this.signature = signature; + } + + public List<String> getNotifyList() { + return notifyList; + } + + public void setNotifyList(List<String> notifyList) { + this.notifyList = notifyList; + } + + public List<String> getStatusNeedNotify() { + return statusNeedNotify; + } + + public void setStatusNeedNotify(List<String> statusNeedNotify) { + this.statusNeedNotify = statusNeedNotify; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + CubeDesc cubeDesc = (CubeDesc) o; + + if (!name.equals(cubeDesc.name)) + return false; + if (!getFactTable().equals(cubeDesc.getFactTable())) + return false; + + return true; + } + + @Override + public int hashCode() { + int result = 0; + result = 31 * result + name.hashCode(); + result = 31 * result + getFactTable().hashCode(); + return result; + } + + @Override + public String toString() { + return "CubeDesc [name=" + name + "]"; + } + + public boolean checkSignature() { + if (StringUtils.isBlank(getSignature())) { + return true; + } + return calculateSignature().equals(getSignature()); + } + + public String calculateSignature() { + MessageDigest md = null; + try { + md = MessageDigest.getInstance("MD5"); + StringBuilder sigString = new StringBuilder(); + sigString.append(this.name).append("|").append(this.getFactTable()).append("|").append(JsonUtil.writeValueAsString(this.model.getPartitionDesc())).append("|").append(JsonUtil.writeValueAsString(this.dimensions)).append("|").append(JsonUtil.writeValueAsString(this.measures)).append("|").append(JsonUtil.writeValueAsString(this.rowkey)).append("|").append(JsonUtil.writeValueAsString(this.hbaseMapping)); + + byte[] signature = md.digest(sigString.toString().getBytes()); + return new String(Base64.encodeBase64(signature)); + } catch (NoSuchAlgorithmException | JsonProcessingException e) { + throw new RuntimeException("Failed to calculate signature"); + } + } + + public Map<String, TblColRef> buildColumnNameAbbreviation() { + Map<String, TblColRef> r = new CaseInsensitiveStringMap<TblColRef>(); + for (TblColRef col : listDimensionColumnsExcludingDerived()) { + r.put(col.getName(), col); + } + return r; + } + + public void init(KylinConfig config, Map<String, TableDesc> tables) { + this.errors.clear(); + this.config = config; + + if (this.modelName == null || this.modelName.length() == 0) { + this.addError("The cubeDesc '" + this.getName() + "' doesn't have data model specified."); + } + + this.model = MetadataManager.getInstance(config).getDataModelDesc(this.modelName); + + if (this.model == null) { + this.addError("No data model found with name '" + modelName + "'."); + } + + for (DimensionDesc dim : dimensions) { + dim.init(this, tables); + } + + sortDimAndMeasure(); + initDimensionColumns(); + initMeasureColumns(); + + rowkey.init(this); + if (hbaseMapping != null) { + hbaseMapping.init(this); + } + + initMeasureReferenceToColumnFamily(); + + // check all dimension columns are presented on rowkey + List<TblColRef> dimCols = listDimensionColumnsExcludingDerived(); + if (rowkey.getRowKeyColumns().length != dimCols.size()) { + addError("RowKey columns count (" + rowkey.getRowKeyColumns().length + ") does not match dimension columns count (" + dimCols.size() + "). "); + } + } + + private void initDimensionColumns() { + for (DimensionDesc dim : dimensions) { + JoinDesc join = dim.getJoin(); + + // init dimension columns + ArrayList<TblColRef> dimCols = Lists.newArrayList(); + String[] colStrs = dim.getColumn(); + + // when column is omitted, special case + if (colStrs == null && dim.isDerived() || ArrayUtils.contains(colStrs, "{FK}")) { + for (TblColRef col : join.getForeignKeyColumns()) { + dimCols.add(initDimensionColRef(col)); + } + } + // normal case + else { + if (colStrs == null || colStrs.length == 0) + throw new IllegalStateException("Dimension column must not be blank " + dim); + + for (String colStr : colStrs) { + dimCols.add(initDimensionColRef(dim, colStr)); + } + + // fill back column ref in hierarchy + if (dim.isHierarchy()) { + for (int i = 0; i < dimCols.size(); i++) + dim.getHierarchy()[i].setColumnRef(dimCols.get(i)); + } + } + + TblColRef[] dimColArray = (TblColRef[]) dimCols.toArray(new TblColRef[dimCols.size()]); + dim.setColumnRefs(dimColArray); + + // init derived columns + TblColRef[] hostCols = dimColArray; + if (dim.isDerived()) { + String[] derived = dim.getDerived(); + String[][] split = splitDerivedColumnAndExtra(derived); + String[] derivedNames = split[0]; + String[] derivedExtra = split[1]; + TblColRef[] derivedCols = new TblColRef[derivedNames.length]; + for (int i = 0; i < derivedNames.length; i++) { + derivedCols[i] = initDimensionColRef(dim, derivedNames[i]); + } + initDerivedMap(hostCols, DeriveType.LOOKUP, dim, derivedCols, derivedExtra); + } + + // PK-FK derive the other side + if (join != null) { + TblColRef[] fk = join.getForeignKeyColumns(); + TblColRef[] pk = join.getPrimaryKeyColumns(); + + allColumns.addAll(Arrays.asList(fk)); + allColumns.addAll(Arrays.asList(pk)); + for (int i = 0; i < fk.length; i++) { + int find = ArrayUtils.indexOf(hostCols, fk[i]); + if (find >= 0) { + TblColRef derivedCol = initDimensionColRef(pk[i]); + initDerivedMap(hostCols[find], DeriveType.PK_FK, dim, derivedCol); + } + } + /** disable this code as we don't need fk be derived from pk + for (int i = 0; i < pk.length; i++) { + int find = ArrayUtils.indexOf(hostCols, pk[i]); + if (find >= 0) { + TblColRef derivedCol = initDimensionColRef(fk[i]); + initDerivedMap(hostCols[find], DeriveType.PK_FK, dim, derivedCol); + } + } + */ + } + } + } + + private String[][] splitDerivedColumnAndExtra(String[] derived) { + String[] cols = new String[derived.length]; + String[] extra = new String[derived.length]; + for (int i = 0; i < derived.length; i++) { + String str = derived[i]; + int cut = str.indexOf(":"); + if (cut >= 0) { + cols[i] = str.substring(0, cut); + extra[i] = str.substring(cut + 1).trim(); + } else { + cols[i] = str; + extra[i] = ""; + } + } + return new String[][] { cols, extra }; + } + + private void initDerivedMap(TblColRef hostCol, DeriveType type, DimensionDesc dimension, TblColRef derivedCol) { + initDerivedMap(new TblColRef[] { hostCol }, type, dimension, new TblColRef[] { derivedCol }, null); + } + + private void initDerivedMap(TblColRef[] hostCols, DeriveType type, DimensionDesc dimension, TblColRef[] derivedCols, String[] extra) { + if (hostCols.length == 0 || derivedCols.length == 0) + throw new IllegalStateException("host/derived columns must not be empty"); + + // Although FK derives PK automatically, user unaware of this can declare PK as derived dimension explicitly. + // In that case, derivedCols[] will contain a FK which is transformed from the PK by initDimensionColRef(). + // Must drop FK from derivedCols[] before continue. + for (int i = 0; i < derivedCols.length; i++) { + if (ArrayUtils.contains(hostCols, derivedCols[i])) { + derivedCols = (TblColRef[]) ArrayUtils.remove(derivedCols, i); + extra = (String[]) ArrayUtils.remove(extra, i); + i--; + } + } + + Array<TblColRef> hostColArray = new Array<TblColRef>(hostCols); + List<DeriveInfo> infoList = hostToDerivedMap.get(hostColArray); + if (infoList == null) { + hostToDerivedMap.put(hostColArray, infoList = new ArrayList<DeriveInfo>()); + } + infoList.add(new DeriveInfo(type, dimension, derivedCols, false)); + + for (int i = 0; i < derivedCols.length; i++) { + TblColRef derivedCol = derivedCols[i]; + boolean isOneToOne = type == DeriveType.PK_FK || ArrayUtils.contains(hostCols, derivedCol) || (extra != null && extra[i].contains("1-1")); + derivedToHostMap.put(derivedCol, new DeriveInfo(type, dimension, hostCols, isOneToOne)); + } + } + + private TblColRef initDimensionColRef(DimensionDesc dim, String colName) { + TableDesc table = dim.getTableDesc(); + ColumnDesc col = table.findColumnByName(colName); + if (col == null) + throw new IllegalArgumentException("No column '" + colName + "' found in table " + table); + + TblColRef ref = new TblColRef(col); + + // always use FK instead PK, FK could be shared by more than one lookup tables + JoinDesc join = dim.getJoin(); + if (join != null) { + int idx = ArrayUtils.indexOf(join.getPrimaryKeyColumns(), ref); + if (idx >= 0) { + ref = join.getForeignKeyColumns()[idx]; + } + } + return initDimensionColRef(ref); + } + + private TblColRef initDimensionColRef(TblColRef ref) { + TblColRef existing = findColumnRef(ref.getTable(), ref.getName()); + if (existing != null) { + return existing; + } + + allColumns.add(ref); + dimensionColumns.add(ref); + + Map<String, TblColRef> cols = columnMap.get(ref.getTable()); + if (cols == null) { + columnMap.put(ref.getTable(), cols = new HashMap<String, TblColRef>()); + } + cols.put(ref.getName(), ref); + return ref; + } + + private void initMeasureColumns() { + if (measures == null || measures.isEmpty()) { + return; + } + + TableDesc factTable = getFactTableDesc(); + for (MeasureDesc m : measures) { + m.setName(m.getName().toUpperCase()); + + if (m.getDependentMeasureRef() != null) { + m.setDependentMeasureRef(m.getDependentMeasureRef().toUpperCase()); + } + + FunctionDesc func = m.getFunction(); + func.init(factTable); + allColumns.addAll(func.getParameter().getColRefs()); + + // verify holistic count distinct as a dependent measure + if (func.isHolisticCountDistinct() && StringUtils.isBlank(m.getDependentMeasureRef())) { + throw new IllegalStateException(m + " is a holistic count distinct but it has no DependentMeasureRef defined!"); + } + } + } + + private void initMeasureReferenceToColumnFamily() { + if (measures == null || measures.size() == 0) + return; + + Map<String, MeasureDesc> measureLookup = new HashMap<String, MeasureDesc>(); + for (MeasureDesc m : measures) + measureLookup.put(m.getName(), m); + Map<String, Integer> measureIndexLookup = new HashMap<String, Integer>(); + for (int i = 0; i < measures.size(); i++) + measureIndexLookup.put(measures.get(i).getName(), i); + + for (HBaseColumnFamilyDesc cf : getHBaseMapping().getColumnFamily()) { + for (HBaseColumnDesc c : cf.getColumns()) { + String[] colMeasureRefs = c.getMeasureRefs(); + MeasureDesc[] measureDescs = new MeasureDesc[colMeasureRefs.length]; + int[] measureIndex = new int[colMeasureRefs.length]; + for (int i = 0; i < colMeasureRefs.length; i++) { + measureDescs[i] = measureLookup.get(colMeasureRefs[i]); + measureIndex[i] = measureIndexLookup.get(colMeasureRefs[i]); + } + c.setMeasures(measureDescs); + c.setMeasureIndex(measureIndex); + c.setColumnFamilyName(cf.getName()); + } + } + } + + private void sortDimAndMeasure() { + sortDimensionsByID(); + sortMeasuresByID(); + for (DimensionDesc dim : dimensions) { + sortHierarchiesByLevel(dim.getHierarchy()); + } + } + + private void sortDimensionsByID() { + Collections.sort(dimensions, new Comparator<DimensionDesc>() { + @Override + public int compare(DimensionDesc d1, DimensionDesc d2) { + Integer id1 = d1.getId(); + Integer id2 = d2.getId(); + return id1.compareTo(id2); + } + }); + } + + private void sortMeasuresByID() { + if (measures == null) { + measures = Lists.newArrayList(); + } + +// Collections.sort(measures, new Comparator<MeasureDesc>() { +// @Override +// public int compare(MeasureDesc m1, MeasureDesc m2) { +// Integer id1 = m1.getId(); +// Integer id2 = m2.getId(); +// return id1.compareTo(id2); +// } +// }); + } + + private void sortHierarchiesByLevel(HierarchyDesc[] hierarchies) { + if (hierarchies != null) { + Arrays.sort(hierarchies, new Comparator<HierarchyDesc>() { + @Override + public int compare(HierarchyDesc h1, HierarchyDesc h2) { + Integer level1 = Integer.parseInt(h1.getLevel()); + Integer level2 = Integer.parseInt(h2.getLevel()); + return level1.compareTo(level2); + } + }); + } + } + + public boolean hasHolisticCountDistinctMeasures() { + for (MeasureDesc measure : measures) { + if (measure.getFunction().isHolisticCountDistinct()) { + return true; + } + } + return false; + } + + public long getRetentionRange() { + return retentionRange; + } + + public void setRetentionRange(long retentionRange) { + this.retentionRange = retentionRange; + } + + public long[] getAutoMergeTimeRanges() { + return autoMergeTimeRanges; + } + + public void setAutoMergeTimeRanges(long[] autoMergeTimeRanges) { + this.autoMergeTimeRanges = autoMergeTimeRanges; + } + + /** + * Add error info and thrown exception out + * + * @param message + */ + public void addError(String message) { + addError(message, false); + } + + /** + * @param message error message + * @param silent if throw exception + */ + public void addError(String message, boolean silent) { + if (!silent) { + throw new IllegalStateException(message); + } else { + this.errors.add(message); + } + } + + public List<String> getError() { + return this.errors; + } + + public HBaseMappingDesc getHbaseMapping() { + return hbaseMapping; + } + + public void setHbaseMapping(HBaseMappingDesc hbaseMapping) { + this.hbaseMapping = hbaseMapping; + } + + public void setNullStrings(String[] nullStrings) { + this.nullStrings = nullStrings; + } + + public int getStorageType() { + return storageType; + } + + void setStorageType(int storageType) { + this.storageType = storageType; + } + + public int getEngineType() { + return engineType; + } + + void setEngineType(int engineType) { + this.engineType = engineType; + } + + public List<TblColRef> getAllColumnsNeedDictionary() { + List<TblColRef> result = Lists.newArrayList(); + + for (RowKeyColDesc rowKeyColDesc : rowkey.getRowKeyColumns()) { + TblColRef colRef = rowKeyColDesc.getColRef(); + if (rowkey.isUseDictionary(colRef)) { + result.add(colRef); + } + } + + for (MeasureDesc measure : measures) { + result.addAll(measure.getColumnsNeedDictionary()); + } + return result; + } + +} http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/DimensionDesc.java ---------------------------------------------------------------------- diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/DimensionDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/DimensionDesc.java new file mode 100644 index 0000000..3432d08 --- /dev/null +++ b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/DimensionDesc.java @@ -0,0 +1,239 @@ +/* + * 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. + */ + +package org.apache.kylin.cube.model.v2; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import org.apache.kylin.common.util.StringUtil; +import org.apache.kylin.metadata.model.JoinDesc; +import org.apache.kylin.metadata.model.LookupDesc; +import org.apache.kylin.metadata.model.TableDesc; +import org.apache.kylin.metadata.model.TblColRef; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + */ +@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE) +public class DimensionDesc { + + @JsonProperty("id") + private int id; + + @JsonProperty("name") + private String name; + + @JsonProperty("hierarchy") + private boolean isHierarchy; + @JsonProperty("table") + private String table; + @JsonProperty("column") + private String[] column; + @JsonProperty("derived") + private String[] derived; + + private TableDesc tableDesc; + private JoinDesc join; + private HierarchyDesc[] hierarchy; + + // computed + private TblColRef[] columnRefs; + private TblColRef[] derivedColRefs; + + public void init(CubeDesc cubeDesc, Map<String, TableDesc> tables) { + if (name != null) + name = name.toUpperCase(); + + if (table != null) + table = table.toUpperCase(); + + tableDesc = tables.get(this.getTable()); + if (tableDesc == null) + throw new IllegalStateException("Can't find table " + table + " for dimension " + name); + + join = null; + for (LookupDesc lookup : cubeDesc.getModel().getLookups()) { + if (lookup.getTable().equalsIgnoreCase(this.getTable())) { + join = lookup.getJoin(); + break; + } + } + + if (isHierarchy && this.column.length > 0) { + List<HierarchyDesc> hierarchyList = new ArrayList<HierarchyDesc>(3); + for (int i = 0, n = this.column.length; i < n; i++) { + String aColumn = this.column[i]; + HierarchyDesc aHierarchy = new HierarchyDesc(); + aHierarchy.setLevel(String.valueOf(i + 1)); + aHierarchy.setColumn(aColumn); + hierarchyList.add(aHierarchy); + } + + this.hierarchy = hierarchyList.toArray(new HierarchyDesc[hierarchyList.size()]); + } + + if (hierarchy != null && hierarchy.length == 0) + hierarchy = null; + if (derived != null && derived.length == 0) + derived = null; + + if (hierarchy != null) { + for (HierarchyDesc h : hierarchy) + h.setColumn(h.getColumn().toUpperCase()); + } + + if (derived != null) { + StringUtil.toUpperCaseArray(derived, derived); + } + + if (derived != null && join == null) { + throw new IllegalStateException("Derived can only be defined on lookup table, cube " + cubeDesc + ", " + this); + } + } + + public boolean isHierarchyColumn(TblColRef col) { + if (hierarchy == null) + return false; + + for (HierarchyDesc hier : hierarchy) { + if (hier.getColumnRef().equals(col)) + return true; + } + return false; + } + + public boolean isDerived() { + return derived != null; + } + + public boolean isHierarchy() { + return isHierarchy; + } + + public void setHierarchy(boolean isHierarchy) { + this.isHierarchy = isHierarchy; + } + + public String getTable() { + return table; + } + + public void setTable(String table) { + this.table = table; + } + + public int getId() { + return id; + } + + public void setId(int id) { + this.id = id; + } + + public JoinDesc getJoin() { + return join; + } + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public TblColRef[] getColumnRefs() { + return this.columnRefs; + } + + public void setColumnRefs(TblColRef[] colRefs) { + this.columnRefs = colRefs; + } + + public String[] getColumn() { + return this.column; + } + + public void setColumn(String[] column) { + this.column = column; + } + + public HierarchyDesc[] getHierarchy() { + return hierarchy; + } + + public void setHierarchy(HierarchyDesc[] hierarchy) { + this.hierarchy = hierarchy; + } + + public String[] getDerived() { + return derived; + } + + public void setDerived(String[] derived) { + this.derived = derived; + } + + public TblColRef[] getDerivedColRefs() { + return derivedColRefs; + } + + public void setDerivedColRefs(TblColRef[] derivedColRefs) { + this.derivedColRefs = derivedColRefs; + } + + public TableDesc getTableDesc() { + return this.tableDesc; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + DimensionDesc that = (DimensionDesc) o; + + if (id != that.id) + return false; + if (!name.equals(that.name)) + return false; + + return true; + } + + @Override + public int hashCode() { + int result = id; + result = 31 * result + name.hashCode(); + return result; + } + + @Override + public String toString() { + return "DimensionDesc [name=" + name + ", join=" + join + ", hierarchy=" + Arrays.toString(hierarchy) + ", table=" + table + ", column=" + Arrays.toString(column) + ", derived=" + Arrays.toString(derived) + "]"; + } + +} http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HBaseColumnDesc.java ---------------------------------------------------------------------- diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HBaseColumnDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HBaseColumnDesc.java new file mode 100644 index 0000000..1912e63 --- /dev/null +++ b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HBaseColumnDesc.java @@ -0,0 +1,138 @@ +/* + * 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. + */ + +package org.apache.kylin.cube.model.v2; + +import java.util.Arrays; + +import org.apache.kylin.metadata.model.FunctionDesc; +import org.apache.kylin.metadata.model.MeasureDesc; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + */ +@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE) +public class HBaseColumnDesc { + + @JsonProperty("qualifier") + private String qualifier; + @JsonProperty("measure_refs") + private String[] measureRefs; + + // these two will be assembled at runtime + private MeasureDesc[] measures; + private int[] measureIndex; // the index on CubeDesc.getMeasures() + private String columnFamilyName; + + public String getQualifier() { + return qualifier; + } + + public void setQualifier(String qualifier) { + this.qualifier = qualifier; + } + + public String[] getMeasureRefs() { + return measureRefs; + } + + public void setMeasureRefs(String[] measureRefs) { + this.measureRefs = measureRefs; + } + + public int[] getMeasureIndex() { + return measureIndex; + } + + public void setMeasureIndex(int[] index) { + this.measureIndex = index; + } + + public MeasureDesc[] getMeasures() { + return measures; + } + + public void setMeasures(MeasureDesc[] measures) { + this.measures = measures; + } + + public String getColumnFamilyName() { + return columnFamilyName; + } + + public void setColumnFamilyName(String columnFamilyName) { + this.columnFamilyName = columnFamilyName; + } + + public int findMeasure(FunctionDesc function) { + for (int i = 0; i < measures.length; i++) { + if (measures[i].getFunction().equals(function)) { + return i; + } + } + return -1; + } + + public boolean containsMeasure(String refName) { + for (String ref : measureRefs) { + if (ref.equals(refName)) + return true; + } + return false; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((columnFamilyName == null) ? 0 : columnFamilyName.hashCode()); + result = prime * result + ((qualifier == null) ? 0 : qualifier.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + HBaseColumnDesc other = (HBaseColumnDesc) obj; + if (columnFamilyName == null) { + if (other.columnFamilyName != null) + return false; + } else if (!columnFamilyName.equals(other.columnFamilyName)) + return false; + if (qualifier == null) { + if (other.qualifier != null) + return false; + } else if (!qualifier.equals(other.qualifier)) + return false; + return true; + } + + @Override + public String toString() { + return "HBaseColumnDesc [qualifier=" + qualifier + ", measureRefs=" + Arrays.toString(measureRefs) + "]"; + } + +} http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HBaseColumnFamilyDesc.java ---------------------------------------------------------------------- diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HBaseColumnFamilyDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HBaseColumnFamilyDesc.java new file mode 100644 index 0000000..73012f3 --- /dev/null +++ b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HBaseColumnFamilyDesc.java @@ -0,0 +1,58 @@ +/* + * 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. + */ + +package org.apache.kylin.cube.model.v2; + +import java.util.Arrays; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + */ +@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE) +public class HBaseColumnFamilyDesc { + + @JsonProperty("name") + private String name; + @JsonProperty("columns") + private HBaseColumnDesc[] columns; + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public HBaseColumnDesc[] getColumns() { + return columns; + } + + public void setColumns(HBaseColumnDesc[] columns) { + this.columns = columns; + } + + @Override + public String toString() { + return "HBaseColumnFamilyDesc [name=" + name + ", columns=" + Arrays.toString(columns) + "]"; + } + +} http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HBaseMappingDesc.java ---------------------------------------------------------------------- diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HBaseMappingDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HBaseMappingDesc.java new file mode 100644 index 0000000..0a15d5f --- /dev/null +++ b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HBaseMappingDesc.java @@ -0,0 +1,96 @@ +/* + * 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. + */ + +package org.apache.kylin.cube.model.v2; + +import java.util.Arrays; +import java.util.Collection; +import java.util.LinkedList; + +import org.apache.kylin.common.util.StringUtil; +import org.apache.kylin.metadata.model.FunctionDesc; +import org.apache.kylin.metadata.model.MeasureDesc; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + */ +@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE) +public class HBaseMappingDesc { + + @JsonProperty("column_family") + private HBaseColumnFamilyDesc[] columnFamily; + + // point to the cube instance which contain this HBaseMappingDesc instance. + private CubeDesc cubeRef; + + public Collection<HBaseColumnDesc> findHBaseColumnByFunction(FunctionDesc function) { + Collection<HBaseColumnDesc> result = new LinkedList<HBaseColumnDesc>(); + HBaseMappingDesc hbaseMapping = cubeRef.getHBaseMapping(); + if (hbaseMapping == null || hbaseMapping.getColumnFamily() == null) { + return result; + } + for (HBaseColumnFamilyDesc cf : hbaseMapping.getColumnFamily()) { + for (HBaseColumnDesc c : cf.getColumns()) { + for (MeasureDesc m : c.getMeasures()) { + if (m.getFunction().equals(function)) { + result.add(c); + } + } + } + } + return result; + } + + public CubeDesc getCubeRef() { + return cubeRef; + } + + public void setCubeRef(CubeDesc cubeRef) { + this.cubeRef = cubeRef; + } + + public HBaseColumnFamilyDesc[] getColumnFamily() { + return columnFamily; + } + + public void setColumnFamily(HBaseColumnFamilyDesc[] columnFamily) { + this.columnFamily = columnFamily; + } + + public void init(CubeDesc cubeDesc) { + cubeRef = cubeDesc; + + for (HBaseColumnFamilyDesc cf : columnFamily) { + cf.setName(cf.getName().toUpperCase()); + + for (HBaseColumnDesc c : cf.getColumns()) { + c.setQualifier(c.getQualifier().toUpperCase()); + StringUtil.toUpperCaseArray(c.getMeasureRefs(), c.getMeasureRefs()); + } + } + } + + @Override + public String toString() { + return "HBaseMappingDesc [columnFamily=" + Arrays.toString(columnFamily) + "]"; + } + +} http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HierarchyDesc.java ---------------------------------------------------------------------- diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HierarchyDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HierarchyDesc.java new file mode 100644 index 0000000..9dcf05b --- /dev/null +++ b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/HierarchyDesc.java @@ -0,0 +1,68 @@ +/* + * 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. + */ + +package org.apache.kylin.cube.model.v2; + +import org.apache.kylin.metadata.model.TblColRef; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + */ +@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE) +public class HierarchyDesc { + + @JsonProperty("level") + private String level; + @JsonProperty("column") + private String column; + + private TblColRef columnRef; + + public String getLevel() { + return level; + } + + public void setLevel(String level) { + this.level = level; + } + + public TblColRef getColumnRef() { + return columnRef; + } + + public void setColumnRef(TblColRef column) { + this.columnRef = column; + } + + public String getColumn() { + return column; + } + + public void setColumn(String columnName) { + this.column = columnName; + } + + @Override + public String toString() { + return "HierarchyDesc [level=" + level + ", column=" + column + "]"; + } + +} http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/RowKeyColDesc.java ---------------------------------------------------------------------- diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/RowKeyColDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/RowKeyColDesc.java new file mode 100644 index 0000000..86e4a53 --- /dev/null +++ b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/RowKeyColDesc.java @@ -0,0 +1,92 @@ +/* + * 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. + */ + +package org.apache.kylin.cube.model.v2; + +import org.apache.kylin.metadata.model.TblColRef; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * @author yangli9 + * + */ +@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE) +public class RowKeyColDesc { + + @JsonProperty("column") + private String column; + @JsonProperty("length") + private int length; + @JsonProperty("dictionary") + private String dictionary; + @JsonProperty("mandatory") + private boolean mandatory = false; + + // computed + private int bitIndex; + private TblColRef colRef; + + public String getDictionary() { + return dictionary; + } + + public String getColumn() { + return column; + } + + void setColumn(String column) { + this.column = column; + } + + public int getLength() { + return length; + } + + public boolean isMandatory() { + return mandatory; + } + + public int getBitIndex() { + return bitIndex; + } + + void setBitIndex(int index) { + this.bitIndex = index; + } + + public TblColRef getColRef() { + return colRef; + } + + void setColRef(TblColRef colRef) { + this.colRef = colRef; + } + + public void setDictionary(String dictionary) { + this.dictionary = dictionary; + } + + @Override + public String toString() { + return "RowKeyColDesc [column=" + column + ", length=" + length + ", dictionary=" + dictionary + ", mandatory=" + mandatory + "]"; + } + +} http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/RowKeyDesc.java ---------------------------------------------------------------------- diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/v2/RowKeyDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/RowKeyDesc.java new file mode 100644 index 0000000..95249ba --- /dev/null +++ b/core-cube/src/main/java/org/apache/kylin/cube/model/v2/RowKeyDesc.java @@ -0,0 +1,295 @@ +/* + * 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. + */ + +package org.apache.kylin.cube.model.v2; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.commons.lang.StringUtils; +import org.apache.kylin.common.util.StringUtil; +import org.apache.kylin.metadata.model.TblColRef; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + */ +@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE) +public class RowKeyDesc { + + public static class HierarchyMask { + public long fullMask; + public long[] allMasks; + } + + public static class AggrGroupMask { + public AggrGroupMask(int size) { + groupOneBitMasks = new long[size]; + } + + public long groupMask; + public long groupOneBitMasks[]; + public long uniqueMask; + public long leftoverMask; + } + + @JsonProperty("rowkey_columns") + private RowKeyColDesc[] rowkeyColumns; + @JsonProperty("aggregation_groups") + private String[][] aggregationGroups; + + // computed content + private CubeDesc cubeDesc; + private Map<TblColRef, RowKeyColDesc> columnMap; + + private long fullMask; + private long mandatoryColumnMask; + private AggrGroupMask[] aggrGroupMasks; + private long aggrGroupFullMask; + private long tailMask; + + private List<HierarchyMask> hierarchyMasks; + + public RowKeyColDesc[] getRowKeyColumns() { + return rowkeyColumns; + } + + // search a specific row key col + public int getRowKeyIndexByColumnName(String columnName) { + if (this.rowkeyColumns == null) + return -1; + + for (int i = 0; i < this.rowkeyColumns.length; ++i) { + RowKeyColDesc desc = this.rowkeyColumns[i]; + if (desc.getColumn().equalsIgnoreCase(columnName)) { + return i; + } + } + return -1; + } + + public int getNCuboidBuildLevels() { + // N aggregation columns requires N levels of cuboid build + // - N columns requires N-1 levels build + // - zero tail cuboid needs one more additional level + Set<String> aggDims = new HashSet<String>(); + for (String[] aggrGroup : aggregationGroups) { + for (String dim : aggrGroup) { + aggDims.add(dim); + } + } + return aggDims.size(); + } + + public String[][] getAggregationGroups() { + return aggregationGroups; + } + + public CubeDesc getCubeRef() { + return cubeDesc; + } + + public void setCubeRef(CubeDesc cubeRef) { + this.cubeDesc = cubeRef; + } + + public long getFullMask() { + return fullMask; + } + + public long getMandatoryColumnMask() { + return mandatoryColumnMask; + } + + public long getAggrGroupFullMask() { + return aggrGroupFullMask; + } + + public AggrGroupMask[] getAggrGroupMasks() { + return aggrGroupMasks; + } + + public List<HierarchyMask> getHierarchyMasks() { + return hierarchyMasks; + } + + public long getTailMask() { + return tailMask; + } + + public int getColumnBitIndex(TblColRef col) { + return getColDesc(col).getBitIndex(); + } + + public int getColumnLength(TblColRef col) { + return getColDesc(col).getLength(); + } + + public String getDictionary(TblColRef col) { + return getColDesc(col).getDictionary(); + } + + private RowKeyColDesc getColDesc(TblColRef col) { + RowKeyColDesc desc = columnMap.get(col); + if (desc == null) + throw new NullPointerException("Column " + col + " does not exist in row key desc"); + return desc; + } + + public boolean isUseDictionary(int index) { + String useDictionary = rowkeyColumns[index].getDictionary(); + return useDictionary(useDictionary); + } + + public boolean isUseDictionary(TblColRef col) { + String useDictionary = getDictionary(col); + return useDictionary(useDictionary); + } + + private boolean useDictionary(String useDictionary) { + return !StringUtils.isBlank(useDictionary) && !"false".equals(useDictionary); + } + + public void init(CubeDesc cube) { + setCubeRef(cube); + Map<String, TblColRef> colNameAbbr = cube.buildColumnNameAbbreviation(); + + buildRowKey(colNameAbbr); + buildAggregationGroups(colNameAbbr); + buildHierarchyMasks(); + } + + @Override + public String toString() { + return "RowKeyDesc [rowkeyColumns=" + Arrays.toString(rowkeyColumns) + ", aggregationGroups=" + Arrays.toString(aggregationGroups) + "]"; + } + + private void buildRowKey(Map<String, TblColRef> colNameAbbr) { + columnMap = new HashMap<TblColRef, RowKeyColDesc>(); + mandatoryColumnMask = 0; + + for (int i = 0; i < rowkeyColumns.length; i++) { + RowKeyColDesc rowKeyColDesc = rowkeyColumns[i]; + String column = rowKeyColDesc.getColumn(); + rowKeyColDesc.setColumn(column.toUpperCase()); + rowKeyColDesc.setBitIndex(rowkeyColumns.length - i - 1); + rowKeyColDesc.setColRef(colNameAbbr.get(column)); + if (rowKeyColDesc.getColRef() == null) { + throw new IllegalArgumentException("Cannot find rowkey column " + column + " in cube " + cubeDesc); + } + + columnMap.put(rowKeyColDesc.getColRef(), rowKeyColDesc); + + if (rowKeyColDesc.isMandatory()) { + mandatoryColumnMask |= 1L << rowKeyColDesc.getBitIndex(); + } + } + } + + private void buildAggregationGroups(Map<String, TblColRef> colNameAbbr) { + if (aggregationGroups == null) { + aggregationGroups = new String[0][]; + } + + for (int i = 0; i < aggregationGroups.length; i++) { + StringUtil.toUpperCaseArray(aggregationGroups[i], this.aggregationGroups[i]); + } + + for (int i = 0; i < this.rowkeyColumns.length; i++) { + int index = rowkeyColumns[i].getBitIndex(); + this.fullMask |= 1L << index; + } + + this.aggrGroupMasks = new AggrGroupMask[aggregationGroups.length]; + for (int i = 0; i < this.aggregationGroups.length; i++) { + String[] aggGrp = this.aggregationGroups[i]; + AggrGroupMask mask = new AggrGroupMask(aggGrp.length); + + for (int j = 0; j < aggGrp.length; j++) { + TblColRef aggCol = colNameAbbr.get(aggGrp[j].toUpperCase()); + if (aggCol == null) { + throw new IllegalArgumentException("Can't find aggregation column " + aggGrp[j] + " in cube " + this.cubeDesc.getName()); + } + Integer index = getColumnBitIndex(aggCol); + mask.groupMask |= 1L << index; + mask.groupOneBitMasks[j] = 1L << index; + this.aggrGroupFullMask |= 1L << index; + } + this.aggrGroupMasks[i] = mask; + } + + this.tailMask = fullMask ^ mandatoryColumnMask ^ aggrGroupFullMask; + + // unique mask = (bits in this group) - (bits in following groups) + // leftover mask = (tail bits) + (bits in following groups) - (bits in + // this group) + for (int i = 0; i < aggrGroupMasks.length; i++) { + AggrGroupMask mask = aggrGroupMasks[i]; + + mask.uniqueMask = mask.groupMask; + for (int j = i + 1; j < aggrGroupMasks.length; j++) { + mask.uniqueMask &= ~aggrGroupMasks[j].groupMask; + } + + mask.leftoverMask = tailMask; + for (int j = i + 1; j < aggrGroupMasks.length; j++) { + mask.leftoverMask |= aggrGroupMasks[j].groupMask; + } + mask.leftoverMask &= ~mask.groupMask; + } + } + + private void buildHierarchyMasks() { + this.hierarchyMasks = new ArrayList<HierarchyMask>(); + + for (DimensionDesc dimension : this.cubeDesc.getDimensions()) { + HierarchyDesc[] hierarchies = dimension.getHierarchy(); + if (hierarchies == null || hierarchies.length == 0) + continue; + + HierarchyMask mask = new HierarchyMask(); + ArrayList<Long> allMaskList = new ArrayList<Long>(); + for (int i = 0; i < hierarchies.length; i++) { + TblColRef hColumn = hierarchies[i].getColumnRef(); + Integer index = getColumnBitIndex(hColumn); + long bit = 1L << index; + + if ((tailMask & bit) > 0) + continue; // ignore levels in tail, they don't participate + // aggregation group combination anyway + + mask.fullMask |= bit; + allMaskList.add(mask.fullMask); + } + + mask.allMasks = new long[allMaskList.size()]; + for (int i = 0; i < allMaskList.size(); i++) + mask.allMasks[i] = allMaskList.get(i); + + this.hierarchyMasks.add(mask); + } + } + +} http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/CubeMetadataValidator.java ---------------------------------------------------------------------- diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/CubeMetadataValidator.java b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/CubeMetadataValidator.java index 2f8fc7b..7d7710c 100644 --- a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/CubeMetadataValidator.java +++ b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/CubeMetadataValidator.java @@ -19,9 +19,8 @@ package org.apache.kylin.cube.model.validation; import org.apache.kylin.cube.model.CubeDesc; -import org.apache.kylin.cube.model.validation.rule.AggregationGroupSizeRule; +import org.apache.kylin.cube.model.validation.rule.AggregationGroupRule; import org.apache.kylin.cube.model.validation.rule.FunctionRule; -import org.apache.kylin.cube.model.validation.rule.MandatoryColumnRule; import org.apache.kylin.cube.model.validation.rule.RowKeyAttrRule; /** @@ -32,16 +31,14 @@ import org.apache.kylin.cube.model.validation.rule.RowKeyAttrRule; */ public class CubeMetadataValidator { @SuppressWarnings("unchecked") - private IValidatorRule<CubeDesc>[] rules = new IValidatorRule[] { new FunctionRule(), new AggregationGroupSizeRule(), new MandatoryColumnRule(), new RowKeyAttrRule() }; + private IValidatorRule<CubeDesc>[] rules = new IValidatorRule[] { new FunctionRule(), new AggregationGroupRule(), new RowKeyAttrRule() }; public ValidateContext validate(CubeDesc cube) { return validate(cube, false); } /** - * @param cubeDesc - * @param inject - * inject error into cube desc + * @param inject inject error into cube desc * @return */ public ValidateContext validate(CubeDesc cube, boolean inject) { http://git-wip-us.apache.org/repos/asf/kylin/blob/07537a9a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/ValidateContext.java ---------------------------------------------------------------------- diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/ValidateContext.java b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/ValidateContext.java index e33ec19..632cbff 100644 --- a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/ValidateContext.java +++ b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/ValidateContext.java @@ -25,9 +25,6 @@ import java.util.List; /** * Context. Supply all dependent objects for validator - * - * @author jianliu - * */ public class ValidateContext { private List<Result> results = new ArrayList<ValidateContext.Result>(); @@ -84,7 +81,7 @@ public class ValidateContext { */ public void print(PrintStream out) { if (results.isEmpty()) { - out.print("The element is perfect."); + out.println("The element is perfect."); } Iterator<Result> it = results.iterator(); while (it.hasNext()) {